aboutsummaryrefslogtreecommitdiff
path: root/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala
blob: 9293239131d52723a8772fe520519b4b05b66df3 (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
/*
 * 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

import org.apache.spark.Logging
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.errors.TreeNodeException
import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRow}
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.{Row, SQLConf, SQLContext}

trait Command {
  /**
   * A concrete command should override this lazy field to wrap up any side effects caused by the
   * command or any other computation that should be evaluated exactly once. The value of this field
   * can be used as the contents of the corresponding RDD generated from the physical plan of this
   * command.
   *
   * The `execute()` method of all the physical command classes should reference `sideEffectResult`
   * so that the command can be executed eagerly right after the command query is created.
   */
  protected[sql] lazy val sideEffectResult: Seq[Any] = Seq.empty[Any]
}

/**
 * :: DeveloperApi ::
 */
@DeveloperApi
case class SetCommand(
    key: Option[String], value: Option[String], output: Seq[Attribute])(
    @transient context: SQLContext)
  extends LeafNode with Command with Logging {

  override protected[sql] lazy val sideEffectResult: Seq[String] = (key, value) match {
    // Set value for key k.
    case (Some(k), Some(v)) =>
      if (k == SQLConf.Deprecated.MAPRED_REDUCE_TASKS) {
        logWarning(s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " +
          s"automatically converted to ${SQLConf.SHUFFLE_PARTITIONS} instead.")
        context.set(SQLConf.SHUFFLE_PARTITIONS, v)
        Array(s"${SQLConf.SHUFFLE_PARTITIONS}=$v")
      } else {
        context.set(k, v)
        Array(s"$k=$v")
      }

    // Query the value bound to key k.
    case (Some(k), _) =>
      // TODO (lian) This is just a workaround to make the Simba ODBC driver work.
      // Should remove this once we get the ODBC driver updated.
      if (k == "-v") {
        val hiveJars = Seq(
          "hive-exec-0.12.0.jar",
          "hive-service-0.12.0.jar",
          "hive-common-0.12.0.jar",
          "hive-hwi-0.12.0.jar",
          "hive-0.12.0.jar").mkString(":")

        Array(
          "system:java.class.path=" + hiveJars,
          "system:sun.java.command=shark.SharkServer2")
      }
      else {
        Array(s"$k=${context.getOption(k).getOrElse("<undefined>")}")
      }

    // Query all key-value pairs that are set in the SQLConf of the context.
    case (None, None) =>
      context.getAll.map { case (k, v) =>
        s"$k=$v"
      }

    case _ =>
      throw new IllegalArgumentException()
  }

  def execute(): RDD[Row] = {
    val rows = sideEffectResult.map { line => new GenericRow(Array[Any](line)) }
    context.sparkContext.parallelize(rows, 1)
  }

  override def otherCopyArgs = context :: Nil
}

/**
 * An explain command for users to see how a command will be executed.
 *
 * Note that this command takes in a logical plan, runs the optimizer on the logical plan
 * (but do NOT actually execute it).
 *
 * :: DeveloperApi ::
 */
@DeveloperApi
case class ExplainCommand(
    logicalPlan: LogicalPlan, output: Seq[Attribute])(
    @transient context: SQLContext)
  extends LeafNode with Command {

  // Run through the optimizer to generate the physical plan.
  override protected[sql] lazy val sideEffectResult: Seq[String] = try {
    "Physical execution plan:" +: context.executePlan(logicalPlan).executedPlan.toString.split("\n")
  } catch { case cause: TreeNodeException[_] =>
    "Error occurred during query planning: " +: cause.getMessage.split("\n")
  }

  def execute(): RDD[Row] = {
    val explanation = sideEffectResult.map(row => new GenericRow(Array[Any](row)))
    context.sparkContext.parallelize(explanation, 1)
  }

  override def otherCopyArgs = context :: Nil
}

/**
 * :: DeveloperApi ::
 */
@DeveloperApi
case class CacheCommand(tableName: String, doCache: Boolean)(@transient context: SQLContext)
  extends LeafNode with Command {

  override protected[sql] lazy val sideEffectResult = {
    if (doCache) {
      context.cacheTable(tableName)
    } else {
      context.uncacheTable(tableName)
    }
    Seq.empty[Any]
  }

  override def execute(): RDD[Row] = {
    sideEffectResult
    context.emptyResult
  }

  override def output: Seq[Attribute] = Seq.empty
}

/**
 * :: DeveloperApi ::
 */
@DeveloperApi
case class DescribeCommand(child: SparkPlan, output: Seq[Attribute])(
    @transient context: SQLContext)
  extends LeafNode with Command {

  override protected[sql] lazy val sideEffectResult: Seq[(String, String, String)] = {
    Seq(("# Registered as a temporary table", null, null)) ++
      child.output.map(field => (field.name, field.dataType.toString, null))
  }

  override def execute(): RDD[Row] = {
    val rows = sideEffectResult.map {
      case (name, dataType, comment) => new GenericRow(Array[Any](name, dataType, comment))
    }
    context.sparkContext.parallelize(rows, 1)
  }
}