Skip to content

Commit 12eca9a

Browse files
author
Jacky Li
committed
solve conflict with master
2 parents 5b0a42c + 39e369c commit 12eca9a

File tree

13 files changed

+163
-38
lines changed

13 files changed

+163
-38
lines changed
Lines changed: 53 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,53 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.sql.catalyst
19+
20+
import scala.collection.immutable
21+
22+
private[spark] object CatalystConf{
23+
val CASE_SENSITIVE = "spark.sql.caseSensitive"
24+
}
25+
26+
private[spark] trait CatalystConf {
27+
def setConf(key: String, value: String) : Unit
28+
def getConf(key: String) : String
29+
def getConf(key: String, defaultValue: String) : String
30+
def getAllConfs: immutable.Map[String, String]
31+
}
32+
33+
/**
34+
* A trivial conf that is empty. Used for testing when all
35+
* relations are already filled in and the analyser needs only to resolve attribute references.
36+
*/
37+
object EmptyConf extends CatalystConf {
38+
def setConf(key: String, value: String) : Unit = {
39+
throw new UnsupportedOperationException
40+
}
41+
42+
def getConf(key: String) : String = {
43+
throw new UnsupportedOperationException
44+
}
45+
46+
def getConf(key: String, defaultValue: String) : String = {
47+
throw new UnsupportedOperationException
48+
}
49+
50+
def getAllConfs: immutable.Map[String, String] = {
51+
throw new UnsupportedOperationException
52+
}
53+
}

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

Lines changed: 11 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -19,18 +19,19 @@ package org.apache.spark.sql.catalyst.analysis
1919

2020
import org.apache.spark.util.collection.OpenHashSet
2121
import org.apache.spark.sql.AnalysisException
22-
import org.apache.spark.sql.catalyst.errors.TreeNodeException
22+
import org.apache.spark.sql.catalyst.CatalystConf
2323
import org.apache.spark.sql.catalyst.expressions._
2424
import org.apache.spark.sql.catalyst.plans.logical._
2525
import org.apache.spark.sql.catalyst.rules._
26+
import org.apache.spark.sql.catalyst.test.SimpleConf
2627
import org.apache.spark.sql.types._
2728

2829
/**
2930
* A trivial [[Analyzer]] with an [[EmptyCatalog]] and [[EmptyFunctionRegistry]]. Used for testing
3031
* when all relations are already filled in and the analyser needs only to resolve attribute
3132
* references.
3233
*/
33-
object SimpleAnalyzer extends Analyzer(EmptyCatalog, EmptyFunctionRegistry, true)
34+
object SimpleAnalyzer extends Analyzer(EmptyCatalog, EmptyFunctionRegistry, new SimpleConf)
3435

3536
/**
3637
* Provides a logical query plan analyzer, which translates [[UnresolvedAttribute]]s and
@@ -39,11 +40,17 @@ object SimpleAnalyzer extends Analyzer(EmptyCatalog, EmptyFunctionRegistry, true
3940
*/
4041
class Analyzer(catalog: Catalog,
4142
registry: FunctionRegistry,
42-
caseSensitive: Boolean,
43+
conf: CatalystConf,
4344
maxIterations: Int = 100)
4445
extends RuleExecutor[LogicalPlan] with HiveTypeCoercion {
4546

46-
val resolver = if (caseSensitive) caseSensitiveResolution else caseInsensitiveResolution
47+
def resolver: Resolver = {
48+
if (conf.getConf(CatalystConf.CASE_SENSITIVE, "true").toBoolean) {
49+
caseSensitiveResolution
50+
} else {
51+
caseInsensitiveResolution
52+
}
53+
}
4754

4855
val fixedPoint = FixedPoint(maxIterations)
4956

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

Lines changed: 7 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,8 @@ package org.apache.spark.sql.catalyst.analysis
2020
import scala.collection.mutable
2121

2222
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery}
23+
import org.apache.spark.sql.catalyst.CatalystConf
24+
import org.apache.spark.sql.catalyst.EmptyConf
2325

2426
/**
2527
* Thrown by a catalog when a table cannot be found. The analzyer will rethrow the exception
@@ -32,7 +34,7 @@ class NoSuchTableException extends Exception
3234
*/
3335
trait Catalog {
3436

35-
def caseSensitive: Boolean
37+
val conf: CatalystConf
3638

3739
def tableExists(tableIdentifier: Seq[String]): Boolean
3840

@@ -55,7 +57,7 @@ trait Catalog {
5557
def unregisterAllTables(): Unit
5658

5759
protected def processTableIdentifier(tableIdentifier: Seq[String]): Seq[String] = {
58-
if (!caseSensitive) {
60+
if (!conf.getConf(CatalystConf.CASE_SENSITIVE, "true").toBoolean) {
5961
tableIdentifier.map(_.toLowerCase)
6062
} else {
6163
tableIdentifier
@@ -76,7 +78,7 @@ trait Catalog {
7678
}
7779
}
7880

79-
class SimpleCatalog(val caseSensitive: Boolean) extends Catalog {
81+
class SimpleCatalog(val conf: CatalystConf) extends Catalog {
8082
val tables = new mutable.HashMap[String, LogicalPlan]()
8183

8284
override def registerTable(
@@ -162,7 +164,7 @@ trait OverrideCatalog extends Catalog {
162164
}
163165

164166
abstract override def getTables(databaseName: Option[String]): Seq[(String, Boolean)] = {
165-
val dbName = if (!caseSensitive) {
167+
val dbName = if (!conf.getConf(CatalystConf.CASE_SENSITIVE).toBoolean) {
166168
if (databaseName.isDefined) Some(databaseName.get.toLowerCase) else None
167169
} else {
168170
databaseName
@@ -205,7 +207,7 @@ trait OverrideCatalog extends Catalog {
205207
*/
206208
object EmptyCatalog extends Catalog {
207209

208-
val caseSensitive: Boolean = true
210+
override val conf: CatalystConf = EmptyConf
209211

210212
def tableExists(tableIdentifier: Seq[String]): Boolean = {
211213
throw new UnsupportedOperationException
Lines changed: 41 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,41 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.sql.catalyst.test
19+
20+
import org.apache.spark.sql.catalyst.CatalystConf
21+
22+
import scala.collection.immutable
23+
import scala.collection.mutable
24+
25+
/** A CatalystConf that can be used for local testing. */
26+
class SimpleConf extends CatalystConf{
27+
val map = mutable.Map[String, String]()
28+
29+
def setConf(key: String, value: String) : Unit = {
30+
map.put(key, value)
31+
}
32+
def getConf(key: String) : String ={
33+
map.get(key).get
34+
}
35+
def getConf(key: String, defaultValue: String) : String = {
36+
map.getOrElse(key, defaultValue)
37+
}
38+
def getAllConfs: immutable.Map[String, String] = {
39+
map.toMap
40+
}
41+
}

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

Lines changed: 10 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -23,17 +23,22 @@ import org.apache.spark.sql.AnalysisException
2323
import org.apache.spark.sql.catalyst.expressions._
2424
import org.apache.spark.sql.catalyst.plans.logical._
2525
import org.apache.spark.sql.types._
26-
26+
import org.apache.spark.sql.catalyst.CatalystConf
27+
import org.apache.spark.sql.catalyst.test.SimpleConf
2728
import org.apache.spark.sql.catalyst.dsl.expressions._
2829
import org.apache.spark.sql.catalyst.dsl.plans._
2930

3031
class AnalysisSuite extends FunSuite with BeforeAndAfter {
31-
val caseSensitiveCatalog = new SimpleCatalog(true)
32-
val caseInsensitiveCatalog = new SimpleCatalog(false)
32+
val caseSensitiveConf = new SimpleConf()
33+
caseSensitiveConf.setConf(CatalystConf.CASE_SENSITIVE, "true")
34+
val caseInsensitiveConf = new SimpleConf()
35+
caseInsensitiveConf.setConf(CatalystConf.CASE_SENSITIVE, "false")
36+
val caseSensitiveCatalog = new SimpleCatalog(caseSensitiveConf)
37+
val caseInsensitiveCatalog = new SimpleCatalog(caseInsensitiveConf)
3338
val caseSensitiveAnalyze =
34-
new Analyzer(caseSensitiveCatalog, EmptyFunctionRegistry, caseSensitive = true)
39+
new Analyzer(caseSensitiveCatalog, EmptyFunctionRegistry, caseSensitiveConf)
3540
val caseInsensitiveAnalyze =
36-
new Analyzer(caseInsensitiveCatalog, EmptyFunctionRegistry, caseSensitive = false)
41+
new Analyzer(caseInsensitiveCatalog, EmptyFunctionRegistry, caseInsensitiveConf)
3742

3843
val testRelation = LocalRelation(AttributeReference("a", IntegerType, nullable = true)())
3944
val testRelation2 = LocalRelation(

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

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -20,11 +20,13 @@ package org.apache.spark.sql.catalyst.analysis
2020
import org.apache.spark.sql.catalyst.expressions._
2121
import org.apache.spark.sql.catalyst.plans.logical.{Project, LocalRelation}
2222
import org.apache.spark.sql.types._
23+
import org.apache.spark.sql.catalyst.test.SimpleConf
2324
import org.scalatest.{BeforeAndAfter, FunSuite}
2425

2526
class DecimalPrecisionSuite extends FunSuite with BeforeAndAfter {
26-
val catalog = new SimpleCatalog(false)
27-
val analyzer = new Analyzer(catalog, EmptyFunctionRegistry, caseSensitive = false)
27+
val conf = new SimpleConf
28+
val catalog = new SimpleCatalog(conf)
29+
val analyzer = new Analyzer(catalog, EmptyFunctionRegistry, conf)
2830

2931
val relation = LocalRelation(
3032
AttributeReference("i", IntegerType)(),

sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,8 @@
1717

1818
package org.apache.spark.sql
1919

20+
import org.apache.spark.sql.catalyst.CatalystConf
21+
2022
import scala.collection.immutable
2123
import scala.collection.JavaConversions._
2224

@@ -69,7 +71,8 @@ private[spark] object SQLConf {
6971
*
7072
* SQLConf is thread-safe (internally synchronized, so safe to be used in multiple threads).
7173
*/
72-
private[sql] class SQLConf extends Serializable {
74+
75+
private[sql] class SQLConf extends Serializable with CatalystConf {
7376
import SQLConf._
7477

7578
/** Only low degree of contention is expected for conf, thus NOT using ConcurrentHashMap. */
@@ -220,4 +223,3 @@ private[sql] class SQLConf extends Serializable {
220223
settings.clear()
221224
}
222225
}
223-

sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -104,14 +104,14 @@ class SQLContext(@transient val sparkContext: SparkContext)
104104
def getAllConfs: immutable.Map[String, String] = conf.getAllConfs
105105

106106
@transient
107-
protected[sql] lazy val catalog: Catalog = new SimpleCatalog(true)
107+
protected[sql] lazy val catalog: Catalog = new SimpleCatalog(conf)
108108

109109
@transient
110110
protected[sql] lazy val functionRegistry: FunctionRegistry = new SimpleFunctionRegistry(true)
111111

112112
@transient
113113
protected[sql] lazy val analyzer: Analyzer =
114-
new Analyzer(catalog, functionRegistry, caseSensitive = true) {
114+
new Analyzer(catalog, functionRegistry, conf) {
115115
override val extendedResolutionRules =
116116
ExtractPythonUdfs ::
117117
sources.PreWriteCheck(catalog) ::

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

Lines changed: 10 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -21,10 +21,10 @@ import org.apache.spark.sql.test.TestSQLContext
2121
import org.scalatest.BeforeAndAfterAll
2222

2323
import org.apache.spark.sql.functions._
24+
import org.apache.spark.sql.catalyst.CatalystConf
2425
import org.apache.spark.sql.catalyst.errors.TreeNodeException
2526
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
2627
import org.apache.spark.sql.types._
27-
2828
import org.apache.spark.sql.TestData._
2929
import org.apache.spark.sql.test.TestSQLContext.{udf => _, _}
3030

@@ -1049,4 +1049,13 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll {
10491049
rdd.toDF().registerTempTable("distinctData")
10501050
checkAnswer(sql("SELECT COUNT(DISTINCT key,value) FROM distinctData"), Row(2))
10511051
}
1052+
1053+
test("SPARK-4699 case sensitivity SQL query") {
1054+
setConf(CatalystConf.CASE_SENSITIVE, "false")
1055+
val data = TestData(1,"val_1") :: TestData(2,"val_2") :: Nil
1056+
val rdd = sparkContext.parallelize((0 to 1).map(i => data(i)))
1057+
rdd.toDF().registerTempTable("testTable1")
1058+
checkAnswer(sql("SELECT VALUE FROM TESTTABLE1 where KEY = 1"), Row("val_1"))
1059+
setConf(CatalystConf.CASE_SENSITIVE, "true")
1060+
}
10521061
}

sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala

Lines changed: 5 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -18,22 +18,13 @@
1818
package org.apache.spark.sql.sources
1919

2020
import org.apache.spark.sql._
21-
import org.apache.spark.sql.catalyst.analysis.Analyzer
21+
import org.apache.spark.sql.catalyst.CatalystConf
2222
import org.apache.spark.sql.test.TestSQLContext
2323
import org.scalatest.BeforeAndAfter
2424

2525
abstract class DataSourceTest extends QueryTest with BeforeAndAfter {
26-
// Case sensitivity is not configurable yet, but we want to test some edge cases.
27-
// TODO: Remove when it is configurable
28-
implicit val caseInsensisitiveContext = new SQLContext(TestSQLContext.sparkContext) {
29-
@transient
30-
override protected[sql] lazy val analyzer: Analyzer =
31-
new Analyzer(catalog, functionRegistry, caseSensitive = false) {
32-
override val extendedResolutionRules =
33-
PreWriteCheck(catalog) ::
34-
PreInsertCastAndRename ::
35-
Nil
36-
}
37-
}
38-
}
26+
// We want to test some edge cases.
27+
implicit val caseInsensisitiveContext = new SQLContext(TestSQLContext.sparkContext)
3928

29+
caseInsensisitiveContext.setConf(CatalystConf.CASE_SENSITIVE, "false")
30+
}

0 commit comments

Comments
 (0)