Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
31 commits
Select commit Hold shift + click to select a range
aebdfc6
[SPARK-19667][SQL]create table with hiveenabled in default database u…
windpiger Feb 20, 2017
825c0ad
rename a conf name
windpiger Feb 20, 2017
a2c9168
fix test faile
windpiger Feb 21, 2017
bacd528
process default database location when create/get database from metas…
windpiger Feb 22, 2017
3f6e061
remove an redundant line
windpiger Feb 22, 2017
96dcc7d
fix empty string location of database
windpiger Feb 22, 2017
f329387
modify the test case
windpiger Feb 22, 2017
83dba73
Merge branch 'master' into defaultDBPathInHive
windpiger Feb 22, 2017
58a0020
fix test failed
windpiger Feb 22, 2017
1dce2d7
add log to find out why jenkins failed
windpiger Feb 22, 2017
12f81d3
add scalastyle:off for println
windpiger Feb 22, 2017
56e83d5
fix test faile
windpiger Feb 22, 2017
901bb1c
make warehouse path qualified for default database
windpiger Feb 23, 2017
99d9746
remove a string s
windpiger Feb 23, 2017
db555e3
modify a comment
windpiger Feb 23, 2017
d327994
fix test failed
windpiger Feb 23, 2017
73c8802
move to sessioncatalog
windpiger Feb 23, 2017
747b31a
remove import
windpiger Feb 23, 2017
8f8063f
remove an import
windpiger Feb 23, 2017
4dc11c1
modify some codestyle and some comment
windpiger Feb 24, 2017
9c0773b
Merge branch 'defaultDBPathInHive' of github.com:windpiger/spark into…
windpiger Feb 24, 2017
80b8133
mv defaultdb path logic to ExternalCatalog
windpiger Feb 27, 2017
41ea115
modify a comment
windpiger Feb 27, 2017
13245e4
modify a comment
windpiger Feb 27, 2017
096ae63
add final def
windpiger Mar 1, 2017
badd61b
modify some code
windpiger Mar 2, 2017
35d2b59
add lazy flag
windpiger Mar 2, 2017
e3a467e
modify test case
windpiger Mar 3, 2017
ae9938a
modify test case
windpiger Mar 3, 2017
7739ccd
mv getdatabase
windpiger Mar 3, 2017
f93f5d3
merge with master
windpiger Mar 8, 2017
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
modify the test case
  • Loading branch information
windpiger committed Feb 22, 2017
commit f329387a6e5c083abd19314f00648f8a855d6a70
Original file line number Diff line number Diff line change
Expand Up @@ -672,14 +672,6 @@ object SQLConf {
.stringConf
.createWithDefault(TimeZone.getDefault().getID())

// for test
val TEST_HIVE_CREATETABLE_DEFAULTDB_USEWAREHOUSE_PATH =
buildConf("spark.hive.test.createTable.defaultDB.location.useWarehousePath")
.doc("Enables test case to use warehouse path instead of db location when " +
"create table in default database.")
.booleanConf
.createWithDefault(false)

object Deprecated {
val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks"
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,6 @@ import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParseException}
import org.apache.spark.sql.execution.QueryExecutionException
import org.apache.spark.sql.execution.command.DDLUtils
import org.apache.spark.sql.hive.client.HiveClientImpl._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.StaticSQLConf._
import org.apache.spark.sql.types._
import org.apache.spark.util.{CircularBuffer, Utils}
Expand Down Expand Up @@ -342,9 +341,7 @@ private[hive] class HiveClientImpl(
override def getDatabase(dbName: String): CatalogDatabase = withHiveState {
Option(client.getDatabase(dbName)).map { d =>
// default database's location always use the warehouse path
// TEST_HIVE_CREATETABLE_DEFAULTDB_USEWAREHOUSE_PATH is a flag fro test
val dbLocation = if (dbName == SessionCatalog.DEFAULT_DATABASE
|| sparkConf.get(SQLConf.TEST_HIVE_CREATETABLE_DEFAULTDB_USEWAREHOUSE_PATH)) {
val dbLocation = if (dbName == SessionCatalog.DEFAULT_DATABASE) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

is it more logical to put this logic in SessionCatalog?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sorry, I don't get the point, if this logic is exactly what we expected, we'd better to replace it at the beginning?

sparkConf.get(WAREHOUSE_PATH)
} else d.getLocationUri

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -334,6 +334,35 @@ class HiveSparkSubmitSuite
runSparkSubmit(argsForShowTables)
}

test("SPARK-19667: create table in default database with HiveEnabled use warehouse path " +
"instead of the location of default database") {
val unusedJar = TestUtils.createJarWithClasses(Seq.empty)
val warehousePath1 = Utils.createTempDir("wh1")
val argsForCreateTable = Seq(
"--class", SPARK_19667_CREATE_TABLE.getClass.getName.stripSuffix("$"),
"--name", "SPARK-19667",
"--master", "local-cluster[2,1,1024]",
"--conf", "spark.ui.enabled=false",
"--conf", "spark.master.rest.enabled=false",
"--conf", s"spark.sql.warehouse.dir=$warehousePath1",
unusedJar.toString)
runSparkSubmit(argsForCreateTable)

val warehousePath2 = Utils.createTempDir("wh2")
val argsForShowTables = Seq(
"--class", SPARK_19667_VERIFY_TABLE_PATH.getClass.getName.stripSuffix("$"),
"--name", "SPARK-19667",
"--master", "local-cluster[2,1,1024]",
"--conf", "spark.ui.enabled=false",
"--conf", "spark.master.rest.enabled=false",
"--conf", s"spark.sql.warehouse.dir=$warehousePath2",
unusedJar.toString)
runSparkSubmit(argsForShowTables)

Utils.deleteRecursively(warehousePath1)
Utils.deleteRecursively(warehousePath2)
}

// NOTE: This is an expensive operation in terms of time (10 seconds+). Use sparingly.
// This is copied from org.apache.spark.deploy.SparkSubmitSuite
private def runSparkSubmit(args: Seq[String]): Unit = {
Expand Down Expand Up @@ -905,3 +934,91 @@ object SPARK_18989_DESC_TABLE {
}
}
}

object SPARK_19667_CREATE_TABLE {
def main(args: Array[String]): Unit = {
val spark = SparkSession.builder().enableHiveSupport().getOrCreate()
try {
val warehousePath = spark.sharedState.warehousePath.stripSuffix("/")
val defaultDB = spark.sessionState.catalog.getDatabaseMetadata("default")
// default database use warehouse path as its location
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

default -> The default
use warehouse path -> uses the warehouse path

assert(defaultDB.locationUri.stripSuffix("/") == warehousePath)
spark.sql("CREATE TABLE t(a string)")

val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t"))
// table in default database use the location of default database which is also warehouse path
assert(table.location.stripSuffix("/") == s"file:$warehousePath/t")
spark.sql("INSERT INTO TABLE t SELECT 1")
assert(spark.sql("SELECT * FROM t").count == 1)

spark.sql("CREATE DATABASE not_default")
spark.sql("USE not_default")
spark.sql("CREATE TABLE t1(b string)")
val table1 = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t1"))
// table in not default database use the location of its own database
assert(table1.location.stripSuffix("/") == s"file:$warehousePath/not_default.db/t1")
} finally {
spark.sql("USE default")
}
}
}

object SPARK_19667_VERIFY_TABLE_PATH {
def main(args: Array[String]): Unit = {
val spark = SparkSession.builder().enableHiveSupport().getOrCreate()
try {
val warehousePath = spark.sharedState.warehousePath.stripSuffix("/")
val defaultDB = spark.sessionState.catalog.getDatabaseMetadata("default")
// default database use warehouse path as its location
assert(defaultDB.locationUri.stripSuffix("/") == warehousePath)

val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t"))
// the table in default database created in job(SPARK_19667_CREATE_TABLE) above,
// which has different warehouse path from this job, its location still equals to
// the location when it's created.
Copy link
Member

@gatorsmile gatorsmile Mar 12, 2017

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

How about?

For the table created by another job in the default database, the location of this table is not changed, even if the current job has a different warehouse path.

assert(table.location.stripSuffix("/") != s"file:$warehousePath/t")
assert(spark.sql("SELECT * FROM t").count == 1)

spark.sql("CREATE TABLE t3(d string)")
val table3 = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t3"))
// the table in default database created here in this job, it will use the warehouse path
// of this job as its location
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

->

When a job creates a table in the default database, the table location is under the warehouse path
that is configured for the local job.

assert(table3.location.stripSuffix("/") == s"file:$warehousePath/t3")

spark.sql("USE not_default")
val table1 = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t1"))
// the table in not default database create in job(SPARK_19667_CREATE_TABLE) above,
// which has different warehouse path from this job, its location still equals to
// the location when it's created.
assert(table1.location.stripSuffix("/") != s"$warehousePath/not_default.db/t1")
assert(!new File(s"$warehousePath/not_default.db/t1").exists())

spark.sql("CREATE TABLE t2(c string)")
val table2 = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t2"))
// the table in not default database created here in this job, it will use the location
// of the database as its location, not the warehouse path in this job
Copy link
Member

@gatorsmile gatorsmile Mar 12, 2017

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

->

The table created in the non-default database (created in another job) is under the database location.

assert(table2.location.stripSuffix("/") != s"file:$warehousePath/not_default.db/t2")

spark.sql("CREATE DATABASE not_default_1")
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

-> non_default_db1

spark.sql("USE not_default_1")
spark.sql("CREATE TABLE t4(e string)")
val table4 = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t4"))
// the table created in the database which created in this job, it will use the location
// of the database.
Copy link
Member

@gatorsmile gatorsmile Mar 12, 2017

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

->

The table created in the non-default database (created in this job) is under the database location.

assert(table4.location.stripSuffix("/") == s"file:$warehousePath/not_default_1.db/t4")

} finally {
spark.sql("DROP TABLE IF EXISTS t4")
spark.sql("DROP DATABASE not_default_1")

spark.sql("USE not_default")
spark.sql("DROP TABLE IF EXISTS t1")
spark.sql("DROP TABLE IF EXISTS t2")
spark.sql("DROP DATABASE not_default")

spark.sql("USE default")
spark.sql("DROP TABLE IF EXISTS t")
spark.sql("DROP TABLE IF EXISTS t3")
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -655,7 +655,7 @@ class VersionsSuite extends QueryTest with SQLTestUtils with TestHiveSingleton w

val tPath = new Path(spark.sessionState.conf.warehousePath, "t")
Seq("1").toDF("a").write.saveAsTable("t")
val expectedPath = s"file:${tPath.toUri.getPath.stripSuffix("/")}"
val expectedPath = tPath.toUri.getPath.stripSuffix("/")
val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t"))

assert(table.location.stripSuffix("/") == expectedPath)
Expand All @@ -665,7 +665,7 @@ class VersionsSuite extends QueryTest with SQLTestUtils with TestHiveSingleton w
val t1Path = new Path(spark.sessionState.conf.warehousePath, "t1")
spark.sql("create table t1 using parquet as select 2 as a")
val table1 = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t1"))
val expectedPath1 = s"file:${t1Path.toUri.getPath.stripSuffix("/")}"
val expectedPath1 = t1Path.toUri.getPath.stripSuffix("/")

assert(table1.location.stripSuffix("/") == expectedPath1)
assert(t1Path.getFileSystem(spark.sessionState.newHadoopConf()).exists(t1Path))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1587,51 +1587,4 @@ class HiveDDLSuite
}
}
}

test("create table with default database use warehouse path instead of database location") {
withTable("t") {
// default database use warehouse path as its location
withTempDir { dir =>
spark.sparkContext.conf
.set(SQLConf.TEST_HIVE_CREATETABLE_DEFAULTDB_USEWAREHOUSE_PATH.key, "true")

val sparkWarehousePath = spark.sharedState.warehousePath.stripSuffix("/")
spark.sql(s"CREATE DATABASE default_test LOCATION '$dir'" )
val db = spark.sessionState.catalog.getDatabaseMetadata("default_test")
assert(db.locationUri.stripSuffix("/") == sparkWarehousePath)
spark.sql("USE default_test")

spark.sql("CREATE TABLE t(a string)")
val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t"))
assert(table.location.stripSuffix("/").stripPrefix("file:") ==
new File(sparkWarehousePath, "t").getAbsolutePath.stripSuffix("/"))

// clear
spark.sparkContext.conf
.remove(SQLConf.TEST_HIVE_CREATETABLE_DEFAULTDB_USEWAREHOUSE_PATH.key)

spark.sql("DROP TABLE t")
spark.sql("DROP DATABASE default_test")
spark.sql("USE DEFAULT")
}

// not default database use its's location from the create command
withTempDir { dir =>
val dirPath = s"file:${dir.getAbsolutePath.stripSuffix("/")}"
spark.sql(s"CREATE DATABASE test_not_default LOCATION '$dir'" )
val db = spark.sessionState.catalog.getDatabaseMetadata("test_not_default")
assert(db.locationUri.stripSuffix("/") == dirPath)
spark.sql("USE test_not_default")

spark.sql("CREATE TABLE t(a string)")
val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t"))
assert(table.location.stripSuffix("/") == s"$dirPath/t" )

// clear
spark.sql("DROP TABLE t")
spark.sql("DROP DATABASE test_not_default")
spark.sql("USE DEFAULT")
}
}
}
}