From f050f398237d781e83ea03b56adde59873f694c4 Mon Sep 17 00:00:00 2001 From: wanghaihua Date: Fri, 9 Jun 2017 13:55:23 +0800 Subject: [PATCH 1/9] Add SQL feature of `reset spark.test.property` support which is compatible with Hive. --- .../spark/sql/catalyst/parser/SqlBase.g4 | 2 +- .../spark/sql/execution/SparkSqlParser.scala | 7 ++++++- .../sql/execution/command/SetCommand.scala | 21 +++++++++++++++++-- .../sql/execution/SparkSqlParserSuite.scala | 9 ++++++-- 4 files changed, 33 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index ef5648c6dbe4..cb0d33f8aaf1 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -160,7 +160,7 @@ statement | op=(ADD | LIST) identifier .*? #manageResource | SET ROLE .*? #failNativeCommand | SET .*? #setConfiguration - | RESET #resetConfiguration + | RESET .*? #resetConfiguration | unsupportedHiveNativeCommands .*? #failNativeCommand ; diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 3c58c6e1b678..9d3c20eac6af 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -86,7 +86,12 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { */ override def visitResetConfiguration( ctx: ResetConfigurationContext): LogicalPlan = withOrigin(ctx) { - ResetCommand + val raw = remainder(ctx.RESET.getSymbol) + if (raw.nonEmpty) { + ResetCommand(Some(raw.trim)) + } else { + ResetCommand(None) + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala index 5f12830ee621..ddd02759ee4c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala @@ -149,13 +149,30 @@ object SetCommand { /** * This command is for resetting SQLConf to the default values. Command that runs * {{{ + * reset key; * reset; * }}} */ -case object ResetCommand extends RunnableCommand with Logging { +case class ResetCommand(key: Option[String]) extends RunnableCommand with Logging { + + private val runFunc: (SparkSession => Unit) = key match { + + case None => + val runFunc = (sparkSession: SparkSession) => { + sparkSession.sessionState.conf.clear() + } + runFunc + + // (In Hive, "UNSET key" clear a sepecific properties.) + case Some(key) => + val runFunc = (sparkSession: SparkSession) => { + sparkSession.sessionState.conf.unsetConf(key) + } + runFunc + } override def run(sparkSession: SparkSession): Seq[Row] = { - sparkSession.sessionState.conf.clear() + runFunc(sparkSession) Seq.empty[Row] } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index b32fb90e1007..576b994529f0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -294,11 +294,16 @@ class SparkSqlParserSuite extends PlanTest { test("pipeline concatenation") { val concat = Concat( Concat(UnresolvedAttribute("a") :: UnresolvedAttribute("b") :: Nil) :: - UnresolvedAttribute("c") :: - Nil + UnresolvedAttribute("c") :: + Nil ) assertEqual( "SELECT a || b || c FROM t", Project(UnresolvedAlias(concat) :: Nil, UnresolvedRelation(TableIdentifier("t")))) } + + test("reset") { + assertEqual("reset", ResetCommand(None)) + assertEqual("reset spark.test.property", ResetCommand(Some("spark.test.property"))) + } } From f2bf940265a3bd07e99b0f25b745d91514dad6cb Mon Sep 17 00:00:00 2001 From: wanghaihua Date: Sun, 11 Jun 2017 15:26:16 +0800 Subject: [PATCH 2/9] Fix typo and add more unit tests. --- .../sql/execution/command/SetCommand.scala | 4 +- .../sql/execution/SparkSqlParserSuite.scala | 4 +- .../spark/sql/internal/SQLConfSuite.scala | 47 +++++++++++++++++++ 3 files changed, 51 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala index ddd02759ee4c..257d144f4398 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala @@ -163,10 +163,10 @@ case class ResetCommand(key: Option[String]) extends RunnableCommand with Loggin } runFunc - // (In Hive, "UNSET key" clear a sepecific properties.) + // (In Hive, "RESET key" clear a specific property.) case Some(key) => val runFunc = (sparkSession: SparkSession) => { - sparkSession.sessionState.conf.unsetConf(key) + sparkSession.conf.unset(key) } runFunc } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index 576b994529f0..70f9516c4b39 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -294,8 +294,8 @@ class SparkSqlParserSuite extends PlanTest { test("pipeline concatenation") { val concat = Concat( Concat(UnresolvedAttribute("a") :: UnresolvedAttribute("b") :: Nil) :: - UnresolvedAttribute("c") :: - Nil + UnresolvedAttribute("c") :: + Nil ) assertEqual( "SELECT a || b || c FROM t", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index a283ff971adc..d3409598e543 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -130,6 +130,22 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { } } + test("reset specific property - public conf") { + spark.sessionState.conf.clear() + val original = spark.conf.get(SQLConf.GROUP_BY_ORDINAL) + try { + assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL) === true) + sql(s"set ${SQLConf.GROUP_BY_ORDINAL.key}=false") + assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL) === false) + assert(sql(s"set").where(s"key = '${SQLConf.GROUP_BY_ORDINAL.key}'").count() == 1) + sql(s"reset ${SQLConf.GROUP_BY_ORDINAL.key}") + assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL) === true) + assert(sql(s"set").where(s"key = '${SQLConf.GROUP_BY_ORDINAL.key}'").count() == 0) + } finally { + sql(s"set ${SQLConf.GROUP_BY_ORDINAL}=$original") + } + } + test("reset - internal conf") { spark.sessionState.conf.clear() val original = spark.conf.get(SQLConf.OPTIMIZER_MAX_ITERATIONS) @@ -146,6 +162,22 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { } } + test("reset specific property - internal conf") { + spark.sessionState.conf.clear() + val original = spark.conf.get(SQLConf.OPTIMIZER_MAX_ITERATIONS) + try { + assert(spark.conf.get(SQLConf.OPTIMIZER_MAX_ITERATIONS) === 100) + sql(s"set ${SQLConf.OPTIMIZER_MAX_ITERATIONS.key}=10") + assert(spark.conf.get(SQLConf.OPTIMIZER_MAX_ITERATIONS) === 10) + assert(sql(s"set").where(s"key = '${SQLConf.OPTIMIZER_MAX_ITERATIONS.key}'").count() == 1) + sql(s"reset ${SQLConf.OPTIMIZER_MAX_ITERATIONS.key}") + assert(spark.conf.get(SQLConf.OPTIMIZER_MAX_ITERATIONS) === 100) + assert(sql(s"set").where(s"key = '${SQLConf.OPTIMIZER_MAX_ITERATIONS.key}'").count() == 0) + } finally { + sql(s"set ${SQLConf.OPTIMIZER_MAX_ITERATIONS}=$original") + } + } + test("reset - user-defined conf") { spark.sessionState.conf.clear() val userDefinedConf = "x.y.z.reset" @@ -161,6 +193,21 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { } } + test("reset specific property - user-defined conf") { + spark.sessionState.conf.clear() + val userDefinedConf = "x.y.z.reset" + try { + assert(spark.conf.getOption(userDefinedConf).isEmpty) + sql(s"set $userDefinedConf=false") + assert(spark.conf.get(userDefinedConf) === "false") + assert(sql(s"set").where(s"key = '$userDefinedConf'").count() == 1) + sql(s"reset $userDefinedConf") + assert(spark.conf.getOption(userDefinedConf).isEmpty) + } finally { + spark.conf.unset(userDefinedConf) + } + } + test("invalid conf value") { spark.sessionState.conf.clear() val e = intercept[IllegalArgumentException] { From 4e76a31b182e51eb26d8506152ac0d1ffa6f2fe8 Mon Sep 17 00:00:00 2001 From: wanghaihua Date: Tue, 13 Jun 2017 11:47:39 +0800 Subject: [PATCH 3/9] Fix typo and style. --- .../spark/sql/execution/SparkSqlParser.scala | 1 + .../sql/execution/command/SetCommand.scala | 22 +++++-------------- 2 files changed, 7 insertions(+), 16 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 9d3c20eac6af..bcbcaa9b19bc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -82,6 +82,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { * Example SQL : * {{{ * RESET; + * RESET key; * }}} */ override def visitResetConfiguration( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala index 257d144f4398..ff98f48d2d99 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala @@ -155,24 +155,14 @@ object SetCommand { */ case class ResetCommand(key: Option[String]) extends RunnableCommand with Logging { - private val runFunc: (SparkSession => Unit) = key match { - - case None => - val runFunc = (sparkSession: SparkSession) => { + override def run(sparkSession: SparkSession): Seq[Row] = { + key match { + case None => sparkSession.sessionState.conf.clear() - } - runFunc - - // (In Hive, "RESET key" clear a specific property.) - case Some(key) => - val runFunc = (sparkSession: SparkSession) => { + // "RESET key" clear a specific property. + case Some(key) => sparkSession.conf.unset(key) - } - runFunc - } - - override def run(sparkSession: SparkSession): Seq[Row] = { - runFunc(sparkSession) + } Seq.empty[Row] } } From 579adf9088967b7489aa18ec9823339a985a44a4 Mon Sep 17 00:00:00 2001 From: wanghaihua Date: Tue, 13 Jun 2017 15:38:08 +0800 Subject: [PATCH 4/9] Fix typo and style according to comment. --- .../spark/sql/execution/SparkSqlParser.scala | 8 +- .../sql/execution/command/SetCommand.scala | 2 +- .../spark/sql/internal/SQLConfSuite.scala | 128 ++++++------------ 3 files changed, 46 insertions(+), 92 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index bcbcaa9b19bc..c3d2298ed916 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -82,17 +82,13 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { * Example SQL : * {{{ * RESET; - * RESET key; + * RESET `special#$!`; * }}} */ override def visitResetConfiguration( ctx: ResetConfigurationContext): LogicalPlan = withOrigin(ctx) { val raw = remainder(ctx.RESET.getSymbol) - if (raw.nonEmpty) { - ResetCommand(Some(raw.trim)) - } else { - ResetCommand(None) - } + if (raw.nonEmpty) ResetCommand(Some(raw.trim)) else ResetCommand(None) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala index ff98f48d2d99..f0264e254267 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala @@ -149,8 +149,8 @@ object SetCommand { /** * This command is for resetting SQLConf to the default values. Command that runs * {{{ - * reset key; * reset; + * reset key; * }}} */ case class ResetCommand(key: Option[String]) extends RunnableCommand with Logging { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index d3409598e543..104e16bac80e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -114,97 +114,55 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { } } - test("reset - public conf") { - spark.sessionState.conf.clear() - val original = spark.conf.get(SQLConf.GROUP_BY_ORDINAL) - try { - assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL) === true) - sql(s"set ${SQLConf.GROUP_BY_ORDINAL.key}=false") - assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL) === false) - assert(sql(s"set").where(s"key = '${SQLConf.GROUP_BY_ORDINAL.key}'").count() == 1) - sql(s"reset") - assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL) === true) - assert(sql(s"set").where(s"key = '${SQLConf.GROUP_BY_ORDINAL.key}'").count() == 0) - } finally { - sql(s"set ${SQLConf.GROUP_BY_ORDINAL}=$original") - } - } - - test("reset specific property - public conf") { - spark.sessionState.conf.clear() - val original = spark.conf.get(SQLConf.GROUP_BY_ORDINAL) - try { - assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL) === true) - sql(s"set ${SQLConf.GROUP_BY_ORDINAL.key}=false") - assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL) === false) - assert(sql(s"set").where(s"key = '${SQLConf.GROUP_BY_ORDINAL.key}'").count() == 1) - sql(s"reset ${SQLConf.GROUP_BY_ORDINAL.key}") - assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL) === true) - assert(sql(s"set").where(s"key = '${SQLConf.GROUP_BY_ORDINAL.key}'").count() == 0) - } finally { - sql(s"set ${SQLConf.GROUP_BY_ORDINAL}=$original") - } - } - - test("reset - internal conf") { - spark.sessionState.conf.clear() - val original = spark.conf.get(SQLConf.OPTIMIZER_MAX_ITERATIONS) - try { - assert(spark.conf.get(SQLConf.OPTIMIZER_MAX_ITERATIONS) === 100) - sql(s"set ${SQLConf.OPTIMIZER_MAX_ITERATIONS.key}=10") - assert(spark.conf.get(SQLConf.OPTIMIZER_MAX_ITERATIONS) === 10) - assert(sql(s"set").where(s"key = '${SQLConf.OPTIMIZER_MAX_ITERATIONS.key}'").count() == 1) - sql(s"reset") - assert(spark.conf.get(SQLConf.OPTIMIZER_MAX_ITERATIONS) === 100) - assert(sql(s"set").where(s"key = '${SQLConf.OPTIMIZER_MAX_ITERATIONS.key}'").count() == 0) - } finally { - sql(s"set ${SQLConf.OPTIMIZER_MAX_ITERATIONS}=$original") - } - } - - test("reset specific property - internal conf") { - spark.sessionState.conf.clear() - val original = spark.conf.get(SQLConf.OPTIMIZER_MAX_ITERATIONS) - try { - assert(spark.conf.get(SQLConf.OPTIMIZER_MAX_ITERATIONS) === 100) - sql(s"set ${SQLConf.OPTIMIZER_MAX_ITERATIONS.key}=10") - assert(spark.conf.get(SQLConf.OPTIMIZER_MAX_ITERATIONS) === 10) - assert(sql(s"set").where(s"key = '${SQLConf.OPTIMIZER_MAX_ITERATIONS.key}'").count() == 1) - sql(s"reset ${SQLConf.OPTIMIZER_MAX_ITERATIONS.key}") - assert(spark.conf.get(SQLConf.OPTIMIZER_MAX_ITERATIONS) === 100) - assert(sql(s"set").where(s"key = '${SQLConf.OPTIMIZER_MAX_ITERATIONS.key}'").count() == 0) - } finally { - sql(s"set ${SQLConf.OPTIMIZER_MAX_ITERATIONS}=$original") + Seq("reset", s"reset ${SQLConf.GROUP_BY_ORDINAL.key}").foreach { resetCmd => + test("reset - public conf") { + spark.sessionState.conf.clear() + val original = spark.conf.get(SQLConf.GROUP_BY_ORDINAL) + try { + assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL) === true) + sql(s"set ${SQLConf.GROUP_BY_ORDINAL.key}=false") + assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL) === false) + assert(sql(s"set").where(s"key = '${SQLConf.GROUP_BY_ORDINAL.key}'").count() == 1) + sql(resetCmd) + assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL) === true) + assert(sql(s"set").where(s"key = '${SQLConf.GROUP_BY_ORDINAL.key}'").count() == 0) + } finally { + sql(s"set ${SQLConf.GROUP_BY_ORDINAL}=$original") + } } } - test("reset - user-defined conf") { - spark.sessionState.conf.clear() - val userDefinedConf = "x.y.z.reset" - try { - assert(spark.conf.getOption(userDefinedConf).isEmpty) - sql(s"set $userDefinedConf=false") - assert(spark.conf.get(userDefinedConf) === "false") - assert(sql(s"set").where(s"key = '$userDefinedConf'").count() == 1) - sql(s"reset") - assert(spark.conf.getOption(userDefinedConf).isEmpty) - } finally { - spark.conf.unset(userDefinedConf) + Seq("reset", s"reset ${SQLConf.OPTIMIZER_MAX_ITERATIONS.key}").foreach { resetCmd => + test("reset - internal conf") { + spark.sessionState.conf.clear() + val original = spark.conf.get(SQLConf.OPTIMIZER_MAX_ITERATIONS) + try { + assert(spark.conf.get(SQLConf.OPTIMIZER_MAX_ITERATIONS) === 100) + sql(s"set ${SQLConf.OPTIMIZER_MAX_ITERATIONS.key}=10") + assert(spark.conf.get(SQLConf.OPTIMIZER_MAX_ITERATIONS) === 10) + assert(sql(s"set").where(s"key = '${SQLConf.OPTIMIZER_MAX_ITERATIONS.key}'").count() == 1) + sql(resetCmd) + assert(spark.conf.get(SQLConf.OPTIMIZER_MAX_ITERATIONS) === 100) + assert(sql(s"set").where(s"key = '${SQLConf.OPTIMIZER_MAX_ITERATIONS.key}'").count() == 0) + } finally { + sql(s"set ${SQLConf.OPTIMIZER_MAX_ITERATIONS}=$original") + } } } - test("reset specific property - user-defined conf") { - spark.sessionState.conf.clear() - val userDefinedConf = "x.y.z.reset" - try { - assert(spark.conf.getOption(userDefinedConf).isEmpty) - sql(s"set $userDefinedConf=false") - assert(spark.conf.get(userDefinedConf) === "false") - assert(sql(s"set").where(s"key = '$userDefinedConf'").count() == 1) - sql(s"reset $userDefinedConf") - assert(spark.conf.getOption(userDefinedConf).isEmpty) - } finally { - spark.conf.unset(userDefinedConf) + Seq("reset", s"reset $testKey").foreach { resetCmd => + test("reset - user-defined conf") { + spark.sessionState.conf.clear() + try { + assert(spark.conf.getOption(testKey).isEmpty) + sql(s"set $testKey=false") + assert(spark.conf.get(testKey) === "false") + assert(sql(s"set").where(s"key = '$testKey'").count() == 1) + sql(resetCmd) + assert(spark.conf.getOption(testKey).isEmpty) + } finally { + spark.conf.unset(testKey) + } } } From cbb424f596c423e15afae987c0ae6efb95858a33 Mon Sep 17 00:00:00 2001 From: wanghaihua Date: Tue, 13 Jun 2017 16:30:00 +0800 Subject: [PATCH 5/9] Fix typo. --- .../scala/org/apache/spark/sql/execution/SparkSqlParser.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index c3d2298ed916..829620d7c033 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -82,7 +82,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { * Example SQL : * {{{ * RESET; - * RESET `special#$!`; + * RESET key; * }}} */ override def visitResetConfiguration( From 99436d624d028fd972d5ec94eaedcf1b639b23a7 Mon Sep 17 00:00:00 2001 From: wanghaihua Date: Thu, 15 Jun 2017 21:16:29 +0800 Subject: [PATCH 6/9] Fix and add more test case. --- .../apache/spark/sql/execution/SparkSqlParserSuite.scala | 1 + .../scala/org/apache/spark/sql/internal/SQLConfSuite.scala | 6 +++--- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index 70f9516c4b39..562efc7b8518 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -305,5 +305,6 @@ class SparkSqlParserSuite extends PlanTest { test("reset") { assertEqual("reset", ResetCommand(None)) assertEqual("reset spark.test.property", ResetCommand(Some("spark.test.property"))) + assertEqual("reset #$a!", ResetCommand(Some("#$a!"))) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 104e16bac80e..0da85067d778 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -115,7 +115,7 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { } Seq("reset", s"reset ${SQLConf.GROUP_BY_ORDINAL.key}").foreach { resetCmd => - test("reset - public conf") { + test(s"reset - public conf $resetCmd") { spark.sessionState.conf.clear() val original = spark.conf.get(SQLConf.GROUP_BY_ORDINAL) try { @@ -133,7 +133,7 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { } Seq("reset", s"reset ${SQLConf.OPTIMIZER_MAX_ITERATIONS.key}").foreach { resetCmd => - test("reset - internal conf") { + test(s"reset - internal conf $resetCmd") { spark.sessionState.conf.clear() val original = spark.conf.get(SQLConf.OPTIMIZER_MAX_ITERATIONS) try { @@ -151,7 +151,7 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { } Seq("reset", s"reset $testKey").foreach { resetCmd => - test("reset - user-defined conf") { + test(s"reset - user-defined conf $resetCmd") { spark.sessionState.conf.clear() try { assert(spark.conf.getOption(testKey).isEmpty) From ecc26504f2d1807809f4ee6e53b540f4bb5317d6 Mon Sep 17 00:00:00 2001 From: wanghaihua Date: Sun, 25 Jun 2017 18:01:30 +0800 Subject: [PATCH 7/9] Add reset multiple config feature. --- .../sql/execution/command/SetCommand.scala | 3 ++- .../sql/execution/SparkSqlParserSuite.scala | 2 +- .../spark/sql/internal/SQLConfSuite.scala | 24 +++++++++++++++++++ 3 files changed, 27 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala index f0264e254267..1dd9923d7829 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala @@ -161,7 +161,8 @@ case class ResetCommand(key: Option[String]) extends RunnableCommand with Loggin sparkSession.sessionState.conf.clear() // "RESET key" clear a specific property. case Some(key) => - sparkSession.conf.unset(key) + key.split("\\s+") + .foreach(confName => if (!confName.isEmpty) sparkSession.conf.unset(confName)) } Seq.empty[Row] } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index 562efc7b8518..e2f341f439ff 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -305,6 +305,6 @@ class SparkSqlParserSuite extends PlanTest { test("reset") { assertEqual("reset", ResetCommand(None)) assertEqual("reset spark.test.property", ResetCommand(Some("spark.test.property"))) - assertEqual("reset #$a!", ResetCommand(Some("#$a!"))) + assertEqual("reset #$a! !a$# \t ", ResetCommand(Some("#$a! !a$#"))) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 0da85067d778..8face23cedf2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -166,6 +166,30 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { } } + Seq("reset", s"reset ${testKey}1 \t ${testKey}2 \t ").foreach { resetCmd => + test(s"reset - multiple conf $resetCmd") { + spark.sessionState.conf.clear() + val key1 = testKey + "1" + val key2 = testKey + "2" + try { + assert(spark.conf.getOption(key1).isEmpty) + assert(spark.conf.getOption(key2).isEmpty) + sql(s"set $key1=false") + sql(s"set $key2=true") + assert(spark.conf.get(key1) === "false") + assert(spark.conf.get(key2) === "true") + assert(sql(s"set").where(s"key = '$key1'").count() == 1) + assert(sql(s"set").where(s"key = '$key2'").count() == 1) + sql(resetCmd) + assert(spark.conf.getOption(key1).isEmpty) + assert(spark.conf.getOption(key2).isEmpty) + } finally { + spark.conf.unset(key1) + spark.conf.unset(key2) + } + } + } + test("invalid conf value") { spark.sessionState.conf.clear() val e = intercept[IllegalArgumentException] { From d8e7106a6e519954eb3935ebd402a9cd37615394 Mon Sep 17 00:00:00 2001 From: wanghaihua Date: Sun, 25 Jun 2017 18:44:34 +0800 Subject: [PATCH 8/9] Add comments for use case. --- .../org/apache/spark/sql/execution/command/SetCommand.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala index 1dd9923d7829..ce23f069232e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala @@ -151,6 +151,7 @@ object SetCommand { * {{{ * reset; * reset key; + * reset key1 key2 ...; * }}} */ case class ResetCommand(key: Option[String]) extends RunnableCommand with Logging { From e4cb39cd4c817066632085fc1ecf1bc81ec6ca5f Mon Sep 17 00:00:00 2001 From: wanghaihua Date: Mon, 26 Jun 2017 17:26:35 +0800 Subject: [PATCH 9/9] Modify according to comment. --- .../org/apache/spark/sql/internal/SQLConfSuite.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 8face23cedf2..5067b8a9619a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -115,7 +115,7 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { } Seq("reset", s"reset ${SQLConf.GROUP_BY_ORDINAL.key}").foreach { resetCmd => - test(s"reset - public conf $resetCmd") { + test(s"$resetCmd - public conf") { spark.sessionState.conf.clear() val original = spark.conf.get(SQLConf.GROUP_BY_ORDINAL) try { @@ -133,7 +133,7 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { } Seq("reset", s"reset ${SQLConf.OPTIMIZER_MAX_ITERATIONS.key}").foreach { resetCmd => - test(s"reset - internal conf $resetCmd") { + test(s"$resetCmd - internal conf") { spark.sessionState.conf.clear() val original = spark.conf.get(SQLConf.OPTIMIZER_MAX_ITERATIONS) try { @@ -151,7 +151,7 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { } Seq("reset", s"reset $testKey").foreach { resetCmd => - test(s"reset - user-defined conf $resetCmd") { + test(s"$resetCmd - user-defined conf") { spark.sessionState.conf.clear() try { assert(spark.conf.getOption(testKey).isEmpty) @@ -167,7 +167,7 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { } Seq("reset", s"reset ${testKey}1 \t ${testKey}2 \t ").foreach { resetCmd => - test(s"reset - multiple conf $resetCmd") { + test(s"$resetCmd - multiple conf") { spark.sessionState.conf.clear() val key1 = testKey + "1" val key2 = testKey + "2"