aboutsummaryrefslogtreecommitdiff
path: root/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala
blob: d3d72089c330318c06d0a5725c99ad22b549531c (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
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
/*
 * 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.sources

import scala.language.implicitConversions

import org.apache.spark.Logging
import org.apache.spark.sql.{DataFrame, SQLContext}
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.AbstractSparkSQLParser
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
import org.apache.spark.sql.catalyst.expressions.AttributeReference
import org.apache.spark.sql.execution.RunnableCommand
import org.apache.spark.sql.types._
import org.apache.spark.util.Utils

/**
 * A parser for foreign DDL commands.
 */
private[sql] class DDLParser extends AbstractSparkSQLParser with Logging {

  def apply(input: String, exceptionOnError: Boolean): Option[LogicalPlan] = {
    try {
      Some(apply(input))
    } catch {
      case ddlException: DDLException => throw ddlException
      case _ if !exceptionOnError => None
      case x: Throwable => throw x
    }
  }

  def parseType(input: String): DataType = {
    lexical.initialize(reservedWords)
    phrase(dataType)(new lexical.Scanner(input)) match {
      case Success(r, x) => r
      case x => throw new DDLException(s"Unsupported dataType: $x")
    }
  }

  // Keyword is a convention with AbstractSparkSQLParser, which will scan all of the `Keyword`
  // properties via reflection the class in runtime for constructing the SqlLexical object
  protected val CREATE = Keyword("CREATE")
  protected val TEMPORARY = Keyword("TEMPORARY")
  protected val TABLE = Keyword("TABLE")
  protected val IF = Keyword("IF")
  protected val NOT = Keyword("NOT")
  protected val EXISTS = Keyword("EXISTS")
  protected val USING = Keyword("USING")
  protected val OPTIONS = Keyword("OPTIONS")
  protected val DESCRIBE = Keyword("DESCRIBE")
  protected val EXTENDED = Keyword("EXTENDED")
  protected val AS = Keyword("AS")
  protected val COMMENT = Keyword("COMMENT")

  // Data types.
  protected val STRING = Keyword("STRING")
  protected val BINARY = Keyword("BINARY")
  protected val BOOLEAN = Keyword("BOOLEAN")
  protected val TINYINT = Keyword("TINYINT")
  protected val SMALLINT = Keyword("SMALLINT")
  protected val INT = Keyword("INT")
  protected val BIGINT = Keyword("BIGINT")
  protected val FLOAT = Keyword("FLOAT")
  protected val DOUBLE = Keyword("DOUBLE")
  protected val DECIMAL = Keyword("DECIMAL")
  protected val DATE = Keyword("DATE")
  protected val TIMESTAMP = Keyword("TIMESTAMP")
  protected val VARCHAR = Keyword("VARCHAR")
  protected val ARRAY = Keyword("ARRAY")
  protected val MAP = Keyword("MAP")
  protected val STRUCT = Keyword("STRUCT")

  protected lazy val ddl: Parser[LogicalPlan] = createTable | describeTable

  protected def start: Parser[LogicalPlan] = ddl

  /**
   * `CREATE [TEMPORARY] TABLE avroTable [IF NOT EXISTS]
   * USING org.apache.spark.sql.avro
   * OPTIONS (path "../hive/src/test/resources/data/files/episodes.avro")`
   * or
   * `CREATE [TEMPORARY] TABLE avroTable(intField int, stringField string...) [IF NOT EXISTS]
   * USING org.apache.spark.sql.avro
   * OPTIONS (path "../hive/src/test/resources/data/files/episodes.avro")`
   * or
   * `CREATE [TEMPORARY] TABLE avroTable [IF NOT EXISTS]
   * USING org.apache.spark.sql.avro
   * OPTIONS (path "../hive/src/test/resources/data/files/episodes.avro")`
   * AS SELECT ...
   */
  protected lazy val createTable: Parser[LogicalPlan] =
  (
    (CREATE ~> TEMPORARY.? <~ TABLE) ~ (IF ~> NOT <~ EXISTS).? ~ ident
      ~ (tableCols).? ~ (USING ~> className) ~ (OPTIONS ~> options).? ~ (AS ~> restInput).? ^^ {
      case temp ~ allowExisting ~ tableName ~ columns ~ provider ~ opts ~ query =>
        if (temp.isDefined && allowExisting.isDefined) {
          throw new DDLException(
            "a CREATE TEMPORARY TABLE statement does not allow IF NOT EXISTS clause.")
        }

        val options = opts.getOrElse(Map.empty[String, String])
        if (query.isDefined) {
          if (columns.isDefined) {
            throw new DDLException(
              "a CREATE TABLE AS SELECT statement does not allow column definitions.")
          }
          // When IF NOT EXISTS clause appears in the query, the save mode will be ignore.
          val mode = if (allowExisting.isDefined) {
            SaveMode.Ignore
          } else if (temp.isDefined) {
            SaveMode.Overwrite
          } else {
            SaveMode.ErrorIfExists
          }

          CreateTableUsingAsSelect(tableName,
            provider,
            temp.isDefined,
            mode,
            options,
            query.get)
        } else {
          val userSpecifiedSchema = columns.flatMap(fields => Some(StructType(fields)))
          CreateTableUsing(
            tableName,
            userSpecifiedSchema,
            provider,
            temp.isDefined,
            options,
            allowExisting.isDefined,
            managedIfNoPath = false)
        }
      }
  )

  protected lazy val tableCols: Parser[Seq[StructField]] =  "(" ~> repsep(column, ",") <~ ")"

  /*
   * describe [extended] table avroTable
   * This will display all columns of table `avroTable` includes column_name,column_type,comment
   */
  protected lazy val describeTable: Parser[LogicalPlan] =
    (DESCRIBE ~> opt(EXTENDED)) ~ (ident <~ ".").? ~ ident  ^^ {
      case e ~ db ~ tbl  =>
        val tblIdentifier = db match {
          case Some(dbName) =>
            Seq(dbName, tbl)
          case None =>
            Seq(tbl)
        }
        DescribeCommand(UnresolvedRelation(tblIdentifier, None), e.isDefined)
   }

  protected lazy val options: Parser[Map[String, String]] =
    "(" ~> repsep(pair, ",") <~ ")" ^^ { case s: Seq[(String, String)] => s.toMap }

  protected lazy val className: Parser[String] = repsep(ident, ".") ^^ { case s => s.mkString(".")}

  protected lazy val pair: Parser[(String, String)] = ident ~ stringLit ^^ { case k ~ v => (k,v) }

  protected lazy val column: Parser[StructField] =
    ident ~ dataType ~ (COMMENT ~> stringLit).?  ^^ { case columnName ~ typ ~ cm =>
      val meta = cm match {
        case Some(comment) =>
          new MetadataBuilder().putString(COMMENT.str.toLowerCase(), comment).build()
        case None => Metadata.empty
      }
      StructField(columnName, typ, true, meta)
    }

  protected lazy val primitiveType: Parser[DataType] =
    STRING ^^^ StringType |
    BINARY ^^^ BinaryType |
    BOOLEAN ^^^ BooleanType |
    TINYINT ^^^ ByteType |
    SMALLINT ^^^ ShortType |
    INT ^^^ IntegerType |
    BIGINT ^^^ LongType |
    FLOAT ^^^ FloatType |
    DOUBLE ^^^ DoubleType |
    fixedDecimalType |                   // decimal with precision/scale
    DECIMAL ^^^ DecimalType.Unlimited |  // decimal with no precision/scale
    DATE ^^^ DateType |
    TIMESTAMP ^^^ TimestampType |
    VARCHAR ~ "(" ~ numericLit ~ ")" ^^^ StringType

  protected lazy val fixedDecimalType: Parser[DataType] =
    (DECIMAL ~ "(" ~> numericLit) ~ ("," ~> numericLit <~ ")") ^^ {
      case precision ~ scale => DecimalType(precision.toInt, scale.toInt)
    }

  protected lazy val arrayType: Parser[DataType] =
    ARRAY ~> "<" ~> dataType <~ ">" ^^ {
      case tpe => ArrayType(tpe)
    }

  protected lazy val mapType: Parser[DataType] =
    MAP ~> "<" ~> dataType ~ "," ~ dataType <~ ">" ^^ {
      case t1 ~ _ ~ t2 => MapType(t1, t2)
    }

  protected lazy val structField: Parser[StructField] =
    ident ~ ":" ~ dataType ^^ {
      case fieldName ~ _ ~ tpe => StructField(fieldName, tpe, nullable = true)
    }

  protected lazy val structType: Parser[DataType] =
    (STRUCT ~> "<" ~> repsep(structField, ",") <~ ">" ^^ {
    case fields => StructType(fields)
    }) |
    (STRUCT ~> "<>" ^^ {
      case fields => StructType(Nil)
    })

  private[sql] lazy val dataType: Parser[DataType] =
    arrayType |
    mapType |
    structType |
    primitiveType
}

private[sql] object ResolvedDataSource {

  private val builtinSources = Map(
    "jdbc" -> classOf[org.apache.spark.sql.jdbc.DefaultSource],
    "json" -> classOf[org.apache.spark.sql.json.DefaultSource],
    "parquet" -> classOf[org.apache.spark.sql.parquet.DefaultSource]
  )

  /** Given a provider name, look up the data source class definition. */
  def lookupDataSource(provider: String): Class[_] = {
    if (builtinSources.contains(provider)) {
      return builtinSources(provider)
    }

    val loader = Utils.getContextOrSparkClassLoader
    try {
      loader.loadClass(provider)
    } catch {
      case cnf: java.lang.ClassNotFoundException =>
        try {
          loader.loadClass(provider + ".DefaultSource")
        } catch {
          case cnf: java.lang.ClassNotFoundException =>
            sys.error(s"Failed to load class for data source: $provider")
        }
    }
  }

  /** Create a [[ResolvedDataSource]] for reading data in. */
  def apply(
      sqlContext: SQLContext,
      userSpecifiedSchema: Option[StructType],
      provider: String,
      options: Map[String, String]): ResolvedDataSource = {
    val clazz: Class[_] = lookupDataSource(provider)
    val relation = userSpecifiedSchema match {
      case Some(schema: StructType) => clazz.newInstance() match {
        case dataSource: SchemaRelationProvider =>
          dataSource.createRelation(sqlContext, new CaseInsensitiveMap(options), schema)
        case dataSource: org.apache.spark.sql.sources.RelationProvider =>
          sys.error(s"${clazz.getCanonicalName} does not allow user-specified schemas.")
      }

      case None => clazz.newInstance() match {
        case dataSource: RelationProvider =>
          dataSource.createRelation(sqlContext, new CaseInsensitiveMap(options))
        case dataSource: org.apache.spark.sql.sources.SchemaRelationProvider =>
          sys.error(s"A schema needs to be specified when using ${clazz.getCanonicalName}.")
      }
    }
    new ResolvedDataSource(clazz, relation)
  }

  /** Create a [[ResolvedDataSource]] for saving the content of the given [[DataFrame]]. */
  def apply(
      sqlContext: SQLContext,
      provider: String,
      mode: SaveMode,
      options: Map[String, String],
      data: DataFrame): ResolvedDataSource = {
    val clazz: Class[_] = lookupDataSource(provider)
    val relation = clazz.newInstance() match {
      case dataSource: CreatableRelationProvider =>
        dataSource.createRelation(sqlContext, mode, options, data)
      case _ =>
        sys.error(s"${clazz.getCanonicalName} does not allow create table as select.")
    }
    new ResolvedDataSource(clazz, relation)
  }
}

private[sql] case class ResolvedDataSource(provider: Class[_], relation: BaseRelation)

/**
 * Returned for the "DESCRIBE [EXTENDED] [dbName.]tableName" command.
 * @param table The table to be described.
 * @param isExtended True if "DESCRIBE EXTENDED" is used. Otherwise, false.
 *                   It is effective only when the table is a Hive table.
 */
private[sql] case class DescribeCommand(
    table: LogicalPlan,
    isExtended: Boolean) extends Command {
  override val output = Seq(
    // Column names are based on Hive.
    AttributeReference("col_name", StringType, nullable = false, 
      new MetadataBuilder().putString("comment", "name of the column").build())(),
    AttributeReference("data_type", StringType, nullable = false, 
      new MetadataBuilder().putString("comment", "data type of the column").build())(),
    AttributeReference("comment", StringType, nullable = false, 
      new MetadataBuilder().putString("comment", "comment of the column").build())())
}

/**
  * Used to represent the operation of create table using a data source.
  * @param tableName
  * @param userSpecifiedSchema
  * @param provider
  * @param temporary
  * @param options
  * @param allowExisting If it is true, we will do nothing when the table already exists.
 *                      If it is false, an exception will be thrown
  * @param managedIfNoPath
  */
private[sql] case class CreateTableUsing(
    tableName: String,
    userSpecifiedSchema: Option[StructType],
    provider: String,
    temporary: Boolean,
    options: Map[String, String],
    allowExisting: Boolean,
    managedIfNoPath: Boolean) extends Command

private[sql] case class CreateTableUsingAsSelect(
    tableName: String,
    provider: String,
    temporary: Boolean,
    mode: SaveMode,
    options: Map[String, String],
    query: String) extends Command

private[sql] case class CreateTableUsingAsLogicalPlan(
    tableName: String,
    provider: String,
    temporary: Boolean,
    mode: SaveMode,
    options: Map[String, String],
    query: LogicalPlan) extends Command

private [sql] case class CreateTempTableUsing(
    tableName: String,
    userSpecifiedSchema: Option[StructType],
    provider: String,
    options: Map[String, String]) extends RunnableCommand {

  def run(sqlContext: SQLContext) = {
    val resolved = ResolvedDataSource(sqlContext, userSpecifiedSchema, provider, options)
    sqlContext.registerRDDAsTable(
      DataFrame(sqlContext, LogicalRelation(resolved.relation)), tableName)
    Seq.empty
  }
}

private [sql] case class CreateTempTableUsingAsSelect(
    tableName: String,
    provider: String,
    mode: SaveMode,
    options: Map[String, String],
    query: LogicalPlan) extends RunnableCommand {

  def run(sqlContext: SQLContext) = {
    val df = DataFrame(sqlContext, query)
    val resolved = ResolvedDataSource(sqlContext, provider, mode, options, df)
    sqlContext.registerRDDAsTable(
      DataFrame(sqlContext, LogicalRelation(resolved.relation)), tableName)

    Seq.empty
  }
}

/**
 * Builds a map in which keys are case insensitive
 */
protected[sql] class CaseInsensitiveMap(map: Map[String, String]) extends Map[String, String]
  with Serializable {

  val baseMap = map.map(kv => kv.copy(_1 = kv._1.toLowerCase))

  override def get(k: String): Option[String] = baseMap.get(k.toLowerCase)

  override def + [B1 >: String](kv: (String, B1)): Map[String, B1] =
    baseMap + kv.copy(_1 = kv._1.toLowerCase)

  override def iterator: Iterator[(String, String)] = baseMap.iterator

  override def -(key: String): Map[String, String] = baseMap - key.toLowerCase()
}

/**
 * The exception thrown from the DDL parser.
 */
protected[sql] class DDLException(message: String) extends Exception(message)