aboutsummaryrefslogtreecommitdiff
path: root/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala
blob: 952a0d676fbc3a11c57f80f74f7a95b621f972ba (plain) (blame)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
/*
 * Licensed to the Apache Software Foundation (ASF) under one or more
 * contributor license agreements.  See the NOTICE file distributed with
 * this work for additional information regarding copyright ownership.
 * The ASF licenses this file to You under the Apache License, Version 2.0
 * (the "License"); you may not use this file except in compliance with
 * the License.  You may obtain a copy of the License at
 *
 *    http://www.apache.org/licenses/LICENSE-2.0
 *
 * Unless required by applicable law or agreed to in writing, software
 * distributed under the License is distributed on an "AS IS" BASIS,
 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */

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
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.{StringType, StructField, StructType}


/**
 * Command that runs
 * {{{
 *   set key = value;
 *   set -v;
 *   set;
 * }}}
 */
case class SetCommand(kv: Option[(String, Option[String])]) extends RunnableCommand with Logging {

  private def keyValueOutput: Seq[Attribute] = {
    val schema = StructType(
      StructField("key", StringType, nullable = false) ::
        StructField("value", StringType, nullable = false) :: Nil)
    schema.toAttributes
  }

  private val (_output, runFunc): (Seq[Attribute], SparkSession => Seq[Row]) = kv match {
    // Configures the deprecated "mapred.reduce.tasks" property.
    case Some((SQLConf.Deprecated.MAPRED_REDUCE_TASKS, Some(value))) =>
      val runFunc = (sparkSession: SparkSession) => {
        logWarning(
          s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " +
            s"automatically converted to ${SQLConf.SHUFFLE_PARTITIONS.key} instead.")
        if (value.toInt < 1) {
          val msg =
            s"Setting negative ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} for automatically " +
              "determining the number of reducers is not supported."
          throw new IllegalArgumentException(msg)
        } else {
          sparkSession.setConf(SQLConf.SHUFFLE_PARTITIONS.key, value)
          Seq(Row(SQLConf.SHUFFLE_PARTITIONS.key, value))
        }
      }
      (keyValueOutput, runFunc)

    case Some((SQLConf.Deprecated.EXTERNAL_SORT, Some(value))) =>
      val runFunc = (sparkSession: SparkSession) => {
        logWarning(
          s"Property ${SQLConf.Deprecated.EXTERNAL_SORT} is deprecated and will be ignored. " +
            s"External sort will continue to be used.")
        Seq(Row(SQLConf.Deprecated.EXTERNAL_SORT, "true"))
      }
      (keyValueOutput, runFunc)

    case Some((SQLConf.Deprecated.USE_SQL_AGGREGATE2, Some(value))) =>
      val runFunc = (sparkSession: SparkSession) => {
        logWarning(
          s"Property ${SQLConf.Deprecated.USE_SQL_AGGREGATE2} is deprecated and " +
            s"will be ignored. ${SQLConf.Deprecated.USE_SQL_AGGREGATE2} will " +
            s"continue to be true.")
        Seq(Row(SQLConf.Deprecated.USE_SQL_AGGREGATE2, "true"))
      }
      (keyValueOutput, runFunc)

    case Some((SQLConf.Deprecated.TUNGSTEN_ENABLED, Some(value))) =>
      val runFunc = (sparkSession: SparkSession) => {
        logWarning(
          s"Property ${SQLConf.Deprecated.TUNGSTEN_ENABLED} is deprecated and " +
            s"will be ignored. Tungsten will continue to be used.")
        Seq(Row(SQLConf.Deprecated.TUNGSTEN_ENABLED, "true"))
      }
      (keyValueOutput, runFunc)

    case Some((SQLConf.Deprecated.CODEGEN_ENABLED, Some(value))) =>
      val runFunc = (sparkSession: SparkSession) => {
        logWarning(
          s"Property ${SQLConf.Deprecated.CODEGEN_ENABLED} is deprecated and " +
            s"will be ignored. Codegen will continue to be used.")
        Seq(Row(SQLConf.Deprecated.CODEGEN_ENABLED, "true"))
      }
      (keyValueOutput, runFunc)

    case Some((SQLConf.Deprecated.UNSAFE_ENABLED, Some(value))) =>
      val runFunc = (sparkSession: SparkSession) => {
        logWarning(
          s"Property ${SQLConf.Deprecated.UNSAFE_ENABLED} is deprecated and " +
            s"will be ignored. Unsafe mode will continue to be used.")
        Seq(Row(SQLConf.Deprecated.UNSAFE_ENABLED, "true"))
      }
      (keyValueOutput, runFunc)

    case Some((SQLConf.Deprecated.SORTMERGE_JOIN, Some(value))) =>
      val runFunc = (sparkSession: SparkSession) => {
        logWarning(
          s"Property ${SQLConf.Deprecated.SORTMERGE_JOIN} is deprecated and " +
            s"will be ignored. Sort merge join will continue to be used.")
        Seq(Row(SQLConf.Deprecated.SORTMERGE_JOIN, "true"))
      }
      (keyValueOutput, runFunc)

    case Some((SQLConf.Deprecated.PARQUET_UNSAFE_ROW_RECORD_READER_ENABLED, Some(value))) =>
      val runFunc = (sparkSession: SparkSession) => {
        logWarning(
          s"Property ${SQLConf.Deprecated.PARQUET_UNSAFE_ROW_RECORD_READER_ENABLED} is " +
            s"deprecated and will be ignored. Vectorized parquet reader will be used instead.")
        Seq(Row(SQLConf.PARQUET_VECTORIZED_READER_ENABLED, "true"))
      }
      (keyValueOutput, runFunc)

    // Configures a single property.
    case Some((key, Some(value))) =>
      val runFunc = (sparkSession: SparkSession) => {
        sparkSession.setConf(key, value)
        Seq(Row(key, value))
      }
      (keyValueOutput, runFunc)

    // (In Hive, "SET" returns all changed properties while "SET -v" returns all properties.)
    // Queries all key-value pairs that are set in the SQLConf of the sparkSession.
    case None =>
      val runFunc = (sparkSession: SparkSession) => {
        sparkSession.getAllConfs.map { case (k, v) => Row(k, v) }.toSeq
      }
      (keyValueOutput, runFunc)

    // Queries all properties along with their default values and docs that are defined in the
    // SQLConf of the sparkSession.
    case Some(("-v", None)) =>
      val runFunc = (sparkSession: SparkSession) => {
        sparkSession.sessionState.conf.getAllDefinedConfs.map { case (key, defaultValue, doc) =>
          Row(key, defaultValue, doc)
        }
      }
      val schema = StructType(
        StructField("key", StringType, nullable = false) ::
          StructField("default", StringType, nullable = false) ::
          StructField("meaning", StringType, nullable = false) :: Nil)
      (schema.toAttributes, runFunc)

    // Queries the deprecated "mapred.reduce.tasks" property.
    case Some((SQLConf.Deprecated.MAPRED_REDUCE_TASKS, None)) =>
      val runFunc = (sparkSession: SparkSession) => {
        logWarning(
          s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " +
            s"showing ${SQLConf.SHUFFLE_PARTITIONS.key} instead.")
        Seq(Row(
          SQLConf.SHUFFLE_PARTITIONS.key,
          sparkSession.sessionState.conf.numShufflePartitions.toString))
      }
      (keyValueOutput, runFunc)

    // Queries a single property.
    case Some((key, None)) =>
      val runFunc = (sparkSession: SparkSession) => {
        val value =
          try sparkSession.getConf(key) catch {
            case _: NoSuchElementException => "<undefined>"
          }
        Seq(Row(key, value))
      }
      (keyValueOutput, runFunc)
  }

  override val output: Seq[Attribute] = _output

  override def run(sparkSession: SparkSession): Seq[Row] = runFunc(sparkSession)

}