Skip to content

Commit 5e16480

Browse files
author
Andrew Or
committed
Fix tests round 2
There were some issues with case sensitivity analysis and error messages not being exactly as expected. The latter is now relaxed where possible.
1 parent 78cbcbd commit 5e16480

File tree

12 files changed

+53
-36
lines changed

12 files changed

+53
-36
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala

Lines changed: 5 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -40,16 +40,14 @@ import org.apache.spark.sql.types._
4040
* Used for testing when all relations are already filled in and the analyzer needs only
4141
* to resolve attribute references.
4242
*/
43-
object SimpleAnalyzer
44-
extends Analyzer(
45-
new SessionCatalog(new InMemoryCatalog),
46-
EmptyFunctionRegistry,
47-
new SimpleCatalystConf(true))
43+
object SimpleAnalyzer extends SimpleAnalyzer(new SimpleCatalystConf(true))
44+
class SimpleAnalyzer(conf: CatalystConf)
45+
extends Analyzer(new SessionCatalog(new InMemoryCatalog, conf), EmptyFunctionRegistry, conf)
4846

4947
/**
5048
* Provides a logical query plan analyzer, which translates [[UnresolvedAttribute]]s and
51-
* [[UnresolvedRelation]]s into fully typed objects using information in a schema [[Catalog]] and
52-
* a [[FunctionRegistry]].
49+
* [[UnresolvedRelation]]s into fully typed objects using information in a
50+
* [[SessionCatalog]] and a [[FunctionRegistry]].
5351
*/
5452
class Analyzer(
5553
catalog: SessionCatalog,

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala

Lines changed: 5 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -64,20 +64,22 @@ class InMemoryCatalog extends ExternalCatalog {
6464

6565
private def requireFunctionExists(db: String, funcName: String): Unit = {
6666
if (!functionExists(db, funcName)) {
67-
throw new AnalysisException(s"Function '$funcName' does not exist in database '$db'")
67+
throw new AnalysisException(
68+
s"Function not found: '$funcName' does not exist in database '$db'")
6869
}
6970
}
7071

7172
private def requireTableExists(db: String, table: String): Unit = {
7273
if (!tableExists(db, table)) {
73-
throw new AnalysisException(s"Table '$table' does not exist in database '$db'")
74+
throw new AnalysisException(
75+
s"Table not found: '$table' does not exist in database '$db'")
7476
}
7577
}
7678

7779
private def requirePartitionExists(db: String, table: String, spec: TablePartitionSpec): Unit = {
7880
if (!partitionExists(db, table, spec)) {
7981
throw new AnalysisException(
80-
s"Partition does not exist in database '$db' table '$table': '$spec'")
82+
s"Partition not found: database '$db' table '$table' does not contain: '$spec'")
8183
}
8284
}
8385

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala

Lines changed: 7 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@ import java.util.concurrent.ConcurrentHashMap
2222
import scala.collection.JavaConverters._
2323

2424
import org.apache.spark.sql.AnalysisException
25+
import org.apache.spark.sql.catalyst.{CatalystConf, SimpleCatalystConf}
2526
import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier}
2627
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
2728

@@ -31,9 +32,13 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
3132
* proxy to the underlying metastore (e.g. Hive Metastore) and it also manages temporary
3233
* tables and functions of the Spark Session that it belongs to.
3334
*/
34-
class SessionCatalog(externalCatalog: ExternalCatalog, caseSensitiveAnalysis: Boolean = true) {
35+
class SessionCatalog(externalCatalog: ExternalCatalog, conf: CatalystConf) {
3536
import ExternalCatalog._
3637

38+
def this(externalCatalog: ExternalCatalog) {
39+
this(externalCatalog, new SimpleCatalystConf(true))
40+
}
41+
3742
protected[this] val tempTables = new ConcurrentHashMap[String, LogicalPlan]
3843
protected[this] val tempFunctions = new ConcurrentHashMap[String, CatalogFunction]
3944

@@ -53,7 +58,7 @@ class SessionCatalog(externalCatalog: ExternalCatalog, caseSensitiveAnalysis: Bo
5358
* Format table name, taking into account case sensitivity.
5459
*/
5560
protected[this] def formatTableName(name: String): String = {
56-
if (caseSensitiveAnalysis) name else name.toLowerCase
61+
if (conf.caseSensitiveAnalysis) name else name.toLowerCase
5762
}
5863

5964
// ----------------------------------------------------------------------------

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -30,7 +30,7 @@ trait AnalysisTest extends PlanTest {
3030

3131
private def makeAnalyzer(caseSensitive: Boolean): Analyzer = {
3232
val conf = new SimpleCatalystConf(caseSensitive)
33-
val catalog = new SessionCatalog(new InMemoryCatalog, caseSensitive)
33+
val catalog = new SessionCatalog(new InMemoryCatalog, conf)
3434
catalog.createTempTable("TaBlE", TestRelations.testRelation, ignoreIfExists = true)
3535
new Analyzer(catalog, EmptyFunctionRegistry, conf) {
3636
override val extendedResolutionRules = EliminateSubqueryAliases :: Nil

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,7 @@ import org.apache.spark.sql.types._
3232

3333
class DecimalPrecisionSuite extends PlanTest with BeforeAndAfter {
3434
private val conf = new SimpleCatalystConf(true)
35-
private val catalog = new SessionCatalog(new InMemoryCatalog, caseSensitiveAnalysis = true)
35+
private val catalog = new SessionCatalog(new InMemoryCatalog, conf)
3636
private val analyzer = new Analyzer(catalog, EmptyFunctionRegistry, conf)
3737

3838
private val relation = LocalRelation(

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala

Lines changed: 4 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -427,9 +427,8 @@ class SessionCatalogSuite extends SparkFunSuite {
427427
TableIdentifier("tbl4"),
428428
TableIdentifier("tbl1", Some("db2")),
429429
TableIdentifier("tbl2", Some("db2"))))
430-
intercept[AnalysisException] {
431-
catalog.listTables("unknown_db")
432-
}
430+
assert(catalog.listTables("unknown_db").toSet ==
431+
Set(TableIdentifier("tbl1"), TableIdentifier("tbl4")))
433432
}
434433

435434
test("list tables with pattern") {
@@ -446,9 +445,8 @@ class SessionCatalogSuite extends SparkFunSuite {
446445
TableIdentifier("tbl2", Some("db2"))))
447446
assert(catalog.listTables("db2", "*1").toSet ==
448447
Set(TableIdentifier("tbl1"), TableIdentifier("tbl1", Some("db2"))))
449-
intercept[AnalysisException] {
450-
catalog.listTables("unknown_db")
451-
}
448+
assert(catalog.listTables("unknown_db", "*").toSet ==
449+
Set(TableIdentifier("tbl1"), TableIdentifier("tbl4")))
452450
}
453451

454452
// --------------------------------------------------------------------------

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -93,6 +93,11 @@ case class CreateTempTableUsing(
9393
provider: String,
9494
options: Map[String, String]) extends RunnableCommand {
9595

96+
if (tableIdent.database.isDefined) {
97+
throw new AnalysisException(
98+
s"Temporary table '$tableIdent' should not have specified a database")
99+
}
100+
96101
def run(sqlContext: SQLContext): Seq[Row] = {
97102
val dataSource = DataSource(
98103
sqlContext,
@@ -116,6 +121,11 @@ case class CreateTempTableUsingAsSelect(
116121
options: Map[String, String],
117122
query: LogicalPlan) extends RunnableCommand {
118123

124+
if (tableIdent.database.isDefined) {
125+
throw new AnalysisException(
126+
s"Temporary table '$tableIdent' should not have specified a database")
127+
}
128+
119129
override def run(sqlContext: SQLContext): Seq[Row] = {
120130
val df = Dataset.newDataFrame(sqlContext, query)
121131
val dataSource = DataSource(

sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -46,7 +46,7 @@ private[sql] class SessionState(ctx: SQLContext) {
4646
/**
4747
* Internal catalog for managing table and database states.
4848
*/
49-
lazy val sessionCatalog = new SessionCatalog(ctx.externalCatalog)
49+
lazy val sessionCatalog = new SessionCatalog(ctx.externalCatalog, conf)
5050

5151
/**
5252
* Internal catalog for managing functions registered by the user.

sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala

Lines changed: 13 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -1395,12 +1395,16 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
13951395
}
13961396

13971397
test("SPARK-4699 case sensitivity SQL query") {
1398-
sqlContext.setConf(SQLConf.CASE_SENSITIVE, false)
1399-
val data = TestData(1, "val_1") :: TestData(2, "val_2") :: Nil
1400-
val rdd = sparkContext.parallelize((0 to 1).map(i => data(i)))
1401-
rdd.toDF().registerTempTable("testTable1")
1402-
checkAnswer(sql("SELECT VALUE FROM TESTTABLE1 where KEY = 1"), Row("val_1"))
1403-
sqlContext.setConf(SQLConf.CASE_SENSITIVE, true)
1398+
val orig = sqlContext.getConf(SQLConf.CASE_SENSITIVE)
1399+
try {
1400+
sqlContext.setConf(SQLConf.CASE_SENSITIVE, false)
1401+
val data = TestData(1, "val_1") :: TestData(2, "val_2") :: Nil
1402+
val rdd = sparkContext.parallelize((0 to 1).map(i => data(i)))
1403+
rdd.toDF().registerTempTable("testTable1")
1404+
checkAnswer(sql("SELECT VALUE FROM TESTTABLE1 where KEY = 1"), Row("val_1"))
1405+
} finally {
1406+
sqlContext.setConf(SQLConf.CASE_SENSITIVE, orig)
1407+
}
14041408
}
14051409

14061410
test("SPARK-6145: ORDER BY test for nested fields") {
@@ -1674,7 +1678,8 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
16741678
.format("parquet")
16751679
.save(path)
16761680

1677-
val message = intercept[AnalysisException] {
1681+
// We don't support creating a temporary table while specifying a database
1682+
intercept[AnalysisException] {
16781683
sqlContext.sql(
16791684
s"""
16801685
|CREATE TEMPORARY TABLE db.t
@@ -1684,9 +1689,8 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
16841689
|)
16851690
""".stripMargin)
16861691
}.getMessage
1687-
assert(message.contains("Specifying database name or other qualifiers are not allowed"))
16881692

1689-
// If you use backticks to quote the name of a temporary table having dot in it.
1693+
// If you use backticks to quote the name then it's OK.
16901694
sqlContext.sql(
16911695
s"""
16921696
|CREATE TEMPORARY TABLE `db.t`

sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -23,15 +23,16 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
2323
import org.apache.spark.sql.catalyst.rules.Rule
2424
import org.apache.spark.sql.execution.datasources.BucketSpec
2525
import org.apache.spark.sql.hive.client.HiveClient
26+
import org.apache.spark.sql.internal.SQLConf
2627
import org.apache.spark.sql.types.StructType
2728

2829

2930
class HiveSessionCatalog(
3031
externalCatalog: HiveCatalog,
3132
client: HiveClient,
3233
context: HiveContext,
33-
caseSensitiveAnalysis: Boolean)
34-
extends SessionCatalog(externalCatalog, caseSensitiveAnalysis) {
34+
conf: SQLConf)
35+
extends SessionCatalog(externalCatalog, conf) {
3536

3637
override def setCurrentDatabase(db: String): Unit = {
3738
super.setCurrentDatabase(db)

0 commit comments

Comments
 (0)