[ https://issues.apache.org/jira/browse/FLINK-8384?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16327307#comment-16327307 ]
ASF GitHub Bot commented on FLINK-8384: --------------------------------------- Github user sunjincheng121 commented on a diff in the pull request: https://github.com/apache/flink/pull/5295#discussion_r161799438 --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/DynamicProcessingTimeSessionWindows.java --- @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.api.windowing.assigners; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.windowing.triggers.Trigger; +import org.apache.flink.streaming.api.windowing.triggers.TypedProcessingTimeTrigger; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; + +import java.util.Collection; +import java.util.Collections; + +/** + * A {@link WindowAssigner} that windows elements into sessions based on the current processing + * time. Windows cannot overlap. + * + * <p>For example, in order to window into windows with a dynamic time gap: + * <pre> {@code + * DataStream<Tuple2<String, Integer>> in = ...; + * KeyedStream<String, Tuple2<String, Integer>> keyed = in.keyBy(...); + * WindowedStream<Tuple2<String, Integer>, String, TimeWindows> windowed = + * keyed.window(DynamicProcessingTimeSessionWindows.withDynamicGap({@link SessionWindowTimeGapExtractor })); + * } </pre> + * + * @param <T> The type of the input elements + */ +public class DynamicProcessingTimeSessionWindows<T> extends MergingWindowAssigner<T, TimeWindow> { + private static final long serialVersionUID = 1L; + + protected SessionWindowTimeGapExtractor<T> sessionWindowTimeGapExtractor; + + protected DynamicProcessingTimeSessionWindows(SessionWindowTimeGapExtractor<T> sessionWindowTimeGapExtractor) { + this.sessionWindowTimeGapExtractor = sessionWindowTimeGapExtractor; + } + + @Override + public Collection<TimeWindow> assignWindows(T element, long timestamp, WindowAssignerContext context) { + long currentProcessingTime = context.getCurrentProcessingTime(); + long sessionTimeout = sessionWindowTimeGapExtractor.extract(element, timestamp, context); + if (sessionTimeout <= 0) { + throw new IllegalArgumentException("Dynamic session time gap must satisfy 0 < gap"); + } + return Collections.singletonList(new TimeWindow(currentProcessingTime, currentProcessingTime + sessionTimeout)); + } + + @Override + public Trigger<T, TimeWindow> getDefaultTrigger(StreamExecutionEnvironment env) { + return TypedProcessingTimeTrigger.create(); --- End diff -- DynamicProcessingTimeSessionWindows. > Session Window Assigner with Dynamic Gaps > ----------------------------------------- > > Key: FLINK-8384 > URL: https://issues.apache.org/jira/browse/FLINK-8384 > Project: Flink > Issue Type: Improvement > Components: Streaming > Reporter: Dyana Rose > Priority: Minor > > *Reason for Improvement* > Currently both Session Window assigners only allow a static inactivity gap. > Given the following scenario, this is too restrictive: > * Given a stream of IoT events from many device types > * Assume each device type could have a different inactivity gap > * Assume each device type gap could change while sessions are in flight > By allowing dynamic inactivity gaps, the correct gap can be determined in the > [assignWindows > function|https://github.com/apache/flink/blob/master/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/EventTimeSessionWindows.java#L59] > by passing the element currently under consideration, the timestamp, and the > context to a user defined function. This eliminates the need to create > unwieldy work arounds if you only have static gaps. > Dynamic Session Window gaps should be available for both Event Time and > Processing Time streams. > (short preliminary discussion: > https://lists.apache.org/thread.html/08a011c0039e826343e9be0b1bb4e0000cfc2e12976bc65f8a43ee88@%3Cdev.flink.apache.org%3E) -- This message was sent by Atlassian JIRA (v7.6.3#76005)