From 9c2a376e413b0701097b0784bd725e4ca87cd837 Mon Sep 17 00:00:00 2001 From: gatorsmile <gatorsmile@gmail.com> Date: Thu, 19 May 2016 10:05:53 +0800 Subject: [PATCH] [SPARK-15297][SQL] Fix Set -V Command #### What changes were proposed in this pull request? The command `SET -v` always outputs the default values even if we set the parameter. This behavior is incorrect. Instead, if users override it, we should output the user-specified value. In addition, the output schema of `SET -v` is wrong. We should use the column `value` instead of `default` for the parameter value. This PR is to fix the above two issues. #### How was this patch tested? Added a test case. Author: gatorsmile <gatorsmile@gmail.com> Closes #13081 from gatorsmile/setVcommand. --- .../sql/execution/command/SetCommand.scala | 4 +--- .../apache/spark/sql/internal/SQLConf.scala | 2 +- .../spark/sql/internal/SQLConfSuite.scala | 23 ++++++++++++++++++- 3 files changed, 24 insertions(+), 5 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 2409b5d203..282f26ce99 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 @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution.command -import java.util.NoSuchElementException - import org.apache.spark.internal.Logging import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.expressions.Attribute @@ -88,7 +86,7 @@ case class SetCommand(kv: Option[(String, Option[String])]) extends RunnableComm } val schema = StructType( StructField("key", StringType, nullable = false) :: - StructField("default", StringType, nullable = false) :: + StructField("value", StringType, nullable = false) :: StructField("meaning", StringType, nullable = false) :: Nil) (schema.toAttributes, runFunc) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 7933d12e28..518430f16d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -752,7 +752,7 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { */ def getAllDefinedConfs: Seq[(String, String, String)] = sqlConfEntries.synchronized { sqlConfEntries.values.asScala.filter(_.isPublic).map { entry => - (entry.key, entry.defaultValueString, entry.doc) + (entry.key, getConfString(entry.key, entry.defaultValueString), entry.doc) }.toSeq } 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 7ead97bbf6..81bc973be7 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 @@ -17,7 +17,7 @@ package org.apache.spark.sql.internal -import org.apache.spark.sql.{QueryTest, SparkSession, SQLContext} +import org.apache.spark.sql.{QueryTest, Row, SparkSession, SQLContext} import org.apache.spark.sql.test.{SharedSQLContext, TestSQLContext} class SQLConfSuite extends QueryTest with SharedSQLContext { @@ -75,6 +75,27 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { spark.wrapped.conf.clear() } + test("set command for display") { + spark.wrapped.conf.clear() + checkAnswer( + sql("SET").where("key = 'spark.sql.groupByOrdinal'").select("key", "value"), + Nil) + + checkAnswer( + sql("SET -v").where("key = 'spark.sql.groupByOrdinal'").select("key", "value"), + Row("spark.sql.groupByOrdinal", "true")) + + sql("SET spark.sql.groupByOrdinal=false") + + checkAnswer( + sql("SET").where("key = 'spark.sql.groupByOrdinal'").select("key", "value"), + Row("spark.sql.groupByOrdinal", "false")) + + checkAnswer( + sql("SET -v").where("key = 'spark.sql.groupByOrdinal'").select("key", "value"), + Row("spark.sql.groupByOrdinal", "false")) + } + test("deprecated property") { spark.wrapped.conf.clear() val original = spark.conf.get(SQLConf.SHUFFLE_PARTITIONS) -- GitLab