Skip to content

Commit fe3e34d

Browse files
HyukjinKwoncloud-fan
authored andcommitted
[SPARK-28273][SQL][PYTHON] Convert and port 'pgSQL/case.sql' into UDF test base
## What changes were proposed in this pull request? This PR adds some tests converted from `pgSQL/case.sql'` to test UDFs. Please see contribution guide of this umbrella ticket - [SPARK-27921](https://issues.apache.org/jira/browse/SPARK-27921). This PR also contains two minor fixes: 1. Change name of Scala UDF from `UDF:name(...)` to `name(...)` to be consistent with Python' 2. Fix Scala UDF at `IntegratedUDFTestUtils.scala ` to handle `null` in strings. <details><summary>Diff comparing to 'pgSQL/case.sql'</summary> <p> ```diff diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/case.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-case.sql.out index fa078d1..55bef64 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/case.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-case.sql.out -115,7 +115,7 struct<> -- !query 13 SELECT '3' AS `One`, CASE - WHEN 1 < 2 THEN 3 + WHEN CAST(udf(1 < 2) AS boolean) THEN 3 END AS `Simple WHEN` -- !query 13 schema struct<One:string,Simple WHEN:int> -126,10 +126,10 struct<One:string,Simple WHEN:int> -- !query 14 SELECT '<NULL>' AS `One`, CASE - WHEN 1 > 2 THEN 3 + WHEN 1 > 2 THEN udf(3) END AS `Simple default` -- !query 14 schema -struct<One:string,Simple default:int> +struct<One:string,Simple default:string> -- !query 14 output <NULL> NULL -137,17 +137,17 struct<One:string,Simple default:int> -- !query 15 SELECT '3' AS `One`, CASE - WHEN 1 < 2 THEN 3 - ELSE 4 + WHEN udf(1) < 2 THEN udf(3) + ELSE udf(4) END AS `Simple ELSE` -- !query 15 schema -struct<One:string,Simple ELSE:int> +struct<One:string,Simple ELSE:string> -- !query 15 output 3 3 -- !query 16 -SELECT '4' AS `One`, +SELECT udf('4') AS `One`, CASE WHEN 1 > 2 THEN 3 ELSE 4 -159,10 +159,10 struct<One:string,ELSE default:int> -- !query 17 -SELECT '6' AS `One`, +SELECT udf('6') AS `One`, CASE - WHEN 1 > 2 THEN 3 - WHEN 4 < 5 THEN 6 + WHEN CAST(udf(1 > 2) AS boolean) THEN 3 + WHEN udf(4) < 5 THEN 6 ELSE 7 END AS `Two WHEN with default` -- !query 17 schema -173,7 +173,7 struct<One:string,Two WHEN with default:int> -- !query 18 SELECT '7' AS `None`, - CASE WHEN rand() < 0 THEN 1 + CASE WHEN rand() < udf(0) THEN 1 END AS `NULL on no matches` -- !query 18 schema struct<None:string,NULL on no matches:int> -182,36 +182,36 struct<None:string,NULL on no matches:int> -- !query 19 -SELECT CASE WHEN 1=0 THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END +SELECT CASE WHEN CAST(udf(1=0) AS boolean) THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END -- !query 19 schema -struct<CASE WHEN (1 = 0) THEN (CAST(1 AS DOUBLE) / CAST(0 AS DOUBLE)) WHEN (1 = 1) THEN CAST(1 AS DOUBLE) ELSE (CAST(2 AS DOUBLE) / CAST(0 AS DOUBLE)) END:double> +struct<CASE WHEN CAST(udf((1 = 0)) AS BOOLEAN) THEN (CAST(1 AS DOUBLE) / CAST(0 AS DOUBLE)) WHEN (1 = 1) THEN CAST(1 AS DOUBLE) ELSE (CAST(2 AS DOUBLE) / CAST(0 AS DOUBLE)) END:double> -- !query 19 output 1.0 -- !query 20 -SELECT CASE 1 WHEN 0 THEN 1/0 WHEN 1 THEN 1 ELSE 2/0 END +SELECT CASE 1 WHEN 0 THEN 1/udf(0) WHEN 1 THEN 1 ELSE 2/0 END -- !query 20 schema -struct<CASE WHEN (1 = 0) THEN (CAST(1 AS DOUBLE) / CAST(0 AS DOUBLE)) WHEN (1 = 1) THEN CAST(1 AS DOUBLE) ELSE (CAST(2 AS DOUBLE) / CAST(0 AS DOUBLE)) END:double> +struct<CASE WHEN (1 = 0) THEN (CAST(1 AS DOUBLE) / CAST(CAST(udf(0) AS DOUBLE) AS DOUBLE)) WHEN (1 = 1) THEN CAST(1 AS DOUBLE) ELSE (CAST(2 AS DOUBLE) / CAST(0 AS DOUBLE)) END:double> -- !query 20 output 1.0 -- !query 21 -SELECT CASE WHEN i > 100 THEN 1/0 ELSE 0 END FROM case_tbl +SELECT CASE WHEN i > 100 THEN udf(1/0) ELSE udf(0) END FROM case_tbl -- !query 21 schema -struct<CASE WHEN (i > 100) THEN (CAST(1 AS DOUBLE) / CAST(0 AS DOUBLE)) ELSE CAST(0 AS DOUBLE) END:double> +struct<CASE WHEN (i > 100) THEN udf((cast(1 as double) / cast(0 as double))) ELSE udf(0) END:string> -- !query 21 output -0.0 -0.0 -0.0 -0.0 +0 +0 +0 +0 -- !query 22 -SELECT CASE 'a' WHEN 'a' THEN 1 ELSE 2 END +SELECT CASE 'a' WHEN 'a' THEN udf(1) ELSE udf(2) END -- !query 22 schema -struct<CASE WHEN (a = a) THEN 1 ELSE 2 END:int> +struct<CASE WHEN (a = a) THEN udf(1) ELSE udf(2) END:string> -- !query 22 output 1 -283,7 +283,7 big -- !query 27 -SELECT * FROM CASE_TBL WHERE COALESCE(f,i) = 4 +SELECT * FROM CASE_TBL WHERE udf(COALESCE(f,i)) = 4 -- !query 27 schema struct<i:int,f:double> -- !query 27 output -291,7 +291,7 struct<i:int,f:double> -- !query 28 -SELECT * FROM CASE_TBL WHERE NULLIF(f,i) = 2 +SELECT * FROM CASE_TBL WHERE udf(NULLIF(f,i)) = 2 -- !query 28 schema struct<i:int,f:double> -- !query 28 output -299,10 +299,10 struct<i:int,f:double> -- !query 29 -SELECT COALESCE(a.f, b.i, b.j) +SELECT udf(COALESCE(a.f, b.i, b.j)) FROM CASE_TBL a, CASE2_TBL b -- !query 29 schema -struct<coalesce(f, CAST(i AS DOUBLE), CAST(j AS DOUBLE)):double> +struct<udf(coalesce(f, cast(i as double), cast(j as double))):string> -- !query 29 output -30.3 -30.3 -332,8 +332,8 struct<coalesce(f, CAST(i AS DOUBLE), CAST(j AS DOUBLE)):double> -- !query 30 SELECT * - FROM CASE_TBL a, CASE2_TBL b - WHERE COALESCE(a.f, b.i, b.j) = 2 + FROM CASE_TBL a, CASE2_TBL b + WHERE udf(COALESCE(a.f, b.i, b.j)) = 2 -- !query 30 schema struct<i:int,f:double,i:int,j:int> -- !query 30 output -342,7 +342,7 struct<i:int,f:double,i:int,j:int> -- !query 31 -SELECT '' AS Five, NULLIF(a.i,b.i) AS `NULLIF(a.i,b.i)`, +SELECT udf('') AS Five, NULLIF(a.i,b.i) AS `NULLIF(a.i,b.i)`, NULLIF(b.i, 4) AS `NULLIF(b.i,4)` FROM CASE_TBL a, CASE2_TBL b -- !query 31 schema -377,7 +377,7 struct<Five:string,NULLIF(a.i,b.i):int,NULLIF(b.i,4):int> -- !query 32 SELECT '' AS `Two`, * FROM CASE_TBL a, CASE2_TBL b - WHERE COALESCE(f,b.i) = 2 + WHERE CAST(udf(COALESCE(f,b.i) = 2) AS boolean) -- !query 32 schema struct<Two:string,i:int,f:double,i:int,j:int> -- !query 32 output -388,15 +388,15 struct<Two:string,i:int,f:double,i:int,j:int> -- !query 33 SELECT CASE (CASE vol('bar') - WHEN 'foo' THEN 'it was foo!' - WHEN vol(null) THEN 'null input' + WHEN udf('foo') THEN 'it was foo!' + WHEN udf(vol(null)) THEN 'null input' WHEN 'bar' THEN 'it was bar!' END ) - WHEN 'it was foo!' THEN 'foo recognized' - WHEN 'it was bar!' THEN 'bar recognized' - ELSE 'unrecognized' END + WHEN udf('it was foo!') THEN 'foo recognized' + WHEN 'it was bar!' THEN udf('bar recognized') + ELSE 'unrecognized' END AS col -- !query 33 schema -struct<CASE WHEN (CASE WHEN (UDF:vol(bar) = foo) THEN it was foo! WHEN (UDF:vol(bar) = UDF:vol(null)) THEN null input WHEN (UDF:vol(bar) = bar) THEN it was bar! END = it was foo!) THEN foo recognized WHEN (CASE WHEN (UDF:vol(bar) = foo) THEN it was foo! WHEN (UDF:vol(bar) = UDF:vol(null)) THEN null input WHEN (UDF:vol(bar) = bar) THEN it was bar! END = it was bar!) THEN bar recognized ELSE unrecognized END:string> +struct<col:string> -- !query 33 output bar recognized ``` </p> </details> #25069 contains the same minor fixes as it's required to write the tests. ## How was this patch tested? Tested as guided in [SPARK-27921](https://issues.apache.org/jira/browse/SPARK-27921). Closes #25070 from HyukjinKwon/SPARK-28273. Authored-by: HyukjinKwon <[email protected]> Signed-off-by: Wenchen Fan <[email protected]>
1 parent a5ff922 commit fe3e34d

File tree

8 files changed

+713
-13
lines changed

8 files changed

+713
-13
lines changed

python/pyspark/sql/udf.py

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -376,17 +376,17 @@ def registerJavaFunction(self, name, javaClassName, returnType=None):
376376
>>> spark.udf.registerJavaFunction(
377377
... "javaStringLength", "test.org.apache.spark.sql.JavaStringLength", IntegerType())
378378
>>> spark.sql("SELECT javaStringLength('test')").collect()
379-
[Row(UDF:javaStringLength(test)=4)]
379+
[Row(javaStringLength(test)=4)]
380380
381381
>>> spark.udf.registerJavaFunction(
382382
... "javaStringLength2", "test.org.apache.spark.sql.JavaStringLength")
383383
>>> spark.sql("SELECT javaStringLength2('test')").collect()
384-
[Row(UDF:javaStringLength2(test)=4)]
384+
[Row(javaStringLength2(test)=4)]
385385
386386
>>> spark.udf.registerJavaFunction(
387387
... "javaStringLength3", "test.org.apache.spark.sql.JavaStringLength", "integer")
388388
>>> spark.sql("SELECT javaStringLength3('test')").collect()
389-
[Row(UDF:javaStringLength3(test)=4)]
389+
[Row(javaStringLength3(test)=4)]
390390
"""
391391

392392
jdt = None

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -57,8 +57,7 @@ case class ScalaUDF(
5757

5858
override lazy val deterministic: Boolean = udfDeterministic && children.forall(_.deterministic)
5959

60-
override def toString: String =
61-
s"${udfName.map(name => s"UDF:$name").getOrElse("UDF")}(${children.mkString(", ")})"
60+
override def toString: String = s"${udfName.getOrElse("UDF")}(${children.mkString(", ")})"
6261

6362
// scalastyle:off line.size.limit
6463

Lines changed: 272 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,272 @@
1+
--
2+
-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group
3+
--
4+
--
5+
-- CASE
6+
-- https://github.com/postgres/postgres/blob/REL_12_BETA1/src/test/regress/sql/case.sql
7+
-- Test the CASE statement
8+
--
9+
-- This test suite contains two Cartesian products without using explicit CROSS JOIN syntax.
10+
-- Thus, we set spark.sql.crossJoin.enabled to true.
11+
12+
-- This test file was converted from pgSQL/case.sql.
13+
-- Note that currently registered UDF returns a string. So there are some differences, for instance
14+
-- in string cast within UDF in Scala and Python.
15+
16+
set spark.sql.crossJoin.enabled=true;
17+
CREATE TABLE CASE_TBL (
18+
i integer,
19+
f double
20+
) USING parquet;
21+
22+
CREATE TABLE CASE2_TBL (
23+
i integer,
24+
j integer
25+
) USING parquet;
26+
27+
INSERT INTO CASE_TBL VALUES (1, 10.1);
28+
INSERT INTO CASE_TBL VALUES (2, 20.2);
29+
INSERT INTO CASE_TBL VALUES (3, -30.3);
30+
INSERT INTO CASE_TBL VALUES (4, NULL);
31+
32+
INSERT INTO CASE2_TBL VALUES (1, -1);
33+
INSERT INTO CASE2_TBL VALUES (2, -2);
34+
INSERT INTO CASE2_TBL VALUES (3, -3);
35+
INSERT INTO CASE2_TBL VALUES (2, -4);
36+
INSERT INTO CASE2_TBL VALUES (1, NULL);
37+
INSERT INTO CASE2_TBL VALUES (NULL, -6);
38+
39+
--
40+
-- Simplest examples without tables
41+
--
42+
43+
SELECT '3' AS `One`,
44+
CASE
45+
WHEN CAST(udf(1 < 2) AS boolean) THEN 3
46+
END AS `Simple WHEN`;
47+
48+
SELECT '<NULL>' AS `One`,
49+
CASE
50+
WHEN 1 > 2 THEN udf(3)
51+
END AS `Simple default`;
52+
53+
SELECT '3' AS `One`,
54+
CASE
55+
WHEN udf(1) < 2 THEN udf(3)
56+
ELSE udf(4)
57+
END AS `Simple ELSE`;
58+
59+
SELECT udf('4') AS `One`,
60+
CASE
61+
WHEN 1 > 2 THEN 3
62+
ELSE 4
63+
END AS `ELSE default`;
64+
65+
SELECT udf('6') AS `One`,
66+
CASE
67+
WHEN CAST(udf(1 > 2) AS boolean) THEN 3
68+
WHEN udf(4) < 5 THEN 6
69+
ELSE 7
70+
END AS `Two WHEN with default`;
71+
72+
SELECT '7' AS `None`,
73+
CASE WHEN rand() < udf(0) THEN 1
74+
END AS `NULL on no matches`;
75+
76+
-- Constant-expression folding shouldn't evaluate unreachable subexpressions
77+
SELECT CASE WHEN CAST(udf(1=0) AS boolean) THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END;
78+
SELECT CASE 1 WHEN 0 THEN 1/udf(0) WHEN 1 THEN 1 ELSE 2/0 END;
79+
80+
-- [SPARK-27923] PostgreSQL throws an exception but Spark SQL is NULL
81+
-- However we do not currently suppress folding of potentially
82+
-- reachable subexpressions
83+
SELECT CASE WHEN i > 100 THEN udf(1/0) ELSE udf(0) END FROM case_tbl;
84+
85+
-- Test for cases involving untyped literals in test expression
86+
SELECT CASE 'a' WHEN 'a' THEN udf(1) ELSE udf(2) END;
87+
88+
--
89+
-- Examples of targets involving tables
90+
--
91+
92+
SELECT '' AS `Five`,
93+
CASE
94+
WHEN i >= 3 THEN i
95+
END AS `>= 3 or Null`
96+
FROM CASE_TBL;
97+
98+
SELECT '' AS `Five`,
99+
CASE WHEN i >= 3 THEN (i + i)
100+
ELSE i
101+
END AS `Simplest Math`
102+
FROM CASE_TBL;
103+
104+
SELECT '' AS `Five`, i AS `Value`,
105+
CASE WHEN (i < 0) THEN 'small'
106+
WHEN (i = 0) THEN 'zero'
107+
WHEN (i = 1) THEN 'one'
108+
WHEN (i = 2) THEN 'two'
109+
ELSE 'big'
110+
END AS `Category`
111+
FROM CASE_TBL;
112+
113+
SELECT '' AS `Five`,
114+
CASE WHEN ((i < 0) or (i < 0)) THEN 'small'
115+
WHEN ((i = 0) or (i = 0)) THEN 'zero'
116+
WHEN ((i = 1) or (i = 1)) THEN 'one'
117+
WHEN ((i = 2) or (i = 2)) THEN 'two'
118+
ELSE 'big'
119+
END AS `Category`
120+
FROM CASE_TBL;
121+
122+
--
123+
-- Examples of qualifications involving tables
124+
--
125+
126+
--
127+
-- NULLIF() and COALESCE()
128+
-- Shorthand forms for typical CASE constructs
129+
-- defined in the SQL standard.
130+
--
131+
132+
SELECT * FROM CASE_TBL WHERE udf(COALESCE(f,i)) = 4;
133+
134+
SELECT * FROM CASE_TBL WHERE udf(NULLIF(f,i)) = 2;
135+
136+
SELECT udf(COALESCE(a.f, b.i, b.j))
137+
FROM CASE_TBL a, CASE2_TBL b;
138+
139+
SELECT *
140+
FROM CASE_TBL a, CASE2_TBL b
141+
WHERE udf(COALESCE(a.f, b.i, b.j)) = 2;
142+
143+
SELECT udf('') AS Five, NULLIF(a.i,b.i) AS `NULLIF(a.i,b.i)`,
144+
NULLIF(b.i, 4) AS `NULLIF(b.i,4)`
145+
FROM CASE_TBL a, CASE2_TBL b;
146+
147+
SELECT '' AS `Two`, *
148+
FROM CASE_TBL a, CASE2_TBL b
149+
WHERE CAST(udf(COALESCE(f,b.i) = 2) AS boolean);
150+
151+
-- We don't support update now.
152+
--
153+
-- Examples of updates involving tables
154+
--
155+
156+
-- UPDATE CASE_TBL
157+
-- SET i = CASE WHEN i >= 3 THEN (- i)
158+
-- ELSE (2 * i) END;
159+
160+
-- SELECT * FROM CASE_TBL;
161+
162+
-- UPDATE CASE_TBL
163+
-- SET i = CASE WHEN i >= 2 THEN (2 * i)
164+
-- ELSE (3 * i) END;
165+
166+
-- SELECT * FROM CASE_TBL;
167+
168+
-- UPDATE CASE_TBL
169+
-- SET i = CASE WHEN b.i >= 2 THEN (2 * j)
170+
-- ELSE (3 * j) END
171+
-- FROM CASE2_TBL b
172+
-- WHERE j = -CASE_TBL.i;
173+
174+
-- SELECT * FROM CASE_TBL;
175+
176+
--
177+
-- Nested CASE expressions
178+
--
179+
180+
-- This test exercises a bug caused by aliasing econtext->caseValue_isNull
181+
-- with the isNull argument of the inner CASE's CaseExpr evaluation. After
182+
-- evaluating the vol(null) expression in the inner CASE's second WHEN-clause,
183+
-- the isNull flag for the case test value incorrectly became true, causing
184+
-- the third WHEN-clause not to match. The volatile function calls are needed
185+
-- to prevent constant-folding in the planner, which would hide the bug.
186+
187+
-- Wrap this in a single transaction so the transient '=' operator doesn't
188+
-- cause problems in concurrent sessions
189+
-- BEGIN;
190+
191+
-- CREATE FUNCTION vol(text) returns text as
192+
-- 'begin return $1; end' language plpgsql volatile;
193+
194+
SELECT CASE
195+
(CASE vol('bar')
196+
WHEN udf('foo') THEN 'it was foo!'
197+
WHEN udf(vol(null)) THEN 'null input'
198+
WHEN 'bar' THEN 'it was bar!' END
199+
)
200+
WHEN udf('it was foo!') THEN 'foo recognized'
201+
WHEN 'it was bar!' THEN udf('bar recognized')
202+
ELSE 'unrecognized' END AS col;
203+
204+
-- We don't support the features below:
205+
-- 1. CREATE DOMAIN ...
206+
-- 2. CREATE OPERATOR ...
207+
-- 3. CREATE TYPE ...
208+
209+
-- In this case, we can't inline the SQL function without confusing things.
210+
-- CREATE DOMAIN foodomain AS text;
211+
212+
-- CREATE FUNCTION volfoo(text) returns foodomain as
213+
-- 'begin return $1::foodomain; end' language plpgsql volatile;
214+
215+
-- CREATE FUNCTION inline_eq(foodomain, foodomain) returns boolean as
216+
-- 'SELECT CASE $2::text WHEN $1::text THEN true ELSE false END' language sql;
217+
218+
-- CREATE OPERATOR = (procedure = inline_eq,
219+
-- leftarg = foodomain, rightarg = foodomain);
220+
221+
-- SELECT CASE volfoo('bar') WHEN 'foo'::foodomain THEN 'is foo' ELSE 'is not foo' END;
222+
223+
-- ROLLBACK;
224+
225+
-- Test multiple evaluation of a CASE arg that is a read/write object (#14472)
226+
-- Wrap this in a single transaction so the transient '=' operator doesn't
227+
-- cause problems in concurrent sessions
228+
-- BEGIN;
229+
230+
-- CREATE DOMAIN arrdomain AS int[];
231+
232+
-- CREATE FUNCTION make_ad(int,int) returns arrdomain as
233+
-- 'declare x arrdomain;
234+
-- begin
235+
-- x := array[$1,$2];
236+
-- return x;
237+
-- end' language plpgsql volatile;
238+
239+
-- CREATE FUNCTION ad_eq(arrdomain, arrdomain) returns boolean as
240+
-- 'begin return array_eq($1, $2); end' language plpgsql;
241+
242+
-- CREATE OPERATOR = (procedure = ad_eq,
243+
-- leftarg = arrdomain, rightarg = arrdomain);
244+
245+
-- SELECT CASE make_ad(1,2)
246+
-- WHEN array[2,4]::arrdomain THEN 'wrong'
247+
-- WHEN array[2,5]::arrdomain THEN 'still wrong'
248+
-- WHEN array[1,2]::arrdomain THEN 'right'
249+
-- END;
250+
251+
-- ROLLBACK;
252+
253+
-- Test interaction of CASE with ArrayCoerceExpr (bug #15471)
254+
-- BEGIN;
255+
256+
-- CREATE TYPE casetestenum AS ENUM ('e', 'f', 'g');
257+
258+
-- SELECT
259+
-- CASE 'foo'::text
260+
-- WHEN 'foo' THEN ARRAY['a', 'b', 'c', 'd'] || enum_range(NULL::casetestenum)::text[]
261+
-- ELSE ARRAY['x', 'y']
262+
-- END;
263+
264+
-- ROLLBACK;
265+
266+
--
267+
-- Clean up
268+
--
269+
270+
DROP TABLE CASE_TBL;
271+
DROP TABLE CASE2_TBL;
272+
set spark.sql.crossJoin.enabled=false;

sql/core/src/test/resources/sql-tests/results/pgSQL/case.sql.out

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -396,7 +396,7 @@ SELECT CASE
396396
WHEN 'it was bar!' THEN 'bar recognized'
397397
ELSE 'unrecognized' END
398398
-- !query 33 schema
399-
struct<CASE WHEN (CASE WHEN (UDF:vol(bar) = foo) THEN it was foo! WHEN (UDF:vol(bar) = UDF:vol(null)) THEN null input WHEN (UDF:vol(bar) = bar) THEN it was bar! END = it was foo!) THEN foo recognized WHEN (CASE WHEN (UDF:vol(bar) = foo) THEN it was foo! WHEN (UDF:vol(bar) = UDF:vol(null)) THEN null input WHEN (UDF:vol(bar) = bar) THEN it was bar! END = it was bar!) THEN bar recognized ELSE unrecognized END:string>
399+
struct<CASE WHEN (CASE WHEN (vol(bar) = foo) THEN it was foo! WHEN (vol(bar) = vol(null)) THEN null input WHEN (vol(bar) = bar) THEN it was bar! END = it was foo!) THEN foo recognized WHEN (CASE WHEN (vol(bar) = foo) THEN it was foo! WHEN (vol(bar) = vol(null)) THEN null input WHEN (vol(bar) = bar) THEN it was bar! END = it was bar!) THEN bar recognized ELSE unrecognized END:string>
400400
-- !query 33 output
401401
bar recognized
402402

0 commit comments

Comments
 (0)