I think the best way to approach this is probably to have an example SQL
statement and to discuss what the relational semantics should be.

Windowing is not really part of SQL (yet) and in a way it just needs very
minimal extensions. See https://arxiv.org/abs/1905.12133. In this proposal
for SQL, windowed aggregation is explicitly be part of the GROUP BY
operation, where you GROUP BY window columns that were added. So it is more
explicit than in Beam. Relations do not have a WindowFn so there is no
problem of them being incompatible.

With Beam SQL there are basically two ways of windowing that work totally
differently:

1. SQL style windowing where you GROUP BY windows. This does not use the
input PCollection windowfn
2. PCollection windowing where the SQL does not do any windowing - this
should apply the SQL expression to each window independently

In order to support a hybrid of these, it might be:

3. SQL style windowing, where when a PCollection has window assigned, the
window columns are added before the SQL is applied. It is a bit strange but
might enable your use.

Kenn

On Mon, Jul 22, 2019 at 10:39 AM rahul patwari <[email protected]>
wrote:

> Hi,
>
> Beam currently doesn't support Join of Unbounded PCollections of different
> WindowFns (
> https://beam.apache.org/documentation/programming-guide/#groupbykey-and-unbounded-pcollections
> ).
>
> BeamSql performs [Unbounded PCollection] JOIN [Bounded PCollection], by
> performing 'SideInputJoin' with Bounded PCollection as a SideInput.
>
> Can we support [Unbounded PCollection] JOIN [Unbounded PCollection], when
> one of the Unbounded PCollection has [GlobalWindows Applied with
> Non-Default Trigger(probably a slow-changing lookup cache
> https://beam.apache.org/documentation/patterns/side-input-patterns/#slowly-updating-global-window-side-inputs)]
> by performing 'SideInputJoin'?
>
> Regards,
> Rahul
>

Reply via email to