Skip to content

Commit

Permalink
fix add UT
Browse files Browse the repository at this point in the history
  • Loading branch information
AngersZhuuuu committed Aug 24, 2020
1 parent 5b17cd9 commit dabae9b
Show file tree
Hide file tree
Showing 4 changed files with 243 additions and 27 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -753,15 +753,18 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
// expects a seq of pairs in which the old parsers' token names are used as keys.
// Transforming the result of visitRowFormatDelimited would be quite a bit messier than
// retrieving the key value pairs ourselves.
def entry(key: String, value: Token): Seq[(String, String)] = {
Option(value).map(t => key -> t.getText).toSeq
}

val entries = entry("TOK_TABLEROWFORMATFIELD", ctx.fieldsTerminatedBy) ++
entry("TOK_TABLEROWFORMATCOLLITEMS", ctx.collectionItemsTerminatedBy) ++
entry("TOK_TABLEROWFORMATMAPKEYS", ctx.keysTerminatedBy) ++
entry("TOK_TABLEROWFORMATLINES", ctx.linesSeparatedBy) ++
entry("TOK_TABLEROWFORMATNULL", ctx.nullDefinedAs)
entry("TOK_TABLEROWFORMATNULL", ctx.nullDefinedAs) ++
Option(ctx.linesSeparatedBy).toSeq.map { token =>
val value = string(token)
validate(
value == "\n",
s"LINES TERMINATED BY only supports newline '\\n' right now: $value",
ctx)
"TOK_TABLEROWFORMATLINES" -> value
}

(entries, None, Seq.empty, None)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1111,16 +1111,16 @@ class PlanParserSuite extends AnalysisTest {
AttributeReference("c", StringType)()),
UnresolvedRelation(TableIdentifier("testData")),
ScriptInputOutputSchema(
Seq(("TOK_TABLEROWFORMATFIELD", "'\\t'"),
("TOK_TABLEROWFORMATCOLLITEMS", "'\u0002'"),
("TOK_TABLEROWFORMATMAPKEYS", "'\u0003'"),
("TOK_TABLEROWFORMATLINES", "'\\n'"),
("TOK_TABLEROWFORMATNULL", "'null'")),
Seq(("TOK_TABLEROWFORMATFIELD", "'\\t'"),
("TOK_TABLEROWFORMATCOLLITEMS", "'\u0004'"),
("TOK_TABLEROWFORMATMAPKEYS", "'\u0005'"),
("TOK_TABLEROWFORMATLINES", "'\\n'"),
("TOK_TABLEROWFORMATNULL", "'NULL'")), None, None,
Seq(("TOK_TABLEROWFORMATFIELD", "\t"),
("TOK_TABLEROWFORMATCOLLITEMS", "\u0002"),
("TOK_TABLEROWFORMATMAPKEYS", "\u0003"),
("TOK_TABLEROWFORMATNULL", "null"),
("TOK_TABLEROWFORMATLINES", "\n")),
Seq(("TOK_TABLEROWFORMATFIELD", "\t"),
("TOK_TABLEROWFORMATCOLLITEMS", "\u0004"),
("TOK_TABLEROWFORMATMAPKEYS", "\u0005"),
("TOK_TABLEROWFORMATNULL", "NULL"),
("TOK_TABLEROWFORMATLINES", "\n")), None, None,
List.empty, List.empty, None, None, false)))

// verify with ROW FORMAT SERDE
Expand Down
89 changes: 85 additions & 4 deletions sql/core/src/test/resources/sql-tests/inputs/transform.sql
Original file line number Diff line number Diff line change
Expand Up @@ -91,24 +91,105 @@ REDUCE a, b USING 'cat' AS (a, b) FROM t;
-- transform with defined row format delimit
SELECT TRANSFORM(a, b, c, null)
ROW FORMAT DELIMITED
FIELDS TERMINATED BY '|'
FIELDS TERMINATED BY '@'
LINES TERMINATED BY '\n'
NULL DEFINED AS 'NULL'
USING 'cat' AS (a, b, c, d)
ROW FORMAT DELIMITED
FIELDS TERMINATED BY '|'
FIELDS TERMINATED BY '@'
LINES TERMINATED BY '\n'
NULL DEFINED AS 'NULL'
FROM t;

SELECT TRANSFORM(a, b, c, null)
ROW FORMAT DELIMITED
FIELDS TERMINATED BY '|'
FIELDS TERMINATED BY '@'
LINES TERMINATED BY '\n'
NULL DEFINED AS 'NULL'
USING 'cat' AS (d)
ROW FORMAT DELIMITED
FIELDS TERMINATED BY '||'
FIELDS TERMINATED BY '@'
LINES TERMINATED BY '\n'
NULL DEFINED AS 'NULL'
FROM t;

-- transform with defined row format delimit handle schema with correct type
SELECT a, b, decode(c, 'UTF-8'), d, e, f, g, h, i, j, k, l FROM (
SELECT TRANSFORM(a, b, c, d, e, f, g, h, i, j, k, l)
ROW FORMAT DELIMITED
FIELDS TERMINATED BY ','
LINES TERMINATED BY '\n'
NULL DEFINED AS 'NULL'
USING 'cat' AS (
a string,
b boolean,
c binary,
d tinyint,
e int,
f smallint,
g long,
h float,
i double,
j decimal(38, 18),
k timestamp,
l date)
ROW FORMAT DELIMITED
FIELDS TERMINATED BY ','
LINES TERMINATED BY '\n'
NULL DEFINED AS 'NULL'
FROM t
) tmp;

-- transform with defined row format delimit handle schema with wrong type
SELECT a, b, decode(c, 'UTF-8'), d, e, f, g, h, i, j, k, l FROM (
SELECT TRANSFORM(a, b, c, d, e, f, g, h, i, j, k, l)
ROW FORMAT DELIMITED
FIELDS TERMINATED BY ','
LINES TERMINATED BY '\n'
NULL DEFINED AS 'NULL'
USING 'cat' AS (
a string,
b long,
c binary,
d tinyint,
e int,
f smallint,
g long,
h float,
i double,
j decimal(38, 18),
k int,
l long)
ROW FORMAT DELIMITED
FIELDS TERMINATED BY ','
LINES TERMINATED BY '\n'
NULL DEFINED AS 'NULL'
FROM t
) tmp;

-- transform with defined row format delimit LINE TERMINATED BY only support '\n'
SELECT a, b, decode(c, 'UTF-8'), d, e, f, g, h, i, j, k, l FROM (
SELECT TRANSFORM(a, b, c, d, e, f, g, h, i, j, k, l)
ROW FORMAT DELIMITED
FIELDS TERMINATED BY ','
LINES TERMINATED BY '@'
NULL DEFINED AS 'NULL'
USING 'cat' AS (
a string,
b string,
c string,
d string,
e string,
f string,
g string,
h string,
i string,
j string,
k string,
l string)
ROW FORMAT DELIMITED
FIELDS TERMINATED BY ','
LINES TERMINATED BY '@'
NULL DEFINED AS 'NULL'
FROM t
) tmp;
146 changes: 139 additions & 7 deletions sql/core/src/test/resources/sql-tests/results/transform.sql.out
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
-- Automatically generated by SQLQueryTestSuite
-- Number of queries: 15
-- Number of queries: 18


-- !query
Expand Down Expand Up @@ -189,31 +189,32 @@ struct<a:string,b:string>
-- !query
SELECT TRANSFORM(a, b, c, null)
ROW FORMAT DELIMITED
FIELDS TERMINATED BY '|'
FIELDS TERMINATED BY '@'
LINES TERMINATED BY '\n'
NULL DEFINED AS 'NULL'
USING 'cat' AS (a, b, c, d)
ROW FORMAT DELIMITED
FIELDS TERMINATED BY '|'
FIELDS TERMINATED BY '@'
LINES TERMINATED BY '\n'
NULL DEFINED AS 'NULL'
FROM t
-- !query schema
struct<a:string,b:string,c:string,d:string>
-- !query output
1 | true |
2 | false |
1 true Spark SQL null
2 false Spark SQL null
3 true Spark SQL null


-- !query
SELECT TRANSFORM(a, b, c, null)
ROW FORMAT DELIMITED
FIELDS TERMINATED BY '|'
FIELDS TERMINATED BY '@'
LINES TERMINATED BY '\n'
NULL DEFINED AS 'NULL'
USING 'cat' AS (d)
ROW FORMAT DELIMITED
FIELDS TERMINATED BY '||'
FIELDS TERMINATED BY '@'
LINES TERMINATED BY '\n'
NULL DEFINED AS 'NULL'
FROM t
Expand All @@ -222,3 +223,134 @@ struct<d:string>
-- !query output
1
2
3


-- !query
SELECT a, b, decode(c, 'UTF-8'), d, e, f, g, h, i, j, k, l FROM (
SELECT TRANSFORM(a, b, c, d, e, f, g, h, i, j, k, l)
ROW FORMAT DELIMITED
FIELDS TERMINATED BY ','
LINES TERMINATED BY '\n'
NULL DEFINED AS 'NULL'
USING 'cat' AS (
a string,
b boolean,
c binary,
d tinyint,
e int,
f smallint,
g long,
h float,
i double,
j decimal(38, 18),
k timestamp,
l date)
ROW FORMAT DELIMITED
FIELDS TERMINATED BY ','
LINES TERMINATED BY '\n'
NULL DEFINED AS 'NULL'
FROM t
) tmp
-- !query schema
struct<a:string,b:boolean,decode(c, UTF-8):string,d:tinyint,e:int,f:smallint,g:bigint,h:float,i:double,j:decimal(38,18),k:timestamp,l:date>
-- !query output
1 true Spark SQL 1 1 100 1 1.0 1.0 1.000000000000000000 1997-01-02 00:00:00 2000-04-01
2 false Spark SQL 2 2 200 2 2.0 2.0 2.000000000000000000 1997-01-02 03:04:05 2000-04-02
3 true Spark SQL 3 3 300 3 3.0 3.0 3.000000000000000000 1997-02-10 17:32:01 2000-04-03


-- !query
SELECT a, b, decode(c, 'UTF-8'), d, e, f, g, h, i, j, k, l FROM (
SELECT TRANSFORM(a, b, c, d, e, f, g, h, i, j, k, l)
ROW FORMAT DELIMITED
FIELDS TERMINATED BY ','
LINES TERMINATED BY '\n'
NULL DEFINED AS 'NULL'
USING 'cat' AS (
a string,
b long,
c binary,
d tinyint,
e int,
f smallint,
g long,
h float,
i double,
j decimal(38, 18),
k int,
l long)
ROW FORMAT DELIMITED
FIELDS TERMINATED BY ','
LINES TERMINATED BY '\n'
NULL DEFINED AS 'NULL'
FROM t
) tmp
-- !query schema
struct<a:string,b:bigint,decode(c, UTF-8):string,d:tinyint,e:int,f:smallint,g:bigint,h:float,i:double,j:decimal(38,18),k:int,l:bigint>
-- !query output
1 NULL Spark SQL 1 1 100 1 1.0 1.0 1.000000000000000000 NULL NULL
2 NULL Spark SQL 2 2 200 2 2.0 2.0 2.000000000000000000 NULL NULL
3 NULL Spark SQL 3 3 300 3 3.0 3.0 3.000000000000000000 NULL NULL


-- !query
SELECT a, b, decode(c, 'UTF-8'), d, e, f, g, h, i, j, k, l FROM (
SELECT TRANSFORM(a, b, c, d, e, f, g, h, i, j, k, l)
ROW FORMAT DELIMITED
FIELDS TERMINATED BY ','
LINES TERMINATED BY '@'
NULL DEFINED AS 'NULL'
USING 'cat' AS (
a string,
b string,
c string,
d string,
e string,
f string,
g string,
h string,
i string,
j string,
k string,
l string)
ROW FORMAT DELIMITED
FIELDS TERMINATED BY ','
LINES TERMINATED BY '@'
NULL DEFINED AS 'NULL'
FROM t
) tmp
-- !query schema
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException

LINES TERMINATED BY only supports newline '\n' right now: @(line 3, pos 4)

== SQL ==
SELECT a, b, decode(c, 'UTF-8'), d, e, f, g, h, i, j, k, l FROM (
SELECT TRANSFORM(a, b, c, d, e, f, g, h, i, j, k, l)
ROW FORMAT DELIMITED
----^^^
FIELDS TERMINATED BY ','
LINES TERMINATED BY '@'
NULL DEFINED AS 'NULL'
USING 'cat' AS (
a string,
b string,
c string,
d string,
e string,
f string,
g string,
h string,
i string,
j string,
k string,
l string)
ROW FORMAT DELIMITED
FIELDS TERMINATED BY ','
LINES TERMINATED BY '@'
NULL DEFINED AS 'NULL'
FROM t
) tmp

0 comments on commit dabae9b

Please sign in to comment.