Skip to content

Commit fabbde4

Browse files
Luan, Xuedongallenma
Luan, Xuedong
authored andcommitted
[CARMEL-3524] Create ReadWriteLock for each database in HiveExternalC… (#24)
* [CARMEL-3524] Create ReadWriteLock for each database in HiveExternalCatalog * fix code style check * fix comment
1 parent c173b47 commit fabbde4

File tree

1 file changed

+60
-40
lines changed

1 file changed

+60
-40
lines changed

sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala

Lines changed: 60 additions & 40 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,8 @@ import java.io.IOException
2121
import java.lang.reflect.InvocationTargetException
2222
import java.util
2323
import java.util.Locale
24+
import java.util.concurrent.ConcurrentHashMap
25+
import java.util.concurrent.locks.ReentrantReadWriteLock
2426

2527
import scala.collection.mutable
2628
import scala.util.control.NonFatal
@@ -68,6 +70,8 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
6870
HiveUtils.newClientForMetadata(conf, hadoopConf)
6971
}
7072

73+
private val clientLocks = new ConcurrentHashMap[String, ReentrantReadWriteLock]()
74+
7175
// Exceptions thrown by the hive client that we would like to wrap
7276
private val clientExceptions = Set(
7377
classOf[HiveException].getCanonicalName,
@@ -94,8 +98,14 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
9498
* Run some code involving `client` in a [[synchronized]] block and wrap certain
9599
* exceptions thrown in the process in [[AnalysisException]].
96100
*/
97-
private def withClient[T](body: => T): T = synchronized {
101+
private def withClient[T](write: Boolean, db: String)(body: => T): T = {
102+
val lock = clientLocks.computeIfAbsent(db, (_: String) => new ReentrantReadWriteLock())
98103
try {
104+
if (write) {
105+
lock.writeLock().lock()
106+
} else {
107+
lock.readLock().lock()
108+
}
99109
body
100110
} catch {
101111
case NonFatal(exception) if isClientException(exception) =>
@@ -107,6 +117,12 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
107117
}
108118
throw new AnalysisException(
109119
e.getClass.getCanonicalName + ": " + e.getMessage, cause = Some(e))
120+
} finally {
121+
if (write) {
122+
lock.writeLock().unlock()
123+
} else {
124+
lock.readLock().unlock()
125+
}
110126
}
111127
}
112128

@@ -186,14 +202,14 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
186202

187203
override def createDatabase(
188204
dbDefinition: CatalogDatabase,
189-
ignoreIfExists: Boolean): Unit = withClient {
205+
ignoreIfExists: Boolean): Unit = withClient(true, dbDefinition.name) {
190206
client.createDatabase(dbDefinition, ignoreIfExists)
191207
}
192208

193209
override def dropDatabase(
194210
db: String,
195211
ignoreIfNotExists: Boolean,
196-
cascade: Boolean): Unit = withClient {
212+
cascade: Boolean): Unit = withClient(true, db) {
197213
client.dropDatabase(db, ignoreIfNotExists, cascade)
198214
}
199215

@@ -203,7 +219,8 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
203219
*
204220
* Note: As of now, this only supports altering database properties!
205221
*/
206-
override def alterDatabase(dbDefinition: CatalogDatabase): Unit = withClient {
222+
override def alterDatabase(dbDefinition: CatalogDatabase): Unit =
223+
withClient(true, dbDefinition.name) {
207224
val existingDb = getDatabase(dbDefinition.name)
208225
if (existingDb.properties == dbDefinition.properties) {
209226
logWarning(s"Request to alter database ${dbDefinition.name} is a no-op because " +
@@ -213,23 +230,23 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
213230
client.alterDatabase(dbDefinition)
214231
}
215232

216-
override def getDatabase(db: String): CatalogDatabase = withClient {
233+
override def getDatabase(db: String): CatalogDatabase = withClient(false, db) {
217234
client.getDatabase(db)
218235
}
219236

220-
override def databaseExists(db: String): Boolean = withClient {
237+
override def databaseExists(db: String): Boolean = withClient(false, db) {
221238
client.databaseExists(db)
222239
}
223240

224-
override def listDatabases(): Seq[String] = withClient {
241+
override def listDatabases(): Seq[String] = withClient(false, "") {
225242
client.listDatabases("*")
226243
}
227244

228-
override def listDatabases(pattern: String): Seq[String] = withClient {
245+
override def listDatabases(pattern: String): Seq[String] = withClient(false, "") {
229246
client.listDatabases(pattern)
230247
}
231248

232-
override def setCurrentDatabase(db: String): Unit = withClient {
249+
override def setCurrentDatabase(db: String): Unit = withClient(false, "") {
233250
client.setCurrentDatabase(db)
234251
}
235252

@@ -239,7 +256,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
239256

240257
override def createTable(
241258
tableDefinition: CatalogTable,
242-
ignoreIfExists: Boolean): Unit = withClient {
259+
ignoreIfExists: Boolean): Unit = withClient(true, tableDefinition.database) {
243260
assert(tableDefinition.identifier.database.isDefined)
244261
val db = tableDefinition.identifier.database.get
245262
val table = tableDefinition.identifier.table
@@ -511,15 +528,15 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
511528
db: String,
512529
table: String,
513530
ignoreIfNotExists: Boolean,
514-
purge: Boolean): Unit = withClient {
531+
purge: Boolean): Unit = withClient(true, db) {
515532
requireDbExists(db)
516533
client.dropTable(db, table, ignoreIfNotExists, purge)
517534
}
518535

519536
override def renameTable(
520537
db: String,
521538
oldName: String,
522-
newName: String): Unit = withClient {
539+
newName: String): Unit = withClient(true, db) {
523540
val rawTable = getRawTable(db, oldName)
524541

525542
// Note that Hive serde tables don't use path option in storage properties to store the value
@@ -567,7 +584,8 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
567584
* Note: As of now, this doesn't support altering table schema, partition column names and bucket
568585
* specification. We will ignore them even if users do specify different values for these fields.
569586
*/
570-
override def alterTable(tableDefinition: CatalogTable): Unit = withClient {
587+
override def alterTable(tableDefinition: CatalogTable): Unit =
588+
withClient(true, tableDefinition.database) {
571589
assert(tableDefinition.identifier.database.isDefined)
572590
val db = tableDefinition.identifier.database.get
573591
requireTableExists(db, tableDefinition.identifier.table)
@@ -666,7 +684,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
666684
override def alterTableDataSchema(
667685
db: String,
668686
table: String,
669-
newDataSchema: StructType): Unit = withClient {
687+
newDataSchema: StructType): Unit = withClient(true, db) {
670688
requireTableExists(db, table)
671689
val oldTable = getTable(db, table)
672690
verifyDataSchema(oldTable.identifier, oldTable.tableType, newDataSchema)
@@ -698,7 +716,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
698716
override def alterTableStats(
699717
db: String,
700718
table: String,
701-
stats: Option[CatalogStatistics]): Unit = withClient {
719+
stats: Option[CatalogStatistics]): Unit = withClient(true, db) {
702720
requireTableExists(db, table)
703721
val rawTable = getRawTable(db, table)
704722

@@ -715,11 +733,12 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
715733
client.alterTable(updatedTable)
716734
}
717735

718-
override def getTable(db: String, table: String): CatalogTable = withClient {
736+
override def getTable(db: String, table: String): CatalogTable = withClient(false, db) {
719737
restoreTableMetadata(getRawTable(db, table))
720738
}
721739

722-
override def getTablesByName(db: String, tables: Seq[String]): Seq[CatalogTable] = withClient {
740+
override def getTablesByName(db: String, tables: Seq[String]): Seq[CatalogTable] =
741+
withClient(false, db) {
723742
getRawTablesByNames(db, tables).map(restoreTableMetadata)
724743
}
725744

@@ -847,21 +866,21 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
847866
properties = table.properties.filterKeys(!HIVE_GENERATED_TABLE_PROPERTIES(_)))
848867
}
849868

850-
override def tableExists(db: String, table: String): Boolean = withClient {
869+
override def tableExists(db: String, table: String): Boolean = withClient(false, db) {
851870
client.tableExists(db, table)
852871
}
853872

854-
override def listTables(db: String): Seq[String] = withClient {
873+
override def listTables(db: String): Seq[String] = withClient(false, db) {
855874
requireDbExists(db)
856875
client.listTables(db)
857876
}
858877

859-
override def listTables(db: String, pattern: String): Seq[String] = withClient {
878+
override def listTables(db: String, pattern: String): Seq[String] = withClient(false, db) {
860879
requireDbExists(db)
861880
client.listTables(db, pattern)
862881
}
863882

864-
override def listViews(db: String, pattern: String): Seq[String] = withClient {
883+
override def listViews(db: String, pattern: String): Seq[String] = withClient(false, db) {
865884
requireDbExists(db)
866885
client.listTablesByType(db, pattern, CatalogTableType.VIEW)
867886
}
@@ -871,7 +890,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
871890
table: String,
872891
loadPath: String,
873892
isOverwrite: Boolean,
874-
isSrcLocal: Boolean): Unit = withClient {
893+
isSrcLocal: Boolean): Unit = withClient(false, db) {
875894
requireTableExists(db, table)
876895
client.loadTable(
877896
loadPath,
@@ -887,7 +906,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
887906
partition: TablePartitionSpec,
888907
isOverwrite: Boolean,
889908
inheritTableSpecs: Boolean,
890-
isSrcLocal: Boolean): Unit = withClient {
909+
isSrcLocal: Boolean): Unit = withClient(false, db) {
891910
requireTableExists(db, table)
892911

893912
val orderedPartitionSpec = new util.LinkedHashMap[String, String]()
@@ -917,7 +936,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
917936
loadPath: String,
918937
partition: TablePartitionSpec,
919938
replace: Boolean,
920-
numDP: Int): Unit = withClient {
939+
numDP: Int): Unit = withClient(false, db) {
921940
requireTableExists(db, table)
922941

923942
val orderedPartitionSpec = new util.LinkedHashMap[String, String]()
@@ -982,7 +1001,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
9821001
db: String,
9831002
table: String,
9841003
parts: Seq[CatalogTablePartition],
985-
ignoreIfExists: Boolean): Unit = withClient {
1004+
ignoreIfExists: Boolean): Unit = withClient(true, db) {
9861005
requireTableExists(db, table)
9871006

9881007
val tableMeta = getTable(db, table)
@@ -1008,7 +1027,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
10081027
parts: Seq[TablePartitionSpec],
10091028
ignoreIfNotExists: Boolean,
10101029
purge: Boolean,
1011-
retainData: Boolean): Unit = withClient {
1030+
retainData: Boolean): Unit = withClient(true, db) {
10121031
requireTableExists(db, table)
10131032
client.dropPartitions(
10141033
db, table, parts.map(lowerCasePartitionSpec), ignoreIfNotExists, purge, retainData)
@@ -1018,7 +1037,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
10181037
db: String,
10191038
table: String,
10201039
specs: Seq[TablePartitionSpec],
1021-
newSpecs: Seq[TablePartitionSpec]): Unit = withClient {
1040+
newSpecs: Seq[TablePartitionSpec]): Unit = withClient(true, db) {
10221041
client.renamePartitions(
10231042
db, table, specs.map(lowerCasePartitionSpec), newSpecs.map(lowerCasePartitionSpec))
10241043

@@ -1145,7 +1164,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
11451164
override def alterPartitions(
11461165
db: String,
11471166
table: String,
1148-
newParts: Seq[CatalogTablePartition]): Unit = withClient {
1167+
newParts: Seq[CatalogTablePartition]): Unit = withClient(true, db) {
11491168
val lowerCasedParts = newParts.map(p => p.copy(spec = lowerCasePartitionSpec(p.spec)))
11501169

11511170
val rawTable = getRawTable(db, table)
@@ -1166,7 +1185,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
11661185
override def getPartition(
11671186
db: String,
11681187
table: String,
1169-
spec: TablePartitionSpec): CatalogTablePartition = withClient {
1188+
spec: TablePartitionSpec): CatalogTablePartition = withClient(false, db) {
11701189
val part = client.getPartition(db, table, lowerCasePartitionSpec(spec))
11711190
restorePartitionMetadata(part, getTable(db, table))
11721191
}
@@ -1204,7 +1223,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
12041223
override def getPartitionOption(
12051224
db: String,
12061225
table: String,
1207-
spec: TablePartitionSpec): Option[CatalogTablePartition] = withClient {
1226+
spec: TablePartitionSpec): Option[CatalogTablePartition] = withClient(false, db) {
12081227
client.getPartitionOption(db, table, lowerCasePartitionSpec(spec)).map { part =>
12091228
restorePartitionMetadata(part, getTable(db, table))
12101229
}
@@ -1216,7 +1235,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
12161235
override def listPartitionNames(
12171236
db: String,
12181237
table: String,
1219-
partialSpec: Option[TablePartitionSpec] = None): Seq[String] = withClient {
1238+
partialSpec: Option[TablePartitionSpec] = None): Seq[String] = withClient(false, db) {
12201239
val catalogTable = getTable(db, table)
12211240
val partColNameMap = buildLowerCasePartColNameMap(catalogTable).mapValues(escapePathName)
12221241
val clientPartitionNames =
@@ -1237,7 +1256,8 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
12371256
override def listPartitions(
12381257
db: String,
12391258
table: String,
1240-
partialSpec: Option[TablePartitionSpec] = None): Seq[CatalogTablePartition] = withClient {
1259+
partialSpec: Option[TablePartitionSpec] = None): Seq[CatalogTablePartition] =
1260+
withClient(false, db) {
12411261
val partColNameMap = buildLowerCasePartColNameMap(getTable(db, table))
12421262
val res = client.getPartitions(db, table, partialSpec.map(lowerCasePartitionSpec)).map { part =>
12431263
part.copy(spec = restorePartitionSpec(part.spec, partColNameMap))
@@ -1258,7 +1278,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
12581278
db: String,
12591279
table: String,
12601280
predicates: Seq[Expression],
1261-
defaultTimeZoneId: String): Seq[CatalogTablePartition] = withClient {
1281+
defaultTimeZoneId: String): Seq[CatalogTablePartition] = withClient(false, db) {
12621282
val rawTable = getRawTable(db, table)
12631283
val catalogTable = restoreTableMetadata(rawTable)
12641284

@@ -1277,7 +1297,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
12771297

12781298
override def createFunction(
12791299
db: String,
1280-
funcDefinition: CatalogFunction): Unit = withClient {
1300+
funcDefinition: CatalogFunction): Unit = withClient(true, db) {
12811301
requireDbExists(db)
12821302
// Hive's metastore is case insensitive. However, Hive's createFunction does
12831303
// not normalize the function name (unlike the getFunction part). So,
@@ -1288,13 +1308,13 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
12881308
client.createFunction(db, funcDefinition.copy(identifier = functionIdentifier))
12891309
}
12901310

1291-
override def dropFunction(db: String, name: String): Unit = withClient {
1311+
override def dropFunction(db: String, name: String): Unit = withClient(true, db) {
12921312
requireFunctionExists(db, name)
12931313
client.dropFunction(db, name)
12941314
}
12951315

12961316
override def alterFunction(
1297-
db: String, funcDefinition: CatalogFunction): Unit = withClient {
1317+
db: String, funcDefinition: CatalogFunction): Unit = withClient(true, db) {
12981318
requireDbExists(db)
12991319
val functionName = funcDefinition.identifier.funcName.toLowerCase(Locale.ROOT)
13001320
requireFunctionExists(db, functionName)
@@ -1305,23 +1325,23 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
13051325
override def renameFunction(
13061326
db: String,
13071327
oldName: String,
1308-
newName: String): Unit = withClient {
1328+
newName: String): Unit = withClient(true, db) {
13091329
requireFunctionExists(db, oldName)
13101330
requireFunctionNotExists(db, newName)
13111331
client.renameFunction(db, oldName, newName)
13121332
}
13131333

1314-
override def getFunction(db: String, funcName: String): CatalogFunction = withClient {
1334+
override def getFunction(db: String, funcName: String): CatalogFunction = withClient(false, db) {
13151335
requireFunctionExists(db, funcName)
13161336
client.getFunction(db, funcName)
13171337
}
13181338

1319-
override def functionExists(db: String, funcName: String): Boolean = withClient {
1339+
override def functionExists(db: String, funcName: String): Boolean = withClient(false, db) {
13201340
requireDbExists(db)
13211341
client.functionExists(db, funcName)
13221342
}
13231343

1324-
override def listFunctions(db: String, pattern: String): Seq[String] = withClient {
1344+
override def listFunctions(db: String, pattern: String): Seq[String] = withClient(false, db) {
13251345
requireDbExists(db)
13261346
client.listFunctions(db, pattern)
13271347
}

0 commit comments

Comments
 (0)