Skip to content

Commit 0f73998

Browse files
committed
Make Analyzer abstract and add TestAnalyzer
1 parent 0f4d9aa commit 0f73998

File tree

13 files changed

+55
-18
lines changed

13 files changed

+55
-18
lines changed

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

Lines changed: 6 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -52,7 +52,11 @@ object SimpleAnalyzer extends Analyzer(
5252
new SQLConf().copy(SQLConf.CASE_SENSITIVE -> true)) {
5353
override def createDatabase(dbDefinition: CatalogDatabase, ignoreIfExists: Boolean) {}
5454
},
55-
new SQLConf().copy(SQLConf.CASE_SENSITIVE -> true))
55+
new SQLConf().copy(SQLConf.CASE_SENSITIVE -> true)) {
56+
57+
override protected def lookupCatalog(name: String): CatalogPlugin =
58+
throw new CatalogNotFoundException("No catalog lookup function")
59+
}
5660

5761
/**
5862
* Provides a way to keep state during the analysis, this enables us to decouple the concerns
@@ -93,7 +97,7 @@ object AnalysisContext {
9397
* Provides a logical query plan analyzer, which translates [[UnresolvedAttribute]]s and
9498
* [[UnresolvedRelation]]s into fully typed objects using information in a [[SessionCatalog]].
9599
*/
96-
class Analyzer(
100+
abstract class Analyzer(
97101
catalog: SessionCatalog,
98102
conf: SQLConf,
99103
maxIterations: Int)
@@ -103,9 +107,6 @@ class Analyzer(
103107
this(catalog, conf, conf.optimizerMaxIterations)
104108
}
105109

106-
override protected def lookupCatalog(name: String): CatalogPlugin =
107-
throw new CatalogNotFoundException("No catalog lookup function")
108-
109110
def executeAndCheck(plan: LogicalPlan, tracker: QueryPlanningTracker): LogicalPlan = {
110111
AnalysisHelper.markInAnalyzer {
111112
val analyzed = executeAndTrack(plan, tracker)

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -43,7 +43,7 @@ class AnalysisExternalCatalogSuite extends AnalysisTest with Matchers {
4343
CatalogStorageFormat.empty,
4444
StructType(Seq(StructField("a", IntegerType, nullable = true)))),
4545
ignoreIfExists = false)
46-
new Analyzer(catalog, conf)
46+
new TestAnalyzer(catalog, conf)
4747
}
4848

4949
test("query builtin functions don't call the external catalog") {

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
@@ -41,7 +41,7 @@ trait AnalysisTest extends PlanTest {
4141
catalog.createTempView("TaBlE", TestRelations.testRelation, overrideIfExists = true)
4242
catalog.createTempView("TaBlE2", TestRelations.testRelation2, overrideIfExists = true)
4343
catalog.createTempView("TaBlE3", TestRelations.testRelation3, overrideIfExists = true)
44-
new Analyzer(catalog, conf) {
44+
new TestAnalyzer(catalog, conf) {
4545
override val extendedResolutionRules = EliminateSubqueryAliases :: Nil
4646
}
4747
}

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
@@ -31,7 +31,7 @@ import org.apache.spark.sql.types._
3131

3232
class DecimalPrecisionSuite extends AnalysisTest with BeforeAndAfter {
3333
private val catalog = new SessionCatalog(new InMemoryCatalog, EmptyFunctionRegistry, conf)
34-
private val analyzer = new Analyzer(catalog, conf)
34+
private val analyzer = new TestAnalyzer(catalog, conf)
3535

3636
private val relation = LocalRelation(
3737
AttributeReference("i", IntegerType)(),

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

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -36,7 +36,7 @@ class LookupFunctionsSuite extends PlanTest {
3636
catalog.createDatabase(
3737
CatalogDatabase("default", "", new URI("loc"), Map.empty),
3838
ignoreIfExists = false)
39-
new Analyzer(catalog, conf)
39+
new TestAnalyzer(catalog, conf)
4040
}
4141

4242
def table(ref: String): LogicalPlan = UnresolvedRelation(TableIdentifier(ref))
@@ -63,7 +63,7 @@ class LookupFunctionsSuite extends PlanTest {
6363
catalog.createDatabase(
6464
CatalogDatabase("default", "", new URI("loc"), Map.empty),
6565
ignoreIfExists = false)
66-
new Analyzer(catalog, conf)
66+
new TestAnalyzer(catalog, conf)
6767
}
6868

6969
def table(ref: String): LogicalPlan = UnresolvedRelation(TableIdentifier(ref))
Lines changed: 30 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,30 @@
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.analysis
19+
20+
import org.apache.spark.sql.catalog.v2.{CatalogNotFoundException, CatalogPlugin}
21+
import org.apache.spark.sql.catalyst.catalog._
22+
import org.apache.spark.sql.internal.SQLConf
23+
24+
class TestAnalyzer(
25+
catalog: SessionCatalog,
26+
conf: SQLConf) extends Analyzer(catalog, conf) {
27+
28+
override protected def lookupCatalog(name: String): CatalogPlugin =
29+
throw new CatalogNotFoundException("No catalog lookup function")
30+
}

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1429,7 +1429,7 @@ abstract class SessionCatalogSuite extends AnalysisTest {
14291429
val catalog = new SessionCatalog(newBasicCatalog(), new SimpleFunctionRegistry, conf)
14301430
catalog.setCurrentDatabase("db1")
14311431
try {
1432-
val analyzer = new Analyzer(catalog, conf)
1432+
val analyzer = new TestAnalyzer(catalog, conf)
14331433

14341434
// The analyzer should report the undefined function rather than the undefined table first.
14351435
val cause = intercept[AnalysisException] {

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala

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

1818
package org.apache.spark.sql.catalyst.optimizer
1919

20-
import org.apache.spark.sql.catalyst.analysis.{Analyzer, EmptyFunctionRegistry}
20+
import org.apache.spark.sql.catalyst.analysis.{EmptyFunctionRegistry, TestAnalyzer}
2121
import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog}
2222
import org.apache.spark.sql.catalyst.dsl.expressions._
2323
import org.apache.spark.sql.catalyst.dsl.plans._
@@ -31,7 +31,7 @@ import org.apache.spark.sql.internal.SQLConf.{CASE_SENSITIVE, GROUP_BY_ORDINAL}
3131
class AggregateOptimizeSuite extends PlanTest {
3232
override val conf = new SQLConf().copy(CASE_SENSITIVE -> false, GROUP_BY_ORDINAL -> false)
3333
val catalog = new SessionCatalog(new InMemoryCatalog, EmptyFunctionRegistry, conf)
34-
val analyzer = new Analyzer(catalog, conf)
34+
val analyzer = new TestAnalyzer(catalog, conf)
3535

3636
object Optimize extends RuleExecutor[LogicalPlan] {
3737
val batches = Batch("Aggregate", FixedPoint(100),

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -189,7 +189,7 @@ class BooleanSimplificationSuite extends PlanTest with ExpressionEvalHelper with
189189
}
190190

191191
private val caseInsensitiveConf = new SQLConf().copy(SQLConf.CASE_SENSITIVE -> false)
192-
private val caseInsensitiveAnalyzer = new Analyzer(
192+
private val caseInsensitiveAnalyzer = new TestAnalyzer(
193193
new SessionCatalog(new InMemoryCatalog, EmptyFunctionRegistry, caseInsensitiveConf),
194194
caseInsensitiveConf)
195195

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsSuite.scala

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

1818
package org.apache.spark.sql.catalyst.optimizer
1919

20-
import org.apache.spark.sql.catalyst.analysis.{Analyzer, EmptyFunctionRegistry}
20+
import org.apache.spark.sql.catalyst.analysis.{EmptyFunctionRegistry, TestAnalyzer}
2121
import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog}
2222
import org.apache.spark.sql.catalyst.dsl.expressions._
2323
import org.apache.spark.sql.catalyst.dsl.plans._
@@ -31,7 +31,7 @@ import org.apache.spark.sql.internal.SQLConf.{CASE_SENSITIVE, ORDER_BY_ORDINAL}
3131
class EliminateSortsSuite extends PlanTest {
3232
override val conf = new SQLConf().copy(CASE_SENSITIVE -> true, ORDER_BY_ORDINAL -> false)
3333
val catalog = new SessionCatalog(new InMemoryCatalog, EmptyFunctionRegistry, conf)
34-
val analyzer = new Analyzer(catalog, conf)
34+
val analyzer = new TestAnalyzer(catalog, conf)
3535

3636
object Optimize extends RuleExecutor[LogicalPlan] {
3737
val batches =

0 commit comments

Comments
 (0)