File tree Expand file tree Collapse file tree 3 files changed +44
-5
lines changed
catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog
hive/src/test/scala/org/apache/spark/sql/hive Expand file tree Collapse file tree 3 files changed +44
-5
lines changed Original file line number Diff line number Diff line change @@ -254,7 +254,18 @@ class SessionCatalog(
254254 val db = formatDatabaseName(tableDefinition.identifier.database.getOrElse(getCurrentDatabase))
255255 val table = formatTableName(tableDefinition.identifier.table)
256256 validateName(table)
257- val newTableDefinition = tableDefinition.copy(identifier = TableIdentifier (table, Some (db)))
257+
258+ val newTableDefinition = if (tableDefinition.storage.locationUri.isDefined) {
259+ // make the location of the table qualified.
260+ val qualifiedTableLocation =
261+ makeQualifiedPath(tableDefinition.storage.locationUri.get).toString
262+ tableDefinition.copy(
263+ storage = tableDefinition.storage.copy(locationUri = Some (qualifiedTableLocation.toString)),
264+ identifier = TableIdentifier (table, Some (db)))
265+ } else {
266+ tableDefinition.copy(identifier = TableIdentifier (table, Some (db)))
267+ }
268+
258269 requireDbExists(db)
259270 externalCatalog.createTable(newTableDefinition, ignoreIfExists)
260271 }
Original file line number Diff line number Diff line change @@ -651,20 +651,19 @@ class VersionsSuite extends QueryTest with SQLTestUtils with TestHiveSingleton w
651651 import spark .implicits ._
652652
653653 val tPath = new Path (spark.sessionState.conf.warehousePath, " t" )
654+ val fs = tPath.getFileSystem(spark.sessionState.newHadoopConf())
654655 Seq (" 1" ).toDF(" a" ).write.saveAsTable(" t" )
655- val expectedPath = s " file: ${tPath.toUri.getPath.stripSuffix(" /" )}"
656656 val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier (" t" ))
657657
658- assert(table.location.stripSuffix( " / " ) == expectedPath )
658+ assert(new Path ( table.location) == fs.makeQualified(tPath) )
659659 assert(tPath.getFileSystem(spark.sessionState.newHadoopConf()).exists(tPath))
660660 checkAnswer(spark.table(" t" ), Row (" 1" ) :: Nil )
661661
662662 val t1Path = new Path (spark.sessionState.conf.warehousePath, " t1" )
663663 spark.sql(" create table t1 using parquet as select 2 as a" )
664664 val table1 = spark.sessionState.catalog.getTableMetadata(TableIdentifier (" t1" ))
665- val expectedPath1 = s " file: ${t1Path.toUri.getPath.stripSuffix(" /" )}"
666665
667- assert(table1.location.stripSuffix( " / " ) == expectedPath1 )
666+ assert(new Path ( table1.location) == fs.makeQualified(t1Path) )
668667 assert(t1Path.getFileSystem(spark.sessionState.newHadoopConf()).exists(t1Path))
669668 checkAnswer(spark.table(" t1" ), Row (2 ) :: Nil )
670669 }
Original file line number Diff line number Diff line change @@ -1587,4 +1587,33 @@ class HiveDDLSuite
15871587 }
15881588 }
15891589 }
1590+
1591+ test(" the qualified path of a datasource table is stored in the catalog" ) {
1592+ withTable(" t" , " t1" ) {
1593+ withTempDir { dir =>
1594+ assert(! dir.getAbsolutePath.startsWith(" file:/" ))
1595+ spark.sql(
1596+ s """
1597+ |CREATE TABLE t(a string)
1598+ |USING parquet
1599+ |LOCATION ' $dir'
1600+ """ .stripMargin)
1601+ val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier (" t" ))
1602+ assert(table.location.startsWith(" file:/" ))
1603+ }
1604+
1605+ withTempDir { dir =>
1606+ assert(! dir.getAbsolutePath.startsWith(" file:/" ))
1607+ spark.sql(
1608+ s """
1609+ |CREATE TABLE t1(a string, b string)
1610+ |USING parquet
1611+ |PARTITIONED BY(b)
1612+ |LOCATION ' $dir'
1613+ """ .stripMargin)
1614+ val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier (" t1" ))
1615+ assert(table.location.startsWith(" file:/" ))
1616+ }
1617+ }
1618+ }
15901619}
You can’t perform that action at this time.
0 commit comments