From 5f7d835d380c1a558a4a6d8366140cd96ee202eb Mon Sep 17 00:00:00 2001
From: jiangxingbo <jiangxb1987@gmail.com>
Date: Wed, 8 Mar 2017 16:18:17 +0100
Subject: [PATCH] [SPARK-19865][SQL] remove the view identifier in
 SubqueryAlias

## What changes were proposed in this pull request?

Since we have a `View` node now, we can remove the view identifier in `SubqueryAlias`, which was used to indicate a view node before.

## How was this patch tested?

Update the related test cases.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #17210 from jiangxb1987/SubqueryAlias.
---
 .../spark/sql/catalyst/analysis/Analyzer.scala   |  4 ++--
 .../sql/catalyst/catalog/SessionCatalog.scala    |  8 ++++----
 .../apache/spark/sql/catalyst/dsl/package.scala  |  4 ++--
 .../spark/sql/catalyst/optimizer/subquery.scala  |  8 ++++----
 .../spark/sql/catalyst/parser/AstBuilder.scala   |  6 +++---
 .../plans/logical/basicLogicalOperators.scala    |  3 +--
 .../sql/catalyst/analysis/AnalysisSuite.scala    | 16 ++++++++--------
 .../catalyst/catalog/SessionCatalogSuite.scala   |  6 +++---
 .../catalyst/optimizer/ColumnPruningSuite.scala  |  8 ++++----
 .../EliminateSubqueryAliasesSuite.scala          |  6 +++---
 .../optimizer/JoinOptimizationSuite.scala        |  8 ++++----
 .../sql/catalyst/parser/PlanParserSuite.scala    |  2 +-
 .../scala/org/apache/spark/sql/Dataset.scala     |  2 +-
 .../sql/execution/joins/BroadcastJoinSuite.scala |  3 ---
 .../sql/hive/HiveMetastoreCatalogSuite.scala     |  2 +-
 .../spark/sql/hive/execution/SQLQuerySuite.scala |  2 +-
 16 files changed, 42 insertions(+), 46 deletions(-)

diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index ffa5aed30e..93666f1495 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -598,7 +598,7 @@ class Analyzer(
           execute(child)
         }
         view.copy(child = newChild)
-      case p @ SubqueryAlias(_, view: View, _) =>
+      case p @ SubqueryAlias(_, view: View) =>
         val newChild = resolveRelation(view)
         p.copy(child = newChild)
       case _ => plan
@@ -2363,7 +2363,7 @@ class Analyzer(
  */
 object EliminateSubqueryAliases extends Rule[LogicalPlan] {
   def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
-    case SubqueryAlias(_, child, _) => child
+    case SubqueryAlias(_, child) => child
   }
 }
 
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
index 498bfbde9d..831e37aac1 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
@@ -578,7 +578,7 @@ class SessionCatalog(
       val table = formatTableName(name.table)
       if (db == globalTempViewManager.database) {
         globalTempViewManager.get(table).map { viewDef =>
-          SubqueryAlias(table, viewDef, None)
+          SubqueryAlias(table, viewDef)
         }.getOrElse(throw new NoSuchTableException(db, table))
       } else if (name.database.isDefined || !tempTables.contains(table)) {
         val metadata = externalCatalog.getTable(db, table)
@@ -591,17 +591,17 @@ class SessionCatalog(
             desc = metadata,
             output = metadata.schema.toAttributes,
             child = parser.parsePlan(viewText))
-          SubqueryAlias(table, child, Some(name.copy(table = table, database = Some(db))))
+          SubqueryAlias(table, child)
         } else {
           val tableRelation = CatalogRelation(
             metadata,
             // we assume all the columns are nullable.
             metadata.dataSchema.asNullable.toAttributes,
             metadata.partitionSchema.asNullable.toAttributes)
-          SubqueryAlias(table, tableRelation, None)
+          SubqueryAlias(table, tableRelation)
         }
       } else {
-        SubqueryAlias(table, tempTables(table), None)
+        SubqueryAlias(table, tempTables(table))
       }
     }
   }
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
index c062e4e84b..0f0d90494f 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
@@ -346,7 +346,7 @@ package object dsl {
           orderSpec: Seq[SortOrder]): LogicalPlan =
         Window(windowExpressions, partitionSpec, orderSpec, logicalPlan)
 
-      def subquery(alias: Symbol): LogicalPlan = SubqueryAlias(alias.name, logicalPlan, None)
+      def subquery(alias: Symbol): LogicalPlan = SubqueryAlias(alias.name, logicalPlan)
 
       def except(otherPlan: LogicalPlan): LogicalPlan = Except(logicalPlan, otherPlan)
 
@@ -368,7 +368,7 @@ package object dsl {
           analysis.UnresolvedRelation(TableIdentifier(tableName)),
           Map.empty, logicalPlan, overwrite, false)
 
-      def as(alias: String): LogicalPlan = SubqueryAlias(alias, logicalPlan, None)
+      def as(alias: String): LogicalPlan = SubqueryAlias(alias, logicalPlan)
 
       def repartition(num: Integer): LogicalPlan =
         Repartition(num, shuffle = true, logicalPlan)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala
index 4d62cce9da..fb7ce6aece 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala
@@ -169,7 +169,7 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
     // and Project operators, followed by an optional Filter, followed by an
     // Aggregate. Traverse the operators recursively.
     def evalPlan(lp : LogicalPlan) : Map[ExprId, Option[Any]] = lp match {
-      case SubqueryAlias(_, child, _) => evalPlan(child)
+      case SubqueryAlias(_, child) => evalPlan(child)
       case Filter(condition, child) =>
         val bindings = evalPlan(child)
         if (bindings.isEmpty) bindings
@@ -227,7 +227,7 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
           topPart += p
           bottomPart = child
 
-        case s @ SubqueryAlias(_, child, _) =>
+        case s @ SubqueryAlias(_, child) =>
           topPart += s
           bottomPart = child
 
@@ -298,8 +298,8 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] {
             topPart.reverse.foreach {
               case Project(projList, _) =>
                 subqueryRoot = Project(projList ++ havingInputs, subqueryRoot)
-              case s @ SubqueryAlias(alias, _, None) =>
-                subqueryRoot = SubqueryAlias(alias, subqueryRoot, None)
+              case s @ SubqueryAlias(alias, _) =>
+                subqueryRoot = SubqueryAlias(alias, subqueryRoot)
               case op => sys.error(s"Unexpected operator $op in corelated subquery")
             }
 
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
index d2e091f4dd..3cf11adc19 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
@@ -108,7 +108,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
    * This is only used for Common Table Expressions.
    */
   override def visitNamedQuery(ctx: NamedQueryContext): SubqueryAlias = withOrigin(ctx) {
-    SubqueryAlias(ctx.name.getText, plan(ctx.query), None)
+    SubqueryAlias(ctx.name.getText, plan(ctx.query))
   }
 
   /**
@@ -666,7 +666,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
 
     val tableWithAlias = Option(ctx.strictIdentifier).map(_.getText) match {
       case Some(strictIdentifier) =>
-        SubqueryAlias(strictIdentifier, table, None)
+        SubqueryAlias(strictIdentifier, table)
       case _ => table
     }
     tableWithAlias.optionalMap(ctx.sample)(withSample)
@@ -731,7 +731,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
    * Create an alias (SubqueryAlias) for a LogicalPlan.
    */
   private def aliasPlan(alias: ParserRuleContext, plan: LogicalPlan): LogicalPlan = {
-    SubqueryAlias(alias.getText, plan, None)
+    SubqueryAlias(alias.getText, plan)
   }
 
   /**
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
index 4d27ff2acd..70c5ed4b07 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
@@ -792,8 +792,7 @@ case class LocalLimit(limitExpr: Expression, child: LogicalPlan) extends UnaryNo
 
 case class SubqueryAlias(
     alias: String,
-    child: LogicalPlan,
-    view: Option[TableIdentifier])
+    child: LogicalPlan)
   extends UnaryNode {
 
   override def output: Seq[Attribute] = child.output.map(_.withQualifier(Some(alias)))
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
index 01737e0a17..893bb1b74c 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
@@ -62,23 +62,23 @@ class AnalysisSuite extends AnalysisTest with ShouldMatchers {
 
     checkAnalysis(
       Project(Seq(UnresolvedAttribute("TbL.a")),
-        SubqueryAlias("TbL", UnresolvedRelation(TableIdentifier("TaBlE")), None)),
+        SubqueryAlias("TbL", UnresolvedRelation(TableIdentifier("TaBlE")))),
       Project(testRelation.output, testRelation))
 
     assertAnalysisError(
       Project(Seq(UnresolvedAttribute("tBl.a")),
-        SubqueryAlias("TbL", UnresolvedRelation(TableIdentifier("TaBlE")), None)),
+        SubqueryAlias("TbL", UnresolvedRelation(TableIdentifier("TaBlE")))),
       Seq("cannot resolve"))
 
     checkAnalysis(
       Project(Seq(UnresolvedAttribute("TbL.a")),
-        SubqueryAlias("TbL", UnresolvedRelation(TableIdentifier("TaBlE")), None)),
+        SubqueryAlias("TbL", UnresolvedRelation(TableIdentifier("TaBlE")))),
       Project(testRelation.output, testRelation),
       caseSensitive = false)
 
     checkAnalysis(
       Project(Seq(UnresolvedAttribute("tBl.a")),
-        SubqueryAlias("TbL", UnresolvedRelation(TableIdentifier("TaBlE")), None)),
+        SubqueryAlias("TbL", UnresolvedRelation(TableIdentifier("TaBlE")))),
       Project(testRelation.output, testRelation),
       caseSensitive = false)
   }
@@ -374,8 +374,8 @@ class AnalysisSuite extends AnalysisTest with ShouldMatchers {
     val query =
       Project(Seq($"x.key", $"y.key"),
         Join(
-          Project(Seq($"x.key"), SubqueryAlias("x", input, None)),
-          Project(Seq($"y.key"), SubqueryAlias("y", input, None)),
+          Project(Seq($"x.key"), SubqueryAlias("x", input)),
+          Project(Seq($"y.key"), SubqueryAlias("y", input)),
           Cross, None))
 
     assertAnalysisSuccess(query)
@@ -435,10 +435,10 @@ class AnalysisSuite extends AnalysisTest with ShouldMatchers {
   test("resolve as with an already existed alias") {
     checkAnalysis(
       Project(Seq(UnresolvedAttribute("tbl2.a")),
-        SubqueryAlias("tbl", testRelation, None).as("tbl2")),
+        SubqueryAlias("tbl", testRelation).as("tbl2")),
       Project(testRelation.output, testRelation),
       caseSensitive = false)
 
-    checkAnalysis(SubqueryAlias("tbl", testRelation, None).as("tbl2"), testRelation)
+    checkAnalysis(SubqueryAlias("tbl", testRelation).as("tbl2"), testRelation)
   }
 }
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala
index ffc272c6c0..328a16c4bf 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala
@@ -437,7 +437,7 @@ class SessionCatalogSuite extends PlanTest {
       .asInstanceOf[CatalogRelation].tableMeta == metastoreTable1)
     // Otherwise, we'll first look up a temporary table with the same name
     assert(sessionCatalog.lookupRelation(TableIdentifier("tbl1"))
-      == SubqueryAlias("tbl1", tempTable1, None))
+      == SubqueryAlias("tbl1", tempTable1))
     // Then, if that does not exist, look up the relation in the current database
     sessionCatalog.dropTable(TableIdentifier("tbl1"), ignoreIfNotExists = false, purge = false)
     assert(sessionCatalog.lookupRelation(TableIdentifier("tbl1")).children.head
@@ -454,11 +454,11 @@ class SessionCatalogSuite extends PlanTest {
     val view = View(desc = metadata, output = metadata.schema.toAttributes,
       child = CatalystSqlParser.parsePlan(metadata.viewText.get))
     comparePlans(sessionCatalog.lookupRelation(TableIdentifier("view1", Some("db3"))),
-      SubqueryAlias("view1", view, Some(TableIdentifier("view1", Some("db3")))))
+      SubqueryAlias("view1", view))
     // Look up a view using current database of the session catalog.
     sessionCatalog.setCurrentDatabase("db3")
     comparePlans(sessionCatalog.lookupRelation(TableIdentifier("view1")),
-      SubqueryAlias("view1", view, Some(TableIdentifier("view1", Some("db3")))))
+      SubqueryAlias("view1", view))
   }
 
   test("table exists") {
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala
index 5bd1bc80c3..589607e3ad 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala
@@ -320,16 +320,16 @@ class ColumnPruningSuite extends PlanTest {
     val query =
       Project(Seq($"x.key", $"y.key"),
         Join(
-          SubqueryAlias("x", input, None),
-          BroadcastHint(SubqueryAlias("y", input, None)), Inner, None)).analyze
+          SubqueryAlias("x", input),
+          BroadcastHint(SubqueryAlias("y", input)), Inner, None)).analyze
 
     val optimized = Optimize.execute(query)
 
     val expected =
       Join(
-        Project(Seq($"x.key"), SubqueryAlias("x", input, None)),
+        Project(Seq($"x.key"), SubqueryAlias("x", input)),
         BroadcastHint(
-          Project(Seq($"y.key"), SubqueryAlias("y", input, None))),
+          Project(Seq($"y.key"), SubqueryAlias("y", input))),
         Inner, None).analyze
 
     comparePlans(optimized, expected)
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSubqueryAliasesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSubqueryAliasesSuite.scala
index a8aeedbd62..9b6d68aee8 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSubqueryAliasesSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSubqueryAliasesSuite.scala
@@ -46,13 +46,13 @@ class EliminateSubqueryAliasesSuite extends PlanTest with PredicateHelper {
 
   test("eliminate top level subquery") {
     val input = LocalRelation('a.int, 'b.int)
-    val query = SubqueryAlias("a", input, None)
+    val query = SubqueryAlias("a", input)
     comparePlans(afterOptimization(query), input)
   }
 
   test("eliminate mid-tree subquery") {
     val input = LocalRelation('a.int, 'b.int)
-    val query = Filter(TrueLiteral, SubqueryAlias("a", input, None))
+    val query = Filter(TrueLiteral, SubqueryAlias("a", input))
     comparePlans(
       afterOptimization(query),
       Filter(TrueLiteral, LocalRelation('a.int, 'b.int)))
@@ -61,7 +61,7 @@ class EliminateSubqueryAliasesSuite extends PlanTest with PredicateHelper {
   test("eliminate multiple subqueries") {
     val input = LocalRelation('a.int, 'b.int)
     val query = Filter(TrueLiteral,
-      SubqueryAlias("c", SubqueryAlias("b", SubqueryAlias("a", input, None), None), None))
+      SubqueryAlias("c", SubqueryAlias("b", SubqueryAlias("a", input))))
     comparePlans(
       afterOptimization(query),
       Filter(TrueLiteral, LocalRelation('a.int, 'b.int)))
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala
index 65dd6225ce..985e49069d 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala
@@ -129,15 +129,15 @@ class JoinOptimizationSuite extends PlanTest {
     val query =
       Project(Seq($"x.key", $"y.key"),
         Join(
-          SubqueryAlias("x", input, None),
-          BroadcastHint(SubqueryAlias("y", input, None)), Cross, None)).analyze
+          SubqueryAlias("x", input),
+          BroadcastHint(SubqueryAlias("y", input)), Cross, None)).analyze
 
     val optimized = Optimize.execute(query)
 
     val expected =
       Join(
-        Project(Seq($"x.key"), SubqueryAlias("x", input, None)),
-        BroadcastHint(Project(Seq($"y.key"), SubqueryAlias("y", input, None))),
+        Project(Seq($"x.key"), SubqueryAlias("x", input)),
+        BroadcastHint(Project(Seq($"y.key"), SubqueryAlias("y", input))),
         Cross, None).analyze
 
     comparePlans(optimized, expected)
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala
index 67d5d2202b..411777d6e8 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala
@@ -79,7 +79,7 @@ class PlanParserSuite extends PlanTest {
     def cte(plan: LogicalPlan, namedPlans: (String, LogicalPlan)*): With = {
       val ctes = namedPlans.map {
         case (name, cte) =>
-          name -> SubqueryAlias(name, cte, None)
+          name -> SubqueryAlias(name, cte)
       }
       With(plan, ctes)
     }
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
index 1b04623596..f00311fc32 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
@@ -1093,7 +1093,7 @@ class Dataset[T] private[sql](
    * @since 1.6.0
    */
   def as(alias: String): Dataset[T] = withTypedPlan {
-    SubqueryAlias(alias, logicalPlan, None)
+    SubqueryAlias(alias, logicalPlan)
   }
 
   /**
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala
index 9c55357ab9..26c45e092d 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala
@@ -22,15 +22,12 @@ import scala.reflect.ClassTag
 import org.apache.spark.AccumulatorSuite
 import org.apache.spark.sql.{Dataset, QueryTest, Row, SparkSession}
 import org.apache.spark.sql.catalyst.expressions.{BitwiseAnd, BitwiseOr, Cast, Literal, ShiftLeft}
-import org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias
-import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.execution.exchange.EnsureRequirements
 import org.apache.spark.sql.execution.SparkPlan
 import org.apache.spark.sql.functions._
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.test.SQLTestUtils
 import org.apache.spark.sql.types.{LongType, ShortType}
-import org.apache.spark.util.Utils
 
 /**
  * Test various broadcast join operators.
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala
index 892a22ddfa..cf552b4a88 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala
@@ -64,7 +64,7 @@ class HiveMetastoreCatalogSuite extends TestHiveSingleton with SQLTestUtils {
       spark.sql("create view vw1 as select 1 as id")
       val plan = spark.sql("select id from vw1").queryExecution.analyzed
       val aliases = plan.collect {
-        case x @ SubqueryAlias("vw1", _, Some(TableIdentifier("vw1", Some("default")))) => x
+        case x @ SubqueryAlias("vw1", _) => x
       }
       assert(aliases.size == 1)
     }
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
index be9a5fd71b..236135dcff 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
@@ -1030,7 +1030,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
     withSQLConf(SQLConf.CONVERT_CTAS.key -> "false") {
       sql("CREATE TABLE explodeTest (key bigInt)")
       table("explodeTest").queryExecution.analyzed match {
-        case SubqueryAlias(_, r: CatalogRelation, _) => // OK
+        case SubqueryAlias(_, r: CatalogRelation) => // OK
         case _ =>
           fail("To correctly test the fix of SPARK-5875, explodeTest should be a MetastoreRelation")
       }
-- 
GitLab