From af55afd8d6839e38337f67e19a614ea3eae9a2cf Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Wed, 17 May 2017 17:21:14 -0700 Subject: [PATCH 01/25] Fix SPARK-12139: REGEX Column Specification for Hive Queries --- .../sql/catalyst/analysis/unresolved.scala | 27 ++++++ .../sql/catalyst/parser/AstBuilder.scala | 44 ++++++++-- .../org/apache/spark/sql/SQLQuerySuite.scala | 88 +++++++++++++++++++ 3 files changed, 152 insertions(+), 7 deletions(-) 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 51bef6e20b9fa..4e1b26dd0bd14 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 @@ -83,6 +83,33 @@ case class UnresolvedTableValuedFunction( override lazy val resolved = false } +/** + * 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: LogicalPlan, 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.output.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.output.filter(_.qualifier.filter(resolver(_, t)).nonEmpty) + .filter(_.name.matches(expr)) + } + + expandedAttributes.zip(input.output).map { + case (n: NamedExpression, _) => n + case (e, originalAttribute) => + Alias(e, originalAttribute.name)() + } + } + + override def toString: String = table.map(_ + ".").getOrElse("") + expr +} + /** * Holds the name of an attribute that has yet to be resolved. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 0e974a02e29d4..ed6a1bdce7ee4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -23,10 +23,9 @@ import javax.xml.bind.DatatypeConverter import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer - import org.antlr.v4.runtime.{ParserRuleContext, Token} import org.antlr.v4.runtime.tree.{ParseTree, RuleNode, TerminalNode} - +import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} @@ -1229,15 +1228,33 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging CaseWhen(branches, Option(ctx.elseExpression).map(expression)) } + def enableHiveSupportQuotedIdentifiers() : Boolean = { + SparkEnv.get != null && + SparkEnv.get.conf != null && + SparkEnv.get.conf.getBoolean("hive.support.quoted.identifiers", false) + } + /** - * Create a dereference expression. The return type depends on the type of the parent, this can - * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an - * [[UnresolvedExtractValue]] if the parent is some expression. + * Create a dereference expression. The return type depends on the type of the parent. + * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or + * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression, + * it can be [[UnresolvedExtractValue]]. */ override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) { val attr = ctx.fieldName.getText expression(ctx.base) match { - case UnresolvedAttribute(nameParts) => + case unresolved_attr @ UnresolvedAttribute(nameParts) => + if (enableHiveSupportQuotedIdentifiers) { + val escapedIdentifier = "`(.+)`".r + val ret = Option(ctx.fieldName.getStart).map(_.getText match { + case r@escapedIdentifier(i) => + UnresolvedRegex(i, Some(unresolved_attr.name)) + case _ => + UnresolvedAttribute(nameParts :+ attr) + }) + return ret.get + } + UnresolvedAttribute(nameParts :+ attr) case e => UnresolvedExtractValue(e, Literal(attr)) @@ -1245,9 +1262,22 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } /** - * Create an [[UnresolvedAttribute]] expression. + * Create an [[UnresolvedAttribute]] expression or a [[UnresolvedRegex]] if it is a regex + * quoted in `` */ override def visitColumnReference(ctx: ColumnReferenceContext): Expression = withOrigin(ctx) { + if (enableHiveSupportQuotedIdentifiers) { + val escapedIdentifier = "`(.+)`".r + val ret = Option(ctx.getStart).map(_.getText match { + case r @ escapedIdentifier(i) => + UnresolvedRegex(i, None) + case _ => + UnresolvedAttribute.quoted(ctx.getText) + }) + + return ret.get + } + UnresolvedAttribute.quoted(ctx.getText) } 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 cd14d24370bad..ef41858a5c6df 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 @@ -2624,4 +2624,92 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { val e = intercept[AnalysisException](sql("SELECT nvl(1, 2, 3)")) assert(e.message.contains("Invalid number of arguments")) } + + test("SPARK-12139: REGEX Column Specification for Hive Queries") { + // hive.support.quoted.identifiers is turned off by default + checkAnswer( + sql( + """ + |SELECT b + |FROM testData2 + |WHERE a = 1 + """.stripMargin), + Row(1) :: Row(2) :: Nil) + + checkAnswer( + sql( + """ + |SELECT t.b + |FROM testData2 t + |WHERE a = 1 + """.stripMargin), + Row(1) :: Row(2) :: Nil) + + intercept[AnalysisException] { + sql( + """ + |SELECT `(a)?+.+` + |FROM testData2 + |WHERE a = 1 + """.stripMargin) + } + + intercept[AnalysisException] { + sql( + """ + |SELECT t.`(a)?+.+` + |FROM testData2 t + |WHERE a = 1 + """.stripMargin) + } + + // now, turn on hive.support.quoted.identifiers + sparkContext.conf.set("hive.support.quoted.identifiers", "true") + + checkAnswer( + sql( + """ + |SELECT b + |FROM testData2 + |WHERE a = 1 + """.stripMargin), + Row(1) :: Row(2) :: Nil) + + checkAnswer( + sql( + """ + |SELECT t.b + |FROM testData2 t + |WHERE a = 1 + """.stripMargin), + Row(1) :: Row(2) :: Nil) + + checkAnswer( + sql( + """ + |SELECT `(a)?+.+` + |FROM testData2 + |WHERE a = 1 + """.stripMargin), + Row(1) :: Row(2) :: Nil) + + checkAnswer( + sql( + """ + |SELECT t.`(a)?+.+` + |FROM testData2 t + |WHERE a = 1 + """.stripMargin), + Row(1) :: Row(2) :: Nil) + + checkAnswer( + sql( + """ + |SELECT p.`(key)?+.+`, b, testdata2.`(b)?+.+` + |FROM testData p join testData2 + |ON p.key = testData2.a + |WHERE key < 3 + """.stripMargin), + Row("1", 1, 1) :: Row("1", 2, 1) :: Row("2", 1, 2) :: Row("2", 2, 2) ::Nil) + } } From 6f9bdb0e935cc609ed86f5c1ddc0d014720c3f56 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Wed, 17 May 2017 17:21:14 -0700 Subject: [PATCH 02/25] Fix SPARK-12139: REGEX Column Specification for Hive Queries --- .../sql/catalyst/analysis/unresolved.scala | 27 ++++++ .../sql/catalyst/parser/AstBuilder.scala | 36 ++++++-- .../apache/spark/sql/internal/SQLConf.scala | 8 ++ .../org/apache/spark/sql/SQLQuerySuite.scala | 88 +++++++++++++++++++ 4 files changed, 154 insertions(+), 5 deletions(-) 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 51bef6e20b9fa..4e1b26dd0bd14 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 @@ -83,6 +83,33 @@ case class UnresolvedTableValuedFunction( override lazy val resolved = false } +/** + * 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: LogicalPlan, 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.output.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.output.filter(_.qualifier.filter(resolver(_, t)).nonEmpty) + .filter(_.name.matches(expr)) + } + + expandedAttributes.zip(input.output).map { + case (n: NamedExpression, _) => n + case (e, originalAttribute) => + Alias(e, originalAttribute.name)() + } + } + + override def toString: String = table.map(_ + ".").getOrElse("") + expr +} + /** * Holds the name of an attribute that has yet to be resolved. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 0e974a02e29d4..fe17ed0f9f154 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -28,6 +28,7 @@ import org.antlr.v4.runtime.{ParserRuleContext, Token} import org.antlr.v4.runtime.tree.{ParseTree, RuleNode, TerminalNode} import org.apache.spark.internal.Logging +import org.apache.spark.SparkEnv import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.analysis._ @@ -1230,14 +1231,26 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } /** - * Create a dereference expression. The return type depends on the type of the parent, this can - * either be a [[UnresolvedAttribute]] (if the parent is an [[UnresolvedAttribute]]), or an - * [[UnresolvedExtractValue]] if the parent is some expression. + * Create a dereference expression. The return type depends on the type of the parent. + * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or + * a [[UnresolvedRegex]] for regex quoted in ``; if the parent is some other expression, + * it can be [[UnresolvedExtractValue]]. */ override def visitDereference(ctx: DereferenceContext): Expression = withOrigin(ctx) { val attr = ctx.fieldName.getText expression(ctx.base) match { - case UnresolvedAttribute(nameParts) => + case unresolved_attr @ UnresolvedAttribute(nameParts) => + if (conf.supportQuotedIdentifiers) { + val escapedIdentifier = "`(.+)`".r + val ret = Option(ctx.fieldName.getStart).map(_.getText match { + case r@escapedIdentifier(i) => + UnresolvedRegex(i, Some(unresolved_attr.name)) + case _ => + UnresolvedAttribute(nameParts :+ attr) + }) + return ret.get + } + UnresolvedAttribute(nameParts :+ attr) case e => UnresolvedExtractValue(e, Literal(attr)) @@ -1245,9 +1258,22 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } /** - * Create an [[UnresolvedAttribute]] expression. + * Create an [[UnresolvedAttribute]] expression or a [[UnresolvedRegex]] if it is a regex + * quoted in `` */ override def visitColumnReference(ctx: ColumnReferenceContext): Expression = withOrigin(ctx) { + if (conf.supportQuotedIdentifiers) { + val escapedIdentifier = "`(.+)`".r + val ret = Option(ctx.getStart).map(_.getText match { + case r @ escapedIdentifier(i) => + UnresolvedRegex(i, None) + case _ => + UnresolvedAttribute.quoted(ctx.getText) + }) + + return ret.get + } + UnresolvedAttribute.quoted(ctx.getText) } 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 b97adf7221d18..b27450529ea99 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 @@ -795,6 +795,12 @@ object SQLConf { .intConf .createWithDefault(UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD.toInt) + val SUPPORT_QUOTED_IDENTIFIERS = buildConf("spark.sql.support.quoted.identifiers") + .internal() + .doc("When true, identifiers specified by regex patterns will be expanded.") + .booleanConf + .createWithDefault(false) + object Deprecated { val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks" } @@ -1051,6 +1057,8 @@ class SQLConf extends Serializable with Logging { def starSchemaFTRatio: Double = getConf(STARSCHEMA_FACT_TABLE_RATIO) + def supportQuotedIdentifiers: Boolean = getConf(SUPPORT_QUOTED_IDENTIFIERS) + /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ 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 cd14d24370bad..118507530d599 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 @@ -2624,4 +2624,92 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { val e = intercept[AnalysisException](sql("SELECT nvl(1, 2, 3)")) assert(e.message.contains("Invalid number of arguments")) } + + test("SPARK-12139: REGEX Column Specification for Hive Queries") { + // hive.support.quoted.identifiers is turned off by default + checkAnswer( + sql( + """ + |SELECT b + |FROM testData2 + |WHERE a = 1 + """.stripMargin), + Row(1) :: Row(2) :: Nil) + + checkAnswer( + sql( + """ + |SELECT t.b + |FROM testData2 t + |WHERE a = 1 + """.stripMargin), + Row(1) :: Row(2) :: Nil) + + intercept[AnalysisException] { + sql( + """ + |SELECT `(a)?+.+` + |FROM testData2 + |WHERE a = 1 + """.stripMargin) + } + + intercept[AnalysisException] { + sql( + """ + |SELECT t.`(a)?+.+` + |FROM testData2 t + |WHERE a = 1 + """.stripMargin) + } + + // now, turn on hive.support.quoted.identifiers + withSQLConf(SQLConf.SUPPORT_QUOTED_IDENTIFIERS.key -> "true") { + checkAnswer( + sql( + """ + |SELECT b + |FROM testData2 + |WHERE a = 1 + """.stripMargin), + Row(1) :: Row(2) :: Nil) + + checkAnswer( + sql( + """ + |SELECT t.b + |FROM testData2 t + |WHERE a = 1 + """.stripMargin), + Row(1) :: Row(2) :: Nil) + + checkAnswer( + sql( + """ + |SELECT `(a)?+.+` + |FROM testData2 + |WHERE a = 1 + """.stripMargin), + Row(1) :: Row(2) :: Nil) + + checkAnswer( + sql( + """ + |SELECT t.`(a)?+.+` + |FROM testData2 t + |WHERE a = 1 + """.stripMargin), + Row(1) :: Row(2) :: Nil) + + checkAnswer( + sql( + """ + |SELECT p.`(key)?+.+`, b, testdata2.`(b)?+.+` + |FROM testData p join testData2 + |ON p.key = testData2.a + |WHERE key < 3 + """.stripMargin), + Row("1", 1, 1) :: Row("1", 2, 1) :: Row("2", 1, 2) :: Row("2", 2, 2) :: Nil) + } + } } From 7699e871a31e37755b35c88b893faf9df8f7664f Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Thu, 18 May 2017 16:06:39 -0700 Subject: [PATCH 03/25] add unittests for DataSet. --- .../sql/catalyst/parser/AstBuilder.scala | 4 +-- .../org/apache/spark/sql/DatasetSuite.scala | 34 +++++++++++++++++++ 2 files changed, 36 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 0e41417bc1548..1b72a395a316f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -23,11 +23,11 @@ import javax.xml.bind.DatatypeConverter import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer + import org.antlr.v4.runtime.{ParserRuleContext, Token} import org.antlr.v4.runtime.tree.{ParseTree, RuleNode, TerminalNode} -import org.apache.spark.SparkEnv + import org.apache.spark.internal.Logging -import org.apache.spark.SparkEnv import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.analysis._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 8eb381b91f46d..176f4539882d4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -244,6 +244,40 @@ class DatasetSuite extends QueryTest with SharedSQLContext { ("a", ClassData("a", 1)), ("b", ClassData("b", 2)), ("c", ClassData("c", 3))) } + test("select 3, regex") { + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDF() + intercept[AnalysisException] { + ds.select(expr("`(_1)?+.+`").as[Int]) + } + + intercept[AnalysisException] { + ds.select(expr("`(_1|_2)`").as[Int]) + } + + withSQLConf(SQLConf.SUPPORT_QUOTED_IDENTIFIERS.key -> "true") { + checkDataset( + ds.select(expr("`(_1)?+.+`").as[Int]), + 1, 2, 3) + val m = ds.select(expr("`(_1|_2)`")) + + checkDataset( + ds.select(expr("`(_1|_2)`")) + .select(expr("named_struct('a', _1, 'b', _2)").as[ClassData]), + ClassData("a", 1), ClassData("b", 2), ClassData("c", 3)) + + checkDataset( + ds.alias("g") + .select(expr("g.`(_1)?+.+`").as[Int]), + 1, 2, 3) + + checkDataset( + ds.alias("g") + .select(expr("g.`(_1|_2)`")) + .select(expr("named_struct('a', _1, 'b', _2)").as[ClassData]), + ClassData("a", 1), ClassData("b", 2), ClassData("c", 3)) + } + } + test("filter") { val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() checkDataset( From 6e375177e68a216cdd53de1e5d600d898b2b59d5 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Thu, 18 May 2017 21:22:11 -0700 Subject: [PATCH 04/25] Address hvanhovell's comments Blame Rev: --- .../sql/catalyst/analysis/unresolved.scala | 19 ++++------- .../sql/catalyst/parser/AstBuilder.scala | 33 ++++++------------- .../sql/catalyst/parser/ParserUtils.scala | 12 +++++++ 3 files changed, 29 insertions(+), 35 deletions(-) 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 4e1b26dd0bd14..d87b60a58d189 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 @@ -90,24 +90,19 @@ case class UnresolvedTableValuedFunction( * @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 { +case class UnresolvedRegex(regexPattern: String, table: Option[String]) + extends Star with Unevaluable { override def expand(input: LogicalPlan, resolver: Resolver): Seq[NamedExpression] = { - val expandedAttributes: Seq[Attribute] = table match { + table match { // If there is no table specified, use all input attributes that match expr - case None => input.output.filter(_.name.matches(expr)) + case None => input.output.filter(_.name.matches(regexPattern)) // If there is a table, pick out attributes that are part of this table that match expr - case Some(t) => input.output.filter(_.qualifier.filter(resolver(_, t)).nonEmpty) - .filter(_.name.matches(expr)) - } - - expandedAttributes.zip(input.output).map { - case (n: NamedExpression, _) => n - case (e, originalAttribute) => - Alias(e, originalAttribute.name)() + case Some(t) => input.output.filter(_.qualifier.exists(resolver(_, t))) + .filter(_.name.matches(regexPattern)) } } - override def toString: String = table.map(_ + ".").getOrElse("") + expr + override def toString: String = table.map(_ + ".").getOrElse("") + regexPattern } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 1b72a395a316f..1ae3e26585d4c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1239,18 +1239,12 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging val attr = ctx.fieldName.getText expression(ctx.base) match { case unresolved_attr @ UnresolvedAttribute(nameParts) => - if (conf.supportQuotedIdentifiers) { - val escapedIdentifier = "`(.+)`".r - val ret = Option(ctx.fieldName.getStart).map(_.getText match { - case r@escapedIdentifier(i) => - UnresolvedRegex(i, Some(unresolved_attr.name)) - case _ => - UnresolvedAttribute(nameParts :+ attr) - }) - return ret.get + matchEscapedIdentifier(ctx.fieldName.getStart.getText) match { + case Some(i) if conf.supportQuotedIdentifiers => + UnresolvedRegex(i, Some(unresolved_attr.name)) + case _ => + UnresolvedAttribute(nameParts :+ attr) } - - UnresolvedAttribute(nameParts :+ attr) case e => UnresolvedExtractValue(e, Literal(attr)) } @@ -1261,19 +1255,12 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * quoted in `` */ override def visitColumnReference(ctx: ColumnReferenceContext): Expression = withOrigin(ctx) { - if (conf.supportQuotedIdentifiers) { - val escapedIdentifier = "`(.+)`".r - val ret = Option(ctx.getStart).map(_.getText match { - case r @ escapedIdentifier(i) => - UnresolvedRegex(i, None) - case _ => - UnresolvedAttribute.quoted(ctx.getText) - }) - - return ret.get + matchEscapedIdentifier(ctx.getStart.getText) match { + case Some(i) if conf.supportQuotedIdentifiers => + UnresolvedRegex(i, None) + case _ => + UnresolvedAttribute.quoted(ctx.getText) } - - UnresolvedAttribute.quoted(ctx.getText) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala index 77fdaa8255aa6..ae3d334766ee3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala @@ -177,6 +177,18 @@ object ParserUtils { sb.toString() } + val escapedIdentifier = "`(.+)`".r + + /** + * Return the substring extracted using regex + */ + def matchEscapedIdentifier(b: String): Option[String] = { + b match { + case escapedIdentifier(i) => Some(i) + case _ => None + } + } + /** Some syntactic sugar which makes it easier to work with optional clauses for LogicalPlans. */ implicit class EnhancedLogicalPlan(val plan: LogicalPlan) extends AnyVal { /** From bee07cd569f60d4b6bfe33bf64dd99e00255cd19 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Fri, 19 May 2017 00:19:39 -0700 Subject: [PATCH 05/25] Address gatorsmile's comments --- .../sql/catalyst/parser/AstBuilder.scala | 8 ++--- .../sql/catalyst/parser/ParserUtils.scala | 12 ++----- .../apache/spark/sql/internal/SQLConf.scala | 6 ++-- .../scala/org/apache/spark/sql/Dataset.scala | 13 ++++--- .../org/apache/spark/sql/DatasetSuite.scala | 35 +++++++++++++++++-- .../org/apache/spark/sql/SQLQuerySuite.scala | 2 +- 6 files changed, 52 insertions(+), 24 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 1ae3e26585d4c..183eb2e4e80ba 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1239,8 +1239,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging val attr = ctx.fieldName.getText expression(ctx.base) match { case unresolved_attr @ UnresolvedAttribute(nameParts) => - matchEscapedIdentifier(ctx.fieldName.getStart.getText) match { - case Some(i) if conf.supportQuotedIdentifiers => + ctx.fieldName.getStart.getText match { + case escapedIdentifier(i) if conf.supportQuotedRegexColumnName => UnresolvedRegex(i, Some(unresolved_attr.name)) case _ => UnresolvedAttribute(nameParts :+ attr) @@ -1255,8 +1255,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * quoted in `` */ override def visitColumnReference(ctx: ColumnReferenceContext): Expression = withOrigin(ctx) { - matchEscapedIdentifier(ctx.getStart.getText) match { - case Some(i) if conf.supportQuotedIdentifiers => + ctx.getStart.getText match { + case escapedIdentifier(i) if conf.supportQuotedRegexColumnName => UnresolvedRegex(i, None) case _ => UnresolvedAttribute.quoted(ctx.getText) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala index ae3d334766ee3..9c1031e8033e7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala @@ -177,17 +177,11 @@ object ParserUtils { sb.toString() } + /** the column name pattern in quoted regex without qualifier */ val escapedIdentifier = "`(.+)`".r - /** - * Return the substring extracted using regex - */ - def matchEscapedIdentifier(b: String): Option[String] = { - b match { - case escapedIdentifier(i) => Some(i) - case _ => None - } - } + /** the column name pattern in quoted regex with qualifier */ + val qualifiedEscapedIdentifier = ("(.+)" + """.""" + "`(.+)`").r /** Some syntactic sugar which makes it easier to work with optional clauses for LogicalPlans. */ implicit class EnhancedLogicalPlan(val plan: LogicalPlan) extends AnyVal { 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 b27450529ea99..e8b15f181f92f 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 @@ -795,9 +795,9 @@ object SQLConf { .intConf .createWithDefault(UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD.toInt) - val SUPPORT_QUOTED_IDENTIFIERS = buildConf("spark.sql.support.quoted.identifiers") + val SUPPORT_QUOTED_REGEX_COLUMN_NAME = buildConf("spark.sql.parser.quotedRegexColumnNames") .internal() - .doc("When true, identifiers specified by regex patterns will be expanded.") + .doc("When true, column names specified by quoted regex pattern will be expanded.") .booleanConf .createWithDefault(false) @@ -1057,7 +1057,7 @@ class SQLConf extends Serializable with Logging { def starSchemaFTRatio: Double = getConf(STARSCHEMA_FACT_TABLE_RATIO) - def supportQuotedIdentifiers: Boolean = getConf(SUPPORT_QUOTED_IDENTIFIERS) + def supportQuotedRegexColumnName: Boolean = getConf(SUPPORT_QUOTED_REGEX_COLUMN_NAME) /** ********************** SQLConf functionality methods ************ */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index c75921e867f64..089db2500533c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -25,9 +25,7 @@ import scala.collection.JavaConverters._ import scala.language.implicitConversions import scala.reflect.runtime.universe.TypeTag import scala.util.control.NonFatal - import org.apache.commons.lang3.StringUtils - import org.apache.spark.annotation.{DeveloperApi, Experimental, InterfaceStability} import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.java.function._ @@ -40,13 +38,13 @@ import org.apache.spark.sql.catalyst.catalog.CatalogRelation import org.apache.spark.sql.catalyst.encoders._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ -import org.apache.spark.sql.catalyst.json.{JacksonGenerator, JSONOptions} +import org.apache.spark.sql.catalyst.json.{JSONOptions, JacksonGenerator} import org.apache.spark.sql.catalyst.optimizer.CombineUnions -import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.catalyst.parser.{ParseException, ParserUtils} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection} -import org.apache.spark.sql.catalyst.util.{usePrettyExpression, DateTimeUtils} +import org.apache.spark.sql.catalyst.util.{DateTimeUtils, usePrettyExpression} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.LogicalRelation @@ -1188,6 +1186,11 @@ class Dataset[T] private[sql]( def col(colName: String): Column = colName match { case "*" => Column(ResolvedStar(queryExecution.analyzed.output)) + case ParserUtils.escapedIdentifier(i) if sqlContext.conf.supportQuotedRegexColumnName => + Column(UnresolvedRegex(i, None)) + case ParserUtils.qualifiedEscapedIdentifier(i, j) + if sqlContext.conf.supportQuotedRegexColumnName => + Column(UnresolvedRegex(j, Some(i))) case _ => val expr = resolve(colName) Column(expr) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 176f4539882d4..111624076b38a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -245,7 +245,8 @@ class DatasetSuite extends QueryTest with SharedSQLContext { } test("select 3, regex") { - val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDF() + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() + intercept[AnalysisException] { ds.select(expr("`(_1)?+.+`").as[Int]) } @@ -254,7 +255,37 @@ class DatasetSuite extends QueryTest with SharedSQLContext { ds.select(expr("`(_1|_2)`").as[Int]) } - withSQLConf(SQLConf.SUPPORT_QUOTED_IDENTIFIERS.key -> "true") { + intercept[AnalysisException] { + ds.select(ds("`(_1)?+.+`")) + } + + intercept[AnalysisException] { + ds.select(ds("`(_1|_2)`")) + } + + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "true") { + checkDataset( + ds.select(ds("`(_1|_2)`")) + .select(expr("named_struct('a', _1, 'b', _2)").as[ClassData]), + ClassData("a", 1), ClassData("b", 2), ClassData("c", 3)) + + checkDataset( + ds.alias("g") + .select(ds("g.`(_1|_2)`")) + .select(expr("named_struct('a', _1, 'b', _2)").as[ClassData]), + ClassData("a", 1), ClassData("b", 2), ClassData("c", 3)) + + checkDataset( + ds.select(ds("`(_1)?+.+`")) + .select(expr("_2").as[Int]), + 1, 2, 3) + + checkDataset( + ds.alias("g") + .select(ds("g.`(_1)?+.+`")) + .select(expr("_2").as[Int]), + 1, 2, 3) + checkDataset( ds.select(expr("`(_1)?+.+`").as[Int]), 1, 2, 3) 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 118507530d599..e2ec9f189d97f 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 @@ -2664,7 +2664,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } // now, turn on hive.support.quoted.identifiers - withSQLConf(SQLConf.SUPPORT_QUOTED_IDENTIFIERS.key -> "true") { + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "true") { checkAnswer( sql( """ From d5e450adb3e305d9630ec09bb6cf5d7e8e148cf0 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Fri, 19 May 2017 11:26:26 -0700 Subject: [PATCH 06/25] address gatorsmile's comment --- .../apache/spark/sql/internal/SQLConf.scala | 1 - .../scala/org/apache/spark/sql/Dataset.scala | 6 +- .../sql-tests/inputs/query_regex_column.sql | 24 +++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 88 ------------------- 4 files changed, 28 insertions(+), 91 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql 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 e8b15f181f92f..7d0df5026d5fc 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 @@ -796,7 +796,6 @@ object SQLConf { .createWithDefault(UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD.toInt) val SUPPORT_QUOTED_REGEX_COLUMN_NAME = buildConf("spark.sql.parser.quotedRegexColumnNames") - .internal() .doc("When true, column names specified by quoted regex pattern will be expanded.") .booleanConf .createWithDefault(false) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 089db2500533c..d031298d31ac7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -25,7 +25,9 @@ import scala.collection.JavaConverters._ import scala.language.implicitConversions import scala.reflect.runtime.universe.TypeTag import scala.util.control.NonFatal + import org.apache.commons.lang3.StringUtils + import org.apache.spark.annotation.{DeveloperApi, Experimental, InterfaceStability} import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.java.function._ @@ -38,13 +40,13 @@ import org.apache.spark.sql.catalyst.catalog.CatalogRelation import org.apache.spark.sql.catalyst.encoders._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ -import org.apache.spark.sql.catalyst.json.{JSONOptions, JacksonGenerator} +import org.apache.spark.sql.catalyst.json.{JacksonGenerator, JSONOptions} import org.apache.spark.sql.catalyst.optimizer.CombineUnions import org.apache.spark.sql.catalyst.parser.{ParseException, ParserUtils} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection} -import org.apache.spark.sql.catalyst.util.{DateTimeUtils, usePrettyExpression} +import org.apache.spark.sql.catalyst.util.{usePrettyExpression, DateTimeUtils} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.LogicalRelation diff --git a/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql b/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql new file mode 100644 index 0000000000000..cf7599184cf9e --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql @@ -0,0 +1,24 @@ +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, "1"), (2, "2"), (3, "3"), (4, "4"), (5, "5"), (6, "6") +AS testData(key, value); + +CREATE OR REPLACE TEMPORARY VIEW testData2 AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2) +AS testData2(a, b); + +-- AnalysisException +SELECT `(a)?+.+` FROM testData2 WHERE a = 1; + +-- AnalysisException +SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1; + +set spark.sql.parser.quotedRegexColumnNames=true; + +-- Regex columns +SELECT `(a)?+.+` FROM testData2 WHERE a = 1; +SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1; +SELECT p.`(key)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3; + +-- Clean-up +DROP VIEW IF EXISTS testData; +DROP VIEW IF EXISTS testData2; 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 e2ec9f189d97f..cd14d24370bad 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 @@ -2624,92 +2624,4 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { val e = intercept[AnalysisException](sql("SELECT nvl(1, 2, 3)")) assert(e.message.contains("Invalid number of arguments")) } - - test("SPARK-12139: REGEX Column Specification for Hive Queries") { - // hive.support.quoted.identifiers is turned off by default - checkAnswer( - sql( - """ - |SELECT b - |FROM testData2 - |WHERE a = 1 - """.stripMargin), - Row(1) :: Row(2) :: Nil) - - checkAnswer( - sql( - """ - |SELECT t.b - |FROM testData2 t - |WHERE a = 1 - """.stripMargin), - Row(1) :: Row(2) :: Nil) - - intercept[AnalysisException] { - sql( - """ - |SELECT `(a)?+.+` - |FROM testData2 - |WHERE a = 1 - """.stripMargin) - } - - intercept[AnalysisException] { - sql( - """ - |SELECT t.`(a)?+.+` - |FROM testData2 t - |WHERE a = 1 - """.stripMargin) - } - - // now, turn on hive.support.quoted.identifiers - withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "true") { - checkAnswer( - sql( - """ - |SELECT b - |FROM testData2 - |WHERE a = 1 - """.stripMargin), - Row(1) :: Row(2) :: Nil) - - checkAnswer( - sql( - """ - |SELECT t.b - |FROM testData2 t - |WHERE a = 1 - """.stripMargin), - Row(1) :: Row(2) :: Nil) - - checkAnswer( - sql( - """ - |SELECT `(a)?+.+` - |FROM testData2 - |WHERE a = 1 - """.stripMargin), - Row(1) :: Row(2) :: Nil) - - checkAnswer( - sql( - """ - |SELECT t.`(a)?+.+` - |FROM testData2 t - |WHERE a = 1 - """.stripMargin), - Row(1) :: Row(2) :: Nil) - - checkAnswer( - sql( - """ - |SELECT p.`(key)?+.+`, b, testdata2.`(b)?+.+` - |FROM testData p join testData2 - |ON p.key = testData2.a - |WHERE key < 3 - """.stripMargin), - Row("1", 1, 1) :: Row("1", 2, 1) :: Row("2", 1, 2) :: Row("2", 2, 2) :: Nil) - } - } } From 979bfb61b10f808fb751a6867ca297e0a9e69bad Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Fri, 19 May 2017 11:28:20 -0700 Subject: [PATCH 07/25] add the gold file --- .../results/query_regex_column.sql.out | 93 +++++++++++++++++++ 1 file changed, 93 insertions(+) create mode 100644 sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out diff --git a/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out b/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out new file mode 100644 index 0000000000000..91d33b2126c4c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out @@ -0,0 +1,93 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 10 + + +-- !query 0 +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, "1"), (2, "2"), (3, "3"), (4, "4"), (5, "5"), (6, "6") +AS testData(key, value) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE OR REPLACE TEMPORARY VIEW testData2 AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2) +AS testData2(a, b) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +SELECT `(a)?+.+` FROM testData2 WHERE a = 1 +-- !query 2 schema +struct<> +-- !query 2 output +org.apache.spark.sql.AnalysisException +cannot resolve '```(a)?+.+```' given input columns: [a, b]; line 1 pos 7 + + +-- !query 3 +SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1 +-- !query 3 schema +struct<> +-- !query 3 output +org.apache.spark.sql.AnalysisException +cannot resolve '`t.``(a)?+.+```' given input columns: [a, b]; line 1 pos 7 + + +-- !query 4 +set spark.sql.parser.quotedRegexColumnNames=true +-- !query 4 schema +struct +-- !query 4 output +spark.sql.parser.quotedRegexColumnNames true + + +-- !query 5 +SELECT `(a)?+.+` FROM testData2 WHERE a = 1 +-- !query 5 schema +struct +-- !query 5 output +1 +2 + + +-- !query 6 +SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1 +-- !query 6 schema +struct +-- !query 6 output +1 +2 + + +-- !query 7 +SELECT p.`(key)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3 +-- !query 7 schema +struct +-- !query 7 output +1 1 1 +1 2 1 +2 1 2 +2 2 2 + + +-- !query 8 +DROP VIEW IF EXISTS testData +-- !query 8 schema +struct<> +-- !query 8 output + + + +-- !query 9 +DROP VIEW IF EXISTS testData2 +-- !query 9 schema +struct<> +-- !query 9 output + From 612bedf9bb1181687fa536d2e927923901c19582 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Fri, 19 May 2017 15:58:47 -0700 Subject: [PATCH 08/25] address gatorsmile's comment --- .../sql/catalyst/parser/AstBuilder.scala | 8 +- .../scala/org/apache/spark/sql/Dataset.scala | 11 +-- .../sql-tests/inputs/query_regex_column.sql | 18 ++--- .../results/query_regex_column.sql.out | 76 ++++++++++++------- 4 files changed, 66 insertions(+), 47 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 183eb2e4e80ba..e2349195aa9dc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1240,8 +1240,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging expression(ctx.base) match { case unresolved_attr @ UnresolvedAttribute(nameParts) => ctx.fieldName.getStart.getText match { - case escapedIdentifier(i) if conf.supportQuotedRegexColumnName => - UnresolvedRegex(i, Some(unresolved_attr.name)) + case escapedIdentifier(columnNameRegex) if conf.supportQuotedRegexColumnName => + UnresolvedRegex(columnNameRegex, Some(unresolved_attr.name)) case _ => UnresolvedAttribute(nameParts :+ attr) } @@ -1256,8 +1256,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging */ override def visitColumnReference(ctx: ColumnReferenceContext): Expression = withOrigin(ctx) { ctx.getStart.getText match { - case escapedIdentifier(i) if conf.supportQuotedRegexColumnName => - UnresolvedRegex(i, None) + case escapedIdentifier(columnNameRegex) if conf.supportQuotedRegexColumnName => + UnresolvedRegex(columnNameRegex, None) case _ => UnresolvedAttribute.quoted(ctx.getText) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index d031298d31ac7..05924458a85ef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -1188,11 +1188,12 @@ class Dataset[T] private[sql]( def col(colName: String): Column = colName match { case "*" => Column(ResolvedStar(queryExecution.analyzed.output)) - case ParserUtils.escapedIdentifier(i) if sqlContext.conf.supportQuotedRegexColumnName => - Column(UnresolvedRegex(i, None)) - case ParserUtils.qualifiedEscapedIdentifier(i, j) - if sqlContext.conf.supportQuotedRegexColumnName => - Column(UnresolvedRegex(j, Some(i))) + case ParserUtils.escapedIdentifier(columnNameRegex) + if sqlContext.conf.supportQuotedRegexColumnName => + Column(UnresolvedRegex(columnNameRegex, None)) + case ParserUtils.qualifiedEscapedIdentifier(nameParts, columnNameRegex) + if sqlContext.conf.supportQuotedRegexColumnName => + Column(UnresolvedRegex(columnNameRegex, Some(nameParts))) case _ => val expr = resolve(colName) Column(expr) diff --git a/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql b/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql index cf7599184cf9e..a6fb4b76b7361 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql @@ -1,24 +1,22 @@ CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES -(1, "1"), (2, "2"), (3, "3"), (4, "4"), (5, "5"), (6, "6") -AS testData(key, value); +(1, "1", "11"), (2, "2", "22"), (3, "3", "33"), (4, "4", "44"), (5, "5", "55"), (6, "6", "66") +AS testData(key, value1, value2); CREATE OR REPLACE TEMPORARY VIEW testData2 AS SELECT * FROM VALUES -(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2) -AS testData2(a, b); +(1, 1, 1, 2), (1, 2, 1, 2), (2, 1, 2, 3), (2, 2, 2, 3), (3, 1, 3, 4), (3, 2, 3, 4) +AS testData2(a, b, c, d); -- AnalysisException SELECT `(a)?+.+` FROM testData2 WHERE a = 1; - --- AnalysisException SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1; +SELECT `(a|b)` FROM testData2 WHERE a = 2; +SELECT `(a|b)?+.+` FROM testData2 WHERE a = 2; set spark.sql.parser.quotedRegexColumnNames=true; -- Regex columns SELECT `(a)?+.+` FROM testData2 WHERE a = 1; SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1; +SELECT `(a|b)` FROM testData2 WHERE a = 2; +SELECT `(a|b)?+.+` FROM testData2 WHERE a = 2; SELECT p.`(key)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3; - --- Clean-up -DROP VIEW IF EXISTS testData; -DROP VIEW IF EXISTS testData2; diff --git a/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out b/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out index 91d33b2126c4c..fcf9021d77095 100644 --- a/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out @@ -1,11 +1,11 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 10 +-- Number of queries: 12 -- !query 0 CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES -(1, "1"), (2, "2"), (3, "3"), (4, "4"), (5, "5"), (6, "6") -AS testData(key, value) +(1, "1", "11"), (2, "2", "22"), (3, "3", "33"), (4, "4", "44"), (5, "5", "55"), (6, "6", "66") +AS testData(key, value1, value2) -- !query 0 schema struct<> -- !query 0 output @@ -14,8 +14,8 @@ struct<> -- !query 1 CREATE OR REPLACE TEMPORARY VIEW testData2 AS SELECT * FROM VALUES -(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2) -AS testData2(a, b) +(1, 1, 1, 2), (1, 2, 1, 2), (2, 1, 2, 3), (2, 2, 2, 3), (3, 1, 3, 4), (3, 2, 3, 4) +AS testData2(a, b, c, d) -- !query 1 schema struct<> -- !query 1 output @@ -28,7 +28,7 @@ SELECT `(a)?+.+` FROM testData2 WHERE a = 1 struct<> -- !query 2 output org.apache.spark.sql.AnalysisException -cannot resolve '```(a)?+.+```' given input columns: [a, b]; line 1 pos 7 +cannot resolve '```(a)?+.+```' given input columns: [a, b, c, d]; line 1 pos 7 -- !query 3 @@ -37,57 +37,77 @@ SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1 struct<> -- !query 3 output org.apache.spark.sql.AnalysisException -cannot resolve '`t.``(a)?+.+```' given input columns: [a, b]; line 1 pos 7 +cannot resolve '`t.``(a)?+.+```' given input columns: [a, b, c, d]; line 1 pos 7 -- !query 4 -set spark.sql.parser.quotedRegexColumnNames=true +SELECT `(a|b)` FROM testData2 WHERE a = 2 -- !query 4 schema -struct +struct<> -- !query 4 output -spark.sql.parser.quotedRegexColumnNames true +org.apache.spark.sql.AnalysisException +cannot resolve '`(a|b)`' given input columns: [a, b, c, d]; line 1 pos 7 -- !query 5 -SELECT `(a)?+.+` FROM testData2 WHERE a = 1 +SELECT `(a|b)?+.+` FROM testData2 WHERE a = 2 -- !query 5 schema -struct +struct<> -- !query 5 output -1 -2 +org.apache.spark.sql.AnalysisException +cannot resolve '```(a|b)?+.+```' given input columns: [a, b, c, d]; line 1 pos 7 -- !query 6 -SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1 +set spark.sql.parser.quotedRegexColumnNames=true -- !query 6 schema -struct +struct -- !query 6 output -1 -2 +spark.sql.parser.quotedRegexColumnNames true -- !query 7 -SELECT p.`(key)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3 +SELECT `(a)?+.+` FROM testData2 WHERE a = 1 -- !query 7 schema -struct +struct -- !query 7 output -1 1 1 -1 2 1 +1 1 2 2 1 2 -2 2 2 -- !query 8 -DROP VIEW IF EXISTS testData +SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1 -- !query 8 schema -struct<> +struct -- !query 8 output - +1 1 2 +2 1 2 -- !query 9 -DROP VIEW IF EXISTS testData2 +SELECT `(a|b)` FROM testData2 WHERE a = 2 -- !query 9 schema -struct<> +struct -- !query 9 output +2 1 +2 2 + + +-- !query 10 +SELECT `(a|b)?+.+` FROM testData2 WHERE a = 2 +-- !query 10 schema +struct +-- !query 10 output +2 3 +2 3 + +-- !query 11 +SELECT p.`(key)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3 +-- !query 11 schema +struct +-- !query 11 output +1 11 1 1 1 2 +1 11 2 1 1 2 +2 22 1 2 2 3 +2 22 2 2 2 3 From 48c54aad51d38c99c5fe39de9d382d2f56f42314 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Fri, 19 May 2017 16:19:56 -0700 Subject: [PATCH 09/25] address gatorsmile's comment --- sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 111624076b38a..5ea912de08835 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -244,7 +244,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { ("a", ClassData("a", 1)), ("b", ClassData("b", 2)), ("c", ClassData("c", 3))) } - test("select 3, regex") { + test("REGEX column specification") { val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() intercept[AnalysisException] { From 0284d01f24cbe2e5362530aa744e2d4e06e9d1e5 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Mon, 22 May 2017 10:12:20 -0700 Subject: [PATCH 10/25] address cloud-fan's commentS --- .../sql/catalyst/analysis/unresolved.scala | 44 +++++++++---------- 1 file changed, 22 insertions(+), 22 deletions(-) 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 d87b60a58d189..30ab1ce13bbdc 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 @@ -83,28 +83,6 @@ case class UnresolvedTableValuedFunction( override lazy val resolved = false } -/** - * 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(regexPattern: String, table: Option[String]) - extends Star with Unevaluable { - override def expand(input: LogicalPlan, resolver: Resolver): Seq[NamedExpression] = { - table match { - // If there is no table specified, use all input attributes that match expr - case None => input.output.filter(_.name.matches(regexPattern)) - // If there is a table, pick out attributes that are part of this table that match expr - case Some(t) => input.output.filter(_.qualifier.exists(resolver(_, t))) - .filter(_.name.matches(regexPattern)) - } - } - - override def toString: String = table.map(_ + ".").getOrElse("") + regexPattern -} - /** * Holds the name of an attribute that has yet to be resolved. */ @@ -310,6 +288,28 @@ 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(regexPattern: String, table: Option[String]) + extends Star with Unevaluable { + override def expand(input: LogicalPlan, resolver: Resolver): Seq[NamedExpression] = { + table match { + // If there is no table specified, use all input attributes that match expr + case None => input.output.filter(_.name.matches(regexPattern)) + // If there is a table, pick out attributes that are part of this table that match expr + case Some(t) => input.output.filter(_.qualifier.exists(resolver(_, t))) + .filter(_.name.matches(regexPattern)) + } + } + + override def toString: String = table.map(_ + "." + regexPattern).getOrElse(regexPattern) +} + /** * 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 From 779724daae609976d6893a4aec7198bbb4f90095 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Fri, 26 May 2017 16:58:46 -0700 Subject: [PATCH 11/25] Address cloud-fan's comments --- .../sql/catalyst/parser/AstBuilder.scala | 21 ++++++++++--------- .../org/apache/spark/sql/DatasetSuite.scala | 2 +- 2 files changed, 12 insertions(+), 11 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 40dbc485e1307..97d58178cd6e9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1239,11 +1239,10 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging val attr = ctx.fieldName.getText expression(ctx.base) match { case unresolved_attr @ UnresolvedAttribute(nameParts) => - ctx.fieldName.getStart.getText match { - case escapedIdentifier(columnNameRegex) if conf.supportQuotedRegexColumnName => - UnresolvedRegex(columnNameRegex, Some(unresolved_attr.name)) - case _ => - UnresolvedAttribute(nameParts :+ attr) + if (conf.supportQuotedRegexColumnName) { + UnresolvedRegex(attr, Some(unresolved_attr.name)) + } else { + UnresolvedAttribute(nameParts :+ attr) } case e => UnresolvedExtractValue(e, Literal(attr)) @@ -1255,12 +1254,14 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * quoted in `` */ override def visitColumnReference(ctx: ColumnReferenceContext): Expression = withOrigin(ctx) { - ctx.getStart.getText match { - case escapedIdentifier(columnNameRegex) if conf.supportQuotedRegexColumnName => - UnresolvedRegex(columnNameRegex, None) - case _ => - UnresolvedAttribute.quoted(ctx.getText) + if (conf.supportQuotedRegexColumnName) { + if (!ctx.getParent().isInstanceOf[DereferenceContext] || + ctx.getParent().asInstanceOf[DereferenceContext].fieldName == this) { + return UnresolvedRegex(ctx.getText, None) + } } + + UnresolvedAttribute.quoted(ctx.getText) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 5ea912de08835..b50c75fe1cdc5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -22,7 +22,7 @@ import java.sql.{Date, Timestamp} import org.apache.spark.sql.catalyst.encoders.{OuterScopes, RowEncoder} import org.apache.spark.sql.catalyst.util.sideBySide -import org.apache.spark.sql.execution.{LogicalRDD, RDDScanExec, SortExec} +import org.apache.spark.sql.execution.{LogicalRDD, RDDScanExec} import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ShuffleExchange} import org.apache.spark.sql.execution.streaming.MemoryStream import org.apache.spark.sql.functions._ From a27023c69ace6026a345dc745206c20e71ac4299 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Fri, 26 May 2017 18:41:45 -0700 Subject: [PATCH 12/25] fix typo --- .../scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 97d58178cd6e9..e02892aa55370 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1256,7 +1256,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging override def visitColumnReference(ctx: ColumnReferenceContext): Expression = withOrigin(ctx) { if (conf.supportQuotedRegexColumnName) { if (!ctx.getParent().isInstanceOf[DereferenceContext] || - ctx.getParent().asInstanceOf[DereferenceContext].fieldName == this) { + ctx.getParent().asInstanceOf[DereferenceContext].fieldName == ctx) { return UnresolvedRegex(ctx.getText, None) } } From a0e3773b67dfd20b0b1c340741ab174969823a6e Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Tue, 30 May 2017 12:38:33 -0700 Subject: [PATCH 13/25] roll back code --- .../sql/catalyst/parser/AstBuilder.scala | 21 +++++++++---------- 1 file changed, 10 insertions(+), 11 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 4c86abeef3393..db6c98b4b2de4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1250,10 +1250,11 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging val attr = ctx.fieldName.getText expression(ctx.base) match { case unresolved_attr @ UnresolvedAttribute(nameParts) => - if (conf.supportQuotedRegexColumnName) { - UnresolvedRegex(attr, Some(unresolved_attr.name)) - } else { - UnresolvedAttribute(nameParts :+ attr) + ctx.fieldName.getStart.getText match { + case escapedIdentifier(columnNameRegex) if conf.supportQuotedRegexColumnName => + UnresolvedRegex(columnNameRegex, Some(unresolved_attr.name)) + case _ => + UnresolvedAttribute(nameParts :+ attr) } case e => UnresolvedExtractValue(e, Literal(attr)) @@ -1265,14 +1266,12 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * quoted in `` */ override def visitColumnReference(ctx: ColumnReferenceContext): Expression = withOrigin(ctx) { - if (conf.supportQuotedRegexColumnName) { - if (!ctx.getParent().isInstanceOf[DereferenceContext] || - ctx.getParent().asInstanceOf[DereferenceContext].fieldName == ctx) { - return UnresolvedRegex(ctx.getText, None) - } + ctx.getStart.getText match { + case escapedIdentifier(columnNameRegex) if conf.supportQuotedRegexColumnName => + UnresolvedRegex(columnNameRegex, None) + case _ => + UnresolvedAttribute.quoted(ctx.getText) } - - UnresolvedAttribute.quoted(ctx.getText) } /** From da60368d0c90d12f4ddd33aa0894615618aa01dc Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Mon, 26 Jun 2017 12:31:08 -0700 Subject: [PATCH 14/25] add gatorsmile's comments --- .../sql/catalyst/analysis/unresolved.scala | 10 ++++++- .../apache/spark/sql/internal/SQLConf.scala | 14 +++++----- .../scala/org/apache/spark/sql/Dataset.scala | 22 +++++++++++---- .../sql-tests/inputs/query_regex_column.sql | 4 ++- .../results/query_regex_column.sql.out | 28 +++++++++++++++---- 5 files changed, 57 insertions(+), 21 deletions(-) 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 9b7a2e2b46f9a..a0ec30ee675b7 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 @@ -22,6 +22,7 @@ import org.apache.spark.sql.catalyst.{FunctionIdentifier, InternalRow, TableIden import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodegenFallback, ExprCode} +import org.apache.spark.sql.catalyst.parser.ParserUtils import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan} import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.util.quoteIdentifier @@ -123,7 +124,14 @@ case class UnresolvedAttribute(nameParts: Seq[String]) extends Attribute with Un override def toString: String = s"'$name" - override def sql: String = quoteIdentifier(name) + override def sql: String = { + name match { + case ParserUtils.escapedIdentifier(_) | + ParserUtils.qualifiedEscapedIdentifier(_, _) => name + case _ => quoteIdentifier(name) + + } + } } object UnresolvedAttribute { 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 abe1137caaf09..13d1a85fc920a 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 @@ -847,11 +847,6 @@ object SQLConf { .intConf .createWithDefault(UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD.toInt) - val SUPPORT_QUOTED_REGEX_COLUMN_NAME = buildConf("spark.sql.parser.quotedRegexColumnNames") - .doc("When true, column names specified by quoted regex pattern will be expanded.") - .booleanConf - .createWithDefault(false) - val ARROW_EXECUTION_ENABLE = buildConf("spark.sql.execution.arrow.enable") .internal() @@ -870,6 +865,11 @@ object SQLConf { .intConf .createWithDefault(10000) + val SUPPORT_QUOTED_REGEX_COLUMN_NAME = buildConf("spark.sql.parser.quotedRegexColumnNames") + .doc("When true, a SELECT statement can take regex-based column specification.") + .booleanConf + .createWithDefault(false) + object Deprecated { val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks" } @@ -1128,12 +1128,12 @@ class SQLConf extends Serializable with Logging { def starSchemaFTRatio: Double = getConf(STARSCHEMA_FACT_TABLE_RATIO) - def supportQuotedRegexColumnName: Boolean = getConf(SUPPORT_QUOTED_REGEX_COLUMN_NAME) - def arrowEnable: Boolean = getConf(ARROW_EXECUTION_ENABLE) def arrowMaxRecordsPerBatch: Int = getConf(ARROW_EXECUTION_MAX_RECORDS_PER_BATCH) + def supportQuotedRegexColumnName: Boolean = getConf(SUPPORT_QUOTED_REGEX_COLUMN_NAME) + /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 9914321bbbdcb..1f88e53077fd6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -1188,15 +1188,25 @@ class Dataset[T] private[sql]( def col(colName: String): Column = colName match { case "*" => Column(ResolvedStar(queryExecution.analyzed.output)) - case ParserUtils.escapedIdentifier(columnNameRegex) - if sqlContext.conf.supportQuotedRegexColumnName => + case _ => + if (sqlContext.conf.supportQuotedRegexColumnName) { + colRegex(colName) + } else { + val expr = resolve(colName) + Column(expr) + } + } + + /** + * Selects column based on the column name specified as a regex and return it as [[Column]]. + */ + def colRegex(colName: String): Column = colName match { + case ParserUtils.escapedIdentifier(columnNameRegex) => Column(UnresolvedRegex(columnNameRegex, None)) - case ParserUtils.qualifiedEscapedIdentifier(nameParts, columnNameRegex) - if sqlContext.conf.supportQuotedRegexColumnName => + case ParserUtils.qualifiedEscapedIdentifier(nameParts, columnNameRegex) => Column(UnresolvedRegex(columnNameRegex, Some(nameParts))) case _ => - val expr = resolve(colName) - Column(expr) + Column(resolve(colName)) } /** diff --git a/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql b/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql index a6fb4b76b7361..8a73770996a9b 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql @@ -19,4 +19,6 @@ SELECT `(a)?+.+` FROM testData2 WHERE a = 1; SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1; SELECT `(a|b)` FROM testData2 WHERE a = 2; SELECT `(a|b)?+.+` FROM testData2 WHERE a = 2; -SELECT p.`(key)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3; +SELECT `(e|f)` FROM testData2; +SELECT t.`(e|f)` FROM testData2 t; +SELECT p.`(key)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3; \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out b/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out index fcf9021d77095..26a3870c675bf 100644 --- a/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 12 +-- Number of queries: 14 -- !query 0 @@ -28,7 +28,7 @@ SELECT `(a)?+.+` FROM testData2 WHERE a = 1 struct<> -- !query 2 output org.apache.spark.sql.AnalysisException -cannot resolve '```(a)?+.+```' given input columns: [a, b, c, d]; line 1 pos 7 +cannot resolve '`(a)?+.+`' given input columns: [a, b, c, d]; line 1 pos 7 -- !query 3 @@ -37,7 +37,7 @@ SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1 struct<> -- !query 3 output org.apache.spark.sql.AnalysisException -cannot resolve '`t.``(a)?+.+```' given input columns: [a, b, c, d]; line 1 pos 7 +cannot resolve 't.`(a)?+.+`' given input columns: [a, b, c, d]; line 1 pos 7 -- !query 4 @@ -55,7 +55,7 @@ SELECT `(a|b)?+.+` FROM testData2 WHERE a = 2 struct<> -- !query 5 output org.apache.spark.sql.AnalysisException -cannot resolve '```(a|b)?+.+```' given input columns: [a, b, c, d]; line 1 pos 7 +cannot resolve '`(a|b)?+.+`' given input columns: [a, b, c, d]; line 1 pos 7 -- !query 6 @@ -103,10 +103,26 @@ struct -- !query 11 -SELECT p.`(key)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3 +SELECT `(e|f)` FROM testData2 -- !query 11 schema -struct +struct<> -- !query 11 output + + + +-- !query 12 +SELECT t.`(e|f)` FROM testData2 t +-- !query 12 schema +struct<> +-- !query 12 output + + + +-- !query 13 +SELECT p.`(key)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3 +-- !query 13 schema +struct +-- !query 13 output 1 11 1 1 1 2 1 11 2 1 1 2 2 22 1 2 2 3 From 79e58f01dbd6c279f85d91a0ceed5479ea33da90 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Tue, 27 Jun 2017 23:01:23 -0700 Subject: [PATCH 15/25] address gatorsmile's comments --- .../sql/catalyst/analysis/unresolved.scala | 10 +++------- .../scala/org/apache/spark/sql/Dataset.scala | 2 ++ .../sql-tests/inputs/query_regex_column.sql | 2 +- .../org/apache/spark/sql/DatasetSuite.scala | 20 ++++++++----------- 4 files changed, 14 insertions(+), 20 deletions(-) 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 a0ec30ee675b7..e2dcd7745e663 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 @@ -124,13 +124,9 @@ case class UnresolvedAttribute(nameParts: Seq[String]) extends Attribute with Un override def toString: String = s"'$name" - override def sql: String = { - name match { - case ParserUtils.escapedIdentifier(_) | - ParserUtils.qualifiedEscapedIdentifier(_, _) => name - case _ => quoteIdentifier(name) - - } + override def sql: String = name match { + case ParserUtils.escapedIdentifier(_) | ParserUtils.qualifiedEscapedIdentifier(_, _) => name + case _ => quoteIdentifier(name) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 1f88e53077fd6..de282348e002f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -1199,6 +1199,8 @@ class Dataset[T] private[sql]( /** * Selects column based on the column name specified as a regex and return it as [[Column]]. + * @group untypedrel + * @since 2.3.0 */ def colRegex(colName: String): Column = colName match { case ParserUtils.escapedIdentifier(columnNameRegex) => diff --git a/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql b/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql index 8a73770996a9b..fe3be8d01625a 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql @@ -21,4 +21,4 @@ SELECT `(a|b)` FROM testData2 WHERE a = 2; SELECT `(a|b)?+.+` FROM testData2 WHERE a = 2; SELECT `(e|f)` FROM testData2; SELECT t.`(e|f)` FROM testData2 t; -SELECT p.`(key)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3; \ No newline at end of file +SELECT p.`(key)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3; diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 14f1b35f63470..e3dd3dd815e72 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -247,21 +247,17 @@ class DatasetSuite extends QueryTest with SharedSQLContext { test("REGEX column specification") { val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() - intercept[AnalysisException] { - ds.select(expr("`(_1)?+.+`").as[Int]) - } + var e = intercept[AnalysisException] { ds.select(expr("`(_1)?+.+`").as[Int]) }.getMessage + assert(e.contains("cannot resolve '`(_1)?+.+`'")) - intercept[AnalysisException] { - ds.select(expr("`(_1|_2)`").as[Int]) - } + e = intercept[AnalysisException] { ds.select(expr("`(_1|_2)`").as[Int]) }.getMessage + assert(e.contains("cannot resolve '`(_1|_2)`'")) - intercept[AnalysisException] { - ds.select(ds("`(_1)?+.+`")) - } + e = intercept[AnalysisException] { ds.select(ds("`(_1)?+.+`")) }.getMessage + assert(e.contains("Cannot resolve column name \"`(_1)?+.+`\"")) - intercept[AnalysisException] { - ds.select(ds("`(_1|_2)`")) - } + e = intercept[AnalysisException] { ds.select(ds("`(_1|_2)`")) }.getMessage + assert(e.contains("Cannot resolve column name \"`(_1|_2)`\"")) withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "true") { checkDataset( From 616b7265dc6b16fcd47567bb777e0b7797fc07cd Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Wed, 28 Jun 2017 00:14:25 -0700 Subject: [PATCH 16/25] made regex case insensitive --- .../sql/catalyst/parser/AstBuilder.scala | 4 +- .../scala/org/apache/spark/sql/Dataset.scala | 3 +- .../sql-tests/inputs/query_regex_column.sql | 5 ++ .../results/query_regex_column.sql.out | 79 +++++++++++++++---- 4 files changed, 72 insertions(+), 19 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 3b4860665631b..df9bf52378ed0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1261,7 +1261,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging val attr = ctx.fieldName.getText expression(ctx.base) match { case unresolved_attr @ UnresolvedAttribute(nameParts) => - ctx.fieldName.getStart.getText match { + ctx.fieldName.getStart.getText.toLowerCase(Locale.ROOT) match { case escapedIdentifier(columnNameRegex) if conf.supportQuotedRegexColumnName => UnresolvedRegex(columnNameRegex, Some(unresolved_attr.name)) case _ => @@ -1277,7 +1277,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * quoted in `` */ override def visitColumnReference(ctx: ColumnReferenceContext): Expression = withOrigin(ctx) { - ctx.getStart.getText match { + ctx.getStart.getText.toLowerCase(Locale.ROOT) match { case escapedIdentifier(columnNameRegex) if conf.supportQuotedRegexColumnName => UnresolvedRegex(columnNameRegex, None) case _ => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index de282348e002f..8d3264afb6ec1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql import java.io.CharArrayWriter import java.sql.{Date, Timestamp} +import java.util.Locale import scala.collection.JavaConverters._ import scala.language.implicitConversions @@ -1202,7 +1203,7 @@ class Dataset[T] private[sql]( * @group untypedrel * @since 2.3.0 */ - def colRegex(colName: String): Column = colName match { + def colRegex(colName: String): Column = colName.toLowerCase(Locale.ROOT) match { case ParserUtils.escapedIdentifier(columnNameRegex) => Column(UnresolvedRegex(columnNameRegex, None)) case ParserUtils.qualifiedEscapedIdentifier(nameParts, columnNameRegex) => diff --git a/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql b/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql index fe3be8d01625a..5dc5e31f34827 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql @@ -16,9 +16,14 @@ set spark.sql.parser.quotedRegexColumnNames=true; -- Regex columns SELECT `(a)?+.+` FROM testData2 WHERE a = 1; +SELECT `(A)?+.+` FROM testData2 WHERE a = 1; SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1; +SELECT t.`(A)?+.+` FROM testData2 t WHERE a = 1; SELECT `(a|b)` FROM testData2 WHERE a = 2; +SELECT `(A|b)` FROM testData2 WHERE a = 2; SELECT `(a|b)?+.+` FROM testData2 WHERE a = 2; +SELECT `(A|b)?+.+` FROM testData2 WHERE a = 2; SELECT `(e|f)` FROM testData2; SELECT t.`(e|f)` FROM testData2 t; +SELECT p.`(KEY)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3; SELECT p.`(key)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3; diff --git a/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out b/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out index 26a3870c675bf..a7452bae4ef55 100644 --- a/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 14 +-- Number of queries: 19 -- !query 0 @@ -76,7 +76,7 @@ struct -- !query 8 -SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1 +SELECT `(A)?+.+` FROM testData2 WHERE a = 1 -- !query 8 schema struct -- !query 8 output @@ -85,44 +85,91 @@ struct -- !query 9 -SELECT `(a|b)` FROM testData2 WHERE a = 2 +SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1 -- !query 9 schema -struct +struct -- !query 9 output +1 1 2 +2 1 2 + + +-- !query 10 +SELECT t.`(A)?+.+` FROM testData2 t WHERE a = 1 +-- !query 10 schema +struct +-- !query 10 output +1 1 2 +2 1 2 + + +-- !query 11 +SELECT `(a|b)` FROM testData2 WHERE a = 2 +-- !query 11 schema +struct +-- !query 11 output 2 1 2 2 --- !query 10 +-- !query 12 +SELECT `(A|b)` FROM testData2 WHERE a = 2 +-- !query 12 schema +struct +-- !query 12 output +2 1 +2 2 + + +-- !query 13 SELECT `(a|b)?+.+` FROM testData2 WHERE a = 2 --- !query 10 schema +-- !query 13 schema struct --- !query 10 output +-- !query 13 output 2 3 2 3 --- !query 11 +-- !query 14 +SELECT `(A|b)?+.+` FROM testData2 WHERE a = 2 +-- !query 14 schema +struct +-- !query 14 output +2 3 +2 3 + + +-- !query 15 SELECT `(e|f)` FROM testData2 --- !query 11 schema +-- !query 15 schema struct<> --- !query 11 output +-- !query 15 output --- !query 12 +-- !query 16 SELECT t.`(e|f)` FROM testData2 t --- !query 12 schema +-- !query 16 schema struct<> --- !query 12 output +-- !query 16 output --- !query 13 +-- !query 17 +SELECT p.`(KEY)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3 +-- !query 17 schema +struct +-- !query 17 output +1 11 1 1 1 2 +1 11 2 1 1 2 +2 22 1 2 2 3 +2 22 2 2 2 3 + + +-- !query 18 SELECT p.`(key)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3 --- !query 13 schema +-- !query 18 schema struct --- !query 13 output +-- !query 18 output 1 11 1 1 1 2 1 11 2 1 1 2 2 22 1 2 2 3 From 321211d39826748e0c81aa73f3873636a1018152 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Wed, 28 Jun 2017 10:24:47 -0700 Subject: [PATCH 17/25] fix regex case insensitive match --- .../sql/catalyst/analysis/unresolved.scala | 4 +-- .../sql/catalyst/parser/AstBuilder.scala | 4 +-- .../scala/org/apache/spark/sql/Dataset.scala | 3 +- .../sql-tests/inputs/query_regex_column.sql | 6 ++-- .../results/query_regex_column.sql.out | 30 +++++++++---------- 5 files changed, 23 insertions(+), 24 deletions(-) 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 e2dcd7745e663..0f42f6add786f 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 @@ -322,10 +322,10 @@ case class UnresolvedRegex(regexPattern: String, table: Option[String]) override def expand(input: LogicalPlan, resolver: Resolver): Seq[NamedExpression] = { table match { // If there is no table specified, use all input attributes that match expr - case None => input.output.filter(_.name.matches(regexPattern)) + case None => input.output.filter(_.name.matches(s"(?i)$regexPattern")) // If there is a table, pick out attributes that are part of this table that match expr case Some(t) => input.output.filter(_.qualifier.exists(resolver(_, t))) - .filter(_.name.matches(regexPattern)) + .filter(_.name.matches(s"(?i)$regexPattern")) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index df9bf52378ed0..3b4860665631b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1261,7 +1261,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging val attr = ctx.fieldName.getText expression(ctx.base) match { case unresolved_attr @ UnresolvedAttribute(nameParts) => - ctx.fieldName.getStart.getText.toLowerCase(Locale.ROOT) match { + ctx.fieldName.getStart.getText match { case escapedIdentifier(columnNameRegex) if conf.supportQuotedRegexColumnName => UnresolvedRegex(columnNameRegex, Some(unresolved_attr.name)) case _ => @@ -1277,7 +1277,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * quoted in `` */ override def visitColumnReference(ctx: ColumnReferenceContext): Expression = withOrigin(ctx) { - ctx.getStart.getText.toLowerCase(Locale.ROOT) match { + ctx.getStart.getText match { case escapedIdentifier(columnNameRegex) if conf.supportQuotedRegexColumnName => UnresolvedRegex(columnNameRegex, None) case _ => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 5efa5bffc1ac6..5f923c350a8eb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql import java.io.CharArrayWriter import java.sql.{Date, Timestamp} -import java.util.Locale import scala.collection.JavaConverters._ import scala.language.implicitConversions @@ -1202,7 +1201,7 @@ class Dataset[T] private[sql]( * @group untypedrel * @since 2.3.0 */ - def colRegex(colName: String): Column = colName.toLowerCase(Locale.ROOT) match { + def colRegex(colName: String): Column = colName match { case ParserUtils.escapedIdentifier(columnNameRegex) => Column(UnresolvedRegex(columnNameRegex, None)) case ParserUtils.qualifiedEscapedIdentifier(nameParts, columnNameRegex) => diff --git a/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql b/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql index 5dc5e31f34827..b2187769e7c45 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql @@ -4,7 +4,7 @@ AS testData(key, value1, value2); CREATE OR REPLACE TEMPORARY VIEW testData2 AS SELECT * FROM VALUES (1, 1, 1, 2), (1, 2, 1, 2), (2, 1, 2, 3), (2, 2, 2, 3), (3, 1, 3, 4), (3, 2, 3, 4) -AS testData2(a, b, c, d); +AS testData2(A, B, c, d); -- AnalysisException SELECT `(a)?+.+` FROM testData2 WHERE a = 1; @@ -19,9 +19,9 @@ SELECT `(a)?+.+` FROM testData2 WHERE a = 1; SELECT `(A)?+.+` FROM testData2 WHERE a = 1; SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1; SELECT t.`(A)?+.+` FROM testData2 t WHERE a = 1; -SELECT `(a|b)` FROM testData2 WHERE a = 2; +SELECT `(a|B)` FROM testData2 WHERE a = 2; SELECT `(A|b)` FROM testData2 WHERE a = 2; -SELECT `(a|b)?+.+` FROM testData2 WHERE a = 2; +SELECT `(a|B)?+.+` FROM testData2 WHERE a = 2; SELECT `(A|b)?+.+` FROM testData2 WHERE a = 2; SELECT `(e|f)` FROM testData2; SELECT t.`(e|f)` FROM testData2 t; diff --git a/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out b/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out index a7452bae4ef55..2ff4578823c14 100644 --- a/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out @@ -15,7 +15,7 @@ struct<> -- !query 1 CREATE OR REPLACE TEMPORARY VIEW testData2 AS SELECT * FROM VALUES (1, 1, 1, 2), (1, 2, 1, 2), (2, 1, 2, 3), (2, 2, 2, 3), (3, 1, 3, 4), (3, 2, 3, 4) -AS testData2(a, b, c, d) +AS testData2(A, B, c, d) -- !query 1 schema struct<> -- !query 1 output @@ -28,7 +28,7 @@ SELECT `(a)?+.+` FROM testData2 WHERE a = 1 struct<> -- !query 2 output org.apache.spark.sql.AnalysisException -cannot resolve '`(a)?+.+`' given input columns: [a, b, c, d]; line 1 pos 7 +cannot resolve '`(a)?+.+`' given input columns: [A, B, c, d]; line 1 pos 7 -- !query 3 @@ -37,7 +37,7 @@ SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1 struct<> -- !query 3 output org.apache.spark.sql.AnalysisException -cannot resolve 't.`(a)?+.+`' given input columns: [a, b, c, d]; line 1 pos 7 +cannot resolve 't.`(a)?+.+`' given input columns: [A, B, c, d]; line 1 pos 7 -- !query 4 @@ -46,7 +46,7 @@ SELECT `(a|b)` FROM testData2 WHERE a = 2 struct<> -- !query 4 output org.apache.spark.sql.AnalysisException -cannot resolve '`(a|b)`' given input columns: [a, b, c, d]; line 1 pos 7 +cannot resolve '`(a|b)`' given input columns: [A, B, c, d]; line 1 pos 7 -- !query 5 @@ -55,7 +55,7 @@ SELECT `(a|b)?+.+` FROM testData2 WHERE a = 2 struct<> -- !query 5 output org.apache.spark.sql.AnalysisException -cannot resolve '`(a|b)?+.+`' given input columns: [a, b, c, d]; line 1 pos 7 +cannot resolve '`(a|b)?+.+`' given input columns: [A, B, c, d]; line 1 pos 7 -- !query 6 @@ -69,7 +69,7 @@ spark.sql.parser.quotedRegexColumnNames true -- !query 7 SELECT `(a)?+.+` FROM testData2 WHERE a = 1 -- !query 7 schema -struct +struct -- !query 7 output 1 1 2 2 1 2 @@ -78,7 +78,7 @@ struct -- !query 8 SELECT `(A)?+.+` FROM testData2 WHERE a = 1 -- !query 8 schema -struct +struct -- !query 8 output 1 1 2 2 1 2 @@ -87,7 +87,7 @@ struct -- !query 9 SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1 -- !query 9 schema -struct +struct -- !query 9 output 1 1 2 2 1 2 @@ -96,16 +96,16 @@ struct -- !query 10 SELECT t.`(A)?+.+` FROM testData2 t WHERE a = 1 -- !query 10 schema -struct +struct -- !query 10 output 1 1 2 2 1 2 -- !query 11 -SELECT `(a|b)` FROM testData2 WHERE a = 2 +SELECT `(a|B)` FROM testData2 WHERE a = 2 -- !query 11 schema -struct +struct -- !query 11 output 2 1 2 2 @@ -114,14 +114,14 @@ struct -- !query 12 SELECT `(A|b)` FROM testData2 WHERE a = 2 -- !query 12 schema -struct +struct -- !query 12 output 2 1 2 2 -- !query 13 -SELECT `(a|b)?+.+` FROM testData2 WHERE a = 2 +SELECT `(a|B)?+.+` FROM testData2 WHERE a = 2 -- !query 13 schema struct -- !query 13 output @@ -157,7 +157,7 @@ struct<> -- !query 17 SELECT p.`(KEY)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3 -- !query 17 schema -struct +struct -- !query 17 output 1 11 1 1 1 2 1 11 2 1 1 2 @@ -168,7 +168,7 @@ struct -- !query 18 SELECT p.`(key)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3 -- !query 18 schema -struct +struct -- !query 18 output 1 11 1 1 1 2 1 11 2 1 1 2 From 4e36ed903973dcf637348825b5726892f2c13f77 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Thu, 29 Jun 2017 23:40:06 -0700 Subject: [PATCH 18/25] Address gatorsmile's comments --- .../sql/catalyst/analysis/unresolved.scala | 7 ++-- .../sql/catalyst/parser/AstBuilder.scala | 4 +- .../scala/org/apache/spark/sql/Dataset.scala | 17 ++++---- .../sql-tests/inputs/query_regex_column.sql | 10 +++++ .../results/query_regex_column.sql.out | 42 ++++++++++++++++++- 5 files changed, 67 insertions(+), 13 deletions(-) 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 0f42f6add786f..fb322697c7c68 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 @@ -317,15 +317,16 @@ case class UnresolvedStar(target: Option[Seq[String]]) extends Star with Unevalu * @param table an optional table that should be the target of the expansion. If omitted all * tables' columns are produced. */ -case class UnresolvedRegex(regexPattern: String, table: Option[String]) +case class UnresolvedRegex(regexPattern: String, table: Option[String], caseSensitive: Boolean) extends Star with Unevaluable { override def expand(input: LogicalPlan, resolver: Resolver): Seq[NamedExpression] = { + val pattern = if (caseSensitive) regexPattern else s"(?i)$regexPattern" table match { // If there is no table specified, use all input attributes that match expr - case None => input.output.filter(_.name.matches(s"(?i)$regexPattern")) + case None => input.output.filter(_.name.matches(pattern)) // If there is a table, pick out attributes that are part of this table that match expr case Some(t) => input.output.filter(_.qualifier.exists(resolver(_, t))) - .filter(_.name.matches(s"(?i)$regexPattern")) + .filter(_.name.matches(pattern)) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 3b4860665631b..d42d256c52466 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1263,7 +1263,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging case unresolved_attr @ UnresolvedAttribute(nameParts) => ctx.fieldName.getStart.getText match { case escapedIdentifier(columnNameRegex) if conf.supportQuotedRegexColumnName => - UnresolvedRegex(columnNameRegex, Some(unresolved_attr.name)) + UnresolvedRegex(columnNameRegex, Some(unresolved_attr.name), conf.caseSensitiveAnalysis) case _ => UnresolvedAttribute(nameParts :+ attr) } @@ -1279,7 +1279,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging override def visitColumnReference(ctx: ColumnReferenceContext): Expression = withOrigin(ctx) { ctx.getStart.getText match { case escapedIdentifier(columnNameRegex) if conf.supportQuotedRegexColumnName => - UnresolvedRegex(columnNameRegex, None) + UnresolvedRegex(columnNameRegex, None, conf.caseSensitiveAnalysis) case _ => UnresolvedAttribute.quoted(ctx.getText) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 5f923c350a8eb..1ca7e620fd402 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -1201,13 +1201,16 @@ class Dataset[T] private[sql]( * @group untypedrel * @since 2.3.0 */ - def colRegex(colName: String): Column = colName match { - case ParserUtils.escapedIdentifier(columnNameRegex) => - Column(UnresolvedRegex(columnNameRegex, None)) - case ParserUtils.qualifiedEscapedIdentifier(nameParts, columnNameRegex) => - Column(UnresolvedRegex(columnNameRegex, Some(nameParts))) - case _ => - Column(resolve(colName)) + def colRegex(colName: String): Column = { + val caseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis + colName match { + case ParserUtils.escapedIdentifier(columnNameRegex) => + Column(UnresolvedRegex(columnNameRegex, None, caseSensitive)) + case ParserUtils.qualifiedEscapedIdentifier(nameParts, columnNameRegex) => + Column(UnresolvedRegex(columnNameRegex, Some(nameParts), caseSensitive)) + case _ => + Column(resolve(colName)) + } } /** diff --git a/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql b/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql index b2187769e7c45..c2724020adaba 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql @@ -27,3 +27,13 @@ SELECT `(e|f)` FROM testData2; SELECT t.`(e|f)` FROM testData2 t; SELECT p.`(KEY)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3; SELECT p.`(key)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3; + +set spark.sql.caseSensitive=true; + +CREATE OR REPLACE TEMPORARY VIEW testdata3 AS SELECT * FROM VALUES +(0, 1), (1, 2), (2, 3), (3, 4) +AS testdata3(a, b); + +-- Regex columns +SELECT `(A)?+.+` FROM testdata3; +SELECT `(a)?+.+` FROM testdata3; diff --git a/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out b/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out index 2ff4578823c14..ee14b55673635 100644 --- a/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 19 +-- Number of queries: 23 -- !query 0 @@ -174,3 +174,43 @@ struct 1 11 2 1 1 2 2 22 1 2 2 3 2 22 2 2 2 3 + + +-- !query 19 +set spark.sql.caseSensitive=true +-- !query 19 schema +struct +-- !query 19 output +spark.sql.caseSensitive true + + +-- !query 20 +CREATE OR REPLACE TEMPORARY VIEW testdata3 AS SELECT * FROM VALUES +(0, 1), (1, 2), (2, 3), (3, 4) +AS testdata3(a, b) +-- !query 20 schema +struct<> +-- !query 20 output + + + +-- !query 21 +SELECT `(A)?+.+` FROM testdata3 +-- !query 21 schema +struct +-- !query 21 output +0 1 +1 2 +2 3 +3 4 + + +-- !query 22 +SELECT `(a)?+.+` FROM testdata3 +-- !query 22 schema +struct +-- !query 22 output +1 +2 +3 +4 From 448c3e2d200ad9530cfd43e8200afc7b7b7f1469 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Fri, 30 Jun 2017 16:51:27 -0700 Subject: [PATCH 19/25] address gatorsmile's comment --- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 4673aaa2f8b6e..d62ffc1ce9e7a 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 @@ -848,7 +848,8 @@ object SQLConf { .createWithDefault(UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD.toInt) val SUPPORT_QUOTED_REGEX_COLUMN_NAME = buildConf("spark.sql.parser.quotedRegexColumnNames") - .doc("When true, a SELECT statement can take regex-based column specification.") + .doc("When true, quoted Identifiers (using backticks) in SELECT statement are interpreted" + + " as regular expressions.") .booleanConf .createWithDefault(false) From d65c462040c07b2eea537b717f6ff28f7f02b031 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Wed, 5 Jul 2017 13:21:08 -0700 Subject: [PATCH 20/25] address gatorsmile's comments and fixed his failed testcases --- .../sql/catalyst/parser/AstBuilder.scala | 16 +++++++++-- .../sql-tests/inputs/query_regex_column.sql | 2 ++ .../results/query_regex_column.sql.out | 28 +++++++++++++++---- 3 files changed, 39 insertions(+), 7 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 61514f1243b0e..eef4a4ad29608 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1258,6 +1258,15 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging CaseWhen(branches, Option(ctx.elseExpression).map(expression)) } + private def isContextNamedExpression(ctx: ParserRuleContext): Boolean = withOrigin(ctx) { + var parent = ctx.getParent + while (parent != null) { + if (parent.isInstanceOf[NamedExpressionContext]) return true + parent = parent.getParent + } + return false + } + /** * Create a dereference expression. The return type depends on the type of the parent. * If the parent is an [[UnresolvedAttribute]], it can be a [[UnresolvedAttribute]] or @@ -1269,7 +1278,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging expression(ctx.base) match { case unresolved_attr @ UnresolvedAttribute(nameParts) => ctx.fieldName.getStart.getText match { - case escapedIdentifier(columnNameRegex) if conf.supportQuotedRegexColumnName => + case escapedIdentifier(columnNameRegex) + if conf.supportQuotedRegexColumnName && isContextNamedExpression(ctx) => UnresolvedRegex(columnNameRegex, Some(unresolved_attr.name), conf.caseSensitiveAnalysis) case _ => UnresolvedAttribute(nameParts :+ attr) @@ -1285,11 +1295,13 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging */ override def visitColumnReference(ctx: ColumnReferenceContext): Expression = withOrigin(ctx) { ctx.getStart.getText match { - case escapedIdentifier(columnNameRegex) if conf.supportQuotedRegexColumnName => + case escapedIdentifier(columnNameRegex) + if conf.supportQuotedRegexColumnName && isContextNamedExpression(ctx) => UnresolvedRegex(columnNameRegex, None, conf.caseSensitiveAnalysis) case _ => UnresolvedAttribute.quoted(ctx.getText) } + } /** diff --git a/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql b/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql index c2724020adaba..4cc4ed77db7b6 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql @@ -37,3 +37,5 @@ AS testdata3(a, b); -- Regex columns SELECT `(A)?+.+` FROM testdata3; SELECT `(a)?+.+` FROM testdata3; +SELECT `(A)?+.+` FROM testdata3 WHERE a > 1; +SELECT `(a)?+.+` FROM testdata3 where `a` > 1; diff --git a/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out b/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out index ee14b55673635..29a0962bd0f92 100644 --- a/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 23 +-- Number of queries: 25 -- !query 0 @@ -28,7 +28,7 @@ SELECT `(a)?+.+` FROM testData2 WHERE a = 1 struct<> -- !query 2 output org.apache.spark.sql.AnalysisException -cannot resolve '`(a)?+.+`' given input columns: [A, B, c, d]; line 1 pos 7 +cannot resolve '`(a)?+.+`' given input columns: [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 7 -- !query 3 @@ -37,7 +37,7 @@ SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1 struct<> -- !query 3 output org.apache.spark.sql.AnalysisException -cannot resolve 't.`(a)?+.+`' given input columns: [A, B, c, d]; line 1 pos 7 +cannot resolve 't.`(a)?+.+`' given input columns: [t.A, t.B, t.c, t.d]; line 1 pos 7 -- !query 4 @@ -46,7 +46,7 @@ SELECT `(a|b)` FROM testData2 WHERE a = 2 struct<> -- !query 4 output org.apache.spark.sql.AnalysisException -cannot resolve '`(a|b)`' given input columns: [A, B, c, d]; line 1 pos 7 +cannot resolve '`(a|b)`' given input columns: [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 7 -- !query 5 @@ -55,7 +55,7 @@ SELECT `(a|b)?+.+` FROM testData2 WHERE a = 2 struct<> -- !query 5 output org.apache.spark.sql.AnalysisException -cannot resolve '`(a|b)?+.+`' given input columns: [A, B, c, d]; line 1 pos 7 +cannot resolve '`(a|b)?+.+`' given input columns: [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 7 -- !query 6 @@ -214,3 +214,21 @@ struct 2 3 4 + + +-- !query 23 +SELECT `(A)?+.+` FROM testdata3 WHERE a > 1 +-- !query 23 schema +struct +-- !query 23 output +2 3 +3 4 + + +-- !query 24 +SELECT `(a)?+.+` FROM testdata3 where `a` > 1 +-- !query 24 schema +struct +-- !query 24 output +3 +4 From 65886cd18c82cc2e5401a5abcc91d06a9765b457 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Wed, 5 Jul 2017 13:33:46 -0700 Subject: [PATCH 21/25] fix build failure --- .../apache/spark/sql/catalyst/parser/AstBuilder.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index f2840dd6068cd..81812d5b59165 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1277,8 +1277,9 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { case unresolved_attr @ UnresolvedAttribute(nameParts) => ctx.fieldName.getStart.getText match { case escapedIdentifier(columnNameRegex) - if conf.supportQuotedRegexColumnName && isContextNamedExpression(ctx) => - UnresolvedRegex(columnNameRegex, Some(unresolved_attr.name), conf.caseSensitiveAnalysis) + if SQLConf.get.supportQuotedRegexColumnName && isContextNamedExpression(ctx) => + UnresolvedRegex(columnNameRegex, Some(unresolved_attr.name), + SQLConf.get.caseSensitiveAnalysis) case _ => UnresolvedAttribute(nameParts :+ attr) } @@ -1294,8 +1295,8 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { override def visitColumnReference(ctx: ColumnReferenceContext): Expression = withOrigin(ctx) { ctx.getStart.getText match { case escapedIdentifier(columnNameRegex) - if conf.supportQuotedRegexColumnName && isContextNamedExpression(ctx) => - UnresolvedRegex(columnNameRegex, None, conf.caseSensitiveAnalysis) + if SQLConf.get.supportQuotedRegexColumnName && isContextNamedExpression(ctx) => + UnresolvedRegex(columnNameRegex, None, SQLConf.get.caseSensitiveAnalysis) case _ => UnresolvedAttribute.quoted(ctx.getText) } From d3eed1a8f87f9897bc55a14d3e0cfa2548eaa920 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Fri, 7 Jul 2017 16:29:35 -0700 Subject: [PATCH 22/25] Address gatorsmile and viirya's comments --- .../sql-tests/inputs/query_regex_column.sql | 9 + .../results/query_regex_column.sql.out | 181 ++++++++++++------ .../org/apache/spark/sql/DatasetSuite.scala | 8 + 3 files changed, 143 insertions(+), 55 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql b/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql index 4cc4ed77db7b6..ed4251c9aafcf 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql @@ -11,6 +11,8 @@ SELECT `(a)?+.+` FROM testData2 WHERE a = 1; SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1; SELECT `(a|b)` FROM testData2 WHERE a = 2; SELECT `(a|b)?+.+` FROM testData2 WHERE a = 2; +SELECT SUM(`(a|b)?+.+`) FROM testData2; +SELECT SUM(`(a)`) FROM testData2; set spark.sql.parser.quotedRegexColumnNames=true; @@ -39,3 +41,10 @@ SELECT `(A)?+.+` FROM testdata3; SELECT `(a)?+.+` FROM testdata3; SELECT `(A)?+.+` FROM testdata3 WHERE a > 1; SELECT `(a)?+.+` FROM testdata3 where `a` > 1; +SELECT SUM(`a`) FROM testdata3; +SELECT SUM(`(a)`) FROM testdata3; +SELECT SUM(`(a)?+.+`) FROM testdata3; +SELECT SUM(a) FROM testdata3 GROUP BY `a`; +-- AnalysisException +SELECT SUM(a) FROM testdata3 GROUP BY `(a)`; +SELECT SUM(a) FROM testdata3 GROUP BY `(a)?+.+`; diff --git a/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out b/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out index 29a0962bd0f92..6ee57b5a60bd4 100644 --- a/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 25 +-- Number of queries: 33 -- !query 0 @@ -59,33 +59,33 @@ cannot resolve '`(a|b)?+.+`' given input columns: [testdata2.A, testdata2.B, tes -- !query 6 -set spark.sql.parser.quotedRegexColumnNames=true +SELECT SUM(`(a|b)?+.+`) FROM testData2 -- !query 6 schema -struct +struct<> -- !query 6 output -spark.sql.parser.quotedRegexColumnNames true +org.apache.spark.sql.AnalysisException +cannot resolve '`(a|b)?+.+`' given input columns: [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 11 -- !query 7 -SELECT `(a)?+.+` FROM testData2 WHERE a = 1 +SELECT SUM(`(a)`) FROM testData2 -- !query 7 schema -struct +struct<> -- !query 7 output -1 1 2 -2 1 2 +org.apache.spark.sql.AnalysisException +cannot resolve '`(a)`' given input columns: [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 11 -- !query 8 -SELECT `(A)?+.+` FROM testData2 WHERE a = 1 +set spark.sql.parser.quotedRegexColumnNames=true -- !query 8 schema -struct +struct -- !query 8 output -1 1 2 -2 1 2 +spark.sql.parser.quotedRegexColumnNames true -- !query 9 -SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1 +SELECT `(a)?+.+` FROM testData2 WHERE a = 1 -- !query 9 schema struct -- !query 9 output @@ -94,7 +94,7 @@ struct -- !query 10 -SELECT t.`(A)?+.+` FROM testData2 t WHERE a = 1 +SELECT `(A)?+.+` FROM testData2 WHERE a = 1 -- !query 10 schema struct -- !query 10 output @@ -103,132 +103,203 @@ struct -- !query 11 -SELECT `(a|B)` FROM testData2 WHERE a = 2 +SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1 -- !query 11 schema -struct +struct -- !query 11 output +1 1 2 +2 1 2 + + +-- !query 12 +SELECT t.`(A)?+.+` FROM testData2 t WHERE a = 1 +-- !query 12 schema +struct +-- !query 12 output +1 1 2 +2 1 2 + + +-- !query 13 +SELECT `(a|B)` FROM testData2 WHERE a = 2 +-- !query 13 schema +struct +-- !query 13 output 2 1 2 2 --- !query 12 +-- !query 14 SELECT `(A|b)` FROM testData2 WHERE a = 2 --- !query 12 schema +-- !query 14 schema struct --- !query 12 output +-- !query 14 output 2 1 2 2 --- !query 13 +-- !query 15 SELECT `(a|B)?+.+` FROM testData2 WHERE a = 2 --- !query 13 schema +-- !query 15 schema struct --- !query 13 output +-- !query 15 output 2 3 2 3 --- !query 14 +-- !query 16 SELECT `(A|b)?+.+` FROM testData2 WHERE a = 2 --- !query 14 schema +-- !query 16 schema struct --- !query 14 output +-- !query 16 output 2 3 2 3 --- !query 15 +-- !query 17 SELECT `(e|f)` FROM testData2 --- !query 15 schema +-- !query 17 schema struct<> --- !query 15 output +-- !query 17 output --- !query 16 +-- !query 18 SELECT t.`(e|f)` FROM testData2 t --- !query 16 schema +-- !query 18 schema struct<> --- !query 16 output +-- !query 18 output --- !query 17 +-- !query 19 SELECT p.`(KEY)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3 --- !query 17 schema +-- !query 19 schema struct --- !query 17 output +-- !query 19 output 1 11 1 1 1 2 1 11 2 1 1 2 2 22 1 2 2 3 2 22 2 2 2 3 --- !query 18 +-- !query 20 SELECT p.`(key)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3 --- !query 18 schema +-- !query 20 schema struct --- !query 18 output +-- !query 20 output 1 11 1 1 1 2 1 11 2 1 1 2 2 22 1 2 2 3 2 22 2 2 2 3 --- !query 19 +-- !query 21 set spark.sql.caseSensitive=true --- !query 19 schema +-- !query 21 schema struct --- !query 19 output +-- !query 21 output spark.sql.caseSensitive true --- !query 20 +-- !query 22 CREATE OR REPLACE TEMPORARY VIEW testdata3 AS SELECT * FROM VALUES (0, 1), (1, 2), (2, 3), (3, 4) AS testdata3(a, b) --- !query 20 schema +-- !query 22 schema struct<> --- !query 20 output +-- !query 22 output --- !query 21 +-- !query 23 SELECT `(A)?+.+` FROM testdata3 --- !query 21 schema +-- !query 23 schema struct --- !query 21 output +-- !query 23 output 0 1 1 2 2 3 3 4 --- !query 22 +-- !query 24 SELECT `(a)?+.+` FROM testdata3 --- !query 22 schema +-- !query 24 schema struct --- !query 22 output +-- !query 24 output 1 2 3 4 --- !query 23 +-- !query 25 SELECT `(A)?+.+` FROM testdata3 WHERE a > 1 --- !query 23 schema +-- !query 25 schema struct --- !query 23 output +-- !query 25 output 2 3 3 4 --- !query 24 +-- !query 26 SELECT `(a)?+.+` FROM testdata3 where `a` > 1 --- !query 24 schema +-- !query 26 schema struct --- !query 24 output +-- !query 26 output 3 4 + + +-- !query 27 +SELECT SUM(`a`) FROM testdata3 +-- !query 27 schema +struct +-- !query 27 output +6 + + +-- !query 28 +SELECT SUM(`(a)`) FROM testdata3 +-- !query 28 schema +struct +-- !query 28 output +6 + + +-- !query 29 +SELECT SUM(`(a)?+.+`) FROM testdata3 +-- !query 29 schema +struct +-- !query 29 output +10 + + +-- !query 30 +SELECT SUM(a) FROM testdata3 GROUP BY `a` +-- !query 30 schema +struct +-- !query 30 output +0 +1 +2 +3 + + +-- !query 31 +SELECT SUM(a) FROM testdata3 GROUP BY `(a)` +-- !query 31 schema +struct<> +-- !query 31 output +org.apache.spark.sql.AnalysisException +cannot resolve '`(a)`' given input columns: [testdata3.a, testdata3.b]; line 1 pos 38 + + +-- !query 32 +SELECT SUM(a) FROM testdata3 GROUP BY `(a)?+.+` +-- !query 32 schema +struct<> +-- !query 32 output +org.apache.spark.sql.AnalysisException +cannot resolve '`(a)?+.+`' given input columns: [testdata3.a, testdata3.b]; line 1 pos 38 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index e3dd3dd815e72..f3ee3728c2191 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -260,6 +260,14 @@ class DatasetSuite extends QueryTest with SharedSQLContext { assert(e.contains("Cannot resolve column name \"`(_1|_2)`\"")) withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "true") { + checkDataset( + ds.select(ds.col("_2")).as[Int], + 1, 2, 3) + + checkDataset( + ds.select(ds.colRegex("`(_1)?+.+`")).as[Int], + 1, 2, 3) + checkDataset( ds.select(ds("`(_1|_2)`")) .select(expr("named_struct('a', _1, 'b', _2)").as[ClassData]), From 956b8493cfd59fd434d7dbd89ec48fce07d715b2 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Mon, 10 Jul 2017 12:29:08 -0700 Subject: [PATCH 23/25] address gatorsmile's comments --- .../sql/catalyst/parser/AstBuilder.scala | 6 +- .../sql-tests/inputs/query_regex_column.sql | 2 + .../results/query_regex_column.sql.out | 182 +++++++------- .../spark/sql/DataFrameAggregateSuite.scala | 10 +- .../spark/sql/DataFrameNaFunctionsSuite.scala | 225 +++++++++--------- .../apache/spark/sql/DataFrameStatSuite.scala | 87 +++---- .../org/apache/spark/sql/DataFrameSuite.scala | 55 +++-- .../org/apache/spark/sql/DatasetSuite.scala | 30 ++- .../org/apache/spark/sql/SQLQuerySuite.scala | 41 ++-- .../datasources/json/JsonSuite.scala | 92 +++---- .../parquet/ParquetFilterSuite.scala | 3 +- .../spark/sql/sources/DataSourceTest.scala | 7 +- .../spark/sql/sources/TableScanSuite.scala | 61 ++--- 13 files changed, 427 insertions(+), 374 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 606bec5d61a2b..fa90ef22ca979 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1258,7 +1258,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { CaseWhen(branches, Option(ctx.elseExpression).map(expression)) } - private def isContextNamedExpression(ctx: ParserRuleContext): Boolean = withOrigin(ctx) { + private def canApplyRegex(ctx: ParserRuleContext): Boolean = withOrigin(ctx) { var parent = ctx.getParent while (parent != null) { if (parent.isInstanceOf[NamedExpressionContext]) return true @@ -1279,7 +1279,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { case unresolved_attr @ UnresolvedAttribute(nameParts) => ctx.fieldName.getStart.getText match { case escapedIdentifier(columnNameRegex) - if SQLConf.get.supportQuotedRegexColumnName && isContextNamedExpression(ctx) => + if SQLConf.get.supportQuotedRegexColumnName && canApplyRegex(ctx) => UnresolvedRegex(columnNameRegex, Some(unresolved_attr.name), SQLConf.get.caseSensitiveAnalysis) case _ => @@ -1297,7 +1297,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { override def visitColumnReference(ctx: ColumnReferenceContext): Expression = withOrigin(ctx) { ctx.getStart.getText match { case escapedIdentifier(columnNameRegex) - if SQLConf.get.supportQuotedRegexColumnName && isContextNamedExpression(ctx) => + if SQLConf.get.supportQuotedRegexColumnName && canApplyRegex(ctx) => UnresolvedRegex(columnNameRegex, None, SQLConf.get.caseSensitiveAnalysis) case _ => UnresolvedAttribute.quoted(ctx.getText) diff --git a/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql b/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql index ed4251c9aafcf..ad96754826a45 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql @@ -1,3 +1,5 @@ +set spark.sql.parser.quotedRegexColumnNames=false; + CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES (1, "1", "11"), (2, "2", "22"), (3, "3", "33"), (4, "4", "44"), (5, "5", "55"), (6, "6", "66") AS testData(key, value1, value2); diff --git a/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out b/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out index 6ee57b5a60bd4..2dade86f35df9 100644 --- a/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out @@ -1,21 +1,19 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 33 +-- Number of queries: 34 -- !query 0 -CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES -(1, "1", "11"), (2, "2", "22"), (3, "3", "33"), (4, "4", "44"), (5, "5", "55"), (6, "6", "66") -AS testData(key, value1, value2) +set spark.sql.parser.quotedRegexColumnNames=false -- !query 0 schema -struct<> +struct -- !query 0 output - +spark.sql.parser.quotedRegexColumnNames false -- !query 1 -CREATE OR REPLACE TEMPORARY VIEW testData2 AS SELECT * FROM VALUES -(1, 1, 1, 2), (1, 2, 1, 2), (2, 1, 2, 3), (2, 2, 2, 3), (3, 1, 3, 4), (3, 2, 3, 4) -AS testData2(A, B, c, d) +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, "1", "11"), (2, "2", "22"), (3, "3", "33"), (4, "4", "44"), (5, "5", "55"), (6, "6", "66") +AS testData(key, value1, value2) -- !query 1 schema struct<> -- !query 1 output @@ -23,78 +21,79 @@ struct<> -- !query 2 -SELECT `(a)?+.+` FROM testData2 WHERE a = 1 +CREATE OR REPLACE TEMPORARY VIEW testData2 AS SELECT * FROM VALUES +(1, 1, 1, 2), (1, 2, 1, 2), (2, 1, 2, 3), (2, 2, 2, 3), (3, 1, 3, 4), (3, 2, 3, 4) +AS testData2(A, B, c, d) -- !query 2 schema struct<> -- !query 2 output -org.apache.spark.sql.AnalysisException -cannot resolve '`(a)?+.+`' given input columns: [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 7 + -- !query 3 -SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1 +SELECT `(a)?+.+` FROM testData2 WHERE a = 1 -- !query 3 schema struct<> -- !query 3 output org.apache.spark.sql.AnalysisException -cannot resolve 't.`(a)?+.+`' given input columns: [t.A, t.B, t.c, t.d]; line 1 pos 7 +cannot resolve '`(a)?+.+`' given input columns: [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 7 -- !query 4 -SELECT `(a|b)` FROM testData2 WHERE a = 2 +SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1 -- !query 4 schema struct<> -- !query 4 output org.apache.spark.sql.AnalysisException -cannot resolve '`(a|b)`' given input columns: [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 7 +cannot resolve 't.`(a)?+.+`' given input columns: [t.A, t.B, t.c, t.d]; line 1 pos 7 -- !query 5 -SELECT `(a|b)?+.+` FROM testData2 WHERE a = 2 +SELECT `(a|b)` FROM testData2 WHERE a = 2 -- !query 5 schema struct<> -- !query 5 output org.apache.spark.sql.AnalysisException -cannot resolve '`(a|b)?+.+`' given input columns: [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 7 +cannot resolve '`(a|b)`' given input columns: [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 7 -- !query 6 -SELECT SUM(`(a|b)?+.+`) FROM testData2 +SELECT `(a|b)?+.+` FROM testData2 WHERE a = 2 -- !query 6 schema struct<> -- !query 6 output org.apache.spark.sql.AnalysisException -cannot resolve '`(a|b)?+.+`' given input columns: [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 11 +cannot resolve '`(a|b)?+.+`' given input columns: [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 7 -- !query 7 -SELECT SUM(`(a)`) FROM testData2 +SELECT SUM(`(a|b)?+.+`) FROM testData2 -- !query 7 schema struct<> -- !query 7 output org.apache.spark.sql.AnalysisException -cannot resolve '`(a)`' given input columns: [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 11 +cannot resolve '`(a|b)?+.+`' given input columns: [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 11 -- !query 8 -set spark.sql.parser.quotedRegexColumnNames=true +SELECT SUM(`(a)`) FROM testData2 -- !query 8 schema -struct +struct<> -- !query 8 output -spark.sql.parser.quotedRegexColumnNames true +org.apache.spark.sql.AnalysisException +cannot resolve '`(a)`' given input columns: [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 11 -- !query 9 -SELECT `(a)?+.+` FROM testData2 WHERE a = 1 +set spark.sql.parser.quotedRegexColumnNames=true -- !query 9 schema -struct +struct -- !query 9 output -1 1 2 -2 1 2 +spark.sql.parser.quotedRegexColumnNames true -- !query 10 -SELECT `(A)?+.+` FROM testData2 WHERE a = 1 +SELECT `(a)?+.+` FROM testData2 WHERE a = 1 -- !query 10 schema struct -- !query 10 output @@ -103,7 +102,7 @@ struct -- !query 11 -SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1 +SELECT `(A)?+.+` FROM testData2 WHERE a = 1 -- !query 11 schema struct -- !query 11 output @@ -112,7 +111,7 @@ struct -- !query 12 -SELECT t.`(A)?+.+` FROM testData2 t WHERE a = 1 +SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1 -- !query 12 schema struct -- !query 12 output @@ -121,16 +120,16 @@ struct -- !query 13 -SELECT `(a|B)` FROM testData2 WHERE a = 2 +SELECT t.`(A)?+.+` FROM testData2 t WHERE a = 1 -- !query 13 schema -struct +struct -- !query 13 output -2 1 -2 2 +1 1 2 +2 1 2 -- !query 14 -SELECT `(A|b)` FROM testData2 WHERE a = 2 +SELECT `(a|B)` FROM testData2 WHERE a = 2 -- !query 14 schema struct -- !query 14 output @@ -139,16 +138,16 @@ struct -- !query 15 -SELECT `(a|B)?+.+` FROM testData2 WHERE a = 2 +SELECT `(A|b)` FROM testData2 WHERE a = 2 -- !query 15 schema -struct +struct -- !query 15 output -2 3 -2 3 +2 1 +2 2 -- !query 16 -SELECT `(A|b)?+.+` FROM testData2 WHERE a = 2 +SELECT `(a|B)?+.+` FROM testData2 WHERE a = 2 -- !query 16 schema struct -- !query 16 output @@ -157,15 +156,16 @@ struct -- !query 17 -SELECT `(e|f)` FROM testData2 +SELECT `(A|b)?+.+` FROM testData2 WHERE a = 2 -- !query 17 schema -struct<> +struct -- !query 17 output - +2 3 +2 3 -- !query 18 -SELECT t.`(e|f)` FROM testData2 t +SELECT `(e|f)` FROM testData2 -- !query 18 schema struct<> -- !query 18 output @@ -173,133 +173,141 @@ struct<> -- !query 19 -SELECT p.`(KEY)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3 +SELECT t.`(e|f)` FROM testData2 t -- !query 19 schema -struct +struct<> -- !query 19 output + + + +-- !query 20 +SELECT p.`(KEY)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3 +-- !query 20 schema +struct +-- !query 20 output 1 11 1 1 1 2 1 11 2 1 1 2 2 22 1 2 2 3 2 22 2 2 2 3 --- !query 20 +-- !query 21 SELECT p.`(key)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3 --- !query 20 schema +-- !query 21 schema struct --- !query 20 output +-- !query 21 output 1 11 1 1 1 2 1 11 2 1 1 2 2 22 1 2 2 3 2 22 2 2 2 3 --- !query 21 +-- !query 22 set spark.sql.caseSensitive=true --- !query 21 schema +-- !query 22 schema struct --- !query 21 output +-- !query 22 output spark.sql.caseSensitive true --- !query 22 +-- !query 23 CREATE OR REPLACE TEMPORARY VIEW testdata3 AS SELECT * FROM VALUES (0, 1), (1, 2), (2, 3), (3, 4) AS testdata3(a, b) --- !query 22 schema +-- !query 23 schema struct<> --- !query 22 output +-- !query 23 output --- !query 23 +-- !query 24 SELECT `(A)?+.+` FROM testdata3 --- !query 23 schema +-- !query 24 schema struct --- !query 23 output +-- !query 24 output 0 1 1 2 2 3 3 4 --- !query 24 +-- !query 25 SELECT `(a)?+.+` FROM testdata3 --- !query 24 schema +-- !query 25 schema struct --- !query 24 output +-- !query 25 output 1 2 3 4 --- !query 25 +-- !query 26 SELECT `(A)?+.+` FROM testdata3 WHERE a > 1 --- !query 25 schema +-- !query 26 schema struct --- !query 25 output +-- !query 26 output 2 3 3 4 --- !query 26 +-- !query 27 SELECT `(a)?+.+` FROM testdata3 where `a` > 1 --- !query 26 schema +-- !query 27 schema struct --- !query 26 output +-- !query 27 output 3 4 --- !query 27 +-- !query 28 SELECT SUM(`a`) FROM testdata3 --- !query 27 schema +-- !query 28 schema struct --- !query 27 output +-- !query 28 output 6 --- !query 28 +-- !query 29 SELECT SUM(`(a)`) FROM testdata3 --- !query 28 schema +-- !query 29 schema struct --- !query 28 output +-- !query 29 output 6 --- !query 29 +-- !query 30 SELECT SUM(`(a)?+.+`) FROM testdata3 --- !query 29 schema +-- !query 30 schema struct --- !query 29 output +-- !query 30 output 10 --- !query 30 +-- !query 31 SELECT SUM(a) FROM testdata3 GROUP BY `a` --- !query 30 schema +-- !query 31 schema struct --- !query 30 output +-- !query 31 output 0 1 2 3 --- !query 31 +-- !query 32 SELECT SUM(a) FROM testdata3 GROUP BY `(a)` --- !query 31 schema +-- !query 32 schema struct<> --- !query 31 output +-- !query 32 output org.apache.spark.sql.AnalysisException cannot resolve '`(a)`' given input columns: [testdata3.a, testdata3.b]; line 1 pos 38 --- !query 32 +-- !query 33 SELECT SUM(a) FROM testdata3 GROUP BY `(a)?+.+` --- !query 32 schema +-- !query 33 schema struct<> --- !query 32 output +-- !query 33 output org.apache.spark.sql.AnalysisException cannot resolve '`(a)?+.+`' given input columns: [testdata3.a, testdata3.b]; line 1 pos 38 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index b52d50b195bcc..4568b67024acb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -543,10 +543,12 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext { test("SPARK-17237 remove backticks in a pivot result schema") { val df = Seq((2, 3, 4), (3, 4, 5)).toDF("a", "x", "y") - checkAnswer( - df.groupBy("a").pivot("x").agg(count("y"), avg("y")).na.fill(0), - Seq(Row(3, 0, 0.0, 1, 5.0), Row(2, 1, 4.0, 0, 0.0)) - ) + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "false") { + checkAnswer( + df.groupBy("a").pivot("x").agg(count("y"), avg("y")).na.fill(0), + Seq(Row(3, 0, 0.0, 1, 5.0), Row(2, 1, 4.0, 0, 0.0)) + ) + } } test("aggregate function in GROUP BY") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala index e63c5cb194d68..47c9ba5847a4f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.sql import scala.collection.JavaConverters._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext - class DataFrameNaFunctionsSuite extends QueryTest with SharedSQLContext { import testImplicits._ @@ -111,119 +111,124 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSQLContext { (null, null) ).toDF("name", "spy") - val fillNumeric = input.na.fill(50.6) - checkAnswer( - fillNumeric, - Row("Bob", 16, 176.5) :: - Row("Alice", 50, 164.3) :: - Row("David", 60, 50.6) :: - Row("Nina", 25, 50.6) :: - Row("Amy", 50, 50.6) :: - Row(null, 50, 50.6) :: Nil) - - // Make sure the columns are properly named. - assert(fillNumeric.columns.toSeq === input.columns.toSeq) - - // string - checkAnswer( - input.na.fill("unknown").select("name"), - Row("Bob") :: Row("Alice") :: Row("David") :: - Row("Nina") :: Row("Amy") :: Row("unknown") :: Nil) - assert(input.na.fill("unknown").columns.toSeq === input.columns.toSeq) - - // boolean - checkAnswer( - boolInput.na.fill(true).select("spy"), - Row(false) :: Row(true) :: Row(true) :: Row(true) :: Nil) - assert(boolInput.na.fill(true).columns.toSeq === boolInput.columns.toSeq) - - // fill double with subset columns - checkAnswer( - input.na.fill(50.6, "age" :: Nil).select("name", "age"), - Row("Bob", 16) :: - Row("Alice", 50) :: - Row("David", 60) :: - Row("Nina", 25) :: - Row("Amy", 50) :: - Row(null, 50) :: Nil) - - // fill boolean with subset columns - checkAnswer( - boolInput.na.fill(true, "spy" :: Nil).select("name", "spy"), - Row("Bob", false) :: - Row("Alice", true) :: - Row("Mallory", true) :: - Row(null, true) :: Nil) - - // fill string with subset columns - checkAnswer( - Seq[(String, String)]((null, null)).toDF("col1", "col2").na.fill("test", "col1" :: Nil), - Row("test", null)) - - checkAnswer( - Seq[(Long, Long)]((1, 2), (-1, -2), (9123146099426677101L, 9123146560113991650L)) - .toDF("a", "b").na.fill(0), - Row(1, 2) :: Row(-1, -2) :: Row(9123146099426677101L, 9123146560113991650L) :: Nil - ) - - checkAnswer( - Seq[(java.lang.Long, java.lang.Double)]((null, 3.14), (9123146099426677101L, null), - (9123146560113991650L, 1.6), (null, null)).toDF("a", "b").na.fill(0.2), - Row(0, 3.14) :: Row(9123146099426677101L, 0.2) :: Row(9123146560113991650L, 1.6) - :: Row(0, 0.2) :: Nil - ) - - checkAnswer( - Seq[(java.lang.Long, java.lang.Float)]((null, 3.14f), (9123146099426677101L, null), - (9123146560113991650L, 1.6f), (null, null)).toDF("a", "b").na.fill(0.2), - Row(0, 3.14f) :: Row(9123146099426677101L, 0.2f) :: Row(9123146560113991650L, 1.6f) - :: Row(0, 0.2f) :: Nil - ) - - checkAnswer( - Seq[(java.lang.Long, java.lang.Double)]((null, 1.23), (3L, null), (4L, 3.45)) - .toDF("a", "b").na.fill(2.34), - Row(2, 1.23) :: Row(3, 2.34) :: Row(4, 3.45) :: Nil - ) - - checkAnswer( - Seq[(java.lang.Long, java.lang.Double)]((null, 1.23), (3L, null), (4L, 3.45)) - .toDF("a", "b").na.fill(5), - Row(5, 1.23) :: Row(3, 5.0) :: Row(4, 3.45) :: Nil - ) + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "false") { + val fillNumeric = input.na.fill(50.6) + checkAnswer( + fillNumeric, + Row("Bob", 16, 176.5) :: + Row("Alice", 50, 164.3) :: + Row("David", 60, 50.6) :: + Row("Nina", 25, 50.6) :: + Row("Amy", 50, 50.6) :: + Row(null, 50, 50.6) :: Nil) + + // Make sure the columns are properly named. + assert(fillNumeric.columns.toSeq === input.columns.toSeq) + + // string + checkAnswer( + input.na.fill("unknown").select("name"), + Row("Bob") :: Row("Alice") :: Row("David") :: + Row("Nina") :: Row("Amy") :: Row("unknown") :: Nil) + assert(input.na.fill("unknown").columns.toSeq === input.columns.toSeq) + + // boolean + checkAnswer( + boolInput.na.fill(true).select("spy"), + Row(false) :: Row(true) :: Row(true) :: Row(true) :: Nil) + assert(boolInput.na.fill(true).columns.toSeq === boolInput.columns.toSeq) + + // fill double with subset columns + checkAnswer( + input.na.fill(50.6, "age" :: Nil).select("name", "age"), + Row("Bob", 16) :: + Row("Alice", 50) :: + Row("David", 60) :: + Row("Nina", 25) :: + Row("Amy", 50) :: + Row(null, 50) :: Nil) + + // fill boolean with subset columns + checkAnswer( + boolInput.na.fill(true, "spy" :: Nil).select("name", "spy"), + Row("Bob", false) :: + Row("Alice", true) :: + Row("Mallory", true) :: + Row(null, true) :: Nil) + + // fill string with subset columns + checkAnswer( + Seq[(String, String)]((null, null)).toDF("col1", "col2").na.fill("test", "col1" :: Nil), + Row("test", null)) + + checkAnswer( + Seq[(Long, Long)]((1, 2), (-1, -2), (9123146099426677101L, 9123146560113991650L)) + .toDF("a", "b").na.fill(0), + Row(1, 2) :: Row(-1, -2) :: Row(9123146099426677101L, 9123146560113991650L) :: Nil + ) + + checkAnswer( + Seq[(java.lang.Long, java.lang.Double)]((null, 3.14), (9123146099426677101L, null), + (9123146560113991650L, 1.6), (null, null)).toDF("a", "b").na.fill(0.2), + Row(0, 3.14) :: Row(9123146099426677101L, 0.2) :: Row(9123146560113991650L, 1.6) + :: Row(0, 0.2) :: Nil + ) + + checkAnswer( + Seq[(java.lang.Long, java.lang.Float)]((null, 3.14f), (9123146099426677101L, null), + (9123146560113991650L, 1.6f), (null, null)).toDF("a", "b").na.fill(0.2), + Row(0, 3.14f) :: Row(9123146099426677101L, 0.2f) :: Row(9123146560113991650L, 1.6f) + :: Row(0, 0.2f) :: Nil + ) + + checkAnswer( + Seq[(java.lang.Long, java.lang.Double)]((null, 1.23), (3L, null), (4L, 3.45)) + .toDF("a", "b").na.fill(2.34), + Row(2, 1.23) :: Row(3, 2.34) :: Row(4, 3.45) :: Nil + ) + + checkAnswer( + Seq[(java.lang.Long, java.lang.Double)]((null, 1.23), (3L, null), (4L, 3.45)) + .toDF("a", "b").na.fill(5), + Row(5, 1.23) :: Row(3, 5.0) :: Row(4, 3.45) :: Nil + ) + } } test("fill with map") { - val df = Seq[(String, String, java.lang.Integer, java.lang.Long, + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "false") { + val df = Seq[(String, String, java.lang.Integer, java.lang.Long, java.lang.Float, java.lang.Double, java.lang.Boolean)]( - (null, null, null, null, null, null, null)) - .toDF("stringFieldA", "stringFieldB", "integerField", "longField", - "floatField", "doubleField", "booleanField") - - val fillMap = Map( - "stringFieldA" -> "test", - "integerField" -> 1, - "longField" -> 2L, - "floatField" -> 3.3f, - "doubleField" -> 4.4d, - "booleanField" -> false) - - val expectedRow = Row("test", null, 1, 2L, 3.3f, 4.4d, false) - - checkAnswer(df.na.fill(fillMap), expectedRow) - checkAnswer(df.na.fill(fillMap.asJava), expectedRow) // Test Java version - - // Ensure replacement values are cast to the column data type. - checkAnswer(df.na.fill(Map( - "integerField" -> 1d, - "longField" -> 2d, - "floatField" -> 3d, - "doubleField" -> 4d)), - Row(null, null, 1, 2L, 3f, 4d, null)) - - // Ensure column types do not change. Columns that have null values replaced - // will no longer be flagged as nullable, so do not compare schemas directly. - assert(df.na.fill(fillMap).schema.fields.map(_.dataType) === df.schema.fields.map(_.dataType)) + (null, null, null, null, null, null, null)) + .toDF("stringFieldA", "stringFieldB", "integerField", "longField", + "floatField", "doubleField", "booleanField") + + val fillMap = Map( + "stringFieldA" -> "test", + "integerField" -> 1, + "longField" -> 2L, + "floatField" -> 3.3f, + "doubleField" -> 4.4d, + "booleanField" -> false) + + val expectedRow = Row("test", null, 1, 2L, 3.3f, 4.4d, false) + + + checkAnswer(df.na.fill(fillMap), expectedRow) + checkAnswer(df.na.fill(fillMap.asJava), expectedRow) // Test Java version + + // Ensure replacement values are cast to the column data type. + checkAnswer(df.na.fill(Map( + "integerField" -> 1d, + "longField" -> 2d, + "floatField" -> 3d, + "doubleField" -> 4d)), + Row(null, null, 1, 2L, 3f, 4d, null)) + + // Ensure column types do not change. Columns that have null values replaced + // will no longer be flagged as nullable, so do not compare schemas directly. + assert(df.na.fill(fillMap).schema.fields.map(_.dataType) === df.schema.fields.map(_.dataType)) + } } test("replace") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala index dd118f88e3bb3..09502d05f7709 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala @@ -24,6 +24,7 @@ import org.scalatest.Matchers._ import org.apache.spark.internal.Logging import org.apache.spark.sql.execution.stat.StatFunctions import org.apache.spark.sql.functions.col +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{DoubleType, StructField, StructType} @@ -263,52 +264,56 @@ class DataFrameStatSuite extends QueryTest with SharedSQLContext { } test("crosstab") { - val rng = new Random() - val data = Seq.tabulate(25)(i => (rng.nextInt(5), rng.nextInt(10))) - val df = data.toDF("a", "b") - val crosstab = df.stat.crosstab("a", "b") - val columnNames = crosstab.schema.fieldNames - assert(columnNames(0) === "a_b") - // reduce by key - val expected = data.map(t => (t, 1)).groupBy(_._1).mapValues(_.length) - val rows = crosstab.collect() - rows.foreach { row => - val i = row.getString(0).toInt - for (col <- 1 until columnNames.length) { - val j = columnNames(col).toInt - assert(row.getLong(col) === expected.getOrElse((i, j), 0).toLong) + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "false") { + val rng = new Random() + val data = Seq.tabulate(25)(i => (rng.nextInt(5), rng.nextInt(10))) + val df = data.toDF("a", "b") + val crosstab = df.stat.crosstab("a", "b") + val columnNames = crosstab.schema.fieldNames + assert(columnNames(0) === "a_b") + // reduce by key + val expected = data.map(t => (t, 1)).groupBy(_._1).mapValues(_.length) + val rows = crosstab.collect() + rows.foreach { row => + val i = row.getString(0).toInt + for (col <- 1 until columnNames.length) { + val j = columnNames(col).toInt + assert(row.getLong(col) === expected.getOrElse((i, j), 0).toLong) + } } } } test("special crosstab elements (., '', null, ``)") { - val data = Seq( - ("a", Double.NaN, "ho"), - (null, 2.0, "ho"), - ("a.b", Double.NegativeInfinity, ""), - ("b", Double.PositiveInfinity, "`ha`"), - ("a", 1.0, null) - ) - val df = data.toDF("1", "2", "3") - val ct1 = df.stat.crosstab("1", "2") - // column fields should be 1 + distinct elements of second column - assert(ct1.schema.fields.length === 6) - assert(ct1.collect().length === 4) - val ct2 = df.stat.crosstab("1", "3") - assert(ct2.schema.fields.length === 5) - assert(ct2.schema.fieldNames.contains("ha")) - assert(ct2.collect().length === 4) - val ct3 = df.stat.crosstab("3", "2") - assert(ct3.schema.fields.length === 6) - assert(ct3.schema.fieldNames.contains("NaN")) - assert(ct3.schema.fieldNames.contains("Infinity")) - assert(ct3.schema.fieldNames.contains("-Infinity")) - assert(ct3.collect().length === 4) - val ct4 = df.stat.crosstab("3", "1") - assert(ct4.schema.fields.length === 5) - assert(ct4.schema.fieldNames.contains("null")) - assert(ct4.schema.fieldNames.contains("a.b")) - assert(ct4.collect().length === 4) + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "false") { + val data = Seq( + ("a", Double.NaN, "ho"), + (null, 2.0, "ho"), + ("a.b", Double.NegativeInfinity, ""), + ("b", Double.PositiveInfinity, "`ha`"), + ("a", 1.0, null) + ) + val df = data.toDF("1", "2", "3") + val ct1 = df.stat.crosstab("1", "2") + // column fields should be 1 + distinct elements of second column + assert(ct1.schema.fields.length === 6) + assert(ct1.collect().length === 4) + val ct2 = df.stat.crosstab("1", "3") + assert(ct2.schema.fields.length === 5) + assert(ct2.schema.fieldNames.contains("ha")) + assert(ct2.collect().length === 4) + val ct3 = df.stat.crosstab("3", "2") + assert(ct3.schema.fields.length === 6) + assert(ct3.schema.fieldNames.contains("NaN")) + assert(ct3.schema.fieldNames.contains("Infinity")) + assert(ct3.schema.fieldNames.contains("-Infinity")) + assert(ct3.collect().length === 4) + val ct4 = df.stat.crosstab("3", "1") + assert(ct4.schema.fields.length === 5) + assert(ct4.schema.fieldNames.contains("null")) + assert(ct4.schema.fieldNames.contains("a.b")) + assert(ct4.collect().length === 4) + } } test("Frequent Items") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 9ea9951c24ef1..451c7b3b6cf63 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -1026,28 +1026,31 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { } test("SPARK-7551: support backticks for DataFrame attribute resolution") { - val df = spark.read.json(Seq("""{"a.b": {"c": {"d..e": {"f": 1}}}}""").toDS()) - checkAnswer( - df.select(df("`a.b`.c.`d..e`.`f`")), - Row(1) - ) - - val df2 = spark.read.json(Seq("""{"a b": {"c": {"d e": {"f": 1}}}}""").toDS()) - checkAnswer( - df2.select(df2("`a b`.c.d e.f")), - Row(1) - ) - - def checkError(testFun: => Unit): Unit = { - val e = intercept[org.apache.spark.sql.AnalysisException] { - testFun + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "false") { + val df = spark.read.json(Seq("""{"a.b": {"c": {"d..e": {"f": 1}}}}""").toDS()) + checkAnswer( + df.select(df("`a.b`.c.`d..e`.`f`")), + Row(1) + ) + + val df2 = spark.read.json(Seq("""{"a b": {"c": {"d e": {"f": 1}}}}""").toDS()) + checkAnswer( + df2.select(df2("`a b`.c.d e.f")), + Row(1) + ) + + def checkError(testFun: => Unit): Unit = { + val e = intercept[org.apache.spark.sql.AnalysisException] { + testFun + } + assert(e.getMessage.contains("syntax error in attribute name:")) } - assert(e.getMessage.contains("syntax error in attribute name:")) + + checkError(df("`abc.`c`")) + checkError(df("`abc`..d")) + checkError(df("`a`.b.")) + checkError(df("`a.b`.c.`d")) } - checkError(df("`abc.`c`")) - checkError(df("`abc`..d")) - checkError(df("`a`.b.")) - checkError(df("`a.b`.c.`d")) } test("SPARK-7324 dropDuplicates") { @@ -1775,11 +1778,13 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { } test("SPARK-17957: outer join + na.fill") { - val df1 = Seq((1, 2), (2, 3)).toDF("a", "b") - val df2 = Seq((2, 5), (3, 4)).toDF("a", "c") - val joinedDf = df1.join(df2, Seq("a"), "outer").na.fill(0) - val df3 = Seq((3, 1)).toDF("a", "d") - checkAnswer(joinedDf.join(df3, "a"), Row(3, 0, 4, 1)) + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "false") { + val df1 = Seq((1, 2), (2, 3)).toDF("a", "b") + val df2 = Seq((2, 5), (3, 4)).toDF("a", "c") + val joinedDf = df1.join(df2, Seq("a"), "outer").na.fill(0) + val df3 = Seq((3, 1)).toDF("a", "d") + checkAnswer(joinedDf.join(df3, "a"), Row(3, 0, 4, 1)) + } } test("SPARK-17123: Performing set operations that combine non-scala native types") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index f3ee3728c2191..825840707d421 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -247,17 +247,27 @@ class DatasetSuite extends QueryTest with SharedSQLContext { test("REGEX column specification") { val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() - var e = intercept[AnalysisException] { ds.select(expr("`(_1)?+.+`").as[Int]) }.getMessage - assert(e.contains("cannot resolve '`(_1)?+.+`'")) - - e = intercept[AnalysisException] { ds.select(expr("`(_1|_2)`").as[Int]) }.getMessage - assert(e.contains("cannot resolve '`(_1|_2)`'")) - - e = intercept[AnalysisException] { ds.select(ds("`(_1)?+.+`")) }.getMessage - assert(e.contains("Cannot resolve column name \"`(_1)?+.+`\"")) + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "false") { + var e = intercept[AnalysisException] { + ds.select(expr("`(_1)?+.+`").as[Int]) + }.getMessage + assert(e.contains("cannot resolve '`(_1)?+.+`'")) + + e = intercept[AnalysisException] { + ds.select(expr("`(_1|_2)`").as[Int]) + }.getMessage + assert(e.contains("cannot resolve '`(_1|_2)`'")) + + e = intercept[AnalysisException] { + ds.select(ds("`(_1)?+.+`")) + }.getMessage + assert(e.contains("Cannot resolve column name \"`(_1)?+.+`\"")) - e = intercept[AnalysisException] { ds.select(ds("`(_1|_2)`")) }.getMessage - assert(e.contains("Cannot resolve column name \"`(_1|_2)`\"")) + e = intercept[AnalysisException] { + ds.select(ds("`(_1|_2)`")) + }.getMessage + assert(e.contains("Cannot resolve column name \"`(_1|_2)`\"")) + } withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "true") { checkDataset( 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 472ff7385b194..c78ec6d9a89ff 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 @@ -1219,7 +1219,9 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-3483 Special chars in column names") { val data = Seq("""{"key?number1": "value1", "key.number2": "value2"}""").toDS() spark.read.json(data).createOrReplaceTempView("records") - sql("SELECT `key?number1`, `key.number2` FROM records") + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "false") { + sql("SELECT `key?number1`, `key.number2` FROM records") + } } test("SPARK-3814 Support Bitwise & operator") { @@ -1339,7 +1341,9 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { .json(Seq("""{"a": {"c.b": 1}, "b.$q": [{"a@!.q": 1}], "q.w": {"w.i&": [1]}}""").toDS()) .createOrReplaceTempView("t") - checkAnswer(sql("SELECT a.`c.b`, `b.$q`[0].`a@!.q`, `q.w`.`w.i&`[0] FROM t"), Row(1, 1, 1)) + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "false") { + checkAnswer(sql("SELECT a.`c.b`, `b.$q`[0].`a@!.q`, `q.w`.`w.i&`[0] FROM t"), Row(1, 1, 1)) + } } test("SPARK-6583 order by aggregated function") { @@ -1835,25 +1839,28 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } // Create paths with unusual characters - val specialCharacterPath = sql( - """ + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "false") { + val specialCharacterPath = sql( + """ | SELECT struct(`col$.a_`, `a.b.c.`) as `r&&b.c` FROM | (SELECT struct(a, b) as `col$.a_`, struct(b, a) as `a.b.c.` FROM testData2) tmp """.stripMargin) - withTempView("specialCharacterTable") { - specialCharacterPath.createOrReplaceTempView("specialCharacterTable") - checkAnswer( - specialCharacterPath.select($"`r&&b.c`.*"), - nestedStructData.select($"record.*")) - checkAnswer( - sql("SELECT `r&&b.c`.`col$.a_` FROM specialCharacterTable"), + withTempView("specialCharacterTable") { + specialCharacterPath.createOrReplaceTempView("specialCharacterTable") + checkAnswer( + specialCharacterPath.select($"`r&&b.c`.*"), + nestedStructData.select($"record.*")) + checkAnswer( + sql( + "SELECT `r&&b.c`.`col$.a_` FROM specialCharacterTable"), nestedStructData.select($"record.r1")) - checkAnswer( - sql("SELECT `r&&b.c`.`a.b.c.` FROM specialCharacterTable"), - nestedStructData.select($"record.r2")) - checkAnswer( - sql("SELECT `r&&b.c`.`col$.a_`.* FROM specialCharacterTable"), - nestedStructData.select($"record.r1.*")) + checkAnswer( + sql("SELECT `r&&b.c`.`a.b.c.` FROM specialCharacterTable"), + nestedStructData.select($"record.r2")) + checkAnswer( + sql("SELECT `r&&b.c`.`col$.a_`.* FROM specialCharacterTable"), + nestedStructData.select($"record.r1.*")) + } } // Try star expanding a scalar. This should fail. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 704823ad516c2..3bce414429309 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -922,59 +922,61 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } test("Applying schemas with MapType") { - val schemaWithSimpleMap = StructType( - StructField("map", MapType(StringType, IntegerType, true), false) :: Nil) - val jsonWithSimpleMap = spark.read.schema(schemaWithSimpleMap).json(mapType1) + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "false") { + val schemaWithSimpleMap = StructType( + StructField("map", MapType(StringType, IntegerType, true), false) :: Nil) + val jsonWithSimpleMap = spark.read.schema(schemaWithSimpleMap).json(mapType1) - jsonWithSimpleMap.createOrReplaceTempView("jsonWithSimpleMap") + jsonWithSimpleMap.createOrReplaceTempView("jsonWithSimpleMap") - checkAnswer( - sql("select `map` from jsonWithSimpleMap"), - Row(Map("a" -> 1)) :: - Row(Map("b" -> 2)) :: - Row(Map("c" -> 3)) :: - Row(Map("c" -> 1, "d" -> 4)) :: - Row(Map("e" -> null)) :: Nil - ) + checkAnswer( + sql("select `map` from jsonWithSimpleMap"), + Row(Map("a" -> 1)) :: + Row(Map("b" -> 2)) :: + Row(Map("c" -> 3)) :: + Row(Map("c" -> 1, "d" -> 4)) :: + Row(Map("e" -> null)) :: Nil + ) - checkAnswer( - sql("select `map`['c'] from jsonWithSimpleMap"), - Row(null) :: - Row(null) :: - Row(3) :: - Row(1) :: - Row(null) :: Nil - ) + checkAnswer( + sql("select `map`['c'] from jsonWithSimpleMap"), + Row(null) :: + Row(null) :: + Row(3) :: + Row(1) :: + Row(null) :: Nil + ) - val innerStruct = StructType( - StructField("field1", ArrayType(IntegerType, true), true) :: - StructField("field2", IntegerType, true) :: Nil) - val schemaWithComplexMap = StructType( - StructField("map", MapType(StringType, innerStruct, true), false) :: Nil) + val innerStruct = StructType( + StructField("field1", ArrayType(IntegerType, true), true) :: + StructField("field2", IntegerType, true) :: Nil) + val schemaWithComplexMap = StructType( + StructField("map", MapType(StringType, innerStruct, true), false) :: Nil) - val jsonWithComplexMap = spark.read.schema(schemaWithComplexMap).json(mapType2) + val jsonWithComplexMap = spark.read.schema(schemaWithComplexMap).json(mapType2) - jsonWithComplexMap.createOrReplaceTempView("jsonWithComplexMap") + jsonWithComplexMap.createOrReplaceTempView("jsonWithComplexMap") - checkAnswer( - sql("select `map` from jsonWithComplexMap"), - Row(Map("a" -> Row(Seq(1, 2, 3, null), null))) :: - Row(Map("b" -> Row(null, 2))) :: - Row(Map("c" -> Row(Seq(), 4))) :: - Row(Map("c" -> Row(null, 3), "d" -> Row(Seq(null), null))) :: - Row(Map("e" -> null)) :: - Row(Map("f" -> Row(null, null))) :: Nil - ) + checkAnswer( + sql("select `map` from jsonWithComplexMap"), + Row(Map("a" -> Row(Seq(1, 2, 3, null), null))) :: + Row(Map("b" -> Row(null, 2))) :: + Row(Map("c" -> Row(Seq(), 4))) :: + Row(Map("c" -> Row(null, 3), "d" -> Row(Seq(null), null))) :: + Row(Map("e" -> null)) :: + Row(Map("f" -> Row(null, null))) :: Nil + ) - checkAnswer( - sql("select `map`['a'].field1, `map`['c'].field2 from jsonWithComplexMap"), - Row(Seq(1, 2, 3, null), null) :: - Row(null, null) :: - Row(null, 4) :: - Row(null, 3) :: - Row(null, null) :: - Row(null, null) :: Nil - ) + checkAnswer( + sql("select `map`['a'].field1, `map`['c'].field2 from jsonWithComplexMap"), + Row(Seq(1, 2, 3, null), null) :: + Row(null, null) :: + Row(null, 4) :: + Row(null, 3) :: + Row(null, null) :: + Row(null, null) :: Nil + ) + } } test("SPARK-2096 Correctly parse dot notations") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala index 98427cfe3031c..c43c1ec8b9a6b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -544,7 +544,8 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex Seq(true, false).foreach { vectorized => withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> vectorized.toString, - SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> true.toString) { + SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> true.toString, + SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "false") { withTempPath { path => Seq(Some(1), None).toDF("col.dots").write.parquet(path.getAbsolutePath) val readBack = spark.read.parquet(path.getAbsolutePath).where("`col.dots` IS NOT NULL") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala index 80868fff897fd..34cbc45830c9d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala @@ -20,14 +20,17 @@ package org.apache.spark.sql.sources import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String private[sql] abstract class DataSourceTest extends QueryTest { - protected def sqlTest(sqlString: String, expectedAnswer: Seq[Row]) { + protected def sqlTest(sqlString: String, expectedAnswer: Seq[Row], enableRegex: String = "true") { test(sqlString) { - checkAnswer(spark.sql(sqlString), expectedAnswer) + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> enableRegex) { + checkAnswer(spark.sql(sqlString), expectedAnswer) + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala index b01d15eb917e2..8fd38328da45e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala @@ -22,6 +22,7 @@ import java.sql.{Date, Timestamp} import org.apache.spark.rdd.RDD import org.apache.spark.sql._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ @@ -247,32 +248,34 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext { assert(expectedSchema == spark.table("tableWithSchema").schema) - checkAnswer( - sql( - """SELECT - | `string$%Field`, - | cast(binaryField as string), - | booleanField, - | byteField, - | shortField, - | int_Field, - | `longField_:,<>=+/~^`, - | floatField, - | doubleField, - | decimalField1, - | decimalField2, - | dateField, - | timestampField, - | varcharField, - | charField, - | arrayFieldSimple, - | arrayFieldComplex, - | mapFieldSimple, - | mapFieldComplex, - | structFieldSimple, - | structFieldComplex FROM tableWithSchema""".stripMargin), - tableWithSchemaExpected - ) + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "false") { + checkAnswer( + sql( + """SELECT + | `string$%Field`, + | cast(binaryField as string), + | booleanField, + | byteField, + | shortField, + | int_Field, + | `longField_:,<>=+/~^`, + | floatField, + | doubleField, + | decimalField1, + | decimalField2, + | dateField, + | timestampField, + | varcharField, + | charField, + | arrayFieldSimple, + | arrayFieldComplex, + | mapFieldSimple, + | mapFieldComplex, + | structFieldSimple, + | structFieldComplex FROM tableWithSchema""".stripMargin), + tableWithSchemaExpected + ) + } } sqlTest( @@ -281,7 +284,7 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext { sqlTest( "SELECT `string$%Field` FROM tableWithSchema", - (1 to 10).map(i => Row(s"str_$i")).toSeq) + (1 to 10).map(i => Row(s"str_$i")).toSeq, "false") sqlTest( "SELECT int_Field FROM tableWithSchema WHERE int_Field < 5", @@ -289,7 +292,7 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext { sqlTest( "SELECT `longField_:,<>=+/~^` * 2 FROM tableWithSchema", - (1 to 10).map(i => Row(i * 2.toLong)).toSeq) + (1 to 10).map(i => Row(i * 2.toLong)).toSeq, "false") sqlTest( "SELECT structFieldSimple.key, arrayFieldSimple[1] FROM tableWithSchema a where int_Field=1", @@ -297,7 +300,7 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext { sqlTest( "SELECT structFieldComplex.Value.`value_(2)` FROM tableWithSchema", - (1 to 10).map(i => Row(Seq(Date.valueOf(s"1970-01-${i + 1}")))).toSeq) + (1 to 10).map(i => Row(Seq(Date.valueOf(s"1970-01-${i + 1}")))).toSeq, "false") test("Caching") { // Cached Query Execution From d613ff90596652d7e22859c0684dbfe3602344e0 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Mon, 10 Jul 2017 22:25:30 -0700 Subject: [PATCH 24/25] fix build failure --- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 95861c6962030..55558ca9f700c 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 @@ -860,7 +860,7 @@ object SQLConf { " as regular expressions.") .booleanConf .createWithDefault(false) - + val ARROW_EXECUTION_ENABLE = buildConf("spark.sql.execution.arrow.enable") .internal() From a5f9c44875b67d9b6590a8cef6778667388fc162 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Tue, 11 Jul 2017 10:47:36 -0700 Subject: [PATCH 25/25] address gaatorsmile's comments --- .../sql/catalyst/parser/AstBuilder.scala | 12 ++- .../datasources/json/JsonSuite.scala | 78 ++++++++++--------- .../spark/sql/sources/DataSourceTest.scala | 4 +- .../spark/sql/sources/TableScanSuite.scala | 6 +- 4 files changed, 53 insertions(+), 47 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 7ef121851b0ff..ad359e714bcc7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1260,6 +1260,10 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging CaseWhen(branches, Option(ctx.elseExpression).map(expression)) } + /** + * Currently only regex in expressions of SELECT statements are supported; in other + * places, e.g., where `(a)?+.+` = 2, regex are not meaningful. + */ private def canApplyRegex(ctx: ParserRuleContext): Boolean = withOrigin(ctx) { var parent = ctx.getParent while (parent != null) { @@ -1281,9 +1285,9 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging case unresolved_attr @ UnresolvedAttribute(nameParts) => ctx.fieldName.getStart.getText match { case escapedIdentifier(columnNameRegex) - if SQLConf.get.supportQuotedRegexColumnName && canApplyRegex(ctx) => + if conf.supportQuotedRegexColumnName && canApplyRegex(ctx) => UnresolvedRegex(columnNameRegex, Some(unresolved_attr.name), - SQLConf.get.caseSensitiveAnalysis) + conf.caseSensitiveAnalysis) case _ => UnresolvedAttribute(nameParts :+ attr) } @@ -1299,8 +1303,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging override def visitColumnReference(ctx: ColumnReferenceContext): Expression = withOrigin(ctx) { ctx.getStart.getText match { case escapedIdentifier(columnNameRegex) - if SQLConf.get.supportQuotedRegexColumnName && canApplyRegex(ctx) => - UnresolvedRegex(columnNameRegex, None, SQLConf.get.caseSensitiveAnalysis) + if conf.supportQuotedRegexColumnName && canApplyRegex(ctx) => + UnresolvedRegex(columnNameRegex, None, conf.caseSensitiveAnalysis) case _ => UnresolvedAttribute.quoted(ctx.getText) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 3bce414429309..1cde137edbb91 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -922,59 +922,61 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } test("Applying schemas with MapType") { - withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "false") { - val schemaWithSimpleMap = StructType( - StructField("map", MapType(StringType, IntegerType, true), false) :: Nil) - val jsonWithSimpleMap = spark.read.schema(schemaWithSimpleMap).json(mapType1) + val schemaWithSimpleMap = StructType( + StructField("map", MapType(StringType, IntegerType, true), false) :: Nil) + val jsonWithSimpleMap = spark.read.schema(schemaWithSimpleMap).json(mapType1) - jsonWithSimpleMap.createOrReplaceTempView("jsonWithSimpleMap") + jsonWithSimpleMap.createOrReplaceTempView("jsonWithSimpleMap") - checkAnswer( - sql("select `map` from jsonWithSimpleMap"), - Row(Map("a" -> 1)) :: - Row(Map("b" -> 2)) :: - Row(Map("c" -> 3)) :: - Row(Map("c" -> 1, "d" -> 4)) :: - Row(Map("e" -> null)) :: Nil - ) + checkAnswer( + sql("select `map` from jsonWithSimpleMap"), + Row(Map("a" -> 1)) :: + Row(Map("b" -> 2)) :: + Row(Map("c" -> 3)) :: + Row(Map("c" -> 1, "d" -> 4)) :: + Row(Map("e" -> null)) :: Nil + ) + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "false") { checkAnswer( sql("select `map`['c'] from jsonWithSimpleMap"), Row(null) :: - Row(null) :: - Row(3) :: - Row(1) :: - Row(null) :: Nil + Row(null) :: + Row(3) :: + Row(1) :: + Row(null) :: Nil ) + } - val innerStruct = StructType( - StructField("field1", ArrayType(IntegerType, true), true) :: - StructField("field2", IntegerType, true) :: Nil) - val schemaWithComplexMap = StructType( - StructField("map", MapType(StringType, innerStruct, true), false) :: Nil) + val innerStruct = StructType( + StructField("field1", ArrayType(IntegerType, true), true) :: + StructField("field2", IntegerType, true) :: Nil) + val schemaWithComplexMap = StructType( + StructField("map", MapType(StringType, innerStruct, true), false) :: Nil) - val jsonWithComplexMap = spark.read.schema(schemaWithComplexMap).json(mapType2) + val jsonWithComplexMap = spark.read.schema(schemaWithComplexMap).json(mapType2) - jsonWithComplexMap.createOrReplaceTempView("jsonWithComplexMap") + jsonWithComplexMap.createOrReplaceTempView("jsonWithComplexMap") - checkAnswer( - sql("select `map` from jsonWithComplexMap"), - Row(Map("a" -> Row(Seq(1, 2, 3, null), null))) :: - Row(Map("b" -> Row(null, 2))) :: - Row(Map("c" -> Row(Seq(), 4))) :: - Row(Map("c" -> Row(null, 3), "d" -> Row(Seq(null), null))) :: - Row(Map("e" -> null)) :: - Row(Map("f" -> Row(null, null))) :: Nil - ) + checkAnswer( + sql("select `map` from jsonWithComplexMap"), + Row(Map("a" -> Row(Seq(1, 2, 3, null), null))) :: + Row(Map("b" -> Row(null, 2))) :: + Row(Map("c" -> Row(Seq(), 4))) :: + Row(Map("c" -> Row(null, 3), "d" -> Row(Seq(null), null))) :: + Row(Map("e" -> null)) :: + Row(Map("f" -> Row(null, null))) :: Nil + ) + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "false") { checkAnswer( sql("select `map`['a'].field1, `map`['c'].field2 from jsonWithComplexMap"), Row(Seq(1, 2, 3, null), null) :: - Row(null, null) :: - Row(null, 4) :: - Row(null, 3) :: - Row(null, null) :: - Row(null, null) :: Nil + Row(null, null) :: + Row(null, 4) :: + Row(null, 3) :: + Row(null, null) :: + Row(null, null) :: Nil ) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala index 34cbc45830c9d..70338670c9f9c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala @@ -26,9 +26,9 @@ import org.apache.spark.unsafe.types.UTF8String private[sql] abstract class DataSourceTest extends QueryTest { - protected def sqlTest(sqlString: String, expectedAnswer: Seq[Row], enableRegex: String = "true") { + protected def sqlTest(sqlString: String, expectedAnswer: Seq[Row], enableRegex: Boolean = false) { test(sqlString) { - withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> enableRegex) { + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> enableRegex.toString) { checkAnswer(spark.sql(sqlString), expectedAnswer) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala index 8fd38328da45e..65474a52dd0b7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala @@ -284,7 +284,7 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext { sqlTest( "SELECT `string$%Field` FROM tableWithSchema", - (1 to 10).map(i => Row(s"str_$i")).toSeq, "false") + (1 to 10).map(i => Row(s"str_$i")).toSeq) sqlTest( "SELECT int_Field FROM tableWithSchema WHERE int_Field < 5", @@ -292,7 +292,7 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext { sqlTest( "SELECT `longField_:,<>=+/~^` * 2 FROM tableWithSchema", - (1 to 10).map(i => Row(i * 2.toLong)).toSeq, "false") + (1 to 10).map(i => Row(i * 2.toLong)).toSeq) sqlTest( "SELECT structFieldSimple.key, arrayFieldSimple[1] FROM tableWithSchema a where int_Field=1", @@ -300,7 +300,7 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext { sqlTest( "SELECT structFieldComplex.Value.`value_(2)` FROM tableWithSchema", - (1 to 10).map(i => Row(Seq(Date.valueOf(s"1970-01-${i + 1}")))).toSeq, "false") + (1 to 10).map(i => Row(Seq(Date.valueOf(s"1970-01-${i + 1}")))).toSeq) test("Caching") { // Cached Query Execution