Skip to content

Commit

Permalink
[SPARK-6618][SQL] HiveMetastoreCatalog.lookupRelation should use fine…
Browse files Browse the repository at this point in the history
…-grained lock

JIRA: https://issues.apache.org/jira/browse/SPARK-6618

Author: Yin Huai <[email protected]>

Closes apache#5281 from yhuai/lookupRelationLock and squashes the following commits:

591b4be [Yin Huai] A test?
b3a9625 [Yin Huai] Just protect client.
  • Loading branch information
yhuai authored and liancheng committed Mar 31, 2015
1 parent b80a030 commit 314afd0
Show file tree
Hide file tree
Showing 2 changed files with 20 additions and 3 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -172,12 +172,16 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with

def lookupRelation(
tableIdentifier: Seq[String],
alias: Option[String]): LogicalPlan = synchronized {
alias: Option[String]): LogicalPlan = {
val tableIdent = processTableIdentifier(tableIdentifier)
val databaseName = tableIdent.lift(tableIdent.size - 2).getOrElse(
hive.sessionState.getCurrentDatabase)
val tblName = tableIdent.last
val table = try client.getTable(databaseName, tblName) catch {
val table = try {
synchronized {
client.getTable(databaseName, tblName)
}
} catch {
case te: org.apache.hadoop.hive.ql.metadata.InvalidTableException =>
throw new NoSuchTableException
}
Expand All @@ -199,7 +203,9 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with
} else {
val partitions: Seq[Partition] =
if (table.isPartitioned) {
HiveShim.getAllPartitionsOf(client, table).toSeq
synchronized {
HiveShim.getAllPartitionsOf(client, table).toSeq
}
} else {
Nil
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -433,4 +433,15 @@ class SQLQuerySuite extends QueryTest {
dropTempTable("data")
setConf("spark.sql.hive.convertCTAS", originalConf)
}

test("sanity test for SPARK-6618") {
(1 to 100).par.map { i =>
val tableName = s"SPARK_6618_table_$i"
sql(s"CREATE TABLE $tableName (col1 string)")
catalog.lookupRelation(Seq(tableName))
table(tableName)
tables()
sql(s"DROP TABLE $tableName")
}
}
}

0 comments on commit 314afd0

Please sign in to comment.