[ https://issues.apache.org/jira/browse/STORM-1757?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15286623#comment-15286623 ]
Satish Duggana commented on STORM-1757: --------------------------------------- [~revans2] Currently, windowing support exists for processing time/event time with watermarks with a single trigger/eviction policy other than count based policies. I think we can enhance core windowing for supporting multiple active windows/panes with multiple triggers/custom eviction policies as mentioned in earlier [comment|https://issues.apache.org/jira/browse/STORM-1757?focusedCommentId=15282971&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15282971]. This should include checkpointing the panes. I plan to work next few days on this and add initial thoughts in STORM-1763. Windowing can take the applied functions like reducer/aggregator etc and make appropriate optimizations in maintaining the pane state. Panes with accumulating eviction policies can have optimizations for reducer/aggregator without the need to store all the tuples in the pane state but by storing only the result value with required context. For ex, Avg function needs to store only the avg value and no of elements, these are sufficient in computing the accumulated value for the next pane. We can explore ways to optimize wherever it is possible. > Apache Beam Runner for Storm > ---------------------------- > > Key: STORM-1757 > URL: https://issues.apache.org/jira/browse/STORM-1757 > Project: Apache Storm > Issue Type: Brainstorming > Reporter: P. Taylor Goetz > Priority: Minor > > This is a call for interested parties to collaborate on an Apache Beam [1] > runner for Storm, and express their thoughts and opinions. > Given the addition of the Windowing API to Apache Storm, we should be able to > map naturally to the Beam API. If not, it may be indicative of shortcomings > of the Storm API that should be addressed. > [1] http://beam.incubator.apache.org -- This message was sent by Atlassian JIRA (v6.3.4#6332)