Skip to content

Commit 6c3832b

Browse files
Andrew Orrxin
authored andcommitted
[SPARK-13080][SQL] Implement new Catalog API using Hive
## What changes were proposed in this pull request? This is a step towards merging `SQLContext` and `HiveContext`. A new internal Catalog API was introduced in #10982 and extended in #11069. This patch introduces an implementation of this API using `HiveClient`, an existing interface to Hive. It also extends `HiveClient` with additional calls to Hive that are needed to complete the catalog implementation. *Where should I start reviewing?* The new catalog introduced is `HiveCatalog`. This class is relatively simple because it just calls `HiveClientImpl`, where most of the new logic is. I would not start with `HiveClient`, `HiveQl`, or `HiveMetastoreCatalog`, which are modified mainly because of a refactor. *Why is this patch so big?* I had to refactor HiveClient to remove an intermediate representation of databases, tables, partitions etc. After this refactor `CatalogTable` convert directly to and from `HiveTable` (etc.). Otherwise we would have to first convert `CatalogTable` to the intermediate representation and then convert that to HiveTable, which is messy. The new class hierarchy is as follows: ``` org.apache.spark.sql.catalyst.catalog.Catalog - org.apache.spark.sql.catalyst.catalog.InMemoryCatalog - org.apache.spark.sql.hive.HiveCatalog ``` Note that, as of this patch, none of these classes are currently used anywhere yet. This will come in the future before the Spark 2.0 release. ## How was the this patch tested? All existing unit tests, and HiveCatalogSuite that extends CatalogTestCases. Author: Andrew Or <[email protected]> Author: Reynold Xin <[email protected]> Closes #11293 from rxin/hive-catalog.
1 parent 7eb83fe commit 6c3832b

File tree

21 files changed

+1483
-700
lines changed

21 files changed

+1483
-700
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,9 @@ package org.apache.spark.sql
1919

2020
import org.apache.spark.annotation.DeveloperApi
2121

22+
23+
// TODO: don't swallow original stack trace if it exists
24+
2225
/**
2326
* :: DeveloperApi ::
2427
* Thrown when a query fails to analyze, usually because the query itself is invalid.

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala

Lines changed: 0 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -20,20 +20,11 @@ package org.apache.spark.sql.catalyst.analysis
2020
import java.util.concurrent.ConcurrentHashMap
2121

2222
import scala.collection.JavaConverters._
23-
import scala.collection.mutable
24-
import scala.collection.mutable.ArrayBuffer
2523

2624
import org.apache.spark.sql.AnalysisException
2725
import org.apache.spark.sql.catalyst.{CatalystConf, EmptyConf, TableIdentifier}
2826
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
2927

30-
/**
31-
* Thrown by a catalog when a table cannot be found. The analyzer will rethrow the exception
32-
* as an AnalysisException with the correct position information.
33-
*/
34-
class NoSuchTableException extends Exception
35-
36-
class NoSuchDatabaseException extends Exception
3728

3829
/**
3930
* An interface for looking up relations by name. Used by an [[Analyzer]].
Lines changed: 52 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,52 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.sql.catalyst.analysis
19+
20+
import org.apache.spark.sql.catalyst.catalog.Catalog.TablePartitionSpec
21+
22+
23+
/**
24+
* Thrown by a catalog when an item cannot be found. The analyzer will rethrow the exception
25+
* as an [[org.apache.spark.sql.AnalysisException]] with the correct position information.
26+
*/
27+
abstract class NoSuchItemException extends Exception {
28+
override def getMessage: String
29+
}
30+
31+
class NoSuchDatabaseException(db: String) extends NoSuchItemException {
32+
override def getMessage: String = s"Database $db not found"
33+
}
34+
35+
class NoSuchTableException(db: String, table: String) extends NoSuchItemException {
36+
override def getMessage: String = s"Table $table not found in database $db"
37+
}
38+
39+
class NoSuchPartitionException(
40+
db: String,
41+
table: String,
42+
spec: TablePartitionSpec)
43+
extends NoSuchItemException {
44+
45+
override def getMessage: String = {
46+
s"Partition not found in table $table database $db:\n" + spec.mkString("\n")
47+
}
48+
}
49+
50+
class NoSuchFunctionException(db: String, func: String) extends NoSuchItemException {
51+
override def getMessage: String = s"Function $func not found in database $db"
52+
}

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala

Lines changed: 80 additions & 74 deletions
Original file line numberDiff line numberDiff line change
@@ -30,15 +30,16 @@ import org.apache.spark.sql.AnalysisException
3030
class InMemoryCatalog extends Catalog {
3131
import Catalog._
3232

33-
private class TableDesc(var table: Table) {
34-
val partitions = new mutable.HashMap[PartitionSpec, TablePartition]
33+
private class TableDesc(var table: CatalogTable) {
34+
val partitions = new mutable.HashMap[TablePartitionSpec, CatalogTablePartition]
3535
}
3636

37-
private class DatabaseDesc(var db: Database) {
37+
private class DatabaseDesc(var db: CatalogDatabase) {
3838
val tables = new mutable.HashMap[String, TableDesc]
39-
val functions = new mutable.HashMap[String, Function]
39+
val functions = new mutable.HashMap[String, CatalogFunction]
4040
}
4141

42+
// Database name -> description
4243
private val catalog = new scala.collection.mutable.HashMap[String, DatabaseDesc]
4344

4445
private def filterPattern(names: Seq[String], pattern: String): Seq[String] = {
@@ -47,39 +48,33 @@ class InMemoryCatalog extends Catalog {
4748
}
4849

4950
private def existsFunction(db: String, funcName: String): Boolean = {
50-
assertDbExists(db)
51+
requireDbExists(db)
5152
catalog(db).functions.contains(funcName)
5253
}
5354

5455
private def existsTable(db: String, table: String): Boolean = {
55-
assertDbExists(db)
56+
requireDbExists(db)
5657
catalog(db).tables.contains(table)
5758
}
5859

59-
private def existsPartition(db: String, table: String, spec: PartitionSpec): Boolean = {
60-
assertTableExists(db, table)
60+
private def existsPartition(db: String, table: String, spec: TablePartitionSpec): Boolean = {
61+
requireTableExists(db, table)
6162
catalog(db).tables(table).partitions.contains(spec)
6263
}
6364

64-
private def assertDbExists(db: String): Unit = {
65-
if (!catalog.contains(db)) {
66-
throw new AnalysisException(s"Database $db does not exist")
67-
}
68-
}
69-
70-
private def assertFunctionExists(db: String, funcName: String): Unit = {
65+
private def requireFunctionExists(db: String, funcName: String): Unit = {
7166
if (!existsFunction(db, funcName)) {
7267
throw new AnalysisException(s"Function $funcName does not exist in $db database")
7368
}
7469
}
7570

76-
private def assertTableExists(db: String, table: String): Unit = {
71+
private def requireTableExists(db: String, table: String): Unit = {
7772
if (!existsTable(db, table)) {
7873
throw new AnalysisException(s"Table $table does not exist in $db database")
7974
}
8075
}
8176

82-
private def assertPartitionExists(db: String, table: String, spec: PartitionSpec): Unit = {
77+
private def requirePartitionExists(db: String, table: String, spec: TablePartitionSpec): Unit = {
8378
if (!existsPartition(db, table, spec)) {
8479
throw new AnalysisException(s"Partition does not exist in database $db table $table: $spec")
8580
}
@@ -90,7 +85,7 @@ class InMemoryCatalog extends Catalog {
9085
// --------------------------------------------------------------------------
9186

9287
override def createDatabase(
93-
dbDefinition: Database,
88+
dbDefinition: CatalogDatabase,
9489
ignoreIfExists: Boolean): Unit = synchronized {
9590
if (catalog.contains(dbDefinition.name)) {
9691
if (!ignoreIfExists) {
@@ -124,17 +119,20 @@ class InMemoryCatalog extends Catalog {
124119
}
125120
}
126121

127-
override def alterDatabase(db: String, dbDefinition: Database): Unit = synchronized {
128-
assertDbExists(db)
129-
assert(db == dbDefinition.name)
130-
catalog(db).db = dbDefinition
122+
override def alterDatabase(dbDefinition: CatalogDatabase): Unit = synchronized {
123+
requireDbExists(dbDefinition.name)
124+
catalog(dbDefinition.name).db = dbDefinition
131125
}
132126

133-
override def getDatabase(db: String): Database = synchronized {
134-
assertDbExists(db)
127+
override def getDatabase(db: String): CatalogDatabase = synchronized {
128+
requireDbExists(db)
135129
catalog(db).db
136130
}
137131

132+
override def databaseExists(db: String): Boolean = synchronized {
133+
catalog.contains(db)
134+
}
135+
138136
override def listDatabases(): Seq[String] = synchronized {
139137
catalog.keySet.toSeq
140138
}
@@ -143,15 +141,17 @@ class InMemoryCatalog extends Catalog {
143141
filterPattern(listDatabases(), pattern)
144142
}
145143

144+
override def setCurrentDatabase(db: String): Unit = { /* no-op */ }
145+
146146
// --------------------------------------------------------------------------
147147
// Tables
148148
// --------------------------------------------------------------------------
149149

150150
override def createTable(
151151
db: String,
152-
tableDefinition: Table,
152+
tableDefinition: CatalogTable,
153153
ignoreIfExists: Boolean): Unit = synchronized {
154-
assertDbExists(db)
154+
requireDbExists(db)
155155
if (existsTable(db, tableDefinition.name)) {
156156
if (!ignoreIfExists) {
157157
throw new AnalysisException(s"Table ${tableDefinition.name} already exists in $db database")
@@ -165,7 +165,7 @@ class InMemoryCatalog extends Catalog {
165165
db: String,
166166
table: String,
167167
ignoreIfNotExists: Boolean): Unit = synchronized {
168-
assertDbExists(db)
168+
requireDbExists(db)
169169
if (existsTable(db, table)) {
170170
catalog(db).tables.remove(table)
171171
} else {
@@ -176,31 +176,30 @@ class InMemoryCatalog extends Catalog {
176176
}
177177

178178
override def renameTable(db: String, oldName: String, newName: String): Unit = synchronized {
179-
assertTableExists(db, oldName)
179+
requireTableExists(db, oldName)
180180
val oldDesc = catalog(db).tables(oldName)
181181
oldDesc.table = oldDesc.table.copy(name = newName)
182182
catalog(db).tables.put(newName, oldDesc)
183183
catalog(db).tables.remove(oldName)
184184
}
185185

186-
override def alterTable(db: String, table: String, tableDefinition: Table): Unit = synchronized {
187-
assertTableExists(db, table)
188-
assert(table == tableDefinition.name)
189-
catalog(db).tables(table).table = tableDefinition
186+
override def alterTable(db: String, tableDefinition: CatalogTable): Unit = synchronized {
187+
requireTableExists(db, tableDefinition.name)
188+
catalog(db).tables(tableDefinition.name).table = tableDefinition
190189
}
191190

192-
override def getTable(db: String, table: String): Table = synchronized {
193-
assertTableExists(db, table)
191+
override def getTable(db: String, table: String): CatalogTable = synchronized {
192+
requireTableExists(db, table)
194193
catalog(db).tables(table).table
195194
}
196195

197196
override def listTables(db: String): Seq[String] = synchronized {
198-
assertDbExists(db)
197+
requireDbExists(db)
199198
catalog(db).tables.keySet.toSeq
200199
}
201200

202201
override def listTables(db: String, pattern: String): Seq[String] = synchronized {
203-
assertDbExists(db)
202+
requireDbExists(db)
204203
filterPattern(listTables(db), pattern)
205204
}
206205

@@ -211,9 +210,9 @@ class InMemoryCatalog extends Catalog {
211210
override def createPartitions(
212211
db: String,
213212
table: String,
214-
parts: Seq[TablePartition],
213+
parts: Seq[CatalogTablePartition],
215214
ignoreIfExists: Boolean): Unit = synchronized {
216-
assertTableExists(db, table)
215+
requireTableExists(db, table)
217216
val existingParts = catalog(db).tables(table).partitions
218217
if (!ignoreIfExists) {
219218
val dupSpecs = parts.collect { case p if existingParts.contains(p.spec) => p.spec }
@@ -229,9 +228,9 @@ class InMemoryCatalog extends Catalog {
229228
override def dropPartitions(
230229
db: String,
231230
table: String,
232-
partSpecs: Seq[PartitionSpec],
231+
partSpecs: Seq[TablePartitionSpec],
233232
ignoreIfNotExists: Boolean): Unit = synchronized {
234-
assertTableExists(db, table)
233+
requireTableExists(db, table)
235234
val existingParts = catalog(db).tables(table).partitions
236235
if (!ignoreIfNotExists) {
237236
val missingSpecs = partSpecs.collect { case s if !existingParts.contains(s) => s }
@@ -244,75 +243,82 @@ class InMemoryCatalog extends Catalog {
244243
partSpecs.foreach(existingParts.remove)
245244
}
246245

247-
override def alterPartition(
246+
override def renamePartitions(
248247
db: String,
249248
table: String,
250-
spec: Map[String, String],
251-
newPart: TablePartition): Unit = synchronized {
252-
assertPartitionExists(db, table, spec)
253-
val existingParts = catalog(db).tables(table).partitions
254-
if (spec != newPart.spec) {
255-
// Also a change in specs; remove the old one and add the new one back
256-
existingParts.remove(spec)
249+
specs: Seq[TablePartitionSpec],
250+
newSpecs: Seq[TablePartitionSpec]): Unit = synchronized {
251+
require(specs.size == newSpecs.size, "number of old and new partition specs differ")
252+
specs.zip(newSpecs).foreach { case (oldSpec, newSpec) =>
253+
val newPart = getPartition(db, table, oldSpec).copy(spec = newSpec)
254+
val existingParts = catalog(db).tables(table).partitions
255+
existingParts.remove(oldSpec)
256+
existingParts.put(newSpec, newPart)
257+
}
258+
}
259+
260+
override def alterPartitions(
261+
db: String,
262+
table: String,
263+
parts: Seq[CatalogTablePartition]): Unit = synchronized {
264+
parts.foreach { p =>
265+
requirePartitionExists(db, table, p.spec)
266+
catalog(db).tables(table).partitions.put(p.spec, p)
257267
}
258-
existingParts.put(newPart.spec, newPart)
259268
}
260269

261270
override def getPartition(
262271
db: String,
263272
table: String,
264-
spec: Map[String, String]): TablePartition = synchronized {
265-
assertPartitionExists(db, table, spec)
273+
spec: TablePartitionSpec): CatalogTablePartition = synchronized {
274+
requirePartitionExists(db, table, spec)
266275
catalog(db).tables(table).partitions(spec)
267276
}
268277

269-
override def listPartitions(db: String, table: String): Seq[TablePartition] = synchronized {
270-
assertTableExists(db, table)
278+
override def listPartitions(
279+
db: String,
280+
table: String): Seq[CatalogTablePartition] = synchronized {
281+
requireTableExists(db, table)
271282
catalog(db).tables(table).partitions.values.toSeq
272283
}
273284

274285
// --------------------------------------------------------------------------
275286
// Functions
276287
// --------------------------------------------------------------------------
277288

278-
override def createFunction(
279-
db: String,
280-
func: Function,
281-
ignoreIfExists: Boolean): Unit = synchronized {
282-
assertDbExists(db)
289+
override def createFunction(db: String, func: CatalogFunction): Unit = synchronized {
290+
requireDbExists(db)
283291
if (existsFunction(db, func.name)) {
284-
if (!ignoreIfExists) {
285-
throw new AnalysisException(s"Function $func already exists in $db database")
286-
}
292+
throw new AnalysisException(s"Function $func already exists in $db database")
287293
} else {
288294
catalog(db).functions.put(func.name, func)
289295
}
290296
}
291297

292298
override def dropFunction(db: String, funcName: String): Unit = synchronized {
293-
assertFunctionExists(db, funcName)
299+
requireFunctionExists(db, funcName)
294300
catalog(db).functions.remove(funcName)
295301
}
296302

297-
override def alterFunction(
298-
db: String,
299-
funcName: String,
300-
funcDefinition: Function): Unit = synchronized {
301-
assertFunctionExists(db, funcName)
302-
if (funcName != funcDefinition.name) {
303-
// Also a rename; remove the old one and add the new one back
304-
catalog(db).functions.remove(funcName)
305-
}
303+
override def renameFunction(db: String, oldName: String, newName: String): Unit = synchronized {
304+
requireFunctionExists(db, oldName)
305+
val newFunc = getFunction(db, oldName).copy(name = newName)
306+
catalog(db).functions.remove(oldName)
307+
catalog(db).functions.put(newName, newFunc)
308+
}
309+
310+
override def alterFunction(db: String, funcDefinition: CatalogFunction): Unit = synchronized {
311+
requireFunctionExists(db, funcDefinition.name)
306312
catalog(db).functions.put(funcDefinition.name, funcDefinition)
307313
}
308314

309-
override def getFunction(db: String, funcName: String): Function = synchronized {
310-
assertFunctionExists(db, funcName)
315+
override def getFunction(db: String, funcName: String): CatalogFunction = synchronized {
316+
requireFunctionExists(db, funcName)
311317
catalog(db).functions(funcName)
312318
}
313319

314320
override def listFunctions(db: String, pattern: String): Seq[String] = synchronized {
315-
assertDbExists(db)
321+
requireDbExists(db)
316322
filterPattern(catalog(db).functions.keysIterator.toSeq, pattern)
317323
}
318324

0 commit comments

Comments
 (0)