aboutsummaryrefslogtreecommitdiff
path: root/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala
blob: a1087edd03fdfd95ec486cf38770659129313ff6 (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
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
/*
 * 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.internal

import scala.collection.JavaConverters._
import scala.reflect.runtime.universe.TypeTag

import org.apache.spark.annotation.Experimental
import org.apache.spark.sql._
import org.apache.spark.sql.catalog.{Catalog, Column, Database, Function, Table}
import org.apache.spark.sql.catalyst.{DefinedByConstructorParams, FunctionIdentifier, TableIdentifier}
import org.apache.spark.sql.catalyst.catalog._
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
import org.apache.spark.sql.execution.datasources.CreateTable
import org.apache.spark.sql.types.StructType


/**
 * Internal implementation of the user-facing [[Catalog]].
 */
class CatalogImpl(sparkSession: SparkSession) extends Catalog {

  private def sessionCatalog: SessionCatalog = sparkSession.sessionState.catalog

  private def requireDatabaseExists(dbName: String): Unit = {
    if (!sessionCatalog.databaseExists(dbName)) {
      throw new AnalysisException(s"Database '$dbName' does not exist.")
    }
  }

  private def requireTableExists(dbName: String, tableName: String): Unit = {
    if (!sessionCatalog.tableExists(TableIdentifier(tableName, Some(dbName)))) {
      throw new AnalysisException(s"Table '$tableName' does not exist in database '$dbName'.")
    }
  }

  /**
   * Returns the current default database in this session.
   */
  override def currentDatabase: String = sessionCatalog.getCurrentDatabase

  /**
   * Sets the current default database in this session.
   */
  @throws[AnalysisException]("database does not exist")
  override def setCurrentDatabase(dbName: String): Unit = {
    requireDatabaseExists(dbName)
    sessionCatalog.setCurrentDatabase(dbName)
  }

  /**
   * Returns a list of databases available across all sessions.
   */
  override def listDatabases(): Dataset[Database] = {
    val databases = sessionCatalog.listDatabases().map { dbName =>
      makeDatabase(sessionCatalog.getDatabaseMetadata(dbName))
    }
    CatalogImpl.makeDataset(databases, sparkSession)
  }

  private def makeDatabase(metadata: CatalogDatabase): Database = {
    new Database(
      name = metadata.name,
      description = metadata.description,
      locationUri = metadata.locationUri)
  }

  /**
   * Returns a list of tables in the current database.
   * This includes all temporary tables.
   */
  override def listTables(): Dataset[Table] = {
    listTables(currentDatabase)
  }

  /**
   * Returns a list of tables in the specified database.
   * This includes all temporary tables.
   */
  @throws[AnalysisException]("database does not exist")
  override def listTables(dbName: String): Dataset[Table] = {
    requireDatabaseExists(dbName)
    val tables = sessionCatalog.listTables(dbName).map { tableIdent =>
      makeTable(tableIdent, tableIdent.database.isEmpty)
    }
    CatalogImpl.makeDataset(tables, sparkSession)
  }

  private def makeTable(tableIdent: TableIdentifier, isTemp: Boolean): Table = {
    val metadata = if (isTemp) None else Some(sessionCatalog.getTableMetadata(tableIdent))
    new Table(
      name = tableIdent.identifier,
      database = metadata.flatMap(_.identifier.database).orNull,
      description = metadata.flatMap(_.comment).orNull,
      tableType = metadata.map(_.tableType.name).getOrElse("TEMPORARY"),
      isTemporary = isTemp)
  }

  /**
   * Returns a list of functions registered in the current database.
   * This includes all temporary functions
   */
  override def listFunctions(): Dataset[Function] = {
    listFunctions(currentDatabase)
  }

  /**
   * Returns a list of functions registered in the specified database.
   * This includes all temporary functions
   */
  @throws[AnalysisException]("database does not exist")
  override def listFunctions(dbName: String): Dataset[Function] = {
    requireDatabaseExists(dbName)
    val functions = sessionCatalog.listFunctions(dbName).map { case (functIdent, _) =>
      makeFunction(functIdent)
    }
    CatalogImpl.makeDataset(functions, sparkSession)
  }

  private def makeFunction(funcIdent: FunctionIdentifier): Function = {
    val metadata = sessionCatalog.lookupFunctionInfo(funcIdent)
    new Function(
      name = funcIdent.identifier,
      database = funcIdent.database.orNull,
      description = null, // for now, this is always undefined
      className = metadata.getClassName,
      isTemporary = funcIdent.database.isEmpty)
  }

  /**
   * Returns a list of columns for the given table in the current database.
   */
  @throws[AnalysisException]("table does not exist")
  override def listColumns(tableName: String): Dataset[Column] = {
    listColumns(TableIdentifier(tableName, None))
  }

  /**
   * Returns a list of columns for the given table in the specified database.
   */
  @throws[AnalysisException]("database or table does not exist")
  override def listColumns(dbName: String, tableName: String): Dataset[Column] = {
    requireTableExists(dbName, tableName)
    listColumns(TableIdentifier(tableName, Some(dbName)))
  }

  private def listColumns(tableIdentifier: TableIdentifier): Dataset[Column] = {
    val tableMetadata = sessionCatalog.getTempViewOrPermanentTableMetadata(tableIdentifier)

    val partitionColumnNames = tableMetadata.partitionColumnNames.toSet
    val bucketColumnNames = tableMetadata.bucketSpec.map(_.bucketColumnNames).getOrElse(Nil).toSet
    val columns = tableMetadata.schema.map { c =>
      new Column(
        name = c.name,
        description = c.getComment().orNull,
        dataType = c.dataType.catalogString,
        nullable = c.nullable,
        isPartition = partitionColumnNames.contains(c.name),
        isBucket = bucketColumnNames.contains(c.name))
    }
    CatalogImpl.makeDataset(columns, sparkSession)
  }

  /**
   * Find the database with the specified name. This throws an [[AnalysisException]] when no
   * [[Database]] can be found.
   */
  override def findDatabase(dbName: String): Database = {
    if (sessionCatalog.databaseExists(dbName)) {
      makeDatabase(sessionCatalog.getDatabaseMetadata(dbName))
    } else {
      throw new AnalysisException(s"The specified database $dbName does not exist.")
    }
  }

  /**
   * Find the table with the specified name. This table can be a temporary table or a table in the
   * current database. This throws an [[AnalysisException]] when no [[Table]] can be found.
   */
  override def findTable(tableName: String): Table = {
    findTable(null, tableName)
  }

  /**
   * Find the table with the specified name in the specified database. This throws an
   * [[AnalysisException]] when no [[Table]] can be found.
   */
  override def findTable(dbName: String, tableName: String): Table = {
    val tableIdent = TableIdentifier(tableName, Option(dbName))
    val isTemporary = sessionCatalog.isTemporaryTable(tableIdent)
    if (isTemporary || sessionCatalog.tableExists(tableIdent)) {
      makeTable(tableIdent, isTemporary)
    } else {
      throw new AnalysisException(s"The specified table $tableIdent does not exist.")
    }
  }

  /**
   * Find the function with the specified name. This function can be a temporary function or a
   * function in the current database. This throws an [[AnalysisException]] when no [[Function]]
   * can be found.
   */
  override def findFunction(functionName: String): Function = {
    findFunction(null, functionName)
  }

  /**
   * Find the function with the specified name. This returns [[None]] when no [[Function]] can be
   * found.
   */
  override def findFunction(dbName: String, functionName: String): Function = {
    val functionIdent = FunctionIdentifier(functionName, Option(dbName))
    if (sessionCatalog.functionExists(functionIdent)) {
      makeFunction(functionIdent)
    } else {
      throw new AnalysisException(s"The specified function $functionIdent does not exist.")
    }
  }

  /**
   * Check if the database with the specified name exists.
   */
  override def databaseExists(dbName: String): Boolean = {
    sessionCatalog.databaseExists(dbName)
  }

  /**
   * Check if the table with the specified name exists. This can either be a temporary table or a
   * table in the current database.
   */
  override def tableExists(tableName: String): Boolean = {
    tableExists(null, tableName)
  }

  /**
   * Check if the table with the specified name exists in the specified database.
   */
  override def tableExists(dbName: String, tableName: String): Boolean = {
    val tableIdent = TableIdentifier(tableName, Option(dbName))
    sessionCatalog.isTemporaryTable(tableIdent) || sessionCatalog.tableExists(tableIdent)
  }

  /**
   * Check if the function with the specified name exists. This can either be a temporary function
   * or a function in the current database.
   */
  override def functionExists(functionName: String): Boolean = {
    functionExists(null, functionName)
  }

  /**
   * Check if the function with the specified name exists in the specified database.
   */
  override def functionExists(dbName: String, functionName: String): Boolean = {
    sessionCatalog.functionExists(FunctionIdentifier(functionName, Option(dbName)))
  }

  /**
   * :: Experimental ::
   * Creates an external table from the given path and returns the corresponding DataFrame.
   * It will use the default data source configured by spark.sql.sources.default.
   *
   * @group ddl_ops
   * @since 2.0.0
   */
  @Experimental
  override def createExternalTable(tableName: String, path: String): DataFrame = {
    val dataSourceName = sparkSession.sessionState.conf.defaultDataSourceName
    createExternalTable(tableName, path, dataSourceName)
  }

  /**
   * :: Experimental ::
   * Creates an external table from the given path based on a data source
   * and returns the corresponding DataFrame.
   *
   * @group ddl_ops
   * @since 2.0.0
   */
  @Experimental
  override def createExternalTable(tableName: String, path: String, source: String): DataFrame = {
    createExternalTable(tableName, source, Map("path" -> path))
  }

  /**
   * :: Experimental ::
   * Creates an external table from the given path based on a data source and a set of options.
   * Then, returns the corresponding DataFrame.
   *
   * @group ddl_ops
   * @since 2.0.0
   */
  @Experimental
  override def createExternalTable(
      tableName: String,
      source: String,
      options: java.util.Map[String, String]): DataFrame = {
    createExternalTable(tableName, source, options.asScala.toMap)
  }

  /**
   * :: Experimental ::
   * (Scala-specific)
   * Creates an external table from the given path based on a data source and a set of options.
   * Then, returns the corresponding DataFrame.
   *
   * @group ddl_ops
   * @since 2.0.0
   */
  @Experimental
  override def createExternalTable(
      tableName: String,
      source: String,
      options: Map[String, String]): DataFrame = {
    createExternalTable(tableName, source, new StructType, options)
  }

  /**
   * :: Experimental ::
   * Create an external table from the given path based on a data source, a schema and
   * a set of options. Then, returns the corresponding DataFrame.
   *
   * @group ddl_ops
   * @since 2.0.0
   */
  @Experimental
  override def createExternalTable(
      tableName: String,
      source: String,
      schema: StructType,
      options: java.util.Map[String, String]): DataFrame = {
    createExternalTable(tableName, source, schema, options.asScala.toMap)
  }

  /**
   * :: Experimental ::
   * (Scala-specific)
   * Create an external table from the given path based on a data source, a schema and
   * a set of options. Then, returns the corresponding DataFrame.
   *
   * @group ddl_ops
   * @since 2.0.0
   */
  @Experimental
  override def createExternalTable(
      tableName: String,
      source: String,
      schema: StructType,
      options: Map[String, String]): DataFrame = {
    if (source.toLowerCase == "hive") {
      throw new AnalysisException("Cannot create hive serde table with createExternalTable API.")
    }

    val tableIdent = sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)
    val tableDesc = CatalogTable(
      identifier = tableIdent,
      tableType = CatalogTableType.EXTERNAL,
      storage = CatalogStorageFormat.empty.copy(properties = options),
      schema = schema,
      provider = Some(source)
    )
    val plan = CreateTable(tableDesc, SaveMode.ErrorIfExists, None)
    sparkSession.sessionState.executePlan(plan).toRdd
    sparkSession.table(tableIdent)
  }

  /**
   * Drops the temporary view with the given view name in the catalog.
   * If the view has been cached/persisted before, it's also unpersisted.
   *
   * @param viewName the name of the view to be dropped.
   * @group ddl_ops
   * @since 2.0.0
   */
  override def dropTempView(viewName: String): Unit = {
    sparkSession.sessionState.catalog.getTempView(viewName).foreach { tempView =>
      sparkSession.sharedState.cacheManager.uncacheQuery(Dataset.ofRows(sparkSession, tempView))
      sessionCatalog.dropTempView(viewName)
    }
  }

  /**
   * Returns true if the table is currently cached in-memory.
   *
   * @group cachemgmt
   * @since 2.0.0
   */
  override def isCached(tableName: String): Boolean = {
    sparkSession.sharedState.cacheManager.lookupCachedData(sparkSession.table(tableName)).nonEmpty
  }

  /**
   * Caches the specified table in-memory.
   *
   * @group cachemgmt
   * @since 2.0.0
   */
  override def cacheTable(tableName: String): Unit = {
    sparkSession.sharedState.cacheManager.cacheQuery(sparkSession.table(tableName), Some(tableName))
  }

  /**
   * Removes the specified table from the in-memory cache.
   *
   * @group cachemgmt
   * @since 2.0.0
   */
  override def uncacheTable(tableName: String): Unit = {
    sparkSession.sharedState.cacheManager.uncacheQuery(query = sparkSession.table(tableName))
  }

  /**
   * Removes all cached tables from the in-memory cache.
   *
   * @group cachemgmt
   * @since 2.0.0
   */
  override def clearCache(): Unit = {
    sparkSession.sharedState.cacheManager.clearCache()
  }

  /**
   * Returns true if the [[Dataset]] is currently cached in-memory.
   *
   * @group cachemgmt
   * @since 2.0.0
   */
  protected[sql] def isCached(qName: Dataset[_]): Boolean = {
    sparkSession.sharedState.cacheManager.lookupCachedData(qName).nonEmpty
  }

  /**
   * Refresh the cache entry for a table, if any. For Hive metastore table, the metadata
   * is refreshed. For data source tables, the schema will not be inferred and refreshed.
   *
   * @group cachemgmt
   * @since 2.0.0
   */
  override def refreshTable(tableName: String): Unit = {
    val tableIdent = sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)
    // Temp tables: refresh (or invalidate) any metadata/data cached in the plan recursively.
    // Non-temp tables: refresh the metadata cache.
    sessionCatalog.refreshTable(tableIdent)

    // If this table is cached as an InMemoryRelation, drop the original
    // cached version and make the new version cached lazily.
    val logicalPlan = sparkSession.sessionState.catalog.lookupRelation(tableIdent)
    // Use lookupCachedData directly since RefreshTable also takes databaseName.
    val isCached = sparkSession.sharedState.cacheManager.lookupCachedData(logicalPlan).nonEmpty
    if (isCached) {
      // Create a data frame to represent the table.
      // TODO: Use uncacheTable once it supports database name.
      val df = Dataset.ofRows(sparkSession, logicalPlan)
      // Uncache the logicalPlan.
      sparkSession.sharedState.cacheManager.uncacheQuery(df, blocking = true)
      // Cache it again.
      sparkSession.sharedState.cacheManager.cacheQuery(df, Some(tableIdent.table))
    }
  }

  /**
   * Refresh the cache entry and the associated metadata for all dataframes (if any), that contain
   * the given data source path.
   *
   * @group cachemgmt
   * @since 2.0.0
   */
  override def refreshByPath(resourcePath: String): Unit = {
    sparkSession.sharedState.cacheManager.invalidateCachedPath(sparkSession, resourcePath)
  }
}


private[sql] object CatalogImpl {

  def makeDataset[T <: DefinedByConstructorParams: TypeTag](
      data: Seq[T],
      sparkSession: SparkSession): Dataset[T] = {
    val enc = ExpressionEncoder[T]()
    val encoded = data.map(d => enc.toRow(d).copy())
    val plan = new LocalRelation(enc.schema.toAttributes, encoded)
    val queryExecution = sparkSession.sessionState.executePlan(plan)
    new Dataset[T](sparkSession, queryExecution, enc)
  }

}