Hi all,

Thanks for starting these discussion - it is very useful.
It does make sense indeed to refactor all these and coordinate a bit the 
efforts not to have overlapping implementations and incompatible solutions.

If you close the 3 jira issues you mentioned - do you plan to redesign them and 
open new ones? Do you need help from our side - we can also pick the redesign 
of some of these new jira issues. For example we already have an implementation 
for this and we can help with the design. Nevertheless, let's coordinate the 
effort.

Regarding the support for the different types of window - I think the best 
option is to split the implementation in small units. We can easily do this 
from the transformation rule class and with this each particular type of window 
(session/sliding/sliderows/processing time/...) will have a clear 
implementation and a corresponding architecture within the jira issue? What do 
you think about such a granularity?

Regarding the issue of " Q4: The implementaion of SlideRows still need a custom 
operator that collects records in a priority queue ordered by the "rowtime", 
which is similar to the design we discussed in FLINK-4697, right? "
Why would you need this operator? The window buffer can act to some extent as a 
priority queue as long as the trigger and evictor is set to work based on the 
rowtime - or maybe I am missing something... Can you please clarify this.


Dr. Radu Tudoran
Senior Research Engineer - Big Data Expert
IT R&D Division


HUAWEI TECHNOLOGIES Duesseldorf GmbH
European Research Center
Riesstrasse 25, 80992 München

E-mail: radu.tudo...@huawei.com
Mobile: +49 15209084330
Telephone: +49 891588344173

HUAWEI TECHNOLOGIES Duesseldorf GmbH
Hansaallee 205, 40549 Düsseldorf, Germany, www.huawei.com
Registered Office: Düsseldorf, Register Court Düsseldorf, HRB 56063,
Managing Director: Bo PENG, Wanzhou MENG, Lifang CHEN
Sitz der Gesellschaft: Düsseldorf, Amtsgericht Düsseldorf, HRB 56063,
Geschäftsführer: Bo PENG, Wanzhou MENG, Lifang CHEN
This e-mail and its attachments contain confidential information from HUAWEI, 
which is intended only for the person or entity whose address is listed above. 
Any use of the information contained herein in any way (including, but not 
limited to, total or partial disclosure, reproduction, or dissemination) by 
persons other than the intended recipient(s) is prohibited. If you receive this 
e-mail in error, please notify the sender by phone or email immediately and 
delete it!


-----Original Message-----
From: Jark Wu [mailto:wuchong...@alibaba-inc.com] 
Sent: Tuesday, January 24, 2017 6:53 AM
To: dev@flink.apache.org
Subject: Re: [DISCUSS] Development of SQL OVER / Table API Row Windows for 
streaming tables

Hi Fabian, 

Thanks for bringing up this discussion and the nice approach to avoid 
overlapping contributions.

All of these make sense to me. But I have some questions.

Q1: If I understand correctly, we will not support TumbleRows and SessionRows 
at the beginning. But maybe support them as a syntax sugar (in Table API) when 
the SlideRows is supported in the future. Right ? 

Q2: How to support SessionRows based on SlideRows ?  I don't get how to 
partition on "gap-separated".

Q3: Should we break down the approach into smaller tasks for streaming tables 
and batch tables ? 

Q4: The implementaion of SlideRows still need a custom operator that collects 
records in a priority queue ordered by the "rowtime", which is similar to the 
design we discussed in FLINK-4697, right? 

+1 not support for OVER ROW for event time at this point.

Regards, Jark


> 在 2017年1月24日,上午10:28,Hongyuhong <hongyuh...@huawei.com> 写道:
> 
> Hi,
> We are also interested in streaming sql and very willing to participate and 
> contribute.
> 
> We are now in progress and we will also contribute to calcite to push forward 
> the window and stream-join support.
> 
> 
> 
> --------------
> Sender: Fabian Hueske [mailto:fhue...@gmail.com] Send Time: 2017年1月24日 
> 5:55
> Receiver: dev@flink.apache.org
> Theme: Re: [DISCUSS] Development of SQL OVER / Table API Row Windows 
> for streaming tables
> 
> Hi Haohui,
> 
> our plan was in fact to piggy-back on Calcite and use the TUMBLE function [1] 
> once is it is available (CALCITE-1345 [2]).
> Unfortunately, this issue does not seem to be very active, so I don't know 
> what the progress is.
> 
> I would suggest to move the discussion about group windows to a separate 
> thread and keep this one focused on the organization of the SQL OVER windows.
> 
> Best,
> Fabian
> 
> [1] http://calcite.apache.org/docs/stream.html)
> [2] https://issues.apache.org/jira/browse/CALCITE-1345
> 
> 2017-01-23 22:42 GMT+01:00 Haohui Mai <ricet...@gmail.com>:
> 
>> Hi Fabian,
>> 
>> FLINK-4692 has added the support for tumbling window and we are 
>> excited to try it out and expose it as a SQL construct.
>> 
>> Just curious -- what's your thought on the SQL syntax on tumbling window?
>> 
>> Implementation wise it might make sense to think tumbling window as a 
>> special case of the sliding window.
>> 
>> The problem I see is that the OVER construct might be insufficient to 
>> support all the use cases of tumbling windows. For example, it fails 
>> to express tumbling windows that have fractional time units (as 
>> pointed out in http://calcite.apache.org/docs/stream.html).
>> 
>> It looks to me that the Calcite / Azure Stream Analytics have 
>> introduced a new construct (TUMBLE / TUMBLINGWINDOW) to address this issue.
>> 
>> Do you think it is a good idea to follow the same conventions? Your 
>> ideas are appreciated.
>> 
>> Regards,
>> Haohui
>> 
>> 
>> On Mon, Jan 23, 2017 at 1:02 PM Haohui Mai <ricet...@gmail.com> wrote:
>> 
>>> +1
>>> 
>>> We are also quite interested in these features and would love to 
>>> participate and contribute.
>>> 
>>> ~Haohui
>>> 
>>> On Mon, Jan 23, 2017 at 7:31 AM Fabian Hueske <fhue...@gmail.com> wrote:
>>> 
>>>> Hi everybody,
>>>> 
>>>> it seems that currently several contributors are working on new 
>>>> features for the streaming Table API / SQL around row windows (as 
>>>> defined in
>>>> FLIP-11
>>>> [1]) and SQL OVER-style window (FLINK-4678, FLINK-4679, FLINK-4680, 
>>>> FLINK-5584).
>>>> Since these efforts overlap quite a bit I spent some time thinking 
>>>> about how we can approach these features and how to avoid 
>>>> overlapping contributions.
>>>> 
>>>> The challenge here is the following. Some of the Table API row 
>>>> windows
>> as
>>>> defined by FLIP-11 [1] are basically SQL OVER windows while other 
>>>> cannot be easily expressed as such (TumbleRows for row-count 
>>>> intervals, SessionRows).
>>>> However, since Calcite already supports SQL OVER windows, we can 
>>>> reuse
>> the
>>>> optimization logic for some of the Table API row windows. I also 
>>>> thought about the semantics of the TumbleRows and SessionRows 
>>>> windows as defined in
>>>> FLIP-11 and came to the conclusion that these are not well defined 
>>>> in
>>>> FLIP-11 and should rather be defined as SlideRows windows with a 
>>>> special PARTITION BY clause.
>>>> 
>>>> I propose to approach SQL OVER windows and Table API row windows as
>>>> follows:
>>>> 
>>>> We start with three simple cases for SQL OVER windows (not Table 
>>>> API
>> yet):
>>>> 
>>>> * OVER RANGE for event time
>>>> * OVER RANGE for processing time
>>>> * OVER ROW for processing time
>>>> 
>>>> All cases fulfill the following restrictions:
>>>> - All aggregations in SELECT must refer to the same window.
>>>> - PARTITION BY may not contain the rowtime attribute.
>>>> - ORDER BY must be on rowtime attribute (for event time) or on a 
>>>> marker function that indicates processing time. Additional sort 
>>>> attributes are not supported initially.
>>>> - only "BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW" and "BETWEEN x 
>>>> PRECEDING AND CURRENT ROW" are supported.
>>>> 
>>>> OVER ROW for event time cannot be easily supported. With event 
>>>> time, we may have late records which need to be injected into the 
>>>> order of records.
>>>> When
>>>> a record in injected in to the order where a row-count window has
>> already
>>>> been computed, this and all following windows will change. We could
>> either
>>>> drop the record or sent out many retraction records. I think it is 
>>>> best
>> to
>>>> not open this can of worms at this point.
>>>> 
>>>> The rational for all of the above restrictions is to have first 
>>>> versions of OVER windows soon.
>>>> Once we have the above cases covered we can extend and remove
>> limitations
>>>> as follows:
>>>> 
>>>> - Table API SlideRow windows (with the same restrictions as above). 
>>>> This will be mostly API work since the execution part has been solved 
>>>> before.
>>>> - Add support for FOLLOWING (except UNBOUNDED FOLLOWING)
>>>> - Add support for different windows in SELECT. All windows must be 
>>>> partitioned and ordered in the same way.
>>>> - Add support for additional ORDER BY attributes (besides time).
>>>> 
>>>> As I said before, TumbleRows and SessionRows windows as in FLIP-11 
>>>> are
>> not
>>>> well defined, IMO.
>>>> They can be expressed as SlideRows windows with special 
>>>> partitioning (partitioning on fixed, non-overlapping time ranges 
>>>> for TumbleRows, and gap-separated, non-overlapping time ranges for
>>>> SessionRows) I would not start to work on those yet.
>>>> 
>>>> I would like to close all related JIRA issues (FLINK-4678, 
>>>> FLINK-4679, FLINK-4680, FLINK-5584) and restructure the development 
>>>> of these
>> features
>>>> as outlined above with corresponding JIRA issues.
>>>> 
>>>> What do others think? (I cc'ed the contributors assigned to the 
>>>> above
>> JIRA
>>>> issues)
>>>> 
>>>> Best, Fabian
>>>> 
>>>> [1]
>>>> 
>>>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-
>> 11%3A+Table+API+Stream+Aggregations
>>>> 
>>> 
>> 

Reply via email to