gearpump-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Kam Kasravi (JIRA)" <j...@apache.org>
Subject [jira] [Comment Edited] (GEARPUMP-23) Add DSL window (time series) support
Date Sun, 10 Apr 2016 18:41:25 GMT

    [ https://issues.apache.org/jira/browse/GEARPUMP-23?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15234206#comment-15234206
] 

Kam Kasravi edited comment on GEARPUMP-23 at 4/10/16 6:41 PM:
--------------------------------------------------------------

Comments from [~mauzhang] moved from https://github.com/gearpump/gearpump/issues/1034
 
We only have `WindowState` now, which limits the usage of windowing to exactly-once processing.
A full windowing support should be oblivious to whether it's at-most-once, at-least-once or
exactly-once. Also, we need a window function in DSL API. 

Besides, [Google's Dataflow model | http://static.googleusercontent.com/media/research.google.com/en//pubs/archive/43864.pdf]
introduced the concept of event-time processing, unaligned window / session (while only aligned
window is supported now)

> Sessions are windows that capture some period of activity
over a subset of the data, in this case per key. Typically
they are defined by a timeout gap. Any events that occur
within a span of time less than the timeout are grouped
together as a session. Sessions are unaligned windows.

and triggers over window for incremental processing as well as use cases that motivates such
concepts

||use cases || requirements||
|session-based analysis for search, ads, analytics, social, and YouTube | unaligned windows
|
|billing pipelines |  triggers and incremental processing support via accumulation and retraction
|
|aggregate statistics calculations | watermark triggers |
|abuse detection pipelines | percentile watermark triggers |
|recommendation generation | processing (system) time triggers |
|anomaly detection | data driven triggers |

It's nice for Gearpump to have these augments, and it shouldn't be hard for Gearpump to do
so since event-time processing and watermark are already built-in. 

In summary, I'd like to implement the following features for windowing support, 

- [ ] window processor in Graph API for all message processing semantics
- [ ] window function in DSL API for all message processing semantics
- [ ] support unaligned sessions
- [ ] trigger API that includes event-time triggers, watermark triggers and data driven triggers
(not sure whether we need processing time triggers)


was (Author: kam kasravi):
Comments from [~mauzhang] moved from https://github.com/gearpump/gearpump/issues/1034
 
We only have `WindowState` now, which limits the usage of windowing to exactly-once processing.
A full windowing support should be oblivious to whether it's at-most-once, at-least-once or
exactly-once. Also, we need a window function in DSL API. 

Besides, [Google's Dataflow model](http://static.googleusercontent.com/media/research.google.com/en//pubs/archive/43864.pdf)
introduced the concept of event-time processing, unaligned window / session (while only aligned
window is supported now)

> Sessions are windows that capture some period of activity
over a subset of the data, in this case per key. Typically
they are defined by a timeout gap. Any events that occur
within a span of time less than the timeout are grouped
together as a session. Sessions are unaligned windows.

and triggers over window for incremental processing as well as use cases that motivates such
concepts

use cases | requirements 
-------------- | -------------------
session-based analysis for search, ads, analytics, social, and YouTube | unaligned windows
 
billing pipelines |  triggers and incremental processing support via accumulation and retraction
aggregate statistics calculations | watermark triggers
abuse detection pipelines | percentile watermark triggers
recommendation generation | processing (system) time triggers
anomaly detection | data driven triggers

It's nice for Gearpump to have these augments, and it shouldn't be hard for Gearpump to do
so since event-time processing and watermark are already built-in. 

In summary, I'd like to implement the following features for windowing support, 

- [ ] window processor in Graph API for all message processing semantics
- [ ] window function in DSL API for all message processing semantics
- [ ] support unaligned sessions
- [ ] trigger API that includes event-time triggers, watermark triggers and data driven triggers
(not sure whether we need processing time triggers)

> Add DSL window (time series) support 
> -------------------------------------
>
>                 Key: GEARPUMP-23
>                 URL: https://issues.apache.org/jira/browse/GEARPUMP-23
>             Project: Apache Gearpump
>          Issue Type: New Feature
>          Components: streaming
>    Affects Versions: 0.8.0
>            Reporter: Kam Kasravi
>             Fix For: 0.8.1
>
>
> Time series support is need for both GEARPUMP-21 and GEARPUMP-22. This was originally
github issue  [1917|https://github.com/gearpump/gearpump/issues/1917]. Other realtime streaming
engines support time windows such as [Flink|https://flink.apache.org/news/2015/12/04/Introducing-windows.html],
[Beam|https://www.oreilly.com/ideas/the-world-beyond-batch-streaming-101] and a future release
of [Spark|https://issues.apache.org/jira/secure/attachment/12793410/StructuredStreamingProgrammingAbstractionSemanticsandAPIs-ApacheJIRA.pdf].
A comparison matrix is found [here|https://docs.google.com/spreadsheets/d/1OM077lZBARrtUi6g0X0O0PHaIbFKCD6v0djRefQRE1I/edit#gid=995660187]



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Mime
View raw message