Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
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
Next Next commit
initial commit
  • Loading branch information
imback82 committed Nov 23, 2020
commit fd2eb372642c55c4823702536c76bfd03fd1f208
Original file line number Diff line number Diff line change
Expand Up @@ -866,11 +866,12 @@ class Analyzer(override val catalogManager: CatalogManager)
u.failAnalysis(s"${ident.quoted} is a temp view. '$cmd' expects a table")
}
u
case u @ UnresolvedTableOrView(ident, allowTempView) =>
case u @ UnresolvedTableOrView(ident, cmd, allowTempView) =>
lookupTempView(ident)
.map { _ =>
if (!allowTempView) {
u.failAnalysis(s"${ident.quoted} is a temp view not table or permanent view.")
u.failAnalysis(
s"${ident.quoted} is a temp view. '$cmd' expects a table or permanent view.")
}
ResolvedView(ident.asIdentifier, isTemp = true)
}
Expand Down Expand Up @@ -955,7 +956,7 @@ class Analyzer(override val catalogManager: CatalogManager)
.map(ResolvedTable(catalog.asTableCatalog, ident, _))
.getOrElse(u)

case u @ UnresolvedTableOrView(NonSessionCatalogAndIdentifier(catalog, ident), _) =>
case u @ UnresolvedTableOrView(NonSessionCatalogAndIdentifier(catalog, ident), _, _) =>
CatalogV2Util.loadTable(catalog, ident)
.map(ResolvedTable(catalog.asTableCatalog, ident, _))
.getOrElse(u)
Expand Down Expand Up @@ -1085,7 +1086,7 @@ class Analyzer(override val catalogManager: CatalogManager)
case table => table
}.getOrElse(u)

case u @ UnresolvedTableOrView(identifier, _) =>
case u @ UnresolvedTableOrView(identifier, _, _) =>
lookupTableOrView(identifier).getOrElse(u)
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -101,7 +101,9 @@ trait CheckAnalysis extends PredicateHelper {
u.failAnalysis(s"Table not found for '${u.commandName}': ${u.multipartIdentifier.quoted}")

case u: UnresolvedTableOrView =>
u.failAnalysis(s"Table or view not found: ${u.multipartIdentifier.quoted}")
val viewStr = if (u.allowTempView) "view" else "permanent view"
u.failAnalysis(
s"Table or $viewStr not found for '${u.commandName}': ${u.multipartIdentifier.quoted}")

case u: UnresolvedRelation =>
u.failAnalysis(s"Table or view not found: ${u.multipartIdentifier.quoted}")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,7 @@ case class UnresolvedTable(
*/
case class UnresolvedTableOrView(
multipartIdentifier: Seq[String],
commandName: String,
allowTempView: Boolean = true) extends LeafNode {
override lazy val resolved: Boolean = false
override def output: Seq[Attribute] = Nil
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2894,7 +2894,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
override def visitDropTable(ctx: DropTableContext): LogicalPlan = withOrigin(ctx) {
// DROP TABLE works with either a table or a temporary view.
DropTable(
UnresolvedTableOrView(visitMultipartIdentifier(ctx.multipartIdentifier())),
UnresolvedTableOrView(visitMultipartIdentifier(ctx.multipartIdentifier()), "DROP TABLE"),
ctx.EXISTS != null,
ctx.PURGE != null)
}
Expand Down Expand Up @@ -3199,12 +3199,15 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
*/
override def visitDescribeRelation(ctx: DescribeRelationContext): LogicalPlan = withOrigin(ctx) {
val isExtended = ctx.EXTENDED != null || ctx.FORMATTED != null
val relation = UnresolvedTableOrView(
visitMultipartIdentifier(ctx.multipartIdentifier()),
"DESCRIBE TABLE")
if (ctx.describeColName != null) {
if (ctx.partitionSpec != null) {
throw new ParseException("DESC TABLE COLUMN for a specific partition is not supported", ctx)
} else {
DescribeColumn(
UnresolvedTableOrView(visitMultipartIdentifier(ctx.multipartIdentifier())),
relation,
ctx.describeColName.nameParts.asScala.map(_.getText).toSeq,
isExtended)
}
Expand All @@ -3219,10 +3222,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
} else {
Map.empty[String, String]
}
DescribeRelation(
UnresolvedTableOrView(visitMultipartIdentifier(ctx.multipartIdentifier())),
partitionSpec,
isExtended)
DescribeRelation(relation, partitionSpec, isExtended)
}
}

Expand Down Expand Up @@ -3260,21 +3260,24 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
val tableName = visitMultipartIdentifier(ctx.multipartIdentifier())
if (ctx.ALL() != null) {
checkPartitionSpec()
AnalyzeColumn(UnresolvedTableOrView(tableName), None, allColumns = true)
AnalyzeColumn(
UnresolvedTableOrView(tableName, "ANALYZE TABLE ... FOR ALL COLUMNS"),
None,
allColumns = true)
} else if (ctx.identifierSeq() == null) {
val partitionSpec = if (ctx.partitionSpec != null) {
visitPartitionSpec(ctx.partitionSpec)
} else {
Map.empty[String, Option[String]]
}
AnalyzeTable(
UnresolvedTableOrView(tableName, allowTempView = false),
UnresolvedTableOrView(tableName, "ANALYZE TABLE", allowTempView = false),
partitionSpec,
noScan = ctx.identifier != null)
} else {
checkPartitionSpec()
AnalyzeColumn(
UnresolvedTableOrView(tableName),
UnresolvedTableOrView(tableName, "ANALYZE TABLE ... FOR COLUMNS ..."),
Option(visitIdentifierSeq(ctx.identifierSeq())),
allColumns = false)
}
Expand Down Expand Up @@ -3318,6 +3321,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
ShowCreateTable(
UnresolvedTableOrView(
visitMultipartIdentifier(ctx.multipartIdentifier()),
"SHOW CREATE TABLE",
allowTempView = false),
ctx.SERDE != null)
}
Expand Down Expand Up @@ -3393,7 +3397,10 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
* }}}
*/
override def visitRefreshTable(ctx: RefreshTableContext): LogicalPlan = withOrigin(ctx) {
RefreshTable(UnresolvedTableOrView(visitMultipartIdentifier(ctx.multipartIdentifier())))
RefreshTable(
UnresolvedTableOrView(
visitMultipartIdentifier(ctx.multipartIdentifier()),
"REFRESH TABLE"))
}

/**
Expand Down Expand Up @@ -3620,7 +3627,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
override def visitShowTblProperties(
ctx: ShowTblPropertiesContext): LogicalPlan = withOrigin(ctx) {
ShowTableProperties(
UnresolvedTableOrView(visitMultipartIdentifier(ctx.table)),
UnresolvedTableOrView(visitMultipartIdentifier(ctx.table), "SHOW TBLPROPERTIES"),
Option(ctx.key).map(visitTablePropertyKey))
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -381,27 +381,27 @@ class DDLParserSuite extends AnalysisTest {
test("drop table") {
parseCompare("DROP TABLE testcat.ns1.ns2.tbl",
DropTable(
UnresolvedTableOrView(Seq("testcat", "ns1", "ns2", "tbl")),
UnresolvedTableOrView(Seq("testcat", "ns1", "ns2", "tbl"), "DROP TABLE"),
ifExists = false,
purge = false))
parseCompare(s"DROP TABLE db.tab",
DropTable(
UnresolvedTableOrView(Seq("db", "tab")), ifExists = false, purge = false))
UnresolvedTableOrView(Seq("db", "tab"), "DROP TABLE"), ifExists = false, purge = false))
parseCompare(s"DROP TABLE IF EXISTS db.tab",
DropTable(
UnresolvedTableOrView(Seq("db", "tab")), ifExists = true, purge = false))
UnresolvedTableOrView(Seq("db", "tab"), "DROP TABLE"), ifExists = true, purge = false))
parseCompare(s"DROP TABLE tab",
DropTable(
UnresolvedTableOrView(Seq("tab")), ifExists = false, purge = false))
UnresolvedTableOrView(Seq("tab"), "DROP TABLE"), ifExists = false, purge = false))
parseCompare(s"DROP TABLE IF EXISTS tab",
DropTable(
UnresolvedTableOrView(Seq("tab")), ifExists = true, purge = false))
UnresolvedTableOrView(Seq("tab"), "DROP TABLE"), ifExists = true, purge = false))
parseCompare(s"DROP TABLE tab PURGE",
DropTable(
UnresolvedTableOrView(Seq("tab")), ifExists = false, purge = true))
UnresolvedTableOrView(Seq("tab"), "DROP TABLE"), ifExists = false, purge = true))
parseCompare(s"DROP TABLE IF EXISTS tab PURGE",
DropTable(
UnresolvedTableOrView(Seq("tab")), ifExists = true, purge = true))
UnresolvedTableOrView(Seq("tab"), "DROP TABLE"), ifExists = true, purge = true))
}

test("drop view") {
Expand Down Expand Up @@ -796,26 +796,26 @@ class DDLParserSuite extends AnalysisTest {
test("describe table column") {
comparePlans(parsePlan("DESCRIBE t col"),
DescribeColumn(
UnresolvedTableOrView(Seq("t")), Seq("col"), isExtended = false))
UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE"), Seq("col"), isExtended = false))
comparePlans(parsePlan("DESCRIBE t `abc.xyz`"),
DescribeColumn(
UnresolvedTableOrView(Seq("t")), Seq("abc.xyz"), isExtended = false))
UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE"), Seq("abc.xyz"), isExtended = false))
comparePlans(parsePlan("DESCRIBE t abc.xyz"),
DescribeColumn(
UnresolvedTableOrView(Seq("t")), Seq("abc", "xyz"), isExtended = false))
UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE"), Seq("abc", "xyz"), isExtended = false))
comparePlans(parsePlan("DESCRIBE t `a.b`.`x.y`"),
DescribeColumn(
UnresolvedTableOrView(Seq("t")), Seq("a.b", "x.y"), isExtended = false))
UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE"), Seq("a.b", "x.y"), isExtended = false))

comparePlans(parsePlan("DESCRIBE TABLE t col"),
DescribeColumn(
UnresolvedTableOrView(Seq("t")), Seq("col"), isExtended = false))
UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE"), Seq("col"), isExtended = false))
comparePlans(parsePlan("DESCRIBE TABLE EXTENDED t col"),
DescribeColumn(
UnresolvedTableOrView(Seq("t")), Seq("col"), isExtended = true))
UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE"), Seq("col"), isExtended = true))
comparePlans(parsePlan("DESCRIBE TABLE FORMATTED t col"),
DescribeColumn(
UnresolvedTableOrView(Seq("t")), Seq("col"), isExtended = true))
UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE"), Seq("col"), isExtended = true))

val caught = intercept[AnalysisException](
parsePlan("DESCRIBE TABLE t PARTITION (ds='1970-01-01') col"))
Expand All @@ -834,13 +834,17 @@ class DDLParserSuite extends AnalysisTest {

test("SPARK-17328 Fix NPE with EXPLAIN DESCRIBE TABLE") {
comparePlans(parsePlan("describe t"),
DescribeRelation(UnresolvedTableOrView(Seq("t")), Map.empty, isExtended = false))
DescribeRelation(
UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE"), Map.empty, isExtended = false))
comparePlans(parsePlan("describe table t"),
DescribeRelation(UnresolvedTableOrView(Seq("t")), Map.empty, isExtended = false))
DescribeRelation(
UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE"), Map.empty, isExtended = false))
comparePlans(parsePlan("describe table extended t"),
DescribeRelation(UnresolvedTableOrView(Seq("t")), Map.empty, isExtended = true))
DescribeRelation(
UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE"), Map.empty, isExtended = true))
comparePlans(parsePlan("describe table formatted t"),
DescribeRelation(UnresolvedTableOrView(Seq("t")), Map.empty, isExtended = true))
DescribeRelation(
UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE"), Map.empty, isExtended = true))
}

test("insert table: basic append") {
Expand Down Expand Up @@ -1453,57 +1457,57 @@ class DDLParserSuite extends AnalysisTest {
test("analyze table statistics") {
comparePlans(parsePlan("analyze table a.b.c compute statistics"),
AnalyzeTable(
UnresolvedTableOrView(Seq("a", "b", "c"), allowTempView = false),
UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE", allowTempView = false),
Map.empty, noScan = false))
comparePlans(parsePlan("analyze table a.b.c compute statistics noscan"),
AnalyzeTable(
UnresolvedTableOrView(Seq("a", "b", "c"), allowTempView = false),
UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE", allowTempView = false),
Map.empty, noScan = true))
comparePlans(parsePlan("analyze table a.b.c partition (a) compute statistics nOscAn"),
AnalyzeTable(
UnresolvedTableOrView(Seq("a", "b", "c"), allowTempView = false),
UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE", allowTempView = false),
Map("a" -> None), noScan = true))

// Partitions specified
comparePlans(
parsePlan("ANALYZE TABLE a.b.c PARTITION(ds='2008-04-09', hr=11) COMPUTE STATISTICS"),
AnalyzeTable(
UnresolvedTableOrView(Seq("a", "b", "c"), allowTempView = false),
UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE", allowTempView = false),
Map("ds" -> Some("2008-04-09"), "hr" -> Some("11")), noScan = false))
comparePlans(
parsePlan("ANALYZE TABLE a.b.c PARTITION(ds='2008-04-09', hr=11) COMPUTE STATISTICS noscan"),
AnalyzeTable(
UnresolvedTableOrView(Seq("a", "b", "c"), allowTempView = false),
UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE", allowTempView = false),
Map("ds" -> Some("2008-04-09"), "hr" -> Some("11")), noScan = true))
comparePlans(
parsePlan("ANALYZE TABLE a.b.c PARTITION(ds='2008-04-09') COMPUTE STATISTICS noscan"),
AnalyzeTable(
UnresolvedTableOrView(Seq("a", "b", "c"), allowTempView = false),
UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE", allowTempView = false),
Map("ds" -> Some("2008-04-09")), noScan = true))
comparePlans(
parsePlan("ANALYZE TABLE a.b.c PARTITION(ds='2008-04-09', hr) COMPUTE STATISTICS"),
AnalyzeTable(
UnresolvedTableOrView(Seq("a", "b", "c"), allowTempView = false),
UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE", allowTempView = false),
Map("ds" -> Some("2008-04-09"), "hr" -> None), noScan = false))
comparePlans(
parsePlan("ANALYZE TABLE a.b.c PARTITION(ds='2008-04-09', hr) COMPUTE STATISTICS noscan"),
AnalyzeTable(
UnresolvedTableOrView(Seq("a", "b", "c"), allowTempView = false),
UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE", allowTempView = false),
Map("ds" -> Some("2008-04-09"), "hr" -> None), noScan = true))
comparePlans(
parsePlan("ANALYZE TABLE a.b.c PARTITION(ds, hr=11) COMPUTE STATISTICS noscan"),
AnalyzeTable(
UnresolvedTableOrView(Seq("a", "b", "c"), allowTempView = false),
UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE", allowTempView = false),
Map("ds" -> None, "hr" -> Some("11")), noScan = true))
comparePlans(
parsePlan("ANALYZE TABLE a.b.c PARTITION(ds, hr) COMPUTE STATISTICS"),
AnalyzeTable(
UnresolvedTableOrView(Seq("a", "b", "c"), allowTempView = false),
UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE", allowTempView = false),
Map("ds" -> None, "hr" -> None), noScan = false))
comparePlans(
parsePlan("ANALYZE TABLE a.b.c PARTITION(ds, hr) COMPUTE STATISTICS noscan"),
AnalyzeTable(
UnresolvedTableOrView(Seq("a", "b", "c"), allowTempView = false),
UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE", allowTempView = false),
Map("ds" -> None, "hr" -> None), noScan = true))

intercept("analyze table a.b.c compute statistics xxxx",
Expand All @@ -1518,7 +1522,9 @@ class DDLParserSuite extends AnalysisTest {
comparePlans(
parsePlan("ANALYZE TABLE a.b.c COMPUTE STATISTICS FOR COLUMNS key, value"),
AnalyzeColumn(
UnresolvedTableOrView(Seq("a", "b", "c")), Option(Seq("key", "value")), allColumns = false))
UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE ... FOR COLUMNS ..."),
Option(Seq("key", "value")),
allColumns = false))

// Partition specified - should be ignored
comparePlans(
Expand All @@ -1528,7 +1534,9 @@ class DDLParserSuite extends AnalysisTest {
|COMPUTE STATISTICS FOR COLUMNS key, value
""".stripMargin),
AnalyzeColumn(
UnresolvedTableOrView(Seq("a", "b", "c")), Option(Seq("key", "value")), allColumns = false))
UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE ... FOR COLUMNS ..."),
Option(Seq("key", "value")),
allColumns = false))

// Partition specified should be ignored in case of COMPUTE STATISTICS FOR ALL COLUMNS
comparePlans(
Expand All @@ -1538,7 +1546,9 @@ class DDLParserSuite extends AnalysisTest {
|COMPUTE STATISTICS FOR ALL COLUMNS
""".stripMargin),
AnalyzeColumn(
UnresolvedTableOrView(Seq("a", "b", "c")), None, allColumns = true))
UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE ... FOR ALL COLUMNS"),
None,
allColumns = true))

intercept("ANALYZE TABLE a.b.c COMPUTE STATISTICS FOR ALL COLUMNS key, value",
"mismatched input 'key' expecting {<EOF>, ';'}")
Expand Down Expand Up @@ -1582,12 +1592,13 @@ class DDLParserSuite extends AnalysisTest {
test("SHOW CREATE table") {
comparePlans(
parsePlan("SHOW CREATE TABLE a.b.c"),
ShowCreateTable(UnresolvedTableOrView(Seq("a", "b", "c"), allowTempView = false)))
ShowCreateTable(
UnresolvedTableOrView(Seq("a", "b", "c"), "SHOW CREATE TABLE", allowTempView = false)))

comparePlans(
parsePlan("SHOW CREATE TABLE a.b.c AS SERDE"),
ShowCreateTable(
UnresolvedTableOrView(Seq("a", "b", "c"), allowTempView = false),
UnresolvedTableOrView(Seq("a", "b", "c"), "SHOW CREATE TABLE", allowTempView = false),
asSerde = true))
}

Expand Down Expand Up @@ -1631,7 +1642,7 @@ class DDLParserSuite extends AnalysisTest {
test("REFRESH TABLE") {
comparePlans(
parsePlan("REFRESH TABLE a.b.c"),
RefreshTable(UnresolvedTableOrView(Seq("a", "b", "c"))))
RefreshTable(UnresolvedTableOrView(Seq("a", "b", "c"), "REFRESH TABLE")))
}

test("show columns") {
Expand Down Expand Up @@ -1982,11 +1993,12 @@ class DDLParserSuite extends AnalysisTest {
test("SHOW TBLPROPERTIES table") {
comparePlans(
parsePlan("SHOW TBLPROPERTIES a.b.c"),
ShowTableProperties(UnresolvedTableOrView(Seq("a", "b", "c")), None))
ShowTableProperties(UnresolvedTableOrView(Seq("a", "b", "c"), "SHOW TBLPROPERTIES"), None))

comparePlans(
parsePlan("SHOW TBLPROPERTIES a.b.c('propKey1')"),
ShowTableProperties(UnresolvedTableOrView(Seq("a", "b", "c")), Some("propKey1")))
ShowTableProperties(
UnresolvedTableOrView(Seq("a", "b", "c"), "SHOW TBLPROPERTIES"), Some("propKey1")))
}

test("DESCRIBE FUNCTION") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -540,7 +540,7 @@ struct<plan:string>
-- !query output
== Parsed Logical Plan ==
'DescribeRelation false
+- 'UnresolvedTableOrView [t], true
+- 'UnresolvedTableOrView [t], DESCRIBE TABLE, true

== Analyzed Logical Plan ==
col_name: string, data_type: string, comment: string
Expand Down
Loading