-
Notifications
You must be signed in to change notification settings - Fork 28k
/
HiveMetastoreCatalog.scala
367 lines (321 loc) · 15.2 KB
/
HiveMetastoreCatalog.scala
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
/*
* 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.hive
import scala.collection.JavaConverters._
import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
import org.apache.hadoop.fs.Path
import org.apache.spark.internal.Logging
import org.apache.spark.sql.{AnalysisException, SaveMode, SparkSession}
import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
import org.apache.spark.sql.catalyst.catalog._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules._
import org.apache.spark.sql.execution.command.DDLUtils
import org.apache.spark.sql.execution.datasources._
import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, ParquetOptions}
import org.apache.spark.sql.hive.orc.OrcFileFormat
import org.apache.spark.sql.types._
/**
* Legacy catalog for interacting with the Hive metastore.
*
* This is still used for things like creating data source tables, but in the future will be
* cleaned up to integrate more nicely with [[HiveExternalCatalog]].
*/
private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Logging {
private val sessionState = sparkSession.sessionState.asInstanceOf[HiveSessionState]
/** A fully qualified identifier for a table (i.e., database.tableName) */
case class QualifiedTableName(database: String, name: String)
private def getCurrentDatabase: String = sessionState.catalog.getCurrentDatabase
def getQualifiedTableName(tableIdent: TableIdentifier): QualifiedTableName = {
QualifiedTableName(
tableIdent.database.getOrElse(getCurrentDatabase).toLowerCase,
tableIdent.table.toLowerCase)
}
private def getQualifiedTableName(t: CatalogTable): QualifiedTableName = {
QualifiedTableName(
t.identifier.database.getOrElse(getCurrentDatabase).toLowerCase,
t.identifier.table.toLowerCase)
}
/** A cache of Spark SQL data source tables that have been accessed. */
protected[hive] val cachedDataSourceTables: LoadingCache[QualifiedTableName, LogicalPlan] = {
val cacheLoader = new CacheLoader[QualifiedTableName, LogicalPlan]() {
override def load(in: QualifiedTableName): LogicalPlan = {
logDebug(s"Creating new cached data source for $in")
val table = sparkSession.sharedState.externalCatalog.getTable(in.database, in.name)
val dataSource =
DataSource(
sparkSession,
userSpecifiedSchema = Some(table.schema),
partitionColumns = table.partitionColumnNames,
bucketSpec = table.bucketSpec,
className = table.provider.get,
options = table.storage.properties,
catalogTable = Some(table))
LogicalRelation(dataSource.resolveRelation(), catalogTable = Some(table))
}
}
CacheBuilder.newBuilder().maximumSize(1000).build(cacheLoader)
}
def refreshTable(tableIdent: TableIdentifier): Unit = {
// refreshTable does not eagerly reload the cache. It just invalidate the cache.
// Next time when we use the table, it will be populated in the cache.
// Since we also cache ParquetRelations converted from Hive Parquet tables and
// adding converted ParquetRelations into the cache is not defined in the load function
// of the cache (instead, we add the cache entry in convertToParquetRelation),
// it is better at here to invalidate the cache to avoid confusing waring logs from the
// cache loader (e.g. cannot find data source provider, which is only defined for
// data source table.).
cachedDataSourceTables.invalidate(getQualifiedTableName(tableIdent))
}
def hiveDefaultTableFilePath(tableIdent: TableIdentifier): String = {
// Code based on: hiveWarehouse.getTablePath(currentDatabase, tableName)
val QualifiedTableName(dbName, tblName) = getQualifiedTableName(tableIdent)
val dbLocation = sparkSession.sharedState.externalCatalog.getDatabase(dbName).locationUri
new Path(new Path(dbLocation), tblName).toString
}
def lookupRelation(
tableIdent: TableIdentifier,
alias: Option[String]): LogicalPlan = {
val qualifiedTableName = getQualifiedTableName(tableIdent)
val table = sparkSession.sharedState.externalCatalog.getTable(
qualifiedTableName.database, qualifiedTableName.name)
if (DDLUtils.isDatasourceTable(table)) {
val dataSourceTable = cachedDataSourceTables(qualifiedTableName)
val qualifiedTable = SubqueryAlias(qualifiedTableName.name, dataSourceTable, None)
// Then, if alias is specified, wrap the table with a Subquery using the alias.
// Otherwise, wrap the table with a Subquery using the table name.
alias.map(a => SubqueryAlias(a, qualifiedTable, None)).getOrElse(qualifiedTable)
} else if (table.tableType == CatalogTableType.VIEW) {
val viewText = table.viewText.getOrElse(sys.error("Invalid view without text."))
SubqueryAlias(
alias.getOrElse(table.identifier.table),
sparkSession.sessionState.sqlParser.parsePlan(viewText),
Option(table.identifier))
} else {
val qualifiedTable =
MetastoreRelation(
qualifiedTableName.database, qualifiedTableName.name)(table, sparkSession)
alias.map(a => SubqueryAlias(a, qualifiedTable, None)).getOrElse(qualifiedTable)
}
}
private def getCached(
tableIdentifier: QualifiedTableName,
pathsInMetastore: Seq[Path],
metastoreRelation: MetastoreRelation,
schemaInMetastore: StructType,
expectedFileFormat: Class[_ <: FileFormat],
expectedBucketSpec: Option[BucketSpec],
partitionSchema: Option[StructType]): Option[LogicalRelation] = {
cachedDataSourceTables.getIfPresent(tableIdentifier) match {
case null => None // Cache miss
case logical @ LogicalRelation(relation: HadoopFsRelation, _, _) =>
val cachedRelationFileFormatClass = relation.fileFormat.getClass
expectedFileFormat match {
case `cachedRelationFileFormatClass` =>
// If we have the same paths, same schema, and same partition spec,
// we will use the cached relation.
val useCached =
relation.location.rootPaths.toSet == pathsInMetastore.toSet &&
logical.schema.sameType(schemaInMetastore) &&
relation.bucketSpec == expectedBucketSpec &&
relation.partitionSchema == partitionSchema.getOrElse(StructType(Nil))
if (useCached) {
Some(logical)
} else {
// If the cached relation is not updated, we invalidate it right away.
cachedDataSourceTables.invalidate(tableIdentifier)
None
}
case _ =>
logWarning(
s"${metastoreRelation.databaseName}.${metastoreRelation.tableName} " +
s"should be stored as $expectedFileFormat. However, we are getting " +
s"a ${relation.fileFormat} from the metastore cache. This cached " +
s"entry will be invalidated.")
cachedDataSourceTables.invalidate(tableIdentifier)
None
}
case other =>
logWarning(
s"${metastoreRelation.databaseName}.${metastoreRelation.tableName} should be stored " +
s"as $expectedFileFormat. However, we are getting a $other from the metastore cache. " +
s"This cached entry will be invalidated.")
cachedDataSourceTables.invalidate(tableIdentifier)
None
}
}
private def convertToLogicalRelation(
metastoreRelation: MetastoreRelation,
options: Map[String, String],
defaultSource: FileFormat,
fileFormatClass: Class[_ <: FileFormat],
fileType: String): LogicalRelation = {
val metastoreSchema = StructType.fromAttributes(metastoreRelation.output)
val tableIdentifier =
QualifiedTableName(metastoreRelation.databaseName, metastoreRelation.tableName)
val bucketSpec = None // We don't support hive bucketed tables, only ones we write out.
val lazyPruningEnabled = sparkSession.sqlContext.conf.filesourcePartitionManagement
val result = if (metastoreRelation.hiveQlTable.isPartitioned) {
val partitionSchema = StructType.fromAttributes(metastoreRelation.partitionKeys)
val rootPaths: Seq[Path] = if (lazyPruningEnabled) {
Seq(metastoreRelation.hiveQlTable.getDataLocation)
} else {
// By convention (for example, see TableFileCatalog), the definition of a
// partitioned table's paths depends on whether that table has any actual partitions.
// Partitioned tables without partitions use the location of the table's base path.
// Partitioned tables with partitions use the locations of those partitions' data
// locations,_omitting_ the table's base path.
val paths = metastoreRelation.getHiveQlPartitions().map { p =>
new Path(p.getLocation)
}
if (paths.isEmpty) {
Seq(metastoreRelation.hiveQlTable.getDataLocation)
} else {
paths
}
}
val cached = getCached(
tableIdentifier,
rootPaths,
metastoreRelation,
metastoreSchema,
fileFormatClass,
bucketSpec,
Some(partitionSchema))
val logicalRelation = cached.getOrElse {
val db = metastoreRelation.databaseName
val table = metastoreRelation.tableName
val sizeInBytes = metastoreRelation.statistics.sizeInBytes.toLong
val fileCatalog = {
val catalog = new TableFileCatalog(
sparkSession, db, table, partitionSchema, sizeInBytes)
if (lazyPruningEnabled) {
catalog
} else {
catalog.filterPartitions(Nil) // materialize all the partitions in memory
}
}
val partitionSchemaColumnNames = partitionSchema.map(_.name.toLowerCase).toSet
val dataSchema =
StructType(metastoreSchema
.filterNot(field => partitionSchemaColumnNames.contains(field.name.toLowerCase)))
val relation = HadoopFsRelation(
location = fileCatalog,
partitionSchema = partitionSchema,
dataSchema = dataSchema,
bucketSpec = bucketSpec,
fileFormat = defaultSource,
options = options)(sparkSession = sparkSession)
val created = LogicalRelation(relation, catalogTable = Some(metastoreRelation.catalogTable))
cachedDataSourceTables.put(tableIdentifier, created)
created
}
logicalRelation
} else {
val rootPath = metastoreRelation.hiveQlTable.getDataLocation
val cached = getCached(tableIdentifier,
Seq(rootPath),
metastoreRelation,
metastoreSchema,
fileFormatClass,
bucketSpec,
None)
val logicalRelation = cached.getOrElse {
val created =
LogicalRelation(
DataSource(
sparkSession = sparkSession,
paths = rootPath.toString :: Nil,
userSpecifiedSchema = Some(metastoreRelation.schema),
bucketSpec = bucketSpec,
options = options,
className = fileType).resolveRelation(),
catalogTable = Some(metastoreRelation.catalogTable))
cachedDataSourceTables.put(tableIdentifier, created)
created
}
logicalRelation
}
result.copy(expectedOutputAttributes = Some(metastoreRelation.output))
}
/**
* When scanning or writing to non-partitioned Metastore Parquet tables, convert them to Parquet
* data source relations for better performance.
*/
object ParquetConversions extends Rule[LogicalPlan] {
private def shouldConvertMetastoreParquet(relation: MetastoreRelation): Boolean = {
relation.tableDesc.getSerdeClassName.toLowerCase.contains("parquet") &&
sessionState.convertMetastoreParquet
}
private def convertToParquetRelation(relation: MetastoreRelation): LogicalRelation = {
val defaultSource = new ParquetFileFormat()
val fileFormatClass = classOf[ParquetFileFormat]
val mergeSchema = sessionState.convertMetastoreParquetWithSchemaMerging
val options = Map(ParquetOptions.MERGE_SCHEMA -> mergeSchema.toString)
convertToLogicalRelation(relation, options, defaultSource, fileFormatClass, "parquet")
}
override def apply(plan: LogicalPlan): LogicalPlan = {
if (!plan.resolved || plan.analyzed) {
return plan
}
plan transformUp {
// Write path
case InsertIntoTable(r: MetastoreRelation, partition, child, overwrite, ifNotExists)
// Inserting into partitioned table is not supported in Parquet data source (yet).
if !r.hiveQlTable.isPartitioned && shouldConvertMetastoreParquet(r) =>
InsertIntoTable(convertToParquetRelation(r), partition, child, overwrite, ifNotExists)
// Read path
case relation: MetastoreRelation if shouldConvertMetastoreParquet(relation) =>
val parquetRelation = convertToParquetRelation(relation)
SubqueryAlias(relation.tableName, parquetRelation, None)
}
}
}
/**
* When scanning Metastore ORC tables, convert them to ORC data source relations
* for better performance.
*/
object OrcConversions extends Rule[LogicalPlan] {
private def shouldConvertMetastoreOrc(relation: MetastoreRelation): Boolean = {
relation.tableDesc.getSerdeClassName.toLowerCase.contains("orc") &&
sessionState.convertMetastoreOrc
}
private def convertToOrcRelation(relation: MetastoreRelation): LogicalRelation = {
val defaultSource = new OrcFileFormat()
val fileFormatClass = classOf[OrcFileFormat]
val options = Map[String, String]()
convertToLogicalRelation(relation, options, defaultSource, fileFormatClass, "orc")
}
override def apply(plan: LogicalPlan): LogicalPlan = {
if (!plan.resolved || plan.analyzed) {
return plan
}
plan transformUp {
// Write path
case InsertIntoTable(r: MetastoreRelation, partition, child, overwrite, ifNotExists)
// Inserting into partitioned table is not supported in Orc data source (yet).
if !r.hiveQlTable.isPartitioned && shouldConvertMetastoreOrc(r) =>
InsertIntoTable(convertToOrcRelation(r), partition, child, overwrite, ifNotExists)
// Read path
case relation: MetastoreRelation if shouldConvertMetastoreOrc(relation) =>
val orcRelation = convertToOrcRelation(relation)
SubqueryAlias(relation.tableName, orcRelation, None)
}
}
}
}