Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -41,15 +41,21 @@
import org.apache.flink.streaming.api.functions.windowing.ReduceApplyWindowFunction;
import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.windowing.assigners.BaseAlignedWindowAssigner;
import org.apache.flink.streaming.api.windowing.assigners.MergingWindowAssigner;
import org.apache.flink.streaming.api.windowing.assigners.SlidingAlignedProcessingTimeWindows;
import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows;
import org.apache.flink.streaming.api.windowing.assigners.TumblingAlignedProcessingTimeWindows;
import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
import org.apache.flink.streaming.api.windowing.evictors.Evictor;
import org.apache.flink.streaming.api.windowing.time.Time;
import org.apache.flink.streaming.api.windowing.triggers.ProcessingTimeTrigger;
import org.apache.flink.streaming.api.windowing.triggers.Trigger;
import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
import org.apache.flink.streaming.api.windowing.windows.Window;
import org.apache.flink.streaming.runtime.operators.windowing.AccumulatingProcessingTimeWindowOperator;
import org.apache.flink.streaming.runtime.operators.windowing.AggregatingProcessingTimeWindowOperator;
import org.apache.flink.streaming.runtime.operators.windowing.EvictingWindowOperator;
import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalIterableWindowFunction;
import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalSingleValueWindowFunction;
Expand Down Expand Up @@ -117,6 +123,10 @@ public WindowedStream<T, K, W> trigger(Trigger<? super T, ? super W> trigger) {
throw new UnsupportedOperationException("A merging window assigner cannot be used with a trigger that does not support merging.");
}

if (windowAssigner instanceof BaseAlignedWindowAssigner) {
throw new UnsupportedOperationException("Cannot use a " + windowAssigner.getClass().getSimpleName() + " with a custom trigger.");
}

this.trigger = trigger;
return this;
}
Expand Down Expand Up @@ -153,6 +163,10 @@ public WindowedStream<T, K, W> evictor(Evictor<? super T, ? super W> evictor) {
if (windowAssigner instanceof MergingWindowAssigner) {
throw new UnsupportedOperationException("Cannot use a merging WindowAssigner with an Evictor.");
}

if (windowAssigner instanceof BaseAlignedWindowAssigner) {
throw new UnsupportedOperationException("Cannot use a " + windowAssigner.getClass().getSimpleName() + " with an Evictor.");
}
this.evictor = evictor;
return this;
}
Expand Down Expand Up @@ -187,6 +201,15 @@ public SingleOutputStreamOperator<T> reduce(ReduceFunction<T> function) {

//clean the closure
function = input.getExecutionEnvironment().clean(function);

String callLocation = Utils.getCallLocationName();
String udfName = "WindowedStream." + callLocation;

SingleOutputStreamOperator<T> result = createFastTimeOperatorIfValid(function, input.getType(), udfName);
if (result != null) {
return result;
}

LegacyWindowOperatorType legacyOpType = getLegacyWindowType(function);
return reduce(function, new PassThroughWindowFunction<K, W, T>(), legacyOpType);
}
Expand Down Expand Up @@ -421,6 +444,11 @@ public <ACC, R> SingleOutputStreamOperator<R> fold(ACC initialValue,
throw new UnsupportedOperationException("Fold cannot be used with a merging WindowAssigner.");
}

if (windowAssigner instanceof BaseAlignedWindowAssigner) {
throw new UnsupportedOperationException("Fold cannot be used with a " +
windowAssigner.getClass().getSimpleName() + " assigner.");
}

//clean the closures
function = input.getExecutionEnvironment().clean(function);
foldFunction = input.getExecutionEnvironment().clean(foldFunction);
Expand Down Expand Up @@ -512,6 +540,11 @@ public <R> SingleOutputStreamOperator<R> apply(WindowFunction<T, R, K, W> functi
String callLocation = Utils.getCallLocationName();
String udfName = "WindowedStream." + callLocation;

SingleOutputStreamOperator<R> result = createFastTimeOperatorIfValid(function, resultType, udfName);
if (result != null) {
return result;
}

LegacyWindowOperatorType legacyWindowOpType = getLegacyWindowType(function);
String opName;
KeySelector<T, K> keySel = input.getKeySelector();
Expand Down Expand Up @@ -977,6 +1010,79 @@ private LegacyWindowOperatorType getLegacyWindowType(Function function) {
return LegacyWindowOperatorType.NONE;
}

private <R> SingleOutputStreamOperator<R> createFastTimeOperatorIfValid(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is simply a copy of the old code, right?

Function function,
TypeInformation<R> resultType,
String functionName) {

if (windowAssigner instanceof SlidingAlignedProcessingTimeWindows && trigger == null && evictor == null) {
SlidingAlignedProcessingTimeWindows timeWindows = (SlidingAlignedProcessingTimeWindows) windowAssigner;
final long windowLength = timeWindows.getSize();
final long windowSlide = timeWindows.getSlide();

String opName = "Fast " + timeWindows + " of " + functionName;

if (function instanceof ReduceFunction) {
@SuppressWarnings("unchecked")
ReduceFunction<T> reducer = (ReduceFunction<T>) function;

@SuppressWarnings("unchecked")
OneInputStreamOperator<T, R> op = (OneInputStreamOperator<T, R>)
new AggregatingProcessingTimeWindowOperator<>(
reducer, input.getKeySelector(),
input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
input.getType().createSerializer(getExecutionEnvironment().getConfig()),
windowLength, windowSlide);
return input.transform(opName, resultType, op);
}
else if (function instanceof WindowFunction) {
@SuppressWarnings("unchecked")
WindowFunction<T, R, K, TimeWindow> wf = (WindowFunction<T, R, K, TimeWindow>) function;

OneInputStreamOperator<T, R> op = new AccumulatingProcessingTimeWindowOperator<>(
wf, input.getKeySelector(),
input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
input.getType().createSerializer(getExecutionEnvironment().getConfig()),
windowLength, windowSlide);
return input.transform(opName, resultType, op);
}
} else if (windowAssigner instanceof TumblingAlignedProcessingTimeWindows && trigger == null && evictor == null) {
TumblingAlignedProcessingTimeWindows timeWindows = (TumblingAlignedProcessingTimeWindows) windowAssigner;
final long windowLength = timeWindows.getSize();
final long windowSlide = timeWindows.getSize();

String opName = "Fast " + timeWindows + " of " + functionName;

if (function instanceof ReduceFunction) {
@SuppressWarnings("unchecked")
ReduceFunction<T> reducer = (ReduceFunction<T>) function;

@SuppressWarnings("unchecked")
OneInputStreamOperator<T, R> op = (OneInputStreamOperator<T, R>)
new AggregatingProcessingTimeWindowOperator<>(
reducer,
input.getKeySelector(),
input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
input.getType().createSerializer(getExecutionEnvironment().getConfig()),
windowLength, windowSlide);
return input.transform(opName, resultType, op);
}
else if (function instanceof WindowFunction) {
@SuppressWarnings("unchecked")
WindowFunction<T, R, K, TimeWindow> wf = (WindowFunction<T, R, K, TimeWindow>) function;

OneInputStreamOperator<T, R> op = new AccumulatingProcessingTimeWindowOperator<>(
wf, input.getKeySelector(),
input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
input.getType().createSerializer(getExecutionEnvironment().getConfig()),
windowLength, windowSlide);
return input.transform(opName, resultType, op);
}
}

return null;
}

public StreamExecutionEnvironment getExecutionEnvironment() {
return input.getExecutionEnvironment();
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,71 @@
/*
* 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.windows.TimeWindow;

import java.util.Collection;

/**
* A base {@link WindowAssigner} used to instantiate one of the deprecated
* {@link org.apache.flink.streaming.runtime.operators.windowing.AccumulatingProcessingTimeWindowOperator
* AccumulatingProcessingTimeWindowOperator} and
* {@link org.apache.flink.streaming.runtime.operators.windowing.AggregatingProcessingTimeWindowOperator
* AggregatingProcessingTimeWindowOperator}.
* <p><p>
* For assigner that extend this one, the user can check the {@link TumblingAlignedProcessingTimeWindows}
* and the {@link SlidingAlignedProcessingTimeWindows}.
* */
public class BaseAlignedWindowAssigner extends WindowAssigner<Object, TimeWindow> {

private static final long serialVersionUID = -6214980179706960234L;

private final long size;

protected BaseAlignedWindowAssigner(long size) {
this.size = size;
}

public long getSize() {
return size;
}

@Override
public Collection<TimeWindow> assignWindows(Object element, long timestamp, WindowAssignerContext context) {
throw new UnsupportedOperationException("This assigner should not be used with the WindowOperator.");
}

@Override
public Trigger<Object, TimeWindow> getDefaultTrigger(StreamExecutionEnvironment env) {
return null;
}

@Override
public TypeSerializer<TimeWindow> getWindowSerializer(ExecutionConfig executionConfig) {
throw new UnsupportedOperationException("This assigner should not be used with the WindowOperator.");
}

@Override
public boolean isEventTime() {
throw new UnsupportedOperationException("This assigner should not be used with the WindowOperator.");
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
/*
* 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.streaming.api.windowing.time.Time;

/**
* A processing time sliding {@link WindowAssigner window assigner} used to perform windowing using the
* {@link org.apache.flink.streaming.runtime.operators.windowing.AccumulatingProcessingTimeWindowOperator
* AccumulatingProcessingTimeWindowOperator} and the
* {@link org.apache.flink.streaming.runtime.operators.windowing.AggregatingProcessingTimeWindowOperator
* AggregatingProcessingTimeWindowOperator}.
* <p><p>
* With this assigner, the {@code trigger} used is a
* {@link org.apache.flink.streaming.api.windowing.triggers.ProcessingTimeTrigger
* ProcessingTimeTrigger} and no {@code evictor} can be specified.
* <p><p>
* <b>WARNING:</b> Bear in mind that no rescaling and no backwards compatibility is supported.
* */
public class SlidingAlignedProcessingTimeWindows extends BaseAlignedWindowAssigner {

private static final long serialVersionUID = 3695562702662473688L;

private final long slide;

public SlidingAlignedProcessingTimeWindows(long size, long slide) {
super(size);
this.slide = slide;
}

public long getSlide() {
return slide;
}

/**
* Creates a new {@code SlidingAlignedProcessingTimeWindows} {@link WindowAssigner} that assigns
* elements to sliding time windows based on the element timestamp.
*
* @param size The size of the generated windows.
* @param slide The slide interval of the generated windows.
*/
public static SlidingAlignedProcessingTimeWindows of(Time size, Time slide) {
return new SlidingAlignedProcessingTimeWindows(size.toMilliseconds(), slide.toMilliseconds());
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,53 @@
/*
* 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.streaming.api.windowing.time.Time;

/**
* A processing time tumbling {@link WindowAssigner window assigner} used to perform windowing using the
* {@link org.apache.flink.streaming.runtime.operators.windowing.AccumulatingProcessingTimeWindowOperator
* AccumulatingProcessingTimeWindowOperator} and the
* {@link org.apache.flink.streaming.runtime.operators.windowing.AggregatingProcessingTimeWindowOperator
* AggregatingProcessingTimeWindowOperator}.
* <p><p>
* With this assigner, the {@code trigger} used is a
* {@link org.apache.flink.streaming.api.windowing.triggers.ProcessingTimeTrigger
* ProcessingTimeTrigger} and no {@code evictor} can be specified.
* <p><p>
* <b>WARNING:</b> Bear in mind that no rescaling and no backwards compatibility is supported.
* */
public class TumblingAlignedProcessingTimeWindows extends BaseAlignedWindowAssigner {

private static final long serialVersionUID = -6217477609512299842L;

protected TumblingAlignedProcessingTimeWindows(long size) {
super(size);
}

/**
* Creates a new {@code TumblingAlignedProcessingTimeWindows} {@link WindowAssigner} that assigns
* elements to time windows based on the element timestamp.
*
* @param size The size of the generated windows.
*/
public static TumblingAlignedProcessingTimeWindows of(Time size) {
return new TumblingAlignedProcessingTimeWindows(size.toMilliseconds());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -108,6 +108,7 @@ public static TumblingProcessingTimeWindows of(Time size) {
public static TumblingProcessingTimeWindows of(Time size, Time offset) {
return new TumblingProcessingTimeWindows(size.toMilliseconds(), offset.toMilliseconds() % size.toMilliseconds());
}

@Override
public TypeSerializer<TimeWindow> getWindowSerializer(ExecutionConfig executionConfig) {
return new TimeWindow.Serializer();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,7 @@
import org.apache.flink.streaming.api.operators.InternalTimerService;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.operators.TimestampedCollector;
import org.apache.flink.streaming.api.windowing.assigners.BaseAlignedWindowAssigner;
import org.apache.flink.streaming.api.windowing.assigners.MergingWindowAssigner;
import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows;
import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
Expand Down Expand Up @@ -214,6 +215,11 @@ public WindowOperator(

super(windowFunction);

checkArgument(!(windowAssigner instanceof BaseAlignedWindowAssigner),
"The " + windowAssigner.getClass().getSimpleName() + " cannot be used with a WindowOperator. " +
"This assigner is only used with the AccumulatingProcessingTimeWindowOperator and " +
"the AggregatingProcessingTimeWindowOperator");

checkArgument(allowedLateness >= 0);

checkArgument(windowStateDescriptor == null || windowStateDescriptor.isSerializerInitialized(),
Expand Down
Loading