Hi Julian,

Thanks for the reply. Tried it with SqlToRelConverter and it seems to be
Cartesian product but not the desired behavior:

> @Test public void testUnnestArrayPlan() {
>     final String sql = "select d.deptno, e2.empno, e3.detail\n"
>         + "from dept_nested as d,\n"
>         + " UNNEST(d.employees) e2, UNNEST(d.employees) e3";
>     sql(sql).with(getExtendedTester()).ok();
>   }
>

> @Test public void testUnnestArrayPlan() {
>     final String sql = "select d.deptno, e2.empno, e3.detail\n"
>         + "from dept_nested as d CROSS JOIN\n"
>         + " UNNEST(d.employees) e2 CROSS JOIN UNNEST(d.employees) e3";
>     sql(sql).with(getExtendedTester()).ok();
>   }


and both of them yield a plan like

LogicalProject(DEPTNO=[$0], EMPNO=[$7], DETAIL=[ROW($12)])
>   LogicalCorrelate(correlation=[$cor1], joinType=[inner],
> requiredColumns=[{6}])
>     LogicalCorrelate(correlation=[$cor0], joinType=[inner],
> requiredColumns=[{6}])
>       LogicalProject(DEPTNO=[$0], NAME=[$1], TYPE=[$2.TYPE],
> DESC=[$2.DESC], A=[$2.OTHERS.A], B=[$2.OTHERS.B], EMPLOYEES=[$3])
>         LogicalTableScan(table=[[CATALOG, SALES, DEPT_NESTED]])
>       Uncollect
>         LogicalProject(EMPLOYEES=[$cor0.EMPLOYEES_6])
>           LogicalValues(tuples=[[{ 0 }]])
>     Uncollect
>       LogicalProject(EMPLOYEES=[$cor1.EMPLOYEES_6])
>         LogicalValues(tuples=[[{ 0 }]])


I only see the desired behavior in Presto, and we are currently using
Calcite to validate (some of) Presto queries. Except the behavior above,
Calcite and Presto treat the following things for structured types
differently:

> CROSS JOIN UNNEST(array_of_structures_with_properties_a_b_and_c) AS t

in Calcite is equivalent to

> CROSS JOIN UNNEST(array_of_structures_with_properties_a_b_and_c) AS
> some_nonused_identifier(t)
>

To resolve such differences, we would like to implement a new conformance
and validation logics to reflect differences above.

Regarding Sql To Rel, I am still not sure what SHOULD BE the right logical
plan for the desired behavior given the above example. Let me know.

Thanks,
Will

On Tue, Feb 11, 2020 at 11:11 PM Julian Hyde <[email protected]> wrote:

> I haven't tried it, but I'd guess that you join to each separately,
> i.e. take the cartesian product:
>
> SELECT numbers, animals, n, a
> FROM (
>   VALUES
>     (ARRAY[2, 5], ARRAY['dog', 'cat', 'bird']),
>     (ARRAY[7, 8, 9], ARRAY['cow', 'pig'])
> ) AS x (numbers, animals)
> CROSS JOIN UNNEST (numbers) AS t1 (n)
> CROSS JOIN UNNEST (animals) AS t2 (a)
>
> This doesn't require an extension to standard SQL.
>
> On Tue, Feb 11, 2020 at 11:07 PM Miao Yu <[email protected]> wrote:
> >
> > Hi Calcite experts:
> >
> > In Presto, multiple array columns could be flattened "together", as
> Presto
> > docs <https://prestodb.io/docs/current/sql/select.html> shows:
> >
> > SELECT numbers, animals, n, aFROM (
> > >   VALUES
> > >     (ARRAY[2, 5], ARRAY['dog', 'cat', 'bird']),
> > >     (ARRAY[7, 8, 9], ARRAY['cow', 'pig'])) AS x (numbers,
> animals)CROSS JOIN UNNEST(numbers, animals) AS t (n, a)
> > >
> > > yields:
> >
> >   numbers  |     animals      |  n   |  a
> > -----------+------------------+------+------
> >  [2, 5]    | [dog, cat, bird] |    2 | dog
> >  [2, 5]    | [dog, cat, bird] |    5 | cat
> >  [2, 5]    | [dog, cat, bird] | NULL | bird
> >  [7, 8, 9] | [cow, pig]       |    7 | cow
> >  [7, 8, 9] | [cow, pig]       |    8 | pig
> >  [7, 8, 9] | [cow, pig]       |    9 | NULL
> >
> >
> > May I know what is the equivalent in Calcite SQL?
> >
> > Thanks!
> > Will
>

Reply via email to