Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
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
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.plans.DescribeCommandSchema
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.util.{escapeSingleQuotedString, quoteIdentifier, CaseInsensitiveMap, CharVarcharUtils}
import org.apache.spark.sql.catalyst.util.{escapeSingleQuotedString, quoteIfNeeded, CaseInsensitiveMap, CharVarcharUtils}
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.TableIdentifierHelper
import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
import org.apache.spark.sql.execution.datasources.DataSource
Expand Down Expand Up @@ -1035,7 +1035,7 @@ trait ShowCreateTableCommandBase {
.map(" COMMENT '" + _ + "'")

// view columns shouldn't have data type info
s"${quoteIdentifier(f.name)}${comment.getOrElse("")}"
s"${quoteIfNeeded(f.name)}${comment.getOrElse("")}"
}
builder ++= concatByMultiLines(viewColumns)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -219,8 +219,8 @@ show create table char_view
struct<createtab_stmt:string>
-- !query output
CREATE VIEW default.char_view (
`c`,
`v`)
c,
v)
AS select * from char_tbl


Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -296,8 +296,8 @@ SHOW CREATE TABLE view_SPARK_30302 AS SERDE
struct<createtab_stmt:string>
-- !query output
CREATE VIEW default.view_SPARK_30302 (
`aaa`,
`bbb`)
aaa,
bbb)
AS SELECT a, b FROM tbl


Expand All @@ -307,8 +307,8 @@ SHOW CREATE TABLE view_SPARK_30302
struct<createtab_stmt:string>
-- !query output
CREATE VIEW default.view_SPARK_30302 (
`aaa`,
`bbb`)
aaa,
bbb)
AS SELECT a, b FROM tbl


Expand Down Expand Up @@ -336,8 +336,8 @@ SHOW CREATE TABLE view_SPARK_30302 AS SERDE
struct<createtab_stmt:string>
-- !query output
CREATE VIEW default.view_SPARK_30302 (
`aaa` COMMENT 'comment with \'quoted text\' for aaa',
`bbb`)
aaa COMMENT 'comment with \'quoted text\' for aaa',
bbb)
COMMENT 'This is a comment with \'quoted text\' for view'
AS SELECT a, b FROM tbl

Expand All @@ -348,8 +348,8 @@ SHOW CREATE TABLE view_SPARK_30302
struct<createtab_stmt:string>
-- !query output
CREATE VIEW default.view_SPARK_30302 (
`aaa` COMMENT 'comment with \'quoted text\' for aaa',
`bbb`)
aaa COMMENT 'comment with \'quoted text\' for aaa',
bbb)
COMMENT 'This is a comment with \'quoted text\' for view'
AS SELECT a, b FROM tbl

Expand Down Expand Up @@ -378,8 +378,8 @@ SHOW CREATE TABLE view_SPARK_30302 AS SERDE
struct<createtab_stmt:string>
-- !query output
CREATE VIEW default.view_SPARK_30302 (
`aaa`,
`bbb`)
aaa,
bbb)
TBLPROPERTIES (
'a' = '1',
'b' = '2')
Expand All @@ -392,8 +392,8 @@ SHOW CREATE TABLE view_SPARK_30302
struct<createtab_stmt:string>
-- !query output
CREATE VIEW default.view_SPARK_30302 (
`aaa`,
`bbb`)
aaa,
bbb)
TBLPROPERTIES (
'a' = '1',
'b' = '2')
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -611,7 +611,7 @@ class PersistedViewTestSuite extends SQLViewTestSuite with SharedSparkSession {
Seq(true, false).foreach { serde =>
withView(viewName) {
createView(viewName, "SELECT 1 AS a")
val expected = s"CREATE VIEW ${formattedViewName(viewName)} ( `a`) AS SELECT 1 AS a"
val expected = s"CREATE VIEW ${formattedViewName(viewName)} ( a) AS SELECT 1 AS a"
assert(getShowCreateDDL(formattedViewName(viewName), serde) == expected)
}
}
Expand All @@ -623,7 +623,7 @@ class PersistedViewTestSuite extends SQLViewTestSuite with SharedSparkSession {
withView(viewName) {
createView(viewName, "SELECT 1 AS a, 2 AS b", Seq("a", "b COMMENT 'b column'"))
val expected = s"CREATE VIEW ${formattedViewName(viewName)}" +
s" ( `a`, `b` COMMENT 'b column') AS SELECT 1 AS a, 2 AS b"
s" ( a, b COMMENT 'b column') AS SELECT 1 AS a, 2 AS b"
assert(getShowCreateDDL(formattedViewName(viewName), serde) == expected)
}
}
Expand All @@ -636,7 +636,7 @@ class PersistedViewTestSuite extends SQLViewTestSuite with SharedSparkSession {
createView(viewName, "SELECT 1 AS c1, '2' AS c2", Seq("c1 COMMENT 'bla'", "c2"),
Seq("COMMENT 'table comment'", "TBLPROPERTIES ( 'prop2' = 'value2', 'prop1' = 'value1')"))

val expected = s"CREATE VIEW ${formattedViewName(viewName)} ( `c1` COMMENT 'bla', `c2`)" +
val expected = s"CREATE VIEW ${formattedViewName(viewName)} ( c1 COMMENT 'bla', c2)" +
" COMMENT 'table comment'" +
" TBLPROPERTIES ( 'prop1' = 'value1', 'prop2' = 'value2')" +
" AS SELECT 1 AS c1, '2' AS c2"
Expand Down