Github user aljoscha commented on a diff in the pull request: https://github.com/apache/flink/pull/2736#discussion_r86385660 --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/evictors/TimeEvictor.java --- @@ -35,23 +37,63 @@ private static final long serialVersionUID = 1L; private final long windowSize; + private final boolean doEvictAfter; public TimeEvictor(long windowSize) { this.windowSize = windowSize; + this.doEvictAfter = false; + } + + public TimeEvictor(long windowSize, boolean doEvictAfter) { + this.windowSize = windowSize; + this.doEvictAfter = doEvictAfter; } + @Override - public int evict(Iterable<StreamRecord<Object>> elements, int size, W window) { - int toEvict = 0; - long currentTime = Iterables.getLast(elements).getTimestamp(); + public void evictBefore(Iterable<TimestampedValue<Object>> elements, int size, W window, EvictorContext ctx) { + if(!doEvictAfter) { + evict(elements,size,ctx); + } + } + + @Override + public void evictAfter(Iterable<TimestampedValue<Object>> elements, int size, W window, EvictorContext ctx) { + if(doEvictAfter) { + evict(elements,size,ctx); + } + } + + private void evict(Iterable<TimestampedValue<Object>> elements, int size, EvictorContext ctx) { + long currentTime = getMaxTimestamp(elements); + + if (currentTime < 0) { + //we don't evict any element if timestamp is not set in the record + return; + } long evictCutoff = currentTime - windowSize; - for (StreamRecord<Object> record: elements) { - if (record.getTimestamp() > evictCutoff) { - break; + + for (Iterator<TimestampedValue<Object>> iterator = elements.iterator(); iterator.hasNext(); ) { + TimestampedValue<Object> record = iterator.next(); + if (record.getTimestamp() <= evictCutoff) { + iterator.remove(); + } + } + } + + /** + * @param elements The elements currently in the pane. + * @return The maximum value of timestamp among the elements + */ + private long getMaxTimestamp(Iterable<TimestampedValue<Object>> elements) { + long currentTime = Long.MIN_VALUE; + for (Iterator<TimestampedValue<Object>> iterator = elements.iterator(); iterator.hasNext();){ + TimestampedValue<Object> record = iterator.next(); + if (record.getTimestamp() > currentTime) { --- End diff -- This could be more succinctly expressed as `currentTime = Math.max(currentTime, record.getTimestamp());`
--- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---