-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-19257][SQL]location for table/partition/database should be java.net.URI #17149
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 5 commits
38436e8
69a1646
890327a
e792cb6
f2b9bd8
109e2b5
dc0a37b
b6bc466
abfb6f5
80f2c40
681db88
5b92620
5b423f5
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,6 +17,8 @@ | |
|
|
||
| package org.apache.spark.sql.catalyst.catalog | ||
|
|
||
| import java.net.URI | ||
|
|
||
| import org.apache.hadoop.conf.Configuration | ||
| import org.apache.hadoop.fs.Path | ||
| import org.scalatest.BeforeAndAfterEach | ||
|
|
@@ -340,8 +342,8 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac | |
| "db1", | ||
| "tbl", | ||
| Map("partCol1" -> "1", "partCol2" -> "2")).location | ||
| val tableLocation = catalog.getTable("db1", "tbl").location | ||
| val defaultPartitionLocation = new Path(new Path(tableLocation, "partCol1=1"), "partCol2=2") | ||
| val tableLocationPath = new Path(catalog.getTable("db1", "tbl").location) | ||
|
||
| val defaultPartitionLocation = new Path(new Path(tableLocationPath, "partCol1=1"), "partCol2=2") | ||
| assert(new Path(partitionLocation) == defaultPartitionLocation) | ||
| } | ||
|
|
||
|
|
@@ -365,10 +367,10 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac | |
|
|
||
| val partition1 = | ||
| CatalogTablePartition(Map("partCol1" -> "1", "partCol2" -> "2"), | ||
| storageFormat.copy(locationUri = Some(newLocationPart1))) | ||
| storageFormat.copy(locationUri = Some(new Path(newLocationPart1).toUri))) | ||
|
||
| val partition2 = | ||
| CatalogTablePartition(Map("partCol1" -> "3", "partCol2" -> "4"), | ||
| storageFormat.copy(locationUri = Some(newLocationPart2))) | ||
| storageFormat.copy(locationUri = Some(new Path(newLocationPart2).toUri))) | ||
| catalog.createPartitions("db1", "tbl", Seq(partition1), ignoreIfExists = false) | ||
| catalog.createPartitions("db1", "tbl", Seq(partition2), ignoreIfExists = false) | ||
|
|
||
|
|
@@ -508,7 +510,7 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac | |
| partitionColumnNames = Seq("partCol1", "partCol2")) | ||
| catalog.createTable(table, ignoreIfExists = false) | ||
|
|
||
| val tableLocation = catalog.getTable("db1", "tbl").location | ||
| val tableLocationPath = new Path(catalog.getTable("db1", "tbl").location) | ||
|
||
|
|
||
| val mixedCasePart1 = CatalogTablePartition( | ||
| Map("partCol1" -> "1", "partCol2" -> "2"), storageFormat) | ||
|
|
@@ -518,12 +520,12 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac | |
| catalog.createPartitions("db1", "tbl", Seq(mixedCasePart1), ignoreIfExists = false) | ||
| assert( | ||
| new Path(catalog.getPartition("db1", "tbl", mixedCasePart1.spec).location) == | ||
| new Path(new Path(tableLocation, "partCol1=1"), "partCol2=2")) | ||
| new Path(new Path(tableLocationPath, "partCol1=1"), "partCol2=2")) | ||
|
|
||
| catalog.renamePartitions("db1", "tbl", Seq(mixedCasePart1.spec), Seq(mixedCasePart2.spec)) | ||
| assert( | ||
| new Path(catalog.getPartition("db1", "tbl", mixedCasePart2.spec).location) == | ||
| new Path(new Path(tableLocation, "partCol1=3"), "partCol2=4")) | ||
| new Path(new Path(tableLocationPath, "partCol1=3"), "partCol2=4")) | ||
|
|
||
| // For external tables, RENAME PARTITION should not update the partition location. | ||
| val existingPartLoc = catalog.getPartition("db2", "tbl2", part1.spec).location | ||
|
|
@@ -553,21 +555,21 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac | |
| test("alter partitions") { | ||
| val catalog = newBasicCatalog() | ||
| try { | ||
| val newLocation = newUriForDatabase() | ||
| val newLocationUri = new Path(newUriForDatabase()).toUri | ||
|
||
| val newSerde = "com.sparkbricks.text.EasySerde" | ||
| val newSerdeProps = Map("spark" -> "bricks", "compressed" -> "false") | ||
| // alter but keep spec the same | ||
| val oldPart1 = catalog.getPartition("db2", "tbl2", part1.spec) | ||
| val oldPart2 = catalog.getPartition("db2", "tbl2", part2.spec) | ||
| catalog.alterPartitions("db2", "tbl2", Seq( | ||
| oldPart1.copy(storage = storageFormat.copy(locationUri = Some(newLocation))), | ||
| oldPart2.copy(storage = storageFormat.copy(locationUri = Some(newLocation))))) | ||
| oldPart1.copy(storage = storageFormat.copy(locationUri = Some(newLocationUri))), | ||
| oldPart2.copy(storage = storageFormat.copy(locationUri = Some(newLocationUri))))) | ||
| val newPart1 = catalog.getPartition("db2", "tbl2", part1.spec) | ||
| val newPart2 = catalog.getPartition("db2", "tbl2", part2.spec) | ||
| assert(newPart1.storage.locationUri == Some(newLocation)) | ||
| assert(newPart2.storage.locationUri == Some(newLocation)) | ||
| assert(oldPart1.storage.locationUri != Some(newLocation)) | ||
| assert(oldPart2.storage.locationUri != Some(newLocation)) | ||
| assert(newPart1.storage.locationUri == Some(newLocationUri)) | ||
| assert(newPart2.storage.locationUri == Some(newLocationUri)) | ||
| assert(oldPart1.storage.locationUri != Some(newLocationUri)) | ||
| assert(oldPart2.storage.locationUri != Some(newLocationUri)) | ||
| // alter other storage information | ||
| catalog.alterPartitions("db2", "tbl2", Seq( | ||
| oldPart1.copy(storage = storageFormat.copy(serde = Some(newSerde))), | ||
|
|
@@ -699,7 +701,7 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac | |
| // File System operations | ||
| // -------------------------------------------------------------------------- | ||
|
|
||
| private def exists(uri: String, children: String*): Boolean = { | ||
| private def exists(uri: URI, children: String*): Boolean = { | ||
| val base = new Path(uri) | ||
| val finalPath = children.foldLeft(base) { | ||
| case (parent, child) => new Path(parent, child) | ||
|
|
@@ -742,7 +744,7 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac | |
| identifier = TableIdentifier("external_table", Some("db1")), | ||
| tableType = CatalogTableType.EXTERNAL, | ||
| storage = CatalogStorageFormat( | ||
| Some(Utils.createTempDir().getAbsolutePath), | ||
| Some(Utils.createTempDir().toURI), | ||
| None, None, None, false, Map.empty), | ||
| schema = new StructType().add("a", "int").add("b", "string"), | ||
| provider = Some(defaultProvider) | ||
|
|
@@ -790,7 +792,7 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac | |
| val partWithExistingDir = CatalogTablePartition( | ||
| Map("partCol1" -> "7", "partCol2" -> "8"), | ||
| CatalogStorageFormat( | ||
| Some(tempPath.toURI.toString), | ||
| Some(tempPath.toURI), | ||
| None, None, None, false, Map.empty)) | ||
| catalog.createPartitions("db1", "tbl", Seq(partWithExistingDir), ignoreIfExists = false) | ||
|
|
||
|
|
@@ -799,7 +801,7 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac | |
| val partWithNonExistingDir = CatalogTablePartition( | ||
| Map("partCol1" -> "9", "partCol2" -> "10"), | ||
| CatalogStorageFormat( | ||
| Some(tempPath.toURI.toString), | ||
| Some(tempPath.toURI), | ||
| None, None, None, false, Map.empty)) | ||
| catalog.createPartitions("db1", "tbl", Seq(partWithNonExistingDir), ignoreIfExists = false) | ||
| assert(tempPath.exists()) | ||
|
|
@@ -883,7 +885,7 @@ abstract class CatalogTestUtils { | |
|
|
||
| def newFunc(): CatalogFunction = newFunc("funcName") | ||
|
|
||
| def newUriForDatabase(): String = Utils.createTempDir().toURI.toString.stripSuffix("/") | ||
| def newUriForDatabase(): URI = new URI(Utils.createTempDir().toURI.toString.stripSuffix("/")) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. we can simply it and write
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @cloud-fan after I try to use Path to Compare , I think stripSuffix here is the simple way. Path with a Path with a So just the Path with But I think it is more complicate, here we normalize the URI with stripSuffix from the Orignal then compare two URI directly, it is more simple. should we must to convert it to Path to compare?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think it's ok if we always compare URI with Path, instead of converting it to string. |
||
|
|
||
| def newDb(name: String): CatalogDatabase = { | ||
| CatalogDatabase(name, name + " description", newUriForDatabase(), Map.empty) | ||
|
|
@@ -895,7 +897,7 @@ abstract class CatalogTestUtils { | |
| CatalogTable( | ||
| identifier = TableIdentifier(name, database), | ||
| tableType = CatalogTableType.EXTERNAL, | ||
| storage = storageFormat.copy(locationUri = Some(Utils.createTempDir().getAbsolutePath)), | ||
| storage = storageFormat.copy(locationUri = Some(Utils.createTempDir().toURI)), | ||
| schema = new StructType() | ||
| .add("col1", "int") | ||
| .add("col2", "string") | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,6 +17,7 @@ | |
|
|
||
| package org.apache.spark.sql.catalog | ||
|
|
||
| import java.net.URI | ||
|
||
| import javax.annotation.Nullable | ||
|
|
||
| import org.apache.spark.annotation.InterfaceStability | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -21,6 +21,7 @@ import scala.collection.JavaConverters._ | |
|
|
||
| import org.antlr.v4.runtime.{ParserRuleContext, Token} | ||
| import org.antlr.v4.runtime.tree.TerminalNode | ||
| import org.apache.hadoop.fs.Path | ||
|
||
|
|
||
| import org.apache.spark.sql.SaveMode | ||
| import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} | ||
|
|
@@ -397,7 +398,8 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { | |
| val tableDesc = CatalogTable( | ||
| identifier = table, | ||
| tableType = tableType, | ||
| storage = storage.copy(locationUri = customLocation), | ||
| storage = storage.copy(locationUri = customLocation.map{ loc => | ||
|
||
| new Path(loc.toString).toUri}), | ||
| schema = schema.getOrElse(new StructType), | ||
| provider = Some(provider), | ||
| partitionColumnNames = partitionColumnNames, | ||
|
|
@@ -1080,8 +1082,10 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { | |
| if (external && location.isEmpty) { | ||
| operationNotAllowed("CREATE EXTERNAL TABLE must be accompanied by LOCATION", ctx) | ||
| } | ||
|
|
||
| val locUri = location.map{ loc => new Path(loc).toUri } | ||
| val storage = CatalogStorageFormat( | ||
| locationUri = location, | ||
| locationUri = locUri, | ||
| inputFormat = fileStorage.inputFormat.orElse(defaultStorage.inputFormat), | ||
| outputFormat = fileStorage.outputFormat.orElse(defaultStorage.outputFormat), | ||
| serde = rowStorage.serde.orElse(fileStorage.serde).orElse(defaultStorage.serde), | ||
|
|
@@ -1132,7 +1136,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { | |
| // At here, both rowStorage.serdeProperties and fileStorage.serdeProperties | ||
| // are empty Maps. | ||
| val newTableDesc = tableDesc.copy( | ||
| storage = CatalogStorageFormat.empty.copy(locationUri = location), | ||
| storage = CatalogStorageFormat.empty.copy(locationUri = locUri), | ||
| provider = Some(conf.defaultDataSourceName)) | ||
| CreateTable(newTableDesc, mode, Some(q)) | ||
| } else { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,6 +17,10 @@ | |
|
|
||
| package org.apache.spark.sql.execution.command | ||
|
|
||
| import java.net.URI | ||
|
|
||
| import org.apache.hadoop.fs.Path | ||
|
|
||
| import org.apache.spark.sql._ | ||
| import org.apache.spark.sql.catalyst.catalog._ | ||
| import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan | ||
|
|
@@ -54,7 +58,7 @@ case class CreateDataSourceTableCommand(table: CatalogTable, ignoreIfExists: Boo | |
|
|
||
| // Create the relation to validate the arguments before writing the metadata to the metastore, | ||
| // and infer the table schema and partition if users didn't specify schema in CREATE TABLE. | ||
| val pathOption = table.storage.locationUri.map("path" -> _) | ||
| val pathOption = table.storage.locationUri.map("path" -> new Path(_).toString) | ||
| // Fill in some default table options from the session conf | ||
| val tableWithDefaultOptions = table.copy( | ||
| identifier = table.identifier.copy( | ||
|
|
@@ -146,7 +150,7 @@ case class CreateDataSourceTableAsSelectCommand( | |
| assert(table.schema.isEmpty) | ||
|
|
||
| val tableLocation = if (table.tableType == CatalogTableType.MANAGED) { | ||
| Some(sessionState.catalog.defaultTablePath(table.identifier)) | ||
| Some(new Path(sessionState.catalog.defaultTablePath(table.identifier)).toUri) | ||
|
||
| } else { | ||
| table.storage.locationUri | ||
| } | ||
|
|
@@ -175,12 +179,12 @@ case class CreateDataSourceTableAsSelectCommand( | |
| private def saveDataIntoTable( | ||
| session: SparkSession, | ||
| table: CatalogTable, | ||
| tableLocation: Option[String], | ||
| tableLocation: Option[URI], | ||
| data: LogicalPlan, | ||
| mode: SaveMode, | ||
| tableExists: Boolean): BaseRelation = { | ||
| // Create the relation based on the input logical plan: `data`. | ||
| val pathOption = tableLocation.map("path" -> _) | ||
| val pathOption = tableLocation.map("path" -> new Path(_).toString) | ||
| val dataSource = DataSource( | ||
| session, | ||
| className = table.provider.get, | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
makeQualifiedPathshould acceptURInow