Hi Julian,

I came up with samples and added them to CALCITE-968. I am exactly not sure
how we can express windows over streams in case of stream-to-stream joins
if OVER is not supported in FROM clause. I only added sliding window join
queries. Given that current support provided by windowing clause is limited
to sliding windows, we have to figure out a way to express hopping and
tumbling windows.

But I like the idea of expressing windowed joins without OVER clause if you
meant it in the above mail.

Thanks
Milinda

On Sun, Nov 15, 2015 at 7:53 PM, Milinda Pathirage <[email protected]>
wrote:

> JIRA ticket to track stream joins can be found at
> https://issues.apache.org/jira/browse/CALCITE-968.
>
> Thanks
> Milinda
>
> On Sat, Nov 14, 2015 at 4:54 PM, Milinda Pathirage <[email protected]>
> wrote:
>
>> Hi Julian,
>>
>> Thanks for the response. Will create a jira ticket and come up with some
>> samples.
>>
>>
>> Milinda
>>
>> On Sat, Nov 14, 2015 at 3:38 AM, Julian Hyde <[email protected]> wrote:
>>
>>> Short answer: yes, we should allow it.
>>>
>>> The design falls into 3 parts:
>>> * Validation. We should allow any combination: table-table, stream-table
>>> and stream-stream joins, as long as the query can make progress. That often
>>> means that where a stream is involved, the join condition should involve a
>>> monotonic expression. If it is a stream-table join you can make progress
>>> without the monotonic expression, but if there are 2 streams you will need
>>> it.
>>> * Translation to relational algebra. Inspired by differential calculus’
>>> product rule[1], "stream(x join y)" becomes "x join stream(y) union all
>>> stream(x) join y". Suppose that products is a table (i.e. we do not receive
>>> notifications of new products); then "stream(products)" is empty. Suppose
>>> that orders is a both a stream and a table; i.e. a stream with history.
>>> Because stream(products) is empty, "stream(products join orders)" is simply
>>> “products join stream(orders)”. These rewrites would happen in a
>>> DeltaJoinTransposeRule.
>>> * Updates to relations. Suppose that the products table is updated two
>>> or three times during each day. How quickly does the end user expect those
>>> updated records to appear in the output of the stream-table join? If the
>>> table is updated at 10am, should the new data be loaded only when
>>> processing transactions from 10am (which might not hit the join until say
>>> 10:07am). There is no ‘right answer’ here; we should offer the end user a
>>> choice of policies. A good basic policy would be “cache for no more than T
>>> seconds” or “cache as long as you like” but give a manual way to flush the
>>> cache.
>>>
>>> Can you please log a jira case to track this? Next step would be to
>>> write some sample queries and decide whether they are valid.
>>>
>>> Julian
>>>
>>> [1] https://en.wikipedia.org/wiki/Product_rule
>>>
>>> > On Nov 13, 2015, at 9:35 PM, Milinda Pathirage <[email protected]>
>>> wrote:
>>> >
>>> > Hi devs,
>>> >
>>> > Current SqlValidatorImpl doesn't allow queries like following:
>>> >
>>> > select stream orders.orderId, orders.productId, products.name from
>>> > orders join products on orders.productId = products.id
>>> >
>>> >
>>> > if the 'products' is a relation. This query fails at the modality
>>> check.
>>> > But I am not sure whether fixing (or changing)  the modality checking
>>> logic
>>> > is enough to solve this. Do we need to change planner rules as well.
>>> Really
>>> > appreciate any ideas on this.
>>> >
>>> > Thanks
>>> > Milinda
>>> >
>>> > p.s. I am trying to get this base case working where every element
>>> from a
>>> > stream is joined with a relation. stream-to-stream joins requires
>>> changes
>>> > to parser as well to support windowing. That's my understanding,
>>> Julian may
>>> > have better ideas.
>>> >
>>> > --
>>> > Milinda Pathirage
>>> >
>>> > PhD Student | Research Assistant
>>> > School of Informatics and Computing | Data to Insight Center
>>> > Indiana University
>>> >
>>> > twitter: milindalakmal
>>> > skype: milinda.pathirage
>>> > blog: http://milinda.pathirage.org
>>>
>>>
>>
>>
>> --
>> Milinda Pathirage
>>
>> PhD Student | Research Assistant
>> School of Informatics and Computing | Data to Insight Center
>> Indiana University
>>
>> twitter: milindalakmal
>> skype: milinda.pathirage
>> blog: http://milinda.pathirage.org
>>
>
>
>
> --
> Milinda Pathirage
>
> PhD Student | Research Assistant
> School of Informatics and Computing | Data to Insight Center
> Indiana University
>
> twitter: milindalakmal
> skype: milinda.pathirage
> blog: http://milinda.pathirage.org
>



-- 
Milinda Pathirage

PhD Student | Research Assistant
School of Informatics and Computing | Data to Insight Center
Indiana University

twitter: milindalakmal
skype: milinda.pathirage
blog: http://milinda.pathirage.org

Reply via email to