Skip to content

Commit

Permalink
Rename: getX -> getXMetadata
Browse files Browse the repository at this point in the history
  • Loading branch information
Andrew Or committed Apr 6, 2016
1 parent 6e6c689 commit a3601fb
Show file tree
Hide file tree
Showing 6 changed files with 46 additions and 47 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -95,7 +95,7 @@ class SessionCatalog(
externalCatalog.alterDatabase(dbDefinition)
}

def getDatabase(db: String): CatalogDatabase = {
def getDatabaseMetadata(db: String): CatalogDatabase = {
externalCatalog.getDatabase(db)
}

Expand Down Expand Up @@ -169,7 +169,7 @@ class SessionCatalog(
* If no database is specified, assume the table is in the current database.
* If the specified table is not found in the database then an [[AnalysisException]] is thrown.
*/
def getTable(name: TableIdentifier): CatalogTable = {
def getTableMetadata(name: TableIdentifier): CatalogTable = {
val db = name.database.getOrElse(currentDb)
val table = formatTableName(name.table)
externalCatalog.getTable(db, table)
Expand Down Expand Up @@ -476,8 +476,7 @@ class SessionCatalog(
* If a database is specified in `name`, this will return the function in that database.
* If no database is specified, this will return the function in the current database.
*/
// TODO: have a better name. This method is actually for fetching the metadata of a function.
def getFunction(name: FunctionIdentifier): CatalogFunction = {
def getFunctionMetadata(name: FunctionIdentifier): CatalogFunction = {
val db = name.database.getOrElse(currentDb)
externalCatalog.getFunction(db, name.funcName)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,15 +62,15 @@ class SessionCatalogSuite extends SparkFunSuite {

test("get database when a database exists") {
val catalog = new SessionCatalog(newBasicCatalog())
val db1 = catalog.getDatabase("db1")
val db1 = catalog.getDatabaseMetadata("db1")
assert(db1.name == "db1")
assert(db1.description.contains("db1"))
}

test("get database should throw exception when the database does not exist") {
val catalog = new SessionCatalog(newBasicCatalog())
intercept[AnalysisException] {
catalog.getDatabase("db_that_does_not_exist")
catalog.getDatabaseMetadata("db_that_does_not_exist")
}
}

Expand Down Expand Up @@ -128,10 +128,10 @@ class SessionCatalogSuite extends SparkFunSuite {

test("alter database") {
val catalog = new SessionCatalog(newBasicCatalog())
val db1 = catalog.getDatabase("db1")
val db1 = catalog.getDatabaseMetadata("db1")
// Note: alter properties here because Hive does not support altering other fields
catalog.alterDatabase(db1.copy(properties = Map("k" -> "v3", "good" -> "true")))
val newDb1 = catalog.getDatabase("db1")
val newDb1 = catalog.getDatabaseMetadata("db1")
assert(db1.properties.isEmpty)
assert(newDb1.properties.size == 2)
assert(newDb1.properties.get("k") == Some("v3"))
Expand Down Expand Up @@ -346,21 +346,21 @@ class SessionCatalogSuite extends SparkFunSuite {
test("get table") {
val externalCatalog = newBasicCatalog()
val sessionCatalog = new SessionCatalog(externalCatalog)
assert(sessionCatalog.getTable(TableIdentifier("tbl1", Some("db2")))
assert(sessionCatalog.getTableMetadata(TableIdentifier("tbl1", Some("db2")))
== externalCatalog.getTable("db2", "tbl1"))
// Get table without explicitly specifying database
sessionCatalog.setCurrentDatabase("db2")
assert(sessionCatalog.getTable(TableIdentifier("tbl1"))
assert(sessionCatalog.getTableMetadata(TableIdentifier("tbl1"))
== externalCatalog.getTable("db2", "tbl1"))
}

test("get table when database/table does not exist") {
val catalog = new SessionCatalog(newBasicCatalog())
intercept[AnalysisException] {
catalog.getTable(TableIdentifier("tbl1", Some("unknown_db")))
catalog.getTableMetadata(TableIdentifier("tbl1", Some("unknown_db")))
}
intercept[AnalysisException] {
catalog.getTable(TableIdentifier("unknown_table", Some("db2")))
catalog.getTableMetadata(TableIdentifier("unknown_table", Some("db2")))
}
}

Expand All @@ -386,7 +386,7 @@ class SessionCatalogSuite extends SparkFunSuite {
test("lookup table relation with alias") {
val catalog = new SessionCatalog(newBasicCatalog())
val alias = "monster"
val tableMetadata = catalog.getTable(TableIdentifier("tbl1", Some("db2")))
val tableMetadata = catalog.getTableMetadata(TableIdentifier("tbl1", Some("db2")))
val relation = SubqueryAlias("tbl1", CatalogRelation("db2", tableMetadata))
val relationWithAlias =
SubqueryAlias(alias,
Expand Down Expand Up @@ -758,19 +758,19 @@ class SessionCatalogSuite extends SparkFunSuite {
val expected =
CatalogFunction(FunctionIdentifier("func1", Some("db2")), funcClass,
Seq.empty[(String, String)])
assert(catalog.getFunction(FunctionIdentifier("func1", Some("db2"))) == expected)
assert(catalog.getFunctionMetadata(FunctionIdentifier("func1", Some("db2"))) == expected)
// Get function without explicitly specifying database
catalog.setCurrentDatabase("db2")
assert(catalog.getFunction(FunctionIdentifier("func1")) == expected)
assert(catalog.getFunctionMetadata(FunctionIdentifier("func1")) == expected)
}

test("get function when database/function does not exist") {
val catalog = new SessionCatalog(newBasicCatalog())
intercept[AnalysisException] {
catalog.getFunction(FunctionIdentifier("func1", Some("does_not_exist")))
catalog.getFunctionMetadata(FunctionIdentifier("func1", Some("does_not_exist")))
}
intercept[AnalysisException] {
catalog.getFunction(FunctionIdentifier("does_not_exist", Some("db2")))
catalog.getFunctionMetadata(FunctionIdentifier("does_not_exist", Some("db2")))
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -407,7 +407,7 @@ case class ShowTablePropertiesCommand(
if (catalog.isTemporaryTable(table)) {
Seq.empty[Row]
} else {
val catalogTable = sqlContext.sessionState.catalog.getTable(table)
val catalogTable = sqlContext.sessionState.catalog.getTableMetadata(table)

propertyKey match {
case Some(p) =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -124,7 +124,7 @@ case class AlterDatabaseProperties(

override def run(sqlContext: SQLContext): Seq[Row] = {
val catalog = sqlContext.sessionState.catalog
val db: CatalogDatabase = catalog.getDatabase(databaseName)
val db: CatalogDatabase = catalog.getDatabaseMetadata(databaseName)
catalog.alterDatabase(db.copy(properties = db.properties ++ props))

Seq.empty[Row]
Expand All @@ -149,7 +149,7 @@ case class DescribeDatabase(
extends RunnableCommand {

override def run(sqlContext: SQLContext): Seq[Row] = {
val dbMetadata: CatalogDatabase = sqlContext.sessionState.catalog.getDatabase(databaseName)
val dbMetadata: CatalogDatabase = sqlContext.sessionState.catalog.getDatabaseMetadata(databaseName)
val result =
Row("Database Name", dbMetadata.name) ::
Row("Description", dbMetadata.description) ::
Expand Down Expand Up @@ -213,7 +213,7 @@ case class AlterTableSetProperties(

override def run(sqlContext: SQLContext): Seq[Row] = {
val catalog = sqlContext.sessionState.catalog
val table = catalog.getTable(tableName)
val table = catalog.getTableMetadata(tableName)
val newProperties = table.properties ++ properties
if (DDLUtils.isDatasourceTable(newProperties)) {
throw new AnalysisException(
Expand Down Expand Up @@ -243,7 +243,7 @@ case class AlterTableUnsetProperties(

override def run(sqlContext: SQLContext): Seq[Row] = {
val catalog = sqlContext.sessionState.catalog
val table = catalog.getTable(tableName)
val table = catalog.getTableMetadata(tableName)
if (DDLUtils.isDatasourceTable(table)) {
throw new AnalysisException(
"alter table properties is not supported for datasource tables")
Expand Down Expand Up @@ -286,7 +286,7 @@ case class AlterTableSerDeProperties(

override def run(sqlContext: SQLContext): Seq[Row] = {
val catalog = sqlContext.sessionState.catalog
val table = catalog.getTable(tableName)
val table = catalog.getTableMetadata(tableName)
// Do not support setting serde for datasource tables
if (serdeClassName.isDefined && DDLUtils.isDatasourceTable(table)) {
throw new AnalysisException(
Expand Down Expand Up @@ -376,7 +376,7 @@ case class AlterTableSetLocation(

override def run(sqlContext: SQLContext): Seq[Row] = {
val catalog = sqlContext.sessionState.catalog
val table = catalog.getTable(tableName)
val table = catalog.getTableMetadata(tableName)
partitionSpec match {
case Some(spec) =>
// Partition spec is specified, so we set the location only for this partition
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -88,7 +88,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach {
val dbNameWithoutBackTicks = cleanIdentifier(dbName)

sql(s"CREATE DATABASE $dbName")
val db1 = catalog.getDatabase(dbNameWithoutBackTicks)
val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks)
assert(db1 == CatalogDatabase(
dbNameWithoutBackTicks,
"",
Expand All @@ -110,7 +110,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach {
try {
val dbNameWithoutBackTicks = cleanIdentifier(dbName)
sql(s"CREATE DATABASE $dbName")
val db1 = catalog.getDatabase(dbNameWithoutBackTicks)
val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks)
assert(db1 == CatalogDatabase(
dbNameWithoutBackTicks,
"",
Expand Down Expand Up @@ -233,14 +233,14 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach {
val tableIdent = TableIdentifier("tab1", Some("dbx"))
createDatabase(catalog, "dbx")
createTable(catalog, tableIdent)
assert(catalog.getTable(tableIdent).properties.isEmpty)
assert(catalog.getTableMetadata(tableIdent).properties.isEmpty)
// set table properties
sql("ALTER TABLE dbx.tab1 SET TBLPROPERTIES ('andrew' = 'or14', 'kor' = 'bel')")
assert(catalog.getTable(tableIdent).properties == Map("andrew" -> "or14", "kor" -> "bel"))
assert(catalog.getTableMetadata(tableIdent).properties == Map("andrew" -> "or14", "kor" -> "bel"))
// set table properties without explicitly specifying database
catalog.setCurrentDatabase("dbx")
sql("ALTER TABLE tab1 SET TBLPROPERTIES ('kor' = 'belle', 'kar' = 'bol')")
assert(catalog.getTable(tableIdent).properties ==
assert(catalog.getTableMetadata(tableIdent).properties ==
Map("andrew" -> "or14", "kor" -> "belle", "kar" -> "bol"))
// table to alter does not exist
intercept[AnalysisException] {
Expand All @@ -262,11 +262,11 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach {
// unset table properties
sql("ALTER TABLE dbx.tab1 SET TBLPROPERTIES ('j' = 'am', 'p' = 'an', 'c' = 'lan')")
sql("ALTER TABLE dbx.tab1 UNSET TBLPROPERTIES ('j')")
assert(catalog.getTable(tableIdent).properties == Map("p" -> "an", "c" -> "lan"))
assert(catalog.getTableMetadata(tableIdent).properties == Map("p" -> "an", "c" -> "lan"))
// unset table properties without explicitly specifying database
catalog.setCurrentDatabase("dbx")
sql("ALTER TABLE tab1 UNSET TBLPROPERTIES ('p')")
assert(catalog.getTable(tableIdent).properties == Map("c" -> "lan"))
assert(catalog.getTableMetadata(tableIdent).properties == Map("c" -> "lan"))
// table to alter does not exist
intercept[AnalysisException] {
sql("ALTER TABLE does_not_exist UNSET TBLPROPERTIES ('c' = 'lan')")
Expand All @@ -278,7 +278,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach {
assert(e.getMessage.contains("xyz"))
// property to unset does not exist, but "IF EXISTS" is specified
sql("ALTER TABLE tab1 UNSET TBLPROPERTIES IF EXISTS ('c', 'xyz')")
assert(catalog.getTable(tableIdent).properties.isEmpty)
assert(catalog.getTableMetadata(tableIdent).properties.isEmpty)
// throw exception for datasource tables
convertToDatasourceTable(catalog, tableIdent)
val e1 = intercept[AnalysisException] {
Expand Down Expand Up @@ -393,7 +393,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach {
private def convertToDatasourceTable(
catalog: SessionCatalog,
tableIdent: TableIdentifier): Unit = {
catalog.alterTable(catalog.getTable(tableIdent).copy(
catalog.alterTable(catalog.getTableMetadata(tableIdent).copy(
properties = Map("spark.sql.sources.provider" -> "csv")))
}

Expand All @@ -407,15 +407,15 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach {
if (isDatasourceTable) {
convertToDatasourceTable(catalog, tableIdent)
}
assert(catalog.getTable(tableIdent).storage.locationUri.isEmpty)
assert(catalog.getTable(tableIdent).storage.serdeProperties.isEmpty)
assert(catalog.getTableMetadata(tableIdent).storage.locationUri.isEmpty)
assert(catalog.getTableMetadata(tableIdent).storage.serdeProperties.isEmpty)
assert(catalog.getPartition(tableIdent, partSpec).storage.locationUri.isEmpty)
assert(catalog.getPartition(tableIdent, partSpec).storage.serdeProperties.isEmpty)
// Verify that the location is set to the expected string
def verifyLocation(expected: String, spec: Option[TablePartitionSpec] = None): Unit = {
val storageFormat = spec
.map { s => catalog.getPartition(tableIdent, s).storage }
.getOrElse { catalog.getTable(tableIdent).storage }
.getOrElse { catalog.getTableMetadata(tableIdent).storage }
if (isDatasourceTable) {
if (spec.isDefined) {
assert(storageFormat.serdeProperties.isEmpty)
Expand Down Expand Up @@ -467,8 +467,8 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach {
if (isDatasourceTable) {
convertToDatasourceTable(catalog, tableIdent)
}
assert(catalog.getTable(tableIdent).storage.serde.isEmpty)
assert(catalog.getTable(tableIdent).storage.serdeProperties.isEmpty)
assert(catalog.getTableMetadata(tableIdent).storage.serde.isEmpty)
assert(catalog.getTableMetadata(tableIdent).storage.serdeProperties.isEmpty)
// set table serde and/or properties (should fail on datasource tables)
if (isDatasourceTable) {
val e1 = intercept[AnalysisException] {
Expand All @@ -482,22 +482,22 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach {
assert(e2.getMessage.contains("datasource"))
} else {
sql("ALTER TABLE dbx.tab1 SET SERDE 'org.apache.jadoop'")
assert(catalog.getTable(tableIdent).storage.serde == Some("org.apache.jadoop"))
assert(catalog.getTable(tableIdent).storage.serdeProperties.isEmpty)
assert(catalog.getTableMetadata(tableIdent).storage.serde == Some("org.apache.jadoop"))
assert(catalog.getTableMetadata(tableIdent).storage.serdeProperties.isEmpty)
sql("ALTER TABLE dbx.tab1 SET SERDE 'org.apache.madoop' " +
"WITH SERDEPROPERTIES ('k' = 'v', 'kay' = 'vee')")
assert(catalog.getTable(tableIdent).storage.serde == Some("org.apache.madoop"))
assert(catalog.getTable(tableIdent).storage.serdeProperties ==
assert(catalog.getTableMetadata(tableIdent).storage.serde == Some("org.apache.madoop"))
assert(catalog.getTableMetadata(tableIdent).storage.serdeProperties ==
Map("k" -> "v", "kay" -> "vee"))
}
// set serde properties only
sql("ALTER TABLE dbx.tab1 SET SERDEPROPERTIES ('k' = 'vvv', 'kay' = 'vee')")
assert(catalog.getTable(tableIdent).storage.serdeProperties ==
assert(catalog.getTableMetadata(tableIdent).storage.serdeProperties ==
Map("k" -> "vvv", "kay" -> "vee"))
// set things without explicitly specifying database
catalog.setCurrentDatabase("dbx")
sql("ALTER TABLE tab1 SET SERDEPROPERTIES ('kay' = 'veee')")
assert(catalog.getTable(tableIdent).storage.serdeProperties ==
assert(catalog.getTableMetadata(tableIdent).storage.serdeProperties ==
Map("k" -> "vvv", "kay" -> "veee"))
// table to alter does not exist
intercept[AnalysisException] {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -83,7 +83,7 @@ class DataSourceWithHiveMetastoreCatalogSuite
.saveAsTable("t")
}

val hiveTable = sessionState.catalog.getTable(TableIdentifier("t", Some("default")))
val hiveTable = sessionState.catalog.getTableMetadata(TableIdentifier("t", Some("default")))
assert(hiveTable.storage.inputFormat === Some(inputFormat))
assert(hiveTable.storage.outputFormat === Some(outputFormat))
assert(hiveTable.storage.serde === Some(serde))
Expand Down Expand Up @@ -114,7 +114,7 @@ class DataSourceWithHiveMetastoreCatalogSuite
.saveAsTable("t")
}

val hiveTable = sessionState.catalog.getTable(TableIdentifier("t", Some("default")))
val hiveTable = sessionState.catalog.getTableMetadata(TableIdentifier("t", Some("default")))
assert(hiveTable.storage.inputFormat === Some(inputFormat))
assert(hiveTable.storage.outputFormat === Some(outputFormat))
assert(hiveTable.storage.serde === Some(serde))
Expand Down Expand Up @@ -144,7 +144,7 @@ class DataSourceWithHiveMetastoreCatalogSuite
|AS SELECT 1 AS d1, "val_1" AS d2
""".stripMargin)

val hiveTable = sessionState.catalog.getTable(TableIdentifier("t", Some("default")))
val hiveTable = sessionState.catalog.getTableMetadata(TableIdentifier("t", Some("default")))
assert(hiveTable.storage.inputFormat === Some(inputFormat))
assert(hiveTable.storage.outputFormat === Some(outputFormat))
assert(hiveTable.storage.serde === Some(serde))
Expand Down

0 comments on commit a3601fb

Please sign in to comment.