diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 783bc5b562ff..12c9a972c1af 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -573,9 +573,8 @@ private[hive] class HiveClientImpl( // If users explicitly alter these Hive-specific properties through ALTER TABLE DDL, we respect // these user-specified values. verifyColumnDataType(table.dataSchema) - val owner = Option(table.owner).filter(_.nonEmpty).getOrElse(userName) val hiveTable = toHiveTable( - table.copy(properties = table.ignoredProperties ++ table.properties), Some(owner)) + table.copy(properties = table.ignoredProperties ++ table.properties), Some(userName)) // Do not use `table.qualifiedName` here because this may be a rename val qualifiedTableName = s"$dbName.$tableName" shim.alterTable(client, qualifiedTableName, hiveTable) @@ -1039,7 +1038,7 @@ private[hive] object HiveClientImpl { } hiveTable.setFields(schema.asJava) hiveTable.setPartCols(partCols.asJava) - userName.foreach(hiveTable.setOwner) + Option(table.owner).filter(_.nonEmpty).orElse(userName).foreach(hiveTable.setOwner) hiveTable.setCreateTime(MILLISECONDS.toSeconds(table.createTime).toInt) hiveTable.setLastAccessTime(MILLISECONDS.toSeconds(table.lastAccessTime).toInt) table.storage.locationUri.map(CatalogUtils.URIToString).foreach { loc => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala index 0a522b6a11c8..46623000405c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala @@ -113,4 +113,19 @@ class HiveExternalCatalogSuite extends ExternalCatalogSuite { catalog.createDatabase(newDb("dbWithNullDesc").copy(description = null), ignoreIfExists = false) assert(catalog.getDatabase("dbWithNullDesc").description == "") } + + test("SPARK-29498 CatalogTable to HiveTable should not change the table's ownership") { + val catalog = newBasicCatalog() + val owner = "SPARK-29498" + val hiveTable = CatalogTable( + identifier = TableIdentifier("spark_29498", Some("db1")), + tableType = CatalogTableType.MANAGED, + storage = storageFormat, + owner = owner, + schema = new StructType().add("i", "int"), + provider = Some("hive")) + + catalog.createTable(hiveTable, ignoreIfExists = false) + assert(catalog.getTable("db1", "spark_29498").owner === owner) + } }