[ https://issues.apache.org/jira/browse/FLINK-3109?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15139097#comment-15139097 ]
ASF GitHub Bot commented on FLINK-3109: --------------------------------------- Github user StephanEwen commented on the pull request: https://github.com/apache/flink/pull/1527#issuecomment-181922703 @wangyangjun We actually merged all the changes concerning the state abstraction. To make this window join work seamlessly on Flink's state backends (memory, or key/value stores, managed memory, ...) you would need to implement it against the key/value state. That means that whenever you store data in the operator, the data should go into the partitioned state that you can obtain from the `AbstractStreamOperator` or the `RuntimeContext`. I think that for this window operator, the `ListState` is a good choice, where you can add values to a key and retrieve the list as a whole once the windows are evaluated. Please write back if you need some more pointers on the state abstraction. > Join two streams with two different buffer time > ----------------------------------------------- > > Key: FLINK-3109 > URL: https://issues.apache.org/jira/browse/FLINK-3109 > Project: Flink > Issue Type: Improvement > Components: Streaming > Affects Versions: 0.10.1 > Reporter: Wang Yangjun > Labels: easyfix, patch > Fix For: 0.10.2 > > Original Estimate: 48h > Remaining Estimate: 48h > > Current Flink streaming only supports join two streams on the same window. > How to solve this problem? > For example, there are two streams. One is advertisements showed to users. > The tuple in which could be described as (id, showed timestamp). The other > one is click stream -- (id, clicked timestamp). We want get a joined stream, > which includes all the advertisement that is clicked by user in 20 minutes > after showed. > It is possible that after an advertisement is shown, some user click it > immediately. It is possible that "click" message arrives server earlier than > "show" message because of Internet delay. We assume that the maximum delay is > one minute. > Then the need is that we should alway keep a buffer(20 mins) of "show" stream > and another buffer(1 min) of "click" stream. > It would be grate that there is such an API like. > showStream.join(clickStream) > .where(keySelector) > .buffer(Time.of(20, TimeUnit.MINUTES)) > .equalTo(keySelector) > .buffer(Time.of(1, TimeUnit.MINUTES)) > .apply(JoinFunction) > http://stackoverflow.com/questions/33849462/how-to-avoid-repeated-tuples-in-flink-slide-window-join/34024149#34024149 -- This message was sent by Atlassian JIRA (v6.3.4#6332)