You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by al...@apache.org on 2015/10/07 22:33:50 UTC
[1/8] flink git commit: [FLINK-2674] Add Fold Window Operation for
new Windowing API
Repository: flink
Updated Branches:
refs/heads/master 47b5cb795 -> f9eea5e5a
[FLINK-2674] Add Fold Window Operation for new Windowing API
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/f9eea5e5
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/f9eea5e5
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/f9eea5e5
Branch: refs/heads/master
Commit: f9eea5e5a7d13d844e06d3eb6849268b3eaf4c08
Parents: ce792b1
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Wed Oct 7 16:49:40 2015 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Wed Oct 7 22:08:25 2015 +0200
----------------------------------------------------------------------
.../api/datastream/AllWindowedStream.java | 34 ++++
.../api/datastream/WindowedStream.java | 34 ++++
.../windowing/FoldAllWindowFunction.java | 97 ++++++++++
.../functions/windowing/FoldWindowFunction.java | 97 ++++++++++
.../windowing/NonKeyedWindowOperator.java | 12 +-
.../operators/windowing/WindowOperator.java | 12 +-
.../operators/windowing/WindowFoldITCase.java | 191 +++++++++++++++++++
.../streaming/api/scala/AllWindowedStream.scala | 43 ++++-
.../streaming/api/scala/WindowedStream.scala | 43 ++++-
.../streaming/api/scala/WindowFoldITCase.scala | 148 ++++++++++++++
10 files changed, 707 insertions(+), 4 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/f9eea5e5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
index a8d7654..c7a70d7 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
@@ -19,6 +19,7 @@
package org.apache.flink.streaming.api.datastream;
import org.apache.commons.lang.SerializationUtils;
+import org.apache.flink.api.common.functions.FoldFunction;
import org.apache.flink.api.common.functions.Function;
import org.apache.flink.api.common.functions.ReduceFunction;
import org.apache.flink.api.common.typeinfo.TypeInformation;
@@ -29,6 +30,7 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction;
import org.apache.flink.streaming.api.functions.aggregation.ComparableAggregator;
import org.apache.flink.streaming.api.functions.aggregation.SumAggregator;
+import org.apache.flink.streaming.api.functions.windowing.FoldAllWindowFunction;
import org.apache.flink.streaming.api.functions.windowing.ReduceAllWindowFunction;
import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
@@ -164,6 +166,38 @@ public class AllWindowedStream<T, W extends Window> {
}
/**
+ * Applies the given fold function to each window. The window function is called for each
+ * evaluation of the window for each key individually. The output of the reduce function is
+ * interpreted as a regular non-windowed stream.
+ *
+ * @param function The fold function.
+ * @return The data stream that is the result of applying the fold function to the window.
+ */
+ public <R> DataStream<R> fold(R initialValue, FoldFunction<T, R> function) {
+ //clean the closure
+ function = input.getExecutionEnvironment().clean(function);
+
+ TypeInformation<R> resultType = TypeExtractor.getFoldReturnTypes(function, input.getType(),
+ Utils.getCallLocationName(), true);
+
+ return apply(new FoldAllWindowFunction<W, T, R>(initialValue, function), resultType);
+ }
+
+ /**
+ * Applies the given fold function to each window. The window function is called for each
+ * evaluation of the window for each key individually. The output of the reduce function is
+ * interpreted as a regular non-windowed stream.
+ *
+ * @param function The fold function.
+ * @return The data stream that is the result of applying the fold function to the window.
+ */
+ public <R> DataStream<R> fold(R initialValue, FoldFunction<T, R> function, TypeInformation<R> resultType) {
+ //clean the closure
+ function = input.getExecutionEnvironment().clean(function);
+ return apply(new FoldAllWindowFunction<W, T, R>(initialValue, function), resultType);
+ }
+
+ /**
* Applies a window function to the window. The window function is called for each evaluation
* of the window for each key individually. The output of the window function is interpreted
* as a regular non-windowed stream.
http://git-wip-us.apache.org/repos/asf/flink/blob/f9eea5e5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
index 99f7d06..42e0bd7 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
@@ -19,6 +19,7 @@
package org.apache.flink.streaming.api.datastream;
import org.apache.commons.lang.SerializationUtils;
+import org.apache.flink.api.common.functions.FoldFunction;
import org.apache.flink.api.common.functions.Function;
import org.apache.flink.api.common.functions.ReduceFunction;
import org.apache.flink.api.common.typeinfo.TypeInformation;
@@ -30,6 +31,7 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction;
import org.apache.flink.streaming.api.functions.aggregation.ComparableAggregator;
import org.apache.flink.streaming.api.functions.aggregation.SumAggregator;
+import org.apache.flink.streaming.api.functions.windowing.FoldWindowFunction;
import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
import org.apache.flink.streaming.api.functions.windowing.ReduceWindowFunction;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
@@ -178,6 +180,38 @@ public class WindowedStream<T, K, W extends Window> {
}
/**
+ * Applies the given fold function to each window. The window function is called for each
+ * evaluation of the window for each key individually. The output of the reduce function is
+ * interpreted as a regular non-windowed stream.
+ *
+ * @param function The fold function.
+ * @return The data stream that is the result of applying the fold function to the window.
+ */
+ public <R> DataStream<R> fold(R initialValue, FoldFunction<T, R> function) {
+ //clean the closure
+ function = input.getExecutionEnvironment().clean(function);
+
+ TypeInformation<R> resultType = TypeExtractor.getFoldReturnTypes(function, input.getType(),
+ Utils.getCallLocationName(), true);
+
+ return apply(new FoldWindowFunction<K, W, T, R>(initialValue, function), resultType);
+ }
+
+ /**
+ * Applies the given fold function to each window. The window function is called for each
+ * evaluation of the window for each key individually. The output of the reduce function is
+ * interpreted as a regular non-windowed stream.
+ *
+ * @param function The fold function.
+ * @return The data stream that is the result of applying the fold function to the window.
+ */
+ public <R> DataStream<R> fold(R initialValue, FoldFunction<T, R> function, TypeInformation<R> resultType) {
+ //clean the closure
+ function = input.getExecutionEnvironment().clean(function);
+ return apply(new FoldWindowFunction<K, W, T, R>(initialValue, function), resultType);
+ }
+
+ /**
* Applies the given window function to each window. The window function is called for each
* evaluation of the window for each key individually. The output of the window function is
* interpreted as a regular non-windowed stream.
http://git-wip-us.apache.org/repos/asf/flink/blob/f9eea5e5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldAllWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldAllWindowFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldAllWindowFunction.java
new file mode 100644
index 0000000..69f24fe
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldAllWindowFunction.java
@@ -0,0 +1,97 @@
+/**
+ * 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.functions.windowing;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.functions.FoldFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.operators.translation.WrappingFunction;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.memory.InputViewDataInputStreamWrapper;
+import org.apache.flink.core.memory.OutputViewDataOutputStreamWrapper;
+import org.apache.flink.streaming.api.operators.OutputTypeConfigurable;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+import org.apache.flink.util.Collector;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+public class FoldAllWindowFunction<W extends Window, T, R>
+ extends WrappingFunction<FoldFunction<T, R>>
+ implements AllWindowFunction<T, R, W>, OutputTypeConfigurable<R> {
+ private static final long serialVersionUID = 1L;
+
+ private byte[] serializedInitialValue;
+ private TypeSerializer<R> outSerializer;
+ private transient R initialValue;
+
+ public FoldAllWindowFunction(R initialValue, FoldFunction<T, R> reduceFunction) {
+ super(reduceFunction);
+ this.initialValue = initialValue;
+ }
+
+ @Override
+ public void open(Configuration configuration) throws Exception {
+ super.open(configuration);
+
+ if (serializedInitialValue == null) {
+ throw new RuntimeException("No initial value was serialized for the fold " +
+ "window function. Probably the setOutputType method was not called.");
+ }
+
+ ByteArrayInputStream bais = new ByteArrayInputStream(serializedInitialValue);
+ InputViewDataInputStreamWrapper in = new InputViewDataInputStreamWrapper(
+ new DataInputStream(bais)
+ );
+ initialValue = outSerializer.deserialize(in);
+ }
+
+ @Override
+ public void apply(W window, Iterable<T> values, Collector<R> out) throws Exception {
+ R result = outSerializer.copy(initialValue);
+
+ for (T val: values) {
+ result = wrappedFunction.fold(result, val);
+ }
+
+ out.collect(result);
+ }
+
+ @Override
+ public void setOutputType(TypeInformation<R> outTypeInfo, ExecutionConfig executionConfig) {
+ outSerializer = outTypeInfo.createSerializer(executionConfig);
+
+ ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ OutputViewDataOutputStreamWrapper out = new OutputViewDataOutputStreamWrapper(
+ new DataOutputStream(baos)
+ );
+
+ try {
+ outSerializer.serialize(initialValue, out);
+ } catch (IOException ioe) {
+ throw new RuntimeException("Unable to serialize initial value of type " +
+ initialValue.getClass().getSimpleName() + " of fold window function.", ioe);
+ }
+
+ serializedInitialValue = baos.toByteArray();
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/f9eea5e5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldWindowFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldWindowFunction.java
new file mode 100644
index 0000000..04d2ac7
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldWindowFunction.java
@@ -0,0 +1,97 @@
+/**
+ * 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.functions.windowing;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.functions.FoldFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.operators.translation.WrappingFunction;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.memory.InputViewDataInputStreamWrapper;
+import org.apache.flink.core.memory.OutputViewDataOutputStreamWrapper;
+import org.apache.flink.streaming.api.operators.OutputTypeConfigurable;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+import org.apache.flink.util.Collector;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+public class FoldWindowFunction<K, W extends Window, T, R>
+ extends WrappingFunction<FoldFunction<T, R>>
+ implements WindowFunction<T, R, K, W>, OutputTypeConfigurable<R> {
+ private static final long serialVersionUID = 1L;
+
+ private byte[] serializedInitialValue;
+ private TypeSerializer<R> outSerializer;
+ private transient R initialValue;
+
+ public FoldWindowFunction(R initialValue, FoldFunction<T, R> reduceFunction) {
+ super(reduceFunction);
+ this.initialValue = initialValue;
+ }
+
+ @Override
+ public void open(Configuration configuration) throws Exception {
+ super.open(configuration);
+
+ if (serializedInitialValue == null) {
+ throw new RuntimeException("No initial value was serialized for the fold " +
+ "window function. Probably the setOutputType method was not called.");
+ }
+
+ ByteArrayInputStream bais = new ByteArrayInputStream(serializedInitialValue);
+ InputViewDataInputStreamWrapper in = new InputViewDataInputStreamWrapper(
+ new DataInputStream(bais)
+ );
+ initialValue = outSerializer.deserialize(in);
+ }
+
+ @Override
+ public void apply(K k, W window, Iterable<T> values, Collector<R> out) throws Exception {
+ R result = outSerializer.copy(initialValue);
+
+ for (T val: values) {
+ result = wrappedFunction.fold(result, val);
+ }
+
+ out.collect(result);
+ }
+
+ @Override
+ public void setOutputType(TypeInformation<R> outTypeInfo, ExecutionConfig executionConfig) {
+ outSerializer = outTypeInfo.createSerializer(executionConfig);
+
+ ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ OutputViewDataOutputStreamWrapper out = new OutputViewDataOutputStreamWrapper(
+ new DataOutputStream(baos)
+ );
+
+ try {
+ outSerializer.serialize(initialValue, out);
+ } catch (IOException ioe) {
+ throw new RuntimeException("Unable to serialize initial value of type " +
+ initialValue.getClass().getSimpleName() + " of fold window function.", ioe);
+ }
+
+ serializedInitialValue = baos.toByteArray();
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/f9eea5e5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperator.java
index a80242d..e6aa53b 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperator.java
@@ -29,6 +29,7 @@ import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.OutputTypeConfigurable;
import org.apache.flink.streaming.api.operators.TimestampedCollector;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
@@ -56,7 +57,7 @@ import java.util.Set;
*/
public class NonKeyedWindowOperator<IN, OUT, W extends Window>
extends AbstractUdfStreamOperator<OUT, AllWindowFunction<IN, OUT, W>>
- implements OneInputStreamOperator<IN, OUT>, Triggerable, InputTypeConfigurable {
+ implements OneInputStreamOperator<IN, OUT>, Triggerable, InputTypeConfigurable, OutputTypeConfigurable<OUT> {
private static final long serialVersionUID = 1L;
@@ -268,6 +269,15 @@ public class NonKeyedWindowOperator<IN, OUT, W extends Window>
return this;
}
+ @Override
+ public void setOutputType(TypeInformation<OUT> outTypeInfo, ExecutionConfig executionConfig) {
+ if (userFunction instanceof OutputTypeConfigurable) {
+ @SuppressWarnings("unchecked")
+ OutputTypeConfigurable<OUT> typeConfigurable = (OutputTypeConfigurable<OUT>) userFunction;
+ typeConfigurable.setOutputType(outTypeInfo, executionConfig);
+ }
+ }
+
// ------------------------------------------------------------------------
// Getters for testing
// ------------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/f9eea5e5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
index 368b8fa..7762101 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
@@ -30,6 +30,7 @@ import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.OutputTypeConfigurable;
import org.apache.flink.streaming.api.operators.TimestampedCollector;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
@@ -74,7 +75,7 @@ import java.util.Set;
*/
public class WindowOperator<K, IN, OUT, W extends Window>
extends AbstractUdfStreamOperator<OUT, WindowFunction<IN, OUT, K, W>>
- implements OneInputStreamOperator<IN, OUT>, Triggerable, InputTypeConfigurable {
+ implements OneInputStreamOperator<IN, OUT>, Triggerable, InputTypeConfigurable, OutputTypeConfigurable<OUT> {
private static final long serialVersionUID = 1L;
@@ -342,6 +343,15 @@ public class WindowOperator<K, IN, OUT, W extends Window>
return this;
}
+ @Override
+ public void setOutputType(TypeInformation<OUT> outTypeInfo, ExecutionConfig executionConfig) {
+ if (userFunction instanceof OutputTypeConfigurable) {
+ @SuppressWarnings("unchecked")
+ OutputTypeConfigurable<OUT> typeConfigurable = (OutputTypeConfigurable<OUT>) userFunction;
+ typeConfigurable.setOutputType(outTypeInfo, executionConfig);
+ }
+ }
+
// ------------------------------------------------------------------------
// Getters for testing
// ------------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/f9eea5e5/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowFoldITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowFoldITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowFoldITCase.java
new file mode 100644
index 0000000..45649bd
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowFoldITCase.java
@@ -0,0 +1,191 @@
+/*
+* 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.runtime.operators.windowing;
+
+import com.google.common.collect.Lists;
+import org.apache.flink.api.common.functions.FoldFunction;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.TimestampExtractor;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Tests for Folds over windows. These also test whether OutputTypeConfigurable functions
+ * work for windows, because FoldWindowFunction is OutputTypeConfigurable.
+ */
+public class WindowFoldITCase extends StreamingMultipleProgramsTestBase {
+
+ private static List<String> testResults;
+
+ @Test
+ public void testFoldWindow() throws Exception {
+
+ testResults = Lists.newArrayList();
+
+ StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+ env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+ env.setParallelism(1);
+
+ DataStream<Tuple2<String, Integer>> source1 = env.addSource(new SourceFunction<Tuple2<String, Integer>>() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public void run(SourceContext<Tuple2<String, Integer>> ctx) throws Exception {
+ ctx.collect(Tuple2.of("a", 0));
+ ctx.collect(Tuple2.of("a", 1));
+ ctx.collect(Tuple2.of("a", 2));
+
+ ctx.collect(Tuple2.of("b", 3));
+ ctx.collect(Tuple2.of("b", 4));
+ ctx.collect(Tuple2.of("b", 5));
+
+ ctx.collect(Tuple2.of("a", 6));
+ ctx.collect(Tuple2.of("a", 7));
+ ctx.collect(Tuple2.of("a", 8));
+ }
+
+ @Override
+ public void cancel() {
+ }
+ }).extractTimestamp(new Tuple2TimestampExtractor());
+
+ source1
+ .keyBy(0)
+ .window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
+ .fold(Tuple2.of("R:", 0), new FoldFunction<Tuple2<String, Integer>, Tuple2<String, Integer>>() {
+ @Override
+ public Tuple2<String, Integer> fold(Tuple2<String, Integer> accumulator,
+ Tuple2<String, Integer> value) throws Exception {
+ accumulator.f0 += value.f0;
+ accumulator.f1 += value.f1;
+ return accumulator;
+ }
+ })
+ .addSink(new SinkFunction<Tuple2<String, Integer>>() {
+ @Override
+ public void invoke(Tuple2<String, Integer> value) throws Exception {
+ testResults.add(value.toString());
+ }
+ });
+
+ env.execute("Fold Window Test");
+
+ List<String> expectedResult = Lists.newArrayList(
+ "(R:aaa,3)",
+ "(R:aaa,21)",
+ "(R:bbb,12)");
+
+ Collections.sort(expectedResult);
+ Collections.sort(testResults);
+
+ Assert.assertEquals(expectedResult, testResults);
+ }
+
+ @Test
+ public void testFoldAllWindow() throws Exception {
+
+ testResults = Lists.newArrayList();
+
+ StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+ env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+ env.setParallelism(1);
+
+ DataStream<Tuple2<String, Integer>> source1 = env.addSource(new SourceFunction<Tuple2<String, Integer>>() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public void run(SourceContext<Tuple2<String, Integer>> ctx) throws Exception {
+ ctx.collect(Tuple2.of("a", 0));
+ ctx.collect(Tuple2.of("a", 1));
+ ctx.collect(Tuple2.of("a", 2));
+
+ ctx.collect(Tuple2.of("b", 3));
+ ctx.collect(Tuple2.of("a", 3));
+ ctx.collect(Tuple2.of("b", 4));
+ ctx.collect(Tuple2.of("a", 4));
+ ctx.collect(Tuple2.of("b", 5));
+ ctx.collect(Tuple2.of("a", 5));
+
+ }
+
+ @Override
+ public void cancel() {
+ }
+ }).extractTimestamp(new Tuple2TimestampExtractor());
+
+ source1
+ .windowAll(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
+ .fold(Tuple2.of("R:", 0), new FoldFunction<Tuple2<String, Integer>, Tuple2<String, Integer>>() {
+ @Override
+ public Tuple2<String, Integer> fold(Tuple2<String, Integer> accumulator,
+ Tuple2<String, Integer> value) throws Exception {
+ accumulator.f0 += value.f0;
+ accumulator.f1 += value.f1;
+ return accumulator;
+ }
+ })
+ .addSink(new SinkFunction<Tuple2<String, Integer>>() {
+ @Override
+ public void invoke(Tuple2<String, Integer> value) throws Exception {
+ testResults.add(value.toString());
+ }
+ });
+
+ env.execute("Fold All-Window Test");
+
+ List<String> expectedResult = Lists.newArrayList(
+ "(R:aaa,3)",
+ "(R:bababa,24)");
+
+ Collections.sort(expectedResult);
+ Collections.sort(testResults);
+
+ Assert.assertEquals(expectedResult, testResults);
+ }
+
+ private static class Tuple2TimestampExtractor implements TimestampExtractor<Tuple2<String, Integer>> {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public long extractTimestamp(Tuple2<String, Integer> element, long currentTimestamp) {
+ return element.f1;
+ }
+
+ @Override
+ public long emitWatermark(Tuple2<String, Integer> element, long currentTimestamp) {
+ return element.f1 - 1;
+ }
+
+ @Override
+ public long getCurrentWatermark() {
+ return Long.MIN_VALUE;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/f9eea5e5/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala
index d2d0a1d..65cafb7 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala
@@ -18,7 +18,7 @@
package org.apache.flink.streaming.api.scala
-import org.apache.flink.api.common.functions.ReduceFunction
+import org.apache.flink.api.common.functions.{FoldFunction, ReduceFunction}
import org.apache.flink.api.common.typeinfo.TypeInformation
import org.apache.flink.streaming.api.datastream.{AllWindowedStream => JavaAllWStream}
import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction.AggregationType
@@ -122,6 +122,47 @@ class AllWindowedStream[T, W <: Window](javaStream: JavaAllWStream[T, W]) {
}
/**
+ * Applies the given fold function to each window. The window function is called for each
+ * evaluation of the window for each key individually. The output of the reduce function is
+ * interpreted as a regular non-windowed stream.
+ *
+ * @param function The fold function.
+ * @return The data stream that is the result of applying the fold function to the window.
+ */
+ def fold[R: TypeInformation: ClassTag](
+ initialValue: R,
+ function: FoldFunction[T,R]): DataStream[R] = {
+ if (function == null) {
+ throw new NullPointerException("Fold function must not be null.")
+ }
+
+ val resultType : TypeInformation[R] = implicitly[TypeInformation[R]]
+
+ javaStream.fold(initialValue, function, resultType)
+ }
+
+ /**
+ * Applies the given fold function to each window. The window function is called for each
+ * evaluation of the window for each key individually. The output of the reduce function is
+ * interpreted as a regular non-windowed stream.
+ *
+ * @param function The fold function.
+ * @return The data stream that is the result of applying the fold function to the window.
+ */
+ def fold[R: TypeInformation: ClassTag](initialValue: R, function: (R, T) => R): DataStream[R] = {
+ if (function == null) {
+ throw new NullPointerException("Fold function must not be null.")
+ }
+ val cleanFun = clean(function)
+ val folder = new FoldFunction[T,R] {
+ def fold(acc: R, v: T) = {
+ cleanFun(acc, v)
+ }
+ }
+ fold(initialValue, folder)
+ }
+
+ /**
* Applies the given window function to each window. The window function is called for each
* evaluation of the window for each key individually. The output of the window function is
* interpreted as a regular non-windowed stream.
http://git-wip-us.apache.org/repos/asf/flink/blob/f9eea5e5/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala
index 3963765..a8ddaf8 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala
@@ -18,7 +18,7 @@
package org.apache.flink.streaming.api.scala
-import org.apache.flink.api.common.functions.ReduceFunction
+import org.apache.flink.api.common.functions.{FoldFunction, ReduceFunction}
import org.apache.flink.api.common.typeinfo.TypeInformation
import org.apache.flink.streaming.api.datastream.{WindowedStream => JavaWStream}
import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction.AggregationType
@@ -125,6 +125,47 @@ class WindowedStream[T, K, W <: Window](javaStream: JavaWStream[T, K, W]) {
}
/**
+ * Applies the given fold function to each window. The window function is called for each
+ * evaluation of the window for each key individually. The output of the reduce function is
+ * interpreted as a regular non-windowed stream.
+ *
+ * @param function The fold function.
+ * @return The data stream that is the result of applying the fold function to the window.
+ */
+ def fold[R: TypeInformation: ClassTag](
+ initialValue: R,
+ function: FoldFunction[T,R]): DataStream[R] = {
+ if (function == null) {
+ throw new NullPointerException("Fold function must not be null.")
+ }
+
+ val resultType : TypeInformation[R] = implicitly[TypeInformation[R]]
+
+ javaStream.fold(initialValue, function, resultType)
+ }
+
+ /**
+ * Applies the given fold function to each window. The window function is called for each
+ * evaluation of the window for each key individually. The output of the reduce function is
+ * interpreted as a regular non-windowed stream.
+ *
+ * @param function The fold function.
+ * @return The data stream that is the result of applying the fold function to the window.
+ */
+ def fold[R: TypeInformation: ClassTag](initialValue: R, function: (R, T) => R): DataStream[R] = {
+ if (function == null) {
+ throw new NullPointerException("Fold function must not be null.")
+ }
+ val cleanFun = clean(function)
+ val folder = new FoldFunction[T,R] {
+ def fold(acc: R, v: T) = {
+ cleanFun(acc, v)
+ }
+ }
+ fold(initialValue, folder)
+ }
+
+ /**
* Applies the given window function to each window. The window function is called for each
* evaluation of the window for each key individually. The output of the window function is
* interpreted as a regular non-windowed stream.
http://git-wip-us.apache.org/repos/asf/flink/blob/f9eea5e5/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowFoldITCase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowFoldITCase.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowFoldITCase.scala
new file mode 100644
index 0000000..dd098a0
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowFoldITCase.scala
@@ -0,0 +1,148 @@
+/*
+ * 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.scala
+
+import java.util.concurrent.TimeUnit
+
+import org.apache.flink.streaming.api.TimeCharacteristic
+import org.apache.flink.streaming.api.functions.TimestampExtractor
+import org.apache.flink.streaming.api.functions.sink.SinkFunction
+import org.apache.flink.streaming.api.functions.source.SourceFunction
+import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows
+import org.apache.flink.streaming.api.windowing.time.Time
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
+import org.junit.Test
+import org.junit.Assert._
+
+import scala.collection.mutable
+
+/**
+ * Tests for Folds over windows. These also test whether OutputTypeConfigurable functions
+ * work for windows, because FoldWindowFunction is OutputTypeConfigurable.
+ */
+class WindowFoldITCase extends StreamingMultipleProgramsTestBase {
+
+ @Test
+ def testFoldWindow(): Unit = {
+ WindowFoldITCase.testResults = mutable.MutableList()
+
+ val env = StreamExecutionEnvironment.getExecutionEnvironment
+ env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+ env.setParallelism(1)
+
+ val source1 = env.addSource(new SourceFunction[(String, Int)]() {
+ def run(ctx: SourceFunction.SourceContext[(String, Int)]) {
+ ctx.collect(("a", 0))
+ ctx.collect(("a", 1))
+ ctx.collect(("a", 2))
+ ctx.collect(("b", 3))
+ ctx.collect(("b", 4))
+ ctx.collect(("b", 5))
+ ctx.collect(("a", 6))
+ ctx.collect(("a", 7))
+ ctx.collect(("a", 8))
+ }
+
+ def cancel() {
+ }
+ }).extractTimestamp(new WindowFoldITCase.Tuple2TimestampExtractor)
+
+ source1
+ .keyBy(0)
+ .window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
+ .fold(("R:", 0), { (acc: (String, Int), v: (String, Int)) => (acc._1 + v._1, acc._2 + v._2) })
+ .addSink(new SinkFunction[(String, Int)]() {
+ def invoke(value: (String, Int)) {
+ WindowFoldITCase.testResults += value.toString
+ }
+ })
+
+ env.execute("Fold Window Test")
+
+ val expectedResult = mutable.MutableList(
+ "(R:aaa,3)",
+ "(R:aaa,21)",
+ "(R:bbb,12)")
+
+ assertEquals(expectedResult.sorted, WindowFoldITCase.testResults.sorted)
+ }
+
+ @Test
+ def testFoldAllWindow(): Unit = {
+ WindowFoldITCase.testResults = mutable.MutableList()
+
+ val env = StreamExecutionEnvironment.getExecutionEnvironment
+ env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+ env.setParallelism(1)
+
+ val source1 = env.addSource(new SourceFunction[(String, Int)]() {
+ def run(ctx: SourceFunction.SourceContext[(String, Int)]) {
+ ctx.collect(("a", 0))
+ ctx.collect(("a", 1))
+ ctx.collect(("a", 2))
+ ctx.collect(("b", 3))
+ ctx.collect(("a", 3))
+ ctx.collect(("b", 4))
+ ctx.collect(("a", 4))
+ ctx.collect(("b", 5))
+ ctx.collect(("a", 5))
+ }
+
+ def cancel() {
+ }
+ }).extractTimestamp(new WindowFoldITCase.Tuple2TimestampExtractor)
+
+ source1
+ .windowAll(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
+ .fold(("R:", 0), { (acc: (String, Int), v: (String, Int)) => (acc._1 + v._1, acc._2 + v._2) })
+ .addSink(new SinkFunction[(String, Int)]() {
+ def invoke(value: (String, Int)) {
+ WindowFoldITCase.testResults += value.toString
+ }
+ })
+
+ env.execute("Fold All-Window Test")
+
+ val expectedResult = mutable.MutableList(
+ "(R:aaa,3)",
+ "(R:bababa,24)")
+
+ assertEquals(expectedResult.sorted, WindowFoldITCase.testResults.sorted)
+ }
+
+}
+
+
+object WindowFoldITCase {
+ private var testResults: mutable.MutableList[String] = null
+
+ private class Tuple2TimestampExtractor extends TimestampExtractor[(String, Int)] {
+ def extractTimestamp(element: (String, Int), currentTimestamp: Long): Long = {
+ element._2
+ }
+
+ def emitWatermark(element: (String, Int), currentTimestamp: Long): Long = {
+ element._2 - 1
+ }
+
+ def getCurrentWatermark: Long = {
+ Long.MinValue
+ }
+ }
+}
[5/8] flink git commit: [FLINK-2819] Add Windowed Join/CoGroup
Operator Based on Tagged Union
Posted by al...@apache.org.
[FLINK-2819] Add Windowed Join/CoGroup Operator Based on Tagged Union
Right now, this does everything in memory, so the JVM will blow if data
for one key becomes too large.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/8634dbbe
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/8634dbbe
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/8634dbbe
Branch: refs/heads/master
Commit: 8634dbbe998af53471bed2f3a6557c722bb37b87
Parents: 47b5cb7
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Tue Oct 6 16:33:04 2015 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Wed Oct 7 22:08:25 2015 +0200
----------------------------------------------------------------------
.../api/datastream/CoGroupedStreams.java | 563 +++++++++++++++++++
.../streaming/api/datastream/DataStream.java | 36 +-
.../streaming/api/datastream/JoinedStreams.java | 328 +++++++++++
.../datastream/temporal/StreamJoinOperator.java | 274 ---------
.../datastream/temporal/TemporalOperator.java | 124 ----
.../api/datastream/temporal/TemporalWindow.java | 45 --
.../windowing/ReduceWindowFunction.java | 32 +-
.../flink/streaming/api/CoGroupJoinITCase.java | 372 ++++++++++++
.../streaming/api/WindowCrossJoinTest.java | 143 -----
.../api/operators/co/SelfConnectionTest.java | 61 --
.../streaming/examples/join/WindowJoin.java | 128 +++--
.../examples/join/util/WindowJoinData.java | 70 +--
.../scala/examples/join/WindowJoin.scala | 43 +-
.../join/WindowJoinITCase.java | 101 ++--
.../join/WindowJoinITCase.java | 101 ++--
.../streaming/api/scala/CoGroupedStreams.scala | 294 ++++++++++
.../flink/streaming/api/scala/DataStream.scala | 24 +-
.../streaming/api/scala/JoinedStreams.scala | 303 ++++++++++
.../api/scala/StreamJoinOperator.scala | 203 -------
.../streaming/api/scala/TemporalOperator.scala | 51 --
.../streaming/api/scala/CoGroupJoinITCase.scala | 274 +++++++++
.../StreamingScalaAPICompletenessTest.scala | 14 +-
22 files changed, 2409 insertions(+), 1175 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java
new file mode 100644
index 0000000..e1f1a96
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java
@@ -0,0 +1,563 @@
+/*
+ * 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.datastream;
+
+import com.google.common.collect.Lists;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.functions.CoGroupFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.operators.translation.WrappingFunction;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
+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.triggers.Trigger;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+import org.apache.flink.util.Collector;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ *{@code CoGroupedStreams} represents two {@link DataStream DataStreams} that have been co-grouped.
+ * A streaming co-group operation is evaluated over elements in a window.
+ *
+ * <p>
+ * To finalize co-group operation you also need to specify a {@link KeySelector} for
+ * both the first and second input and a {@link WindowAssigner}.
+ *
+ * <p>
+ * Note: Right now, the groups are being built in memory so you need to ensure that they don't
+ * get too big. Otherwise the JVM might crash.
+ *
+ * <p>
+ * Example:
+ *
+ * <pre> {@code
+ * DataStream<Tuple2<String, Integer>> one = ...;
+ * DataStream<Tuple2<String, Integer>> two = ...;
+ *
+ * DataStream<T> result = one.coGroup(two)
+ * .where(new MyFirstKeySelector())
+ * .equalTo(new MyFirstKeySelector())
+ * .window(TumblingTimeWindows.of(Time.of(5, TimeUnit.SECONDS)))
+ * .apply(new MyCoGroupFunction());
+ * } </pre>
+ */
+public class CoGroupedStreams {
+
+ /**
+ * A co-group operation that does not yet have its {@link KeySelector KeySelectors} defined.
+ *
+ * @param <T1> Type of the elements from the first input
+ * @param <T2> Type of the elements from the second input
+ */
+ public static class Unspecified<T1, T2> {
+ DataStream<T1> input1;
+ DataStream<T2> input2;
+
+ protected Unspecified(DataStream<T1> input1,
+ DataStream<T2> input2) {
+ this.input1 = input1;
+ this.input2 = input2;
+ }
+
+ /**
+ * Specifies a {@link KeySelector} for elements from the first input.
+ */
+ public <KEY> WithKey<T1, T2, KEY> where(KeySelector<T1, KEY> keySelector) {
+ return new WithKey<>(input1, input2, input1.clean(keySelector), null);
+ }
+
+ /**
+ * Specifies a {@link KeySelector} for elements from the second input.
+ */
+ public <KEY> WithKey<T1, T2, KEY> equalTo(KeySelector<T2, KEY> keySelector) {
+ return new WithKey<>(input1, input2, null, input1.clean(keySelector));
+ }
+ }
+
+ /**
+ * A co-group operation that has {@link KeySelector KeySelectors} defined for either both or
+ * one input.
+ *
+ * <p>
+ * You need to specify a {@code KeySelector} for both inputs using {@link #where(KeySelector)}
+ * and {@link #equalTo(KeySelector)} before you can proceeed with specifying a
+ * {@link WindowAssigner} using {@link #window(WindowAssigner)}.
+ *
+ * @param <T1> Type of the elements from the first input
+ * @param <T2> Type of the elements from the second input
+ * @param <KEY> Type of the key. This must be the same for both inputs
+ */
+ public static class WithKey<T1, T2, KEY> {
+ DataStream<T1> input1;
+ DataStream<T2> input2;
+
+ KeySelector<T1, KEY> keySelector1;
+ KeySelector<T2, KEY> keySelector2;
+
+ protected WithKey(DataStream<T1> input1, DataStream<T2> input2, KeySelector<T1, KEY> keySelector1, KeySelector<T2, KEY> keySelector2) {
+ this.input1 = input1;
+ this.input2 = input2;
+
+ this.keySelector1 = keySelector1;
+ this.keySelector2 = keySelector2;
+ }
+
+ /**
+ * Specifies a {@link KeySelector} for elements from the first input.
+ */
+ public WithKey<T1, T2, KEY> where(KeySelector<T1, KEY> keySelector) {
+ return new CoGroupedStreams.WithKey<>(input1, input2, input1.clean(keySelector), keySelector2);
+ }
+
+ /**
+ * Specifies a {@link KeySelector} for elements from the second input.
+ */
+ public CoGroupedStreams.WithKey<T1, T2, KEY> equalTo(KeySelector<T2, KEY> keySelector) {
+ return new CoGroupedStreams.WithKey<>(input1, input2, keySelector1, input1.clean(keySelector));
+ }
+
+ /**
+ * Specifies the window on which the co-group operation works.
+ */
+ public <W extends Window> CoGroupedStreams.WithWindow<T1, T2, KEY, W> window(WindowAssigner<? super TaggedUnion<T1, T2>, W> assigner) {
+ if (keySelector1 == null || keySelector2 == null) {
+ throw new UnsupportedOperationException("You first need to specify KeySelectors for both inputs using where() and equalTo().");
+
+ }
+ return new WithWindow<>(input1, input2, keySelector1, keySelector2, assigner, null, null);
+ }
+ }
+
+ /**
+ * A co-group operation that has {@link KeySelector KeySelectors} defined for both inputs as
+ * well as a {@link WindowAssigner}.
+ *
+ * @param <T1> Type of the elements from the first input
+ * @param <T2> Type of the elements from the second input
+ * @param <KEY> Type of the key. This must be the same for both inputs
+ * @param <W> Type of {@link Window} on which the co-group operation works.
+ */
+ public static class WithWindow<T1, T2, KEY, W extends Window> {
+ private final DataStream<T1> input1;
+ private final DataStream<T2> input2;
+
+ private final KeySelector<T1, KEY> keySelector1;
+ private final KeySelector<T2, KEY> keySelector2;
+
+ private final WindowAssigner<? super TaggedUnion<T1, T2>, W> windowAssigner;
+
+ private final Trigger<? super TaggedUnion<T1, T2>, ? super W> trigger;
+
+ private final Evictor<? super TaggedUnion<T1, T2>, ? super W> evictor;
+
+ protected WithWindow(DataStream<T1> input1,
+ DataStream<T2> input2,
+ KeySelector<T1, KEY> keySelector1,
+ KeySelector<T2, KEY> keySelector2,
+ WindowAssigner<? super TaggedUnion<T1, T2>, W> windowAssigner,
+ Trigger<? super TaggedUnion<T1, T2>, ? super W> trigger,
+ Evictor<? super TaggedUnion<T1, T2>, ? super W> evictor) {
+ this.input1 = input1;
+ this.input2 = input2;
+
+ this.keySelector1 = keySelector1;
+ this.keySelector2 = keySelector2;
+
+ this.windowAssigner = windowAssigner;
+ this.trigger = trigger;
+ this.evictor = evictor;
+ }
+
+ /**
+ * Sets the {@code Trigger} that should be used to trigger window emission.
+ */
+ public WithWindow<T1, T2, KEY, W> trigger(Trigger<? super TaggedUnion<T1, T2>, ? super W> newTrigger) {
+ return new WithWindow<>(input1, input2, keySelector1, keySelector2, windowAssigner, newTrigger, evictor);
+ }
+
+ /**
+ * Sets the {@code Evictor} that should be used to evict elements from a window before emission.
+ *
+ * <p>
+ * Note: When using an evictor window performance will degrade significantly, since
+ * pre-aggregation of window results cannot be used.
+ */
+ public WithWindow<T1, T2, KEY, W> evictor(Evictor<? super TaggedUnion<T1, T2>, ? super W> newEvictor) {
+ return new WithWindow<>(input1, input2, keySelector1, keySelector2, windowAssigner, trigger, newEvictor);
+ }
+
+ /**
+ * Completes the co-group operation with the user function that is executed
+ * for windowed groups.
+ */
+ public <T> DataStream<T> apply(CoGroupFunction<T1, T2, T> function) {
+
+ TypeInformation<T> resultType = TypeExtractor.getBinaryOperatorReturnType(
+ function,
+ CoGroupFunction.class,
+ true,
+ true,
+ input1.getType(),
+ input2.getType(),
+ "CoGroup",
+ false);
+
+ return apply(function, resultType);
+ }
+
+ /**
+ * Completes the co-group operation with the user function that is executed
+ * for windowed groups.
+ */
+ public <T> DataStream<T> apply(CoGroupFunction<T1, T2, T> function, TypeInformation<T> resultType) {
+ //clean the closure
+ function = input1.getExecutionEnvironment().clean(function);
+
+ DataStream<TaggedUnion<T1, T2>> taggedInput1 = input1
+ .map(new Input1Tagger<T1, T2>())
+ .returns(new UnionTypeInfo<>(input1.getType(), input2.getType()));
+ DataStream<TaggedUnion<T1, T2>> taggedInput2 = input2
+ .map(new Input2Tagger<T1, T2>())
+ .returns(new UnionTypeInfo<>(input1.getType(), input2.getType()));
+
+ WindowedStream<TaggedUnion<T1, T2>, KEY, W> windowOp = taggedInput1
+ .union(taggedInput2)
+ .keyBy(new UnionKeySelector<>(keySelector1, keySelector2))
+ .window(windowAssigner);
+
+ if (trigger != null) {
+ windowOp.trigger(trigger);
+ }
+ if (evictor != null) {
+ windowOp.evictor(evictor);
+ }
+
+ return windowOp.apply(new CoGroupWindowFunction<T1, T2, T, KEY, W>(function), resultType);
+ }
+ }
+
+ /**
+ * Creates a new co-group operation from the two given inputs.
+ */
+ public static <T1, T2> Unspecified<T1, T2> createCoGroup(DataStream<T1> input1, DataStream<T2> input2) {
+ return new Unspecified<>(input1, input2);
+ }
+
+ /**
+ * Internal class for implementing tagged union co-group.
+ */
+ public static class TaggedUnion<T1, T2> {
+ private final T1 one;
+ private final T2 two;
+
+ private TaggedUnion(T1 one, T2 two) {
+ this.one = one;
+ this.two = two;
+ }
+
+ public boolean isOne() {
+ return one != null;
+ }
+
+ public boolean isTwo() {
+ return two != null;
+ }
+
+ public T1 getOne() {
+ return one;
+ }
+
+ public T2 getTwo() {
+ return two;
+ }
+
+ public static <T1, T2> TaggedUnion<T1, T2> one(T1 one) {
+ return new TaggedUnion<>(one, null);
+ }
+
+ public static <T1, T2> TaggedUnion<T1, T2> two(T2 two) {
+ return new TaggedUnion<>(null, two);
+ }
+ }
+
+ private static class UnionTypeInfo<T1, T2> extends TypeInformation<TaggedUnion<T1, T2>> {
+ private static final long serialVersionUID = 1L;
+
+ TypeInformation<T1> oneType;
+ TypeInformation<T2> twoType;
+
+ public UnionTypeInfo(TypeInformation<T1> oneType,
+ TypeInformation<T2> twoType) {
+ this.oneType = oneType;
+ this.twoType = twoType;
+ }
+
+ @Override
+ public boolean isBasicType() {
+ return false;
+ }
+
+ @Override
+ public boolean isTupleType() {
+ return false;
+ }
+
+ @Override
+ public int getArity() {
+ return 2;
+ }
+
+ @Override
+ public int getTotalFields() {
+ return 2;
+ }
+
+ @Override
+ @SuppressWarnings("unchecked, rawtypes")
+ public Class<TaggedUnion<T1, T2>> getTypeClass() {
+ return (Class) TaggedUnion.class;
+ }
+
+ @Override
+ public boolean isKeyType() {
+ return true;
+ }
+
+ @Override
+ public TypeSerializer<TaggedUnion<T1, T2>> createSerializer(ExecutionConfig config) {
+ return new UnionSerializer<>(oneType.createSerializer(config), twoType.createSerializer(config));
+ }
+
+ @Override
+ public String toString() {
+ return "TaggedUnion<" + oneType + ", " + twoType + ">";
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof UnionTypeInfo) {
+ @SuppressWarnings("unchecked")
+ UnionTypeInfo<T1, T2> unionTypeInfo = (UnionTypeInfo<T1, T2>) obj;
+
+ return unionTypeInfo.canEqual(this) && oneType.equals(unionTypeInfo.oneType) && twoType.equals(unionTypeInfo.twoType);
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public int hashCode() {
+ return 31 * oneType.hashCode() + twoType.hashCode();
+ }
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof UnionTypeInfo;
+ }
+ }
+
+ private static class UnionSerializer<T1, T2> extends TypeSerializer<TaggedUnion<T1, T2>> {
+ private static final long serialVersionUID = 1L;
+
+ private final TypeSerializer<T1> oneSerializer;
+ private final TypeSerializer<T2> twoSerializer;
+
+ public UnionSerializer(TypeSerializer<T1> oneSerializer,
+ TypeSerializer<T2> twoSerializer) {
+ this.oneSerializer = oneSerializer;
+ this.twoSerializer = twoSerializer;
+ }
+
+ @Override
+ public boolean isImmutableType() {
+ return false;
+ }
+
+ @Override
+ public TypeSerializer<TaggedUnion<T1, T2>> duplicate() {
+ return this;
+ }
+
+ @Override
+ public TaggedUnion<T1, T2> createInstance() {
+ return null;
+ }
+
+ @Override
+ public TaggedUnion<T1, T2> copy(TaggedUnion<T1, T2> from) {
+ if (from.isOne()) {
+ return TaggedUnion.one(oneSerializer.copy(from.getOne()));
+ } else {
+ return TaggedUnion.two(twoSerializer.copy(from.getTwo()));
+ }
+ }
+
+ @Override
+ public TaggedUnion<T1, T2> copy(TaggedUnion<T1, T2> from, TaggedUnion<T1, T2> reuse) {
+ if (from.isOne()) {
+ return TaggedUnion.one(oneSerializer.copy(from.getOne()));
+ } else {
+ return TaggedUnion.two(twoSerializer.copy(from.getTwo()));
+ } }
+
+ @Override
+ public int getLength() {
+ return 0;
+ }
+
+ @Override
+ public void serialize(TaggedUnion<T1, T2> record, DataOutputView target) throws IOException {
+ if (record.isOne()) {
+ target.writeByte(1);
+ oneSerializer.serialize(record.getOne(), target);
+ } else {
+ target.writeByte(2);
+ twoSerializer.serialize(record.getTwo(), target);
+ }
+ }
+
+ @Override
+ public TaggedUnion<T1, T2> deserialize(DataInputView source) throws IOException {
+ byte tag = source.readByte();
+ if (tag == 1) {
+ return TaggedUnion.one(oneSerializer.deserialize(source));
+ } else {
+ return TaggedUnion.two(twoSerializer.deserialize(source));
+ }
+ }
+
+ @Override
+ public TaggedUnion<T1, T2> deserialize(TaggedUnion<T1, T2> reuse,
+ DataInputView source) throws IOException {
+ byte tag = source.readByte();
+ if (tag == 1) {
+ return TaggedUnion.one(oneSerializer.deserialize(source));
+ } else {
+ return TaggedUnion.two(twoSerializer.deserialize(source));
+ }
+ }
+
+ @Override
+ public void copy(DataInputView source, DataOutputView target) throws IOException {
+ byte tag = source.readByte();
+ target.writeByte(tag);
+ if (tag == 1) {
+ oneSerializer.copy(source, target);
+ } else {
+ twoSerializer.copy(source, target);
+ }
+ }
+
+ @Override
+ public int hashCode() {
+ return 31 * oneSerializer.hashCode() + twoSerializer.hashCode();
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public boolean equals(Object obj) {
+ if (obj instanceof UnionSerializer) {
+ UnionSerializer<T1, T2> other = (UnionSerializer<T1, T2>) obj;
+
+ return other.canEqual(this) && oneSerializer.equals(other.oneSerializer) && twoSerializer.equals(other.twoSerializer);
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof UnionSerializer;
+ }
+ }
+
+ private static class Input1Tagger<T1, T2> implements MapFunction<T1, TaggedUnion<T1, T2>> {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public TaggedUnion<T1, T2> map(T1 value) throws Exception {
+ return TaggedUnion.one(value);
+ }
+ }
+
+ private static class Input2Tagger<T1, T2> implements MapFunction<T2, TaggedUnion<T1, T2>> {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public TaggedUnion<T1, T2> map(T2 value) throws Exception {
+ return TaggedUnion.two(value);
+ }
+ }
+
+ private static class UnionKeySelector<T1, T2, KEY> implements KeySelector<TaggedUnion<T1, T2>, KEY> {
+ private static final long serialVersionUID = 1L;
+
+ private final KeySelector<T1, KEY> keySelector1;
+ private final KeySelector<T2, KEY> keySelector2;
+
+ public UnionKeySelector(KeySelector<T1, KEY> keySelector1,
+ KeySelector<T2, KEY> keySelector2) {
+ this.keySelector1 = keySelector1;
+ this.keySelector2 = keySelector2;
+ }
+
+ @Override
+ public KEY getKey(TaggedUnion<T1, T2> value) throws Exception{
+ if (value.isOne()) {
+ return keySelector1.getKey(value.getOne());
+ } else {
+ return keySelector2.getKey(value.getTwo());
+ }
+ }
+ }
+
+ private static class CoGroupWindowFunction<T1, T2, T, KEY, W extends Window>
+ extends WrappingFunction<CoGroupFunction<T1, T2, T>>
+ implements WindowFunction<TaggedUnion<T1, T2>, T, KEY, W> {
+ private static final long serialVersionUID = 1L;
+
+ public CoGroupWindowFunction(CoGroupFunction<T1, T2, T> userFunction) {
+ super(userFunction);
+ }
+
+ @Override
+ public void apply(KEY key,
+ W window,
+ Iterable<TaggedUnion<T1, T2>> values,
+ Collector<T> out) throws Exception {
+ List<T1> oneValues = Lists.newArrayList();
+ List<T2> twoValues = Lists.newArrayList();
+ for (TaggedUnion<T1, T2> val: values) {
+ if (val.isOne()) {
+ oneValues.add(val.getOne());
+ } else {
+ twoValues.add(val.getTwo());
+ }
+ }
+ wrappedFunction.coGroup(oneValues, twoValues, out);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
index 8de1a0d..0be1d56 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
@@ -44,7 +44,6 @@ import org.apache.flink.api.java.typeutils.TypeExtractor;
import org.apache.flink.core.fs.FileSystem.WriteMode;
import org.apache.flink.core.fs.Path;
import org.apache.flink.streaming.api.collector.selector.OutputSelector;
-import org.apache.flink.streaming.api.datastream.temporal.StreamJoinOperator;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.TimestampExtractor;
import org.apache.flink.streaming.api.functions.sink.FileSinkFunctionByMillis;
@@ -618,30 +617,19 @@ public class DataStream<T> {
}
/**
- * Initiates a temporal Join transformation. <br/>
- * A temporal Join transformation joins the elements of two
- * {@link DataStream}s on key equality over a specified time window.
- *
- * <p>
- * This method returns a {@link StreamJoinOperator} on which the
- * {@link StreamJoinOperator#onWindow(long, java.util.concurrent.TimeUnit)}
- * should be called to define the window, and then the
- * {@link StreamJoinOperator.JoinWindow#where(int...)} and
- * {@link StreamJoinOperator.JoinPredicate#equalTo(int...)} can be used to define
- * the join keys.
- * <p>
- * The user can also use the
- * {@link org.apache.flink.streaming.api.datastream.temporal.StreamJoinOperator.JoinPredicate.JoinedStream#with}
- * to apply a custom join function.
- *
- * @param dataStreamToJoin
- * The other DataStream with which this DataStream is joined.
- * @return A {@link StreamJoinOperator} to continue the definition of the
- * Join transformation.
- *
+ * Creates a join operation. See {@link CoGroupedStreams} for an example of how the keys
+ * and window can be specified.
+ */
+ public <T2> CoGroupedStreams.Unspecified<T, T2> coGroup(DataStream<T2> otherStream) {
+ return CoGroupedStreams.createCoGroup(this, otherStream);
+ }
+
+ /**
+ * Creates a join operation. See {@link JoinedStreams} for an example of how the keys
+ * and window can be specified.
*/
- public <IN2> StreamJoinOperator<T, IN2> join(DataStream<IN2> dataStreamToJoin) {
- return new StreamJoinOperator<T, IN2>(this, dataStreamToJoin);
+ public <T2> JoinedStreams.Unspecified<T, T2> join(DataStream<T2> otherStream) {
+ return JoinedStreams.createJoin(this, otherStream);
}
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/JoinedStreams.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/JoinedStreams.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/JoinedStreams.java
new file mode 100644
index 0000000..ee848e3
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/JoinedStreams.java
@@ -0,0 +1,328 @@
+/*
+ * 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.datastream;
+
+import org.apache.flink.api.common.functions.CoGroupFunction;
+import org.apache.flink.api.common.functions.FlatJoinFunction;
+import org.apache.flink.api.common.functions.JoinFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.operators.translation.WrappingFunction;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+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.triggers.Trigger;
+import org.apache.flink.streaming.api.windowing.windows.Window;
+import org.apache.flink.util.Collector;
+
+/**
+ *{@code JoinedStreams} represents two {@link DataStream DataStreams} that have been joined.
+ * A streaming join operation is evaluated over elements in a window.
+ *
+ * <p>
+ * To finalize the join operation you also need to specify a {@link KeySelector} for
+ * both the first and second input and a {@link WindowAssigner}.
+ *
+ * <p>
+ * Note: Right now, the the join is being evaluated in memory so you need to ensure that the number
+ * of elements per key does not get too high. Otherwise the JVM might crash.
+ *
+ * <p>
+ * Example:
+ *
+ * <pre> {@code
+ * DataStream<Tuple2<String, Integer>> one = ...;
+ * DataStream<Tuple2<String, Integer>> twp = ...;
+ *
+ * DataStream<T> result = one.join(two)
+ * .where(new MyFirstKeySelector())
+ * .equalTo(new MyFirstKeySelector())
+ * .window(TumblingTimeWindows.of(Time.of(5, TimeUnit.SECONDS)))
+ * .apply(new MyJoinFunction());
+ * } </pre>
+ */
+public class JoinedStreams extends CoGroupedStreams{
+
+ /**
+ * A join operation that does not yet have its {@link KeySelector KeySelectors} defined.
+ *
+ * @param <T1> Type of the elements from the first input
+ * @param <T2> Type of the elements from the second input
+ */
+ public static class Unspecified<T1, T2> {
+ DataStream<T1> input1;
+ DataStream<T2> input2;
+
+ protected Unspecified(DataStream<T1> input1,
+ DataStream<T2> input2) {
+ this.input1 = input1;
+ this.input2 = input2;
+ }
+
+ /**
+ * Specifies a {@link KeySelector} for elements from the first input.
+ */
+ public <KEY> WithKey<T1, T2, KEY> where(KeySelector<T1, KEY> keySelector) {
+ return new WithKey<>(input1, input2, keySelector, null);
+ }
+
+ /**
+ * Specifies a {@link KeySelector} for elements from the second input.
+ */
+ public <KEY> WithKey<T1, T2, KEY> equalTo(KeySelector<T2, KEY> keySelector) {
+ return new WithKey<>(input1, input2, null, keySelector);
+ }
+ }
+
+ /**
+ * A join operation that has {@link KeySelector KeySelectors} defined for either both or
+ * one input.
+ *
+ * <p>
+ * You need to specify a {@code KeySelector} for both inputs using {@link #where(KeySelector)}
+ * and {@link #equalTo(KeySelector)} before you can proceeed with specifying a
+ * {@link WindowAssigner} using {@link #window(WindowAssigner)}.
+ *
+ * @param <T1> Type of the elements from the first input
+ * @param <T2> Type of the elements from the second input
+ * @param <KEY> Type of the key. This must be the same for both inputs
+ */
+ public static class WithKey<T1, T2, KEY> {
+ DataStream<T1> input1;
+ DataStream<T2> input2;
+
+ KeySelector<T1, KEY> keySelector1;
+ KeySelector<T2, KEY> keySelector2;
+
+ protected WithKey(DataStream<T1> input1, DataStream<T2> input2, KeySelector<T1, KEY> keySelector1, KeySelector<T2, KEY> keySelector2) {
+ this.input1 = input1;
+ this.input2 = input2;
+
+ this.keySelector1 = keySelector1;
+ this.keySelector2 = keySelector2;
+ }
+
+ /**
+ * Specifies a {@link KeySelector} for elements from the first input.
+ */
+ public WithKey<T1, T2, KEY> where(KeySelector<T1, KEY> keySelector) {
+ return new JoinedStreams.WithKey<>(input1, input2, keySelector, keySelector2);
+ }
+
+ /**
+ * Specifies a {@link KeySelector} for elements from the second input.
+ */
+ public JoinedStreams.WithKey<T1, T2, KEY> equalTo(KeySelector<T2, KEY> keySelector) {
+ return new JoinedStreams.WithKey<>(input1, input2, keySelector1, keySelector);
+ }
+
+ /**
+ * Specifies the window on which the join operation works.
+ */
+ public <W extends Window> JoinedStreams.WithWindow<T1, T2, KEY, W> window(WindowAssigner<? super TaggedUnion<T1, T2>, W> assigner) {
+ if (keySelector1 == null || keySelector2 == null) {
+ throw new UnsupportedOperationException("You first need to specify KeySelectors for both inputs using where() and equalTo().");
+
+ }
+ return new WithWindow<>(input1, input2, keySelector1, keySelector2, assigner, null, null);
+ }
+ }
+
+ /**
+ * A join operation that has {@link KeySelector KeySelectors} defined for both inputs as
+ * well as a {@link WindowAssigner}.
+ *
+ * @param <T1> Type of the elements from the first input
+ * @param <T2> Type of the elements from the second input
+ * @param <KEY> Type of the key. This must be the same for both inputs
+ * @param <W> Type of {@link Window} on which the join operation works.
+ */
+ public static class WithWindow<T1, T2, KEY, W extends Window> {
+ private final DataStream<T1> input1;
+ private final DataStream<T2> input2;
+
+ private final KeySelector<T1, KEY> keySelector1;
+ private final KeySelector<T2, KEY> keySelector2;
+
+ private final WindowAssigner<? super TaggedUnion<T1, T2>, W> windowAssigner;
+
+ private final Trigger<? super TaggedUnion<T1, T2>, ? super W> trigger;
+
+ private final Evictor<? super TaggedUnion<T1, T2>, ? super W> evictor;
+
+ protected WithWindow(DataStream<T1> input1,
+ DataStream<T2> input2,
+ KeySelector<T1, KEY> keySelector1,
+ KeySelector<T2, KEY> keySelector2,
+ WindowAssigner<? super TaggedUnion<T1, T2>, W> windowAssigner,
+ Trigger<? super TaggedUnion<T1, T2>, ? super W> trigger,
+ Evictor<? super TaggedUnion<T1, T2>, ? super W> evictor) {
+ this.input1 = input1;
+ this.input2 = input2;
+
+ this.keySelector1 = keySelector1;
+ this.keySelector2 = keySelector2;
+
+ this.windowAssigner = windowAssigner;
+ this.trigger = trigger;
+ this.evictor = evictor;
+ }
+
+ /**
+ * Sets the {@code Trigger} that should be used to trigger window emission.
+ */
+ public WithWindow<T1, T2, KEY, W> trigger(Trigger<? super TaggedUnion<T1, T2>, ? super W> newTrigger) {
+ return new WithWindow<>(input1, input2, keySelector1, keySelector2, windowAssigner, newTrigger, evictor);
+ }
+
+ /**
+ * Sets the {@code Evictor} that should be used to evict elements from a window before emission.
+ *
+ * <p>
+ * Note: When using an evictor window performance will degrade significantly, since
+ * pre-aggregation of window results cannot be used.
+ */
+ public WithWindow<T1, T2, KEY, W> evictor(Evictor<? super TaggedUnion<T1, T2>, ? super W> newEvictor) {
+ return new WithWindow<>(input1, input2, keySelector1, keySelector2, windowAssigner, trigger, newEvictor);
+ }
+
+ /**
+ * Completes the join operation with the user function that is executed
+ * for each combination of elements with the same key in a window.
+ */
+ public <T> DataStream<T> apply(JoinFunction<T1, T2, T> function) {
+ TypeInformation<T> resultType = TypeExtractor.getBinaryOperatorReturnType(
+ function,
+ JoinFunction.class,
+ true,
+ true,
+ input1.getType(),
+ input2.getType(),
+ "CoGroup",
+ false);
+
+ return apply(function, resultType);
+ }
+
+ /**
+ * Completes the join operation with the user function that is executed
+ * for each combination of elements with the same key in a window.
+ */
+ public <T> DataStream<T> apply(FlatJoinFunction<T1, T2, T> function, TypeInformation<T> resultType) {
+ //clean the closure
+ function = input1.getExecutionEnvironment().clean(function);
+
+ return input1.coGroup(input2)
+ .where(keySelector1)
+ .equalTo(keySelector2)
+ .window(windowAssigner)
+ .trigger(trigger)
+ .evictor(evictor)
+ .apply(new FlatJoinCoGroupFunction<>(function), resultType);
+
+ }
+
+ /**
+ * Completes the join operation with the user function that is executed
+ * for each combination of elements with the same key in a window.
+ */
+ public <T> DataStream<T> apply(FlatJoinFunction<T1, T2, T> function) {
+ TypeInformation<T> resultType = TypeExtractor.getBinaryOperatorReturnType(
+ function,
+ JoinFunction.class,
+ true,
+ true,
+ input1.getType(),
+ input2.getType(),
+ "CoGroup",
+ false);
+
+ return apply(function, resultType);
+ }
+
+ /**
+ * Completes the join operation with the user function that is executed
+ * for each combination of elements with the same key in a window.
+ */
+ public <T> DataStream<T> apply(JoinFunction<T1, T2, T> function, TypeInformation<T> resultType) {
+ //clean the closure
+ function = input1.getExecutionEnvironment().clean(function);
+
+ return input1.coGroup(input2)
+ .where(keySelector1)
+ .equalTo(keySelector2)
+ .window(windowAssigner)
+ .trigger(trigger)
+ .evictor(evictor)
+ .apply(new JoinCoGroupFunction<>(function), resultType);
+
+ }
+ }
+
+ /**
+ * Creates a new join operation from the two given inputs.
+ */
+ public static <T1, T2> Unspecified<T1, T2> createJoin(DataStream<T1> input1, DataStream<T2> input2) {
+ return new Unspecified<>(input1, input2);
+ }
+
+ /**
+ * CoGroup function that does a nested-loop join to get the join result.
+ */
+ private static class JoinCoGroupFunction<T1, T2, T>
+ extends WrappingFunction<JoinFunction<T1, T2, T>>
+ implements CoGroupFunction<T1, T2, T> {
+ private static final long serialVersionUID = 1L;
+
+ public JoinCoGroupFunction(JoinFunction<T1, T2, T> wrappedFunction) {
+ super(wrappedFunction);
+ }
+
+ @Override
+ public void coGroup(Iterable<T1> first, Iterable<T2> second, Collector<T> out) throws Exception {
+ for (T1 val1: first) {
+ for (T2 val2: second) {
+ out.collect(wrappedFunction.join(val1, val2));
+ }
+ }
+ }
+ }
+
+ /**
+ * CoGroup function that does a nested-loop join to get the join result. (FlatJoin version)
+ */
+ private static class FlatJoinCoGroupFunction<T1, T2, T>
+ extends WrappingFunction<FlatJoinFunction<T1, T2, T>>
+ implements CoGroupFunction<T1, T2, T> {
+ private static final long serialVersionUID = 1L;
+
+ public FlatJoinCoGroupFunction(FlatJoinFunction<T1, T2, T> wrappedFunction) {
+ super(wrappedFunction);
+ }
+
+ @Override
+ public void coGroup(Iterable<T1> first, Iterable<T2> second, Collector<T> out) throws Exception {
+ for (T1 val1: first) {
+ for (T2 val2: second) {
+ wrappedFunction.join(val1, val2, out);
+ }
+ }
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamJoinOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamJoinOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamJoinOperator.java
deleted file mode 100644
index 4a5622d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/StreamJoinOperator.java
+++ /dev/null
@@ -1,274 +0,0 @@
-/*
- * 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.datastream.temporal;
-
-import java.util.concurrent.TimeUnit;
-
-import org.apache.flink.api.common.functions.JoinFunction;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.operators.Keys;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
-import org.apache.flink.streaming.util.keys.KeySelectorUtil;
-
-public class StreamJoinOperator<I1, I2> extends
- TemporalOperator<I1, I2, StreamJoinOperator.JoinWindow<I1, I2>> {
-
- public StreamJoinOperator(DataStream<I1> input1, DataStream<I2> input2) {
- super(input1, input2);
- }
-
- @Override
- protected JoinWindow<I1, I2> createNextWindowOperator() {
- return new JoinWindow<I1, I2>(this);
- }
-
- public static class JoinWindow<I1, I2> implements TemporalWindow<JoinWindow<I1, I2>> {
-
- private StreamJoinOperator<I1, I2> op;
- private TypeInformation<I1> type1;
-
- private JoinWindow(StreamJoinOperator<I1, I2> operator) {
- this.op = operator;
- this.type1 = op.input1.getType();
- }
-
- /**
- * Continues a temporal Join transformation. <br/>
- * Defines the {@link Tuple} fields of the first join {@link DataStream}
- * that should be used as join keys.<br/>
- * <b>Note: Fields can only be selected as join keys on Tuple
- * DataStreams.</b><br/>
- *
- * @param fields
- * The indexes of the other Tuple fields of the first join
- * DataStreams that should be used as keys.
- * @return An incomplete Join transformation. Call
- * {@link JoinPredicate#equalTo} to continue the Join.
- */
- public JoinPredicate<I1, I2> where(int... fields) {
- return new JoinPredicate<I1, I2>(op, KeySelectorUtil.getSelectorForKeys(
- new Keys.ExpressionKeys<I1>(fields, type1), type1, op.input1.getExecutionEnvironment().getConfig()));
- }
-
- /**
- * Continues a temporal join transformation. <br/>
- * Defines the fields of the first join {@link DataStream} that should
- * be used as grouping keys. Fields are the names of member fields of
- * the underlying type of the data stream.
- *
- * @param fields
- * The fields of the first join DataStream that should be
- * used as keys.
- * @return An incomplete Join transformation. Call
- * {@link JoinPredicate#equalTo} to continue the Join.
- */
- public JoinPredicate<I1, I2> where(String... fields) {
- return new JoinPredicate<I1, I2>(op, KeySelectorUtil.getSelectorForKeys(
- new Keys.ExpressionKeys<I1>(fields, type1), type1, op.input1.getExecutionEnvironment().getConfig()));
- }
-
- /**
- * Continues a temporal Join transformation and defines a
- * {@link KeySelector} function for the first join {@link DataStream}
- * .</br> The KeySelector function is called for each element of the
- * first DataStream and extracts a single key value on which the
- * DataStream is joined. </br>
- *
- * @param keySelector
- * The KeySelector function which extracts the key values
- * from the DataStream on which it is joined.
- * @return An incomplete Join transformation. Call
- * {@link JoinPredicate#equalTo} to continue the Join.
- */
- public <K> JoinPredicate<I1, I2> where(KeySelector<I1, K> keySelector) {
- return new JoinPredicate<I1, I2>(op, keySelector);
- }
-
- @Override
- public JoinWindow<I1, I2> every(long length, TimeUnit timeUnit) {
- return every(timeUnit.toMillis(length));
- }
-
- @Override
- public JoinWindow<I1, I2> every(long length) {
- op.slideInterval = length;
- return this;
- }
-
- // ----------------------------------------------------------------------------------------
-
- }
-
- /**
- * Intermediate step of a temporal Join transformation. <br/>
- * To continue the Join transformation, select the join key of the second
- * input {@link DataStream} by calling {@link JoinPredicate#equalTo}
- *
- */
- public static class JoinPredicate<I1, I2> {
-
- private StreamJoinOperator<I1, I2> op;
- private KeySelector<I1, ?> keys1;
- private KeySelector<I2, ?> keys2;
- private TypeInformation<I2> type2;
-
- private JoinPredicate(StreamJoinOperator<I1, I2> operator, KeySelector<I1, ?> keys1) {
- this.op = operator;
- this.keys1 = keys1;
- this.type2 = op.input2.getType();
- }
-
- /**
- * Creates a temporal Join transformation and defines the {@link Tuple}
- * fields of the second join {@link DataStream} that should be used as
- * join keys.<br/>
- * </p> The resulting operator wraps each pair of joining elements in a
- * Tuple2<I1,I2>(first, second). To use a different wrapping function
- * use {@link JoinedStream#with(JoinFunction)}
- *
- * @param fields
- * The indexes of the Tuple fields of the second join
- * DataStream that should be used as keys.
- * @return A streaming join operator. Call {@link JoinedStream#with} to
- * apply a custom wrapping
- */
- public JoinedStream<I1, I2, Tuple2<I1, I2>> equalTo(int... fields) {
- keys2 = KeySelectorUtil.getSelectorForKeys(new Keys.ExpressionKeys<I2>(fields, type2),
- type2, op.input1.getExecutionEnvironment().getConfig());
- return createJoinOperator();
- }
-
- /**
- * Creates a temporal Join transformation and defines the fields of the
- * second join {@link DataStream} that should be used as join keys. </p>
- * The resulting operator wraps each pair of joining elements in a
- * Tuple2<I1,I2>(first, second). To use a different wrapping function
- * use {@link JoinedStream#with(JoinFunction)}
- *
- * @param fields
- * The fields of the second join DataStream that should be
- * used as keys.
- * @return A streaming join operator. Call {@link JoinedStream#with} to
- * apply a custom wrapping
- */
- public JoinedStream<I1, I2, Tuple2<I1, I2>> equalTo(String... fields) {
- this.keys2 = KeySelectorUtil.getSelectorForKeys(new Keys.ExpressionKeys<I2>(fields,
- type2), type2, op.input1.getExecutionEnvironment().getConfig());
- return createJoinOperator();
- }
-
- /**
- * Creates a temporal Join transformation and defines a
- * {@link KeySelector} function for the second join {@link DataStream}
- * .</br> The KeySelector function is called for each element of the
- * second DataStream and extracts a single key value on which the
- * DataStream is joined. </p> The resulting operator wraps each pair of
- * joining elements in a Tuple2<I1,I2>(first, second). To use a
- * different wrapping function use
- * {@link JoinedStream#with(JoinFunction)}
- *
- *
- * @param keySelector
- * The KeySelector function which extracts the key values
- * from the second DataStream on which it is joined.
- * @return A streaming join operator. Call {@link JoinedStream#with} to
- * apply a custom wrapping
- */
- public <K> JoinedStream<I1, I2, Tuple2<I1, I2>> equalTo(KeySelector<I2, K> keySelector) {
- this.keys2 = keySelector;
- return createJoinOperator();
- }
-
- private JoinedStream<I1, I2, Tuple2<I1, I2>> createJoinOperator() {
-
-// JoinFunction<I1, I2, Tuple2<I1, I2>> joinFunction = new DefaultJoinFunction<I1, I2>();
-//
-// JoinWindowFunction<I1, I2, Tuple2<I1, I2>> joinWindowFunction = getJoinWindowFunction(
-// joinFunction, this);
-//
-// TypeInformation<Tuple2<I1, I2>> outType = new TupleTypeInfo<Tuple2<I1, I2>>(
-// op.input1.getType(), op.input2.getType());
-
-// return new JoinedStream<I1, I2, Tuple2<I1, I2>>(this, op.input1
-// .keyBy(keys1)
-// .connect(op.input2.keyBy(keys2))
-// .addGeneralWindowCombine(joinWindowFunction, outType, op.windowSize,
-// op.slideInterval, op.timeStamp1, op.timeStamp2));
- return null;
- }
-
- public static class JoinedStream<I1, I2, R> extends
- SingleOutputStreamOperator<R, JoinedStream<I1, I2, R>> {
- private final JoinPredicate<I1, I2> predicate;
-
- private JoinedStream(JoinPredicate<I1, I2> predicate, DataStream<R> ds) {
- super(ds.getExecutionEnvironment(), ds.getTransformation());
- this.predicate = predicate;
- }
-
- /**
- * Completes a stream join. </p> The resulting operator wraps each pair
- * of joining elements using the user defined {@link JoinFunction}
- *
- * @return The joined data stream.
- */
- @SuppressWarnings("unchecked")
- public <OUT> SingleOutputStreamOperator<OUT, ?> with(JoinFunction<I1, I2, OUT> joinFunction) {
-
- TypeInformation<OUT> outType = TypeExtractor.getJoinReturnTypes(joinFunction,
- predicate.op.input1.getType(), predicate.op.input2.getType());
-
-// JoinWindowFunction<I1, I2, OUT> joinWindowFunction = getJoinWindowFunction(joinFunction, predicate);
-//
-
-// return new JoinedStream<I1, I2, OUT>(
-// predicate, predicate.op.input1
-// .keyBy(predicate.keys1)
-// .connect(predicate.op.input2.keyBy(predicate.keys2))
-// .addGeneralWindowCombine(joinWindowFunction, outType, predicate.op.windowSize,
-// predicate.op.slideInterval, predicate.op.timeStamp1, predicate.op.timeStamp2));
- return null;
- }
- }
- }
-
- public static final class DefaultJoinFunction<I1, I2> implements
- JoinFunction<I1, I2, Tuple2<I1, I2>> {
-
- private static final long serialVersionUID = 1L;
- private final Tuple2<I1, I2> outTuple = new Tuple2<I1, I2>();
-
- @Override
- public Tuple2<I1, I2> join(I1 first, I2 second) throws Exception {
- outTuple.f0 = first;
- outTuple.f1 = second;
- return outTuple;
- }
- }
-
-// private static <I1, I2, OUT> JoinWindowFunction<I1, I2, OUT> getJoinWindowFunction(
-// JoinFunction<I1, I2, OUT> joinFunction, JoinPredicate<I1, I2> predicate) {
-// return new JoinWindowFunction<I1, I2, OUT>(predicate.keys1, predicate.keys2, joinFunction);
-// }
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/TemporalOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/TemporalOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/TemporalOperator.java
deleted file mode 100644
index 9da00f2..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/TemporalOperator.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/*
- * 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.datastream.temporal;
-
-import java.util.concurrent.TimeUnit;
-
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.windowing.helper.SystemTimestamp;
-import org.apache.flink.streaming.api.windowing.helper.Timestamp;
-import org.apache.flink.streaming.api.windowing.helper.TimestampWrapper;
-
-public abstract class TemporalOperator<I1, I2, OP extends TemporalWindow<OP>> {
-
- public final DataStream<I1> input1;
- public final DataStream<I2> input2;
-
- public long windowSize;
- public long slideInterval;
-
- public TimestampWrapper<I1> timeStamp1;
- public TimestampWrapper<I2> timeStamp2;
-
- public TemporalOperator(DataStream<I1> input1, DataStream<I2> input2) {
- if (input1 == null || input2 == null) {
- throw new NullPointerException();
- }
- this.input1 = input1;
- this.input2 = input2;
- }
-
- /**
- * Continues a temporal transformation.<br/>
- * Defines the window size on which the two DataStreams will be transformed.
- * To define sliding windows call {@link TemporalWindow#every} on the
- * resulting operator.
- *
- * @param length
- * The size of the window in milliseconds.
- * @param timeUnit
- * The unit if time to be used
- * @return An incomplete temporal transformation.
- */
- @SuppressWarnings("unchecked")
- public OP onWindow(long length, TimeUnit timeUnit) {
- return onWindow(timeUnit.toMillis(length),
- (TimestampWrapper<I1>) SystemTimestamp.getWrapper(),
- (TimestampWrapper<I2>) SystemTimestamp.getWrapper());
- }
-
- /**
- * Continues a temporal transformation.<br/>
- * Defines the window size on which the two DataStreams will be
- * transformed.To define sliding windows call {@link TemporalWindow#every}
- * on the resulting operator.
- *
- * @param length
- * The size of the window in milliseconds.
- * @param timeStamp1
- * The timestamp used to extract time from the elements of the
- * first data stream.
- * @param timeStamp2
- * The timestamp used to extract time from the elements of the
- * second data stream.
- * @return An incomplete temporal transformation.
- */
- public OP onWindow(long length, Timestamp<I1> timeStamp1, Timestamp<I2> timeStamp2) {
- return onWindow(length, timeStamp1, timeStamp2, 0);
- }
-
- /**
- * Continues a temporal transformation.<br/>
- * Defines the window size on which the two DataStreams will be
- * transformed.To define sliding windows call {@link TemporalWindow#every}
- * on the resulting operator.
- *
- * @param length
- * The size of the window in milliseconds.
- * @param timeStamp1
- * The timestamp used to extract time from the elements of the
- * first data stream.
- * @param timeStamp2
- * The timestamp used to extract time from the elements of the
- * second data stream.
- * @param startTime
- * The start time to measure the first window
- * @return An incomplete temporal transformation.
- */
- public OP onWindow(long length, Timestamp<I1> timeStamp1, Timestamp<I2> timeStamp2,
- long startTime) {
- return onWindow(length, new TimestampWrapper<I1>(timeStamp1, startTime),
- new TimestampWrapper<I2>(timeStamp2, startTime));
- }
-
- private OP onWindow(long length, TimestampWrapper<I1> timeStamp1,
- TimestampWrapper<I2> timeStamp2) {
-
- this.windowSize = length;
- this.slideInterval = length;
-
- this.timeStamp1 = timeStamp1;
- this.timeStamp2 = timeStamp2;
-
- return createNextWindowOperator();
- }
-
- protected abstract OP createNextWindowOperator();
-
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/TemporalWindow.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/TemporalWindow.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/TemporalWindow.java
deleted file mode 100644
index 49c75c4..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporal/TemporalWindow.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.datastream.temporal;
-
-import java.util.concurrent.TimeUnit;
-
-public interface TemporalWindow<T> {
-
- /**
- * Defines the slide interval for this temporal operator
- *
- * @param length
- * Length of the window
- * @param timeUnit
- * Unit of time
- * @return The temporal operator with slide interval specified
- */
- public T every(long length, TimeUnit timeUnit);
-
- /**
- * Defines the slide interval for this temporal operator
- *
- * @param length
- * Length of the window
- * @return The temporal operator with slide interval specified
- */
- public T every(long length);
-
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceWindowFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceWindowFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceWindowFunction.java
index 042fe18..edd8a34 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceWindowFunction.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceWindowFunction.java
@@ -18,37 +18,17 @@
package org.apache.flink.streaming.api.functions.windowing;
import org.apache.flink.api.common.functions.ReduceFunction;
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.api.common.functions.util.FunctionUtils;
-import org.apache.flink.configuration.Configuration;
+import org.apache.flink.api.java.operators.translation.WrappingFunction;
import org.apache.flink.streaming.api.windowing.windows.Window;
import org.apache.flink.util.Collector;
-public class ReduceWindowFunction<K, W extends Window, T> extends RichWindowFunction<T, T, K, W> {
+public class ReduceWindowFunction<K, W extends Window, T>
+ extends WrappingFunction<ReduceFunction<T>>
+ implements WindowFunction<T, T, K, W> {
private static final long serialVersionUID = 1L;
- private final ReduceFunction<T> reduceFunction;
-
public ReduceWindowFunction(ReduceFunction<T> reduceFunction) {
- this.reduceFunction = reduceFunction;
- }
-
- @Override
- public void setRuntimeContext(RuntimeContext ctx) {
- super.setRuntimeContext(ctx);
- FunctionUtils.setFunctionRuntimeContext(reduceFunction, ctx);
- }
-
- @Override
- public void open(Configuration parameters) throws Exception {
- super.open(parameters);
- FunctionUtils.openFunction(reduceFunction, parameters);
- }
-
- @Override
- public void close() throws Exception {
- super.close();
- FunctionUtils.closeFunction(reduceFunction);
+ super(reduceFunction);
}
@Override
@@ -59,7 +39,7 @@ public class ReduceWindowFunction<K, W extends Window, T> extends RichWindowFunc
if (result == null) {
result = v;
} else {
- result = reduceFunction.reduce(result, v);
+ result = wrappedFunction.reduce(result, v);
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoGroupJoinITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoGroupJoinITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoGroupJoinITCase.java
new file mode 100644
index 0000000..c06a608
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoGroupJoinITCase.java
@@ -0,0 +1,372 @@
+/*
+* 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;
+
+import com.google.common.collect.Lists;
+import org.apache.flink.api.common.functions.CoGroupFunction;
+import org.apache.flink.api.common.functions.JoinFunction;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.TimestampExtractor;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
+import org.apache.flink.util.Collector;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+public class CoGroupJoinITCase extends StreamingMultipleProgramsTestBase {
+
+ private static List<String> testResults;
+
+ @Test
+ public void testCoGroup() throws Exception {
+
+ testResults = Lists.newArrayList();
+
+ StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+ env.setParallelism(1);
+ env.getConfig().enableTimestamps();
+
+ DataStream<Tuple2<String, Integer>> source1 = env.addSource(new SourceFunction<Tuple2<String, Integer>>() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public void run(SourceContext<Tuple2<String, Integer>> ctx) throws Exception {
+ ctx.collect(Tuple2.of("a", 0));
+ ctx.collect(Tuple2.of("a", 1));
+ ctx.collect(Tuple2.of("a", 2));
+
+ ctx.collect(Tuple2.of("b", 3));
+ ctx.collect(Tuple2.of("b", 4));
+ ctx.collect(Tuple2.of("b", 5));
+
+ ctx.collect(Tuple2.of("a", 6));
+ ctx.collect(Tuple2.of("a", 7));
+ ctx.collect(Tuple2.of("a", 8));
+ }
+
+ @Override
+ public void cancel() {
+ }
+ }).extractTimestamp(new Tuple2TimestampExtractor());
+
+ DataStream<Tuple2<String, Integer>> source2 = env.addSource(new SourceFunction<Tuple2<String, Integer>>() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public void run(SourceContext<Tuple2<String, Integer>> ctx) throws Exception {
+ ctx.collect(Tuple2.of("a", 0));
+ ctx.collect(Tuple2.of("a", 1));
+
+ ctx.collect(Tuple2.of("b", 3));
+
+ ctx.collect(Tuple2.of("c", 6));
+ ctx.collect(Tuple2.of("c", 7));
+ ctx.collect(Tuple2.of("c", 8));
+ }
+
+ @Override
+ public void cancel() {
+ }
+ }).extractTimestamp(new Tuple2TimestampExtractor());
+
+
+ source1.coGroup(source2)
+ .where(new Tuple2KeyExtractor())
+ .equalTo(new Tuple2KeyExtractor())
+ .window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
+ .apply(new CoGroupFunction<Tuple2<String,Integer>, Tuple2<String,Integer>, String>() {
+ @Override
+ public void coGroup(Iterable<Tuple2<String, Integer>> first,
+ Iterable<Tuple2<String, Integer>> second,
+ Collector<String> out) throws Exception {
+ StringBuilder result = new StringBuilder();
+ result.append("F:");
+ for (Tuple2<String, Integer> t: first) {
+ result.append(t.toString());
+ }
+ result.append(" S:");
+ for (Tuple2<String, Integer> t: second) {
+ result.append(t.toString());
+ }
+ out.collect(result.toString());
+ }
+ })
+ .addSink(new SinkFunction<String>() {
+ @Override
+ public void invoke(String value) throws Exception {
+ testResults.add(value);
+ }
+ });
+
+ env.execute("CoGroup Test");
+
+ List<String> expectedResult = Lists.newArrayList(
+ "F:(a,0)(a,1)(a,2) S:(a,0)(a,1)",
+ "F:(b,3)(b,4)(b,5) S:(b,3)",
+ "F:(a,6)(a,7)(a,8) S:",
+ "F: S:(c,6)(c,7)(c,8)");
+
+ Collections.sort(expectedResult);
+ Collections.sort(testResults);
+
+ Assert.assertEquals(expectedResult, testResults);
+ }
+
+ @Test
+ public void testJoin() throws Exception {
+
+ testResults = Lists.newArrayList();
+
+ StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+ env.setParallelism(1);
+ env.getConfig().enableTimestamps();
+
+ DataStream<Tuple3<String, String, Integer>> source1 = env.addSource(new SourceFunction<Tuple3<String, String, Integer>>() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public void run(SourceContext<Tuple3<String, String, Integer>> ctx) throws Exception {
+ ctx.collect(Tuple3.of("a", "x", 0));
+ ctx.collect(Tuple3.of("a", "y", 1));
+ ctx.collect(Tuple3.of("a", "z", 2));
+
+ ctx.collect(Tuple3.of("b", "u", 3));
+ ctx.collect(Tuple3.of("b", "w", 5));
+
+ ctx.collect(Tuple3.of("a", "i", 6));
+ ctx.collect(Tuple3.of("a", "j", 7));
+ ctx.collect(Tuple3.of("a", "k", 8));
+ }
+
+ @Override
+ public void cancel() {
+ }
+ }).extractTimestamp(new Tuple3TimestampExtractor());
+
+ DataStream<Tuple3<String, String, Integer>> source2 = env.addSource(new SourceFunction<Tuple3<String, String, Integer>>() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public void run(SourceContext<Tuple3<String, String, Integer>> ctx) throws Exception {
+ ctx.collect(Tuple3.of("a", "u", 0));
+ ctx.collect(Tuple3.of("a", "w", 1));
+
+ ctx.collect(Tuple3.of("b", "i", 3));
+ ctx.collect(Tuple3.of("b", "k", 5));
+
+ ctx.collect(Tuple3.of("a", "x", 6));
+ ctx.collect(Tuple3.of("a", "z", 8));
+ }
+
+ @Override
+ public void cancel() {
+ }
+ }).extractTimestamp(new Tuple3TimestampExtractor());
+
+
+ source1.join(source2)
+ .where(new Tuple3KeyExtractor())
+ .equalTo(new Tuple3KeyExtractor())
+ .window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
+ .apply(new JoinFunction<Tuple3<String, String, Integer>, Tuple3<String, String, Integer>, String>() {
+ @Override
+ public String join(Tuple3<String, String, Integer> first, Tuple3<String, String, Integer> second) throws Exception {
+ return first + ":" + second;
+ }
+ })
+ .addSink(new SinkFunction<String>() {
+ @Override
+ public void invoke(String value) throws Exception {
+ testResults.add(value);
+ }
+ });
+
+ env.execute("Join Test");
+
+ List<String> expectedResult = Lists.newArrayList(
+ "(a,x,0):(a,u,0)",
+ "(a,x,0):(a,w,1)",
+ "(a,y,1):(a,u,0)",
+ "(a,y,1):(a,w,1)",
+ "(a,z,2):(a,u,0)",
+ "(a,z,2):(a,w,1)",
+ "(b,u,3):(b,i,3)",
+ "(b,u,3):(b,k,5)",
+ "(b,w,5):(b,i,3)",
+ "(b,w,5):(b,k,5)",
+ "(a,i,6):(a,x,6)",
+ "(a,i,6):(a,z,8)",
+ "(a,j,7):(a,x,6)",
+ "(a,j,7):(a,z,8)",
+ "(a,k,8):(a,x,6)",
+ "(a,k,8):(a,z,8)");
+
+ Collections.sort(expectedResult);
+ Collections.sort(testResults);
+
+ Assert.assertEquals(expectedResult, testResults);
+ }
+
+ @Test
+ public void testSelfJoin() throws Exception {
+
+ testResults = Lists.newArrayList();
+
+ StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+ env.setParallelism(1);
+ env.getConfig().enableTimestamps();
+
+ DataStream<Tuple3<String, String, Integer>> source1 = env.addSource(new SourceFunction<Tuple3<String, String, Integer>>() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public void run(SourceContext<Tuple3<String, String, Integer>> ctx) throws Exception {
+ ctx.collect(Tuple3.of("a", "x", 0));
+ ctx.collect(Tuple3.of("a", "y", 1));
+ ctx.collect(Tuple3.of("a", "z", 2));
+
+ ctx.collect(Tuple3.of("b", "u", 3));
+ ctx.collect(Tuple3.of("b", "w", 5));
+
+ ctx.collect(Tuple3.of("a", "i", 6));
+ ctx.collect(Tuple3.of("a", "j", 7));
+ ctx.collect(Tuple3.of("a", "k", 8));
+ }
+
+ @Override
+ public void cancel() {
+ }
+ }).extractTimestamp(new Tuple3TimestampExtractor());
+
+ source1.join(source1)
+ .where(new Tuple3KeyExtractor())
+ .equalTo(new Tuple3KeyExtractor())
+ .window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
+ .apply(new JoinFunction<Tuple3<String, String, Integer>, Tuple3<String, String, Integer>, String>() {
+ @Override
+ public String join(Tuple3<String, String, Integer> first, Tuple3<String, String, Integer> second) throws Exception {
+ return first + ":" + second;
+ }
+ })
+ .addSink(new SinkFunction<String>() {
+ @Override
+ public void invoke(String value) throws Exception {
+ testResults.add(value);
+ }
+ });
+
+ env.execute("Self-Join Test");
+
+ List<String> expectedResult = Lists.newArrayList(
+ "(a,x,0):(a,x,0)",
+ "(a,x,0):(a,y,1)",
+ "(a,x,0):(a,z,2)",
+ "(a,y,1):(a,x,0)",
+ "(a,y,1):(a,y,1)",
+ "(a,y,1):(a,z,2)",
+ "(a,z,2):(a,x,0)",
+ "(a,z,2):(a,y,1)",
+ "(a,z,2):(a,z,2)",
+ "(b,u,3):(b,u,3)",
+ "(b,u,3):(b,w,5)",
+ "(b,w,5):(b,u,3)",
+ "(b,w,5):(b,w,5)",
+ "(a,i,6):(a,i,6)",
+ "(a,i,6):(a,j,7)",
+ "(a,i,6):(a,k,8)",
+ "(a,j,7):(a,i,6)",
+ "(a,j,7):(a,j,7)",
+ "(a,j,7):(a,k,8)",
+ "(a,k,8):(a,i,6)",
+ "(a,k,8):(a,j,7)",
+ "(a,k,8):(a,k,8)");
+
+ Collections.sort(expectedResult);
+ Collections.sort(testResults);
+
+ Assert.assertEquals(expectedResult, testResults);
+ }
+
+ private static class Tuple2TimestampExtractor implements TimestampExtractor<Tuple2<String, Integer>> {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public long extractTimestamp(Tuple2<String, Integer> element, long currentTimestamp) {
+ return element.f1;
+ }
+
+ @Override
+ public long emitWatermark(Tuple2<String, Integer> element, long currentTimestamp) {
+ return element.f1 - 1;
+ }
+
+ @Override
+ public long getCurrentWatermark() {
+ return Long.MIN_VALUE;
+ }
+ }
+
+ private static class Tuple3TimestampExtractor implements TimestampExtractor<Tuple3<String, String, Integer>> {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public long extractTimestamp(Tuple3<String, String, Integer> element, long currentTimestamp) {
+ return element.f2;
+ }
+
+ @Override
+ public long emitWatermark(Tuple3<String, String, Integer> element, long currentTimestamp) {
+ return element.f2 - 1;
+ }
+
+ @Override
+ public long getCurrentWatermark() {
+ return Long.MIN_VALUE;
+ }
+ }
+
+ private static class Tuple2KeyExtractor implements KeySelector<Tuple2<String,Integer>, String> {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public String getKey(Tuple2<String, Integer> value) throws Exception {
+ return value.f0;
+ }
+ }
+
+ private static class Tuple3KeyExtractor implements KeySelector<Tuple3<String, String, Integer>, String> {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public String getKey(Tuple3<String, String, Integer> value) throws Exception {
+ return value.f0;
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WindowCrossJoinTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WindowCrossJoinTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WindowCrossJoinTest.java
deleted file mode 100644
index 7d2a131..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/WindowCrossJoinTest.java
+++ /dev/null
@@ -1,143 +0,0 @@
-/*
- * 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;
-
-import static org.junit.Assert.assertEquals;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.HashSet;
-
-import org.apache.flink.api.common.functions.CrossFunction;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.windowing.helper.Timestamp;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.apache.flink.streaming.util.TestListResultSink;
-import org.apache.flink.streaming.util.TestStreamEnvironment;
-import org.junit.Ignore;
-import org.junit.Test;
-
-public class WindowCrossJoinTest extends StreamingMultipleProgramsTestBase {
-
- private static ArrayList<Tuple2<Tuple2<Integer, String>, Integer>> joinExpectedResults = new ArrayList<Tuple2<Tuple2<Integer, String>, Integer>>();
- private static ArrayList<Tuple2<Tuple2<Integer, String>, Integer>> crossExpectedResults = new ArrayList<Tuple2<Tuple2<Integer, String>, Integer>>();
-
- private static class MyTimestamp<T> implements Timestamp<T> {
-
- private static final long serialVersionUID = 1L;
-
- @Override
- public long getTimestamp(T value) {
- return 101L;
- }
- }
-
- /**
- * TODO: enable once new join operator is ready
- * @throws Exception
- */
- @Ignore
- @Test
- public void test() throws Exception {
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.setParallelism(1);
- env.setBufferTimeout(1);
-
- TestListResultSink<Tuple2<Tuple2<Integer, String>, Integer>> joinResultSink =
- new TestListResultSink<Tuple2<Tuple2<Integer, String>, Integer>>();
- TestListResultSink<Tuple2<Tuple2<Integer, String>, Integer>> crossResultSink =
- new TestListResultSink<Tuple2<Tuple2<Integer, String>, Integer>>();
-
- ArrayList<Tuple2<Integer, String>> in1 = new ArrayList<Tuple2<Integer, String>>();
- ArrayList<Tuple1<Integer>> in2 = new ArrayList<Tuple1<Integer>>();
-
- in1.add(new Tuple2<Integer, String>(10, "a"));
- in1.add(new Tuple2<Integer, String>(20, "b"));
- in1.add(new Tuple2<Integer, String>(20, "x"));
- in1.add(new Tuple2<Integer, String>(0, "y"));
-
- in2.add(new Tuple1<Integer>(0));
- in2.add(new Tuple1<Integer>(5));
- in2.add(new Tuple1<Integer>(20));
-
- joinExpectedResults.add(new Tuple2<Tuple2<Integer, String>, Integer>(
- new Tuple2<Integer, String>(20, "b"), 20));
- joinExpectedResults.add(new Tuple2<Tuple2<Integer, String>, Integer>(
- new Tuple2<Integer, String>(20, "x"), 20));
- joinExpectedResults.add(new Tuple2<Tuple2<Integer, String>, Integer>(
- new Tuple2<Integer, String>(0, "y"), 0));
-
- crossExpectedResults.add(new Tuple2<Tuple2<Integer, String>, Integer>(
- new Tuple2<Integer, String>(10, "a"), 0));
- crossExpectedResults.add(new Tuple2<Tuple2<Integer, String>, Integer>(
- new Tuple2<Integer, String>(10, "a"), 5));
- crossExpectedResults.add(new Tuple2<Tuple2<Integer, String>, Integer>(
- new Tuple2<Integer, String>(10, "a"), 20));
- crossExpectedResults.add(new Tuple2<Tuple2<Integer, String>, Integer>(
- new Tuple2<Integer, String>(20, "b"), 0));
- crossExpectedResults.add(new Tuple2<Tuple2<Integer, String>, Integer>(
- new Tuple2<Integer, String>(20, "b"), 5));
- crossExpectedResults.add(new Tuple2<Tuple2<Integer, String>, Integer>(
- new Tuple2<Integer, String>(20, "b"), 20));
- crossExpectedResults.add(new Tuple2<Tuple2<Integer, String>, Integer>(
- new Tuple2<Integer, String>(20, "x"), 0));
- crossExpectedResults.add(new Tuple2<Tuple2<Integer, String>, Integer>(
- new Tuple2<Integer, String>(20, "x"), 5));
- crossExpectedResults.add(new Tuple2<Tuple2<Integer, String>, Integer>(
- new Tuple2<Integer, String>(20, "x"), 20));
- crossExpectedResults.add(new Tuple2<Tuple2<Integer, String>, Integer>(
- new Tuple2<Integer, String>(0, "y"), 0));
- crossExpectedResults.add(new Tuple2<Tuple2<Integer, String>, Integer>(
- new Tuple2<Integer, String>(0, "y"), 5));
- crossExpectedResults.add(new Tuple2<Tuple2<Integer, String>, Integer>(
- new Tuple2<Integer, String>(0, "y"), 20));
-
- DataStream<Tuple2<Integer, String>> inStream1 = env.fromCollection(in1);
- DataStream<Tuple1<Integer>> inStream2 = env.fromCollection(in2);
-
- inStream1
- .join(inStream2)
- .onWindow(1000, new MyTimestamp<Tuple2<Integer, String>>(),
- new MyTimestamp<Tuple1<Integer>>(), 100).where(0).equalTo(0)
- .map(new ResultMap())
- .addSink(joinResultSink);
-
- env.execute();
-
- assertEquals(new HashSet<Tuple2<Tuple2<Integer, String>, Integer>>(joinExpectedResults),
- new HashSet<Tuple2<Tuple2<Integer, String>, Integer>>(joinResultSink.getResult()));
- assertEquals(new HashSet<Tuple2<Tuple2<Integer, String>, Integer>>(crossExpectedResults),
- new HashSet<Tuple2<Tuple2<Integer, String>, Integer>>(crossResultSink.getResult()));
- }
-
- private static class ResultMap implements
- MapFunction<Tuple2<Tuple2<Integer, String>, Tuple1<Integer>>,
- Tuple2<Tuple2<Integer, String>, Integer>> {
- private static final long serialVersionUID = 1L;
-
- @Override
- public Tuple2<Tuple2<Integer, String>, Integer> map(Tuple2<Tuple2<Integer, String>, Tuple1<Integer>> value) throws Exception {
- return new Tuple2<Tuple2<Integer, String>, Integer>(value.f0, value.f1.f0);
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/SelfConnectionTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/SelfConnectionTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/SelfConnectionTest.java
index fc9de1d..c116c01 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/SelfConnectionTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/SelfConnectionTest.java
@@ -47,67 +47,6 @@ public class SelfConnectionTest extends StreamingMultipleProgramsTestBase {
private static List<String> expected;
/**
- * TODO: enable once new join operator is implemented
- */
- @SuppressWarnings({ "unchecked", "rawtypes" })
- @Ignore
- @Test
- public void sameDataStreamTest() {
-
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.setParallelism(3);
-
- TestListResultSink<String> resultSink = new TestListResultSink<String>();
-
- Timestamp<Integer> timeStamp = new IntegerTimestamp();
-
- KeySelector keySelector = new KeySelector<Integer, Integer>() {
-
- private static final long serialVersionUID = 1L;
-
- @Override
- public Integer getKey(Integer value) throws Exception {
- return value;
- }
- };
-
- DataStream<Integer> src = env.fromElements(1, 3, 5);
-
- @SuppressWarnings("unused")
- DataStreamSink<Tuple2<Integer, Integer>> dataStream =
- src.join(src).onWindow(50L, timeStamp, timeStamp).where(keySelector).equalTo(keySelector)
- .map(new MapFunction<Tuple2<Integer, Integer>, String>() {
-
- private static final long serialVersionUID = 1L;
-
- @Override
- public String map(Tuple2<Integer, Integer> value) throws Exception {
- return value.toString();
- }
- })
- .addSink(resultSink);
-
-
- try {
- env.execute();
-
- expected = new ArrayList<String>();
-
- expected.addAll(Arrays.asList("(1,1)", "(3,3)", "(5,5)"));
-
- List<String> result = resultSink.getResult();
-
- Collections.sort(expected);
- Collections.sort(result);
-
- assertEquals(expected, result);
- } catch (Exception e) {
- e.printStackTrace();
- fail();
- }
- }
-
- /**
* We connect two different data streams in a chain to a CoMap.
*/
@Test
[8/8] flink git commit: [FLINK-2550] Add Window Aggregations to new
Windowing API
Posted by al...@apache.org.
[FLINK-2550] Add Window Aggregations to new Windowing API
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/28a38bb7
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/28a38bb7
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/28a38bb7
Branch: refs/heads/master
Commit: 28a38bb7dedbc10ceab9d4ae1dbcc15789e33211
Parents: 0bac272
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Tue Oct 6 17:37:39 2015 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Wed Oct 7 22:08:25 2015 +0200
----------------------------------------------------------------------
.../api/datastream/AllWindowedStream.java | 205 +++++++++++++++++++
.../api/datastream/WindowedStream.java | 205 +++++++++++++++++++
.../aggregation/ComparableAggregator.java | 11 +-
.../functions/aggregation/SumAggregator.java | 2 +-
.../windowing/util/SessionWindowingData.java | 2 +-
.../streaming/api/scala/AllWindowedStream.scala | 96 +++++++++
.../streaming/api/scala/WindowedStream.scala | 96 +++++++++
.../StreamingScalaAPICompletenessTest.scala | 3 +
8 files changed, 612 insertions(+), 8 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/28a38bb7/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
index 0cc1854..89c4857 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
@@ -25,6 +25,9 @@ import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.Utils;
import org.apache.flink.api.java.typeutils.TypeExtractor;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction;
+import org.apache.flink.streaming.api.functions.aggregation.ComparableAggregator;
+import org.apache.flink.streaming.api.functions.aggregation.SumAggregator;
import org.apache.flink.streaming.api.functions.windowing.ReduceAllWindowFunction;
import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
@@ -222,6 +225,204 @@ public class AllWindowedStream<T, W extends Window> {
}
// ------------------------------------------------------------------------
+ // Aggregations on the windows
+ // ------------------------------------------------------------------------
+
+ /**
+ * Applies an aggregation that sums every window of the data stream at the
+ * given position.
+ *
+ * @param positionToSum The position in the tuple/array to sum
+ * @return The transformed DataStream.
+ */
+ public DataStream<T> sum(int positionToSum) {
+ return aggregate(new SumAggregator<>(positionToSum, input.getType(), input.getExecutionConfig()));
+ }
+
+ /**
+ * Applies an aggregation that sums every window of the pojo data stream at
+ * the given field for every window.
+ *
+ * <p>
+ * A field expression is either
+ * the name of a public field or a getter method with parentheses of the
+ * stream's underlying type. A dot can be used to drill down into objects,
+ * as in {@code "field1.getInnerField2()" }.
+ *
+ * @param field The field to sum
+ * @return The transformed DataStream.
+ */
+ public DataStream<T> sum(String field) {
+ return aggregate(new SumAggregator<>(field, input.getType(), input.getExecutionConfig()));
+ }
+
+ /**
+ * Applies an aggregation that that gives the minimum value of every window
+ * of the data stream at the given position.
+ *
+ * @param positionToMin The position to minimize
+ * @return The transformed DataStream.
+ */
+ public DataStream<T> min(int positionToMin) {
+ return aggregate(new ComparableAggregator<>(positionToMin, input.getType(), AggregationFunction.AggregationType.MIN, input.getExecutionConfig()));
+ }
+
+ /**
+ * Applies an aggregation that that gives the minimum value of the pojo data
+ * stream at the given field expression for every window.
+ *
+ * <p>
+ * A field
+ * expression is either the name of a public field or a getter method with
+ * parentheses of the {@link DataStream}S underlying type. A dot can be used
+ * to drill down into objects, as in {@code "field1.getInnerField2()" }.
+ *
+ * @param field The field expression based on which the aggregation will be applied.
+ * @return The transformed DataStream.
+ */
+ public DataStream<T> min(String field) {
+ return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MIN, false, input.getExecutionConfig()));
+ }
+
+ /**
+ * Applies an aggregation that gives the minimum element of every window of
+ * the data stream by the given position. If more elements have the same
+ * minimum value the operator returns the first element by default.
+ *
+ * @param positionToMinBy
+ * The position to minimize by
+ * @return The transformed DataStream.
+ */
+ public DataStream<T> minBy(int positionToMinBy) {
+ return this.minBy(positionToMinBy, true);
+ }
+
+ /**
+ * Applies an aggregation that gives the minimum element of every window of
+ * the data stream by the given position. If more elements have the same
+ * minimum value the operator returns the first element by default.
+ *
+ * @param positionToMinBy The position to minimize by
+ * @return The transformed DataStream.
+ */
+ public DataStream<T> minBy(String positionToMinBy) {
+ return this.minBy(positionToMinBy, true);
+ }
+
+ /**
+ * Applies an aggregation that gives the minimum element of every window of
+ * the data stream by the given position. If more elements have the same
+ * minimum value the operator returns either the first or last one depending
+ * on the parameter setting.
+ *
+ * @param positionToMinBy The position to minimize
+ * @param first If true, then the operator return the first element with the minimum value, otherwise returns the last
+ * @return The transformed DataStream.
+ */
+ public DataStream<T> minBy(int positionToMinBy, boolean first) {
+ return aggregate(new ComparableAggregator<>(positionToMinBy, input.getType(), AggregationFunction.AggregationType.MINBY, first, input.getExecutionConfig()));
+ }
+
+ /**
+ * Applies an aggregation that that gives the minimum element of the pojo
+ * data stream by the given field expression for every window. A field
+ * expression is either the name of a public field or a getter method with
+ * parentheses of the {@link DataStream DataStreams} underlying type. A dot can be used
+ * to drill down into objects, as in {@code "field1.getInnerField2()" }.
+ *
+ * @param field The field expression based on which the aggregation will be applied.
+ * @param first If True then in case of field equality the first object will be returned
+ * @return The transformed DataStream.
+ */
+ public DataStream<T> minBy(String field, boolean first) {
+ return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MINBY, first, input.getExecutionConfig()));
+ }
+
+ /**
+ * Applies an aggregation that gives the maximum value of every window of
+ * the data stream at the given position.
+ *
+ * @param positionToMax The position to maximize
+ * @return The transformed DataStream.
+ */
+ public DataStream<T> max(int positionToMax) {
+ return aggregate(new ComparableAggregator<>(positionToMax, input.getType(), AggregationFunction.AggregationType.MAX, input.getExecutionConfig()));
+ }
+
+ /**
+ * Applies an aggregation that that gives the maximum value of the pojo data
+ * stream at the given field expression for every window. A field expression
+ * is either the name of a public field or a getter method with parentheses
+ * of the {@link DataStream DataStreams} underlying type. A dot can be used to drill
+ * down into objects, as in {@code "field1.getInnerField2()" }.
+ *
+ * @param field The field expression based on which the aggregation will be applied.
+ * @return The transformed DataStream.
+ */
+ public DataStream<T> max(String field) {
+ return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MAX, false, input.getExecutionConfig()));
+ }
+
+ /**
+ * Applies an aggregation that gives the maximum element of every window of
+ * the data stream by the given position. If more elements have the same
+ * maximum value the operator returns the first by default.
+ *
+ * @param positionToMaxBy
+ * The position to maximize by
+ * @return The transformed DataStream.
+ */
+ public DataStream<T> maxBy(int positionToMaxBy) {
+ return this.maxBy(positionToMaxBy, true);
+ }
+
+ /**
+ * Applies an aggregation that gives the maximum element of every window of
+ * the data stream by the given position. If more elements have the same
+ * maximum value the operator returns the first by default.
+ *
+ * @param positionToMaxBy
+ * The position to maximize by
+ * @return The transformed DataStream.
+ */
+ public DataStream<T> maxBy(String positionToMaxBy) {
+ return this.maxBy(positionToMaxBy, true);
+ }
+
+ /**
+ * Applies an aggregation that gives the maximum element of every window of
+ * the data stream by the given position. If more elements have the same
+ * maximum value the operator returns either the first or last one depending
+ * on the parameter setting.
+ *
+ * @param positionToMaxBy The position to maximize by
+ * @param first If true, then the operator return the first element with the maximum value, otherwise returns the last
+ * @return The transformed DataStream.
+ */
+ public DataStream<T> maxBy(int positionToMaxBy, boolean first) {
+ return aggregate(new ComparableAggregator<>(positionToMaxBy, input.getType(), AggregationFunction.AggregationType.MAXBY, first, input.getExecutionConfig()));
+ }
+
+ /**
+ * Applies an aggregation that that gives the maximum element of the pojo
+ * data stream by the given field expression for every window. A field
+ * expression is either the name of a public field or a getter method with
+ * parentheses of the {@link DataStream}S underlying type. A dot can be used
+ * to drill down into objects, as in {@code "field1.getInnerField2()" }.
+ *
+ * @param field The field expression based on which the aggregation will be applied.
+ * @param first If True then in case of field equality the first object will be returned
+ * @return The transformed DataStream.
+ */
+ public DataStream<T> maxBy(String field, boolean first) {
+ return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MAXBY, first, input.getExecutionConfig()));
+ }
+
+ private DataStream<T> aggregate(AggregationFunction<T> aggregator) {
+ return reduce(aggregator);
+ }
+
+ // ------------------------------------------------------------------------
// Utilities
// ------------------------------------------------------------------------
@@ -238,4 +439,8 @@ public class AllWindowedStream<T, W extends Window> {
public StreamExecutionEnvironment getExecutionEnvironment() {
return input.getExecutionEnvironment();
}
+
+ public TypeInformation<T> getInputType() {
+ return input.getType();
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/28a38bb7/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
index 0ea9cad..1273b42 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
@@ -26,6 +26,9 @@ import org.apache.flink.api.java.Utils;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.api.java.typeutils.TypeExtractor;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction;
+import org.apache.flink.streaming.api.functions.aggregation.ComparableAggregator;
+import org.apache.flink.streaming.api.functions.aggregation.SumAggregator;
import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
import org.apache.flink.streaming.api.functions.windowing.ReduceWindowFunction;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
@@ -241,6 +244,204 @@ public class WindowedStream<T, K, W extends Window> {
}
// ------------------------------------------------------------------------
+ // Aggregations on the keyed windows
+ // ------------------------------------------------------------------------
+
+ /**
+ * Applies an aggregation that sums every window of the data stream at the
+ * given position.
+ *
+ * @param positionToSum The position in the tuple/array to sum
+ * @return The transformed DataStream.
+ */
+ public DataStream<T> sum(int positionToSum) {
+ return aggregate(new SumAggregator<>(positionToSum, input.getType(), input.getExecutionConfig()));
+ }
+
+ /**
+ * Applies an aggregation that sums every window of the pojo data stream at
+ * the given field for every window.
+ *
+ * <p>
+ * A field expression is either
+ * the name of a public field or a getter method with parentheses of the
+ * stream's underlying type. A dot can be used to drill down into objects,
+ * as in {@code "field1.getInnerField2()" }.
+ *
+ * @param field The field to sum
+ * @return The transformed DataStream.
+ */
+ public DataStream<T> sum(String field) {
+ return aggregate(new SumAggregator<>(field, input.getType(), input.getExecutionConfig()));
+ }
+
+ /**
+ * Applies an aggregation that that gives the minimum value of every window
+ * of the data stream at the given position.
+ *
+ * @param positionToMin The position to minimize
+ * @return The transformed DataStream.
+ */
+ public DataStream<T> min(int positionToMin) {
+ return aggregate(new ComparableAggregator<>(positionToMin, input.getType(), AggregationFunction.AggregationType.MIN, input.getExecutionConfig()));
+ }
+
+ /**
+ * Applies an aggregation that that gives the minimum value of the pojo data
+ * stream at the given field expression for every window.
+ *
+ * <p>
+ * A field
+ * expression is either the name of a public field or a getter method with
+ * parentheses of the {@link DataStream}S underlying type. A dot can be used
+ * to drill down into objects, as in {@code "field1.getInnerField2()" }.
+ *
+ * @param field The field expression based on which the aggregation will be applied.
+ * @return The transformed DataStream.
+ */
+ public DataStream<T> min(String field) {
+ return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MIN, false, input.getExecutionConfig()));
+ }
+
+ /**
+ * Applies an aggregation that gives the minimum element of every window of
+ * the data stream by the given position. If more elements have the same
+ * minimum value the operator returns the first element by default.
+ *
+ * @param positionToMinBy
+ * The position to minimize by
+ * @return The transformed DataStream.
+ */
+ public DataStream<T> minBy(int positionToMinBy) {
+ return this.minBy(positionToMinBy, true);
+ }
+
+ /**
+ * Applies an aggregation that gives the minimum element of every window of
+ * the data stream by the given position. If more elements have the same
+ * minimum value the operator returns the first element by default.
+ *
+ * @param positionToMinBy The position to minimize by
+ * @return The transformed DataStream.
+ */
+ public DataStream<T> minBy(String positionToMinBy) {
+ return this.minBy(positionToMinBy, true);
+ }
+
+ /**
+ * Applies an aggregation that gives the minimum element of every window of
+ * the data stream by the given position. If more elements have the same
+ * minimum value the operator returns either the first or last one depending
+ * on the parameter setting.
+ *
+ * @param positionToMinBy The position to minimize
+ * @param first If true, then the operator return the first element with the minimum value, otherwise returns the last
+ * @return The transformed DataStream.
+ */
+ public DataStream<T> minBy(int positionToMinBy, boolean first) {
+ return aggregate(new ComparableAggregator<>(positionToMinBy, input.getType(), AggregationFunction.AggregationType.MINBY, first, input.getExecutionConfig()));
+ }
+
+ /**
+ * Applies an aggregation that that gives the minimum element of the pojo
+ * data stream by the given field expression for every window. A field
+ * expression is either the name of a public field or a getter method with
+ * parentheses of the {@link DataStream DataStreams} underlying type. A dot can be used
+ * to drill down into objects, as in {@code "field1.getInnerField2()" }.
+ *
+ * @param field The field expression based on which the aggregation will be applied.
+ * @param first If True then in case of field equality the first object will be returned
+ * @return The transformed DataStream.
+ */
+ public DataStream<T> minBy(String field, boolean first) {
+ return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MINBY, first, input.getExecutionConfig()));
+ }
+
+ /**
+ * Applies an aggregation that gives the maximum value of every window of
+ * the data stream at the given position.
+ *
+ * @param positionToMax The position to maximize
+ * @return The transformed DataStream.
+ */
+ public DataStream<T> max(int positionToMax) {
+ return aggregate(new ComparableAggregator<>(positionToMax, input.getType(), AggregationFunction.AggregationType.MAX, input.getExecutionConfig()));
+ }
+
+ /**
+ * Applies an aggregation that that gives the maximum value of the pojo data
+ * stream at the given field expression for every window. A field expression
+ * is either the name of a public field or a getter method with parentheses
+ * of the {@link DataStream DataStreams} underlying type. A dot can be used to drill
+ * down into objects, as in {@code "field1.getInnerField2()" }.
+ *
+ * @param field The field expression based on which the aggregation will be applied.
+ * @return The transformed DataStream.
+ */
+ public DataStream<T> max(String field) {
+ return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MAX, false, input.getExecutionConfig()));
+ }
+
+ /**
+ * Applies an aggregation that gives the maximum element of every window of
+ * the data stream by the given position. If more elements have the same
+ * maximum value the operator returns the first by default.
+ *
+ * @param positionToMaxBy
+ * The position to maximize by
+ * @return The transformed DataStream.
+ */
+ public DataStream<T> maxBy(int positionToMaxBy) {
+ return this.maxBy(positionToMaxBy, true);
+ }
+
+ /**
+ * Applies an aggregation that gives the maximum element of every window of
+ * the data stream by the given position. If more elements have the same
+ * maximum value the operator returns the first by default.
+ *
+ * @param positionToMaxBy
+ * The position to maximize by
+ * @return The transformed DataStream.
+ */
+ public DataStream<T> maxBy(String positionToMaxBy) {
+ return this.maxBy(positionToMaxBy, true);
+ }
+
+ /**
+ * Applies an aggregation that gives the maximum element of every window of
+ * the data stream by the given position. If more elements have the same
+ * maximum value the operator returns either the first or last one depending
+ * on the parameter setting.
+ *
+ * @param positionToMaxBy The position to maximize by
+ * @param first If true, then the operator return the first element with the maximum value, otherwise returns the last
+ * @return The transformed DataStream.
+ */
+ public DataStream<T> maxBy(int positionToMaxBy, boolean first) {
+ return aggregate(new ComparableAggregator<>(positionToMaxBy, input.getType(), AggregationFunction.AggregationType.MAXBY, first, input.getExecutionConfig()));
+ }
+
+ /**
+ * Applies an aggregation that that gives the maximum element of the pojo
+ * data stream by the given field expression for every window. A field
+ * expression is either the name of a public field or a getter method with
+ * parentheses of the {@link DataStream}S underlying type. A dot can be used
+ * to drill down into objects, as in {@code "field1.getInnerField2()" }.
+ *
+ * @param field The field expression based on which the aggregation will be applied.
+ * @param first If True then in case of field equality the first object will be returned
+ * @return The transformed DataStream.
+ */
+ public DataStream<T> maxBy(String field, boolean first) {
+ return aggregate(new ComparableAggregator<>(field, input.getType(), AggregationFunction.AggregationType.MAXBY, first, input.getExecutionConfig()));
+ }
+
+ private DataStream<T> aggregate(AggregationFunction<T> aggregator) {
+ return reduce(aggregator);
+ }
+
+ // ------------------------------------------------------------------------
// Utilities
// ------------------------------------------------------------------------
@@ -307,4 +508,8 @@ public class WindowedStream<T, K, W extends Window> {
public StreamExecutionEnvironment getExecutionEnvironment() {
return input.getExecutionEnvironment();
}
+
+ public TypeInformation<T> getInputType() {
+ return input.getType();
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/28a38bb7/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/ComparableAggregator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/ComparableAggregator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/ComparableAggregator.java
index 766a59e..e5501a0 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/ComparableAggregator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/ComparableAggregator.java
@@ -33,8 +33,7 @@ public class ComparableAggregator<T> extends AggregationFunction<T> {
private ComparableAggregator(int pos, AggregationType aggregationType, boolean first) {
super(pos);
this.comparator = Comparator.getForAggregation(aggregationType);
- this.byAggregate = (aggregationType == AggregationType.MAXBY)
- || (aggregationType == AggregationType.MINBY);
+ this.byAggregate = (aggregationType == AggregationType.MAXBY) || (aggregationType == AggregationType.MINBY);
this.first = first;
}
@@ -61,7 +60,7 @@ public class ComparableAggregator<T> extends AggregationFunction<T> {
@SuppressWarnings("unchecked")
@Override
public T reduce(T value1, T value2) throws Exception {
- Comparable<Object> o1 = (Comparable<Object>)fieldAccessor.get(value1);
+ Comparable<Object> o1 = (Comparable<Object>) fieldAccessor.get(value1);
Object o2 = fieldAccessor.get(value2);
int c = comparator.isExtremal(o1, o2);
@@ -79,10 +78,10 @@ public class ComparableAggregator<T> extends AggregationFunction<T> {
return value2;
} else {
- if (c == 1) {
- value2 = fieldAccessor.set(value2, o1);
+ if (c == 0) {
+ value1 = fieldAccessor.set(value1, o2);
}
- return value2;
+ return value1;
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/28a38bb7/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/SumAggregator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/SumAggregator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/SumAggregator.java
index c23695e..b045233 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/SumAggregator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/aggregation/SumAggregator.java
@@ -43,6 +43,6 @@ public class SumAggregator<T> extends AggregationFunction<T> {
@SuppressWarnings("unchecked")
@Override
public T reduce(T value1, T value2) throws Exception {
- return fieldAccessor.set(value2, adder.add(fieldAccessor.get(value1), fieldAccessor.get(value2)));
+ return fieldAccessor.set(value1, adder.add(fieldAccessor.get(value1), fieldAccessor.get(value2)));
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/28a38bb7/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/util/SessionWindowingData.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/util/SessionWindowingData.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/util/SessionWindowingData.java
index bb4a123..c1a99a8 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/util/SessionWindowingData.java
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/util/SessionWindowingData.java
@@ -19,7 +19,7 @@ package org.apache.flink.streaming.examples.windowing.util;
public class SessionWindowingData {
- public static final String EXPECTED = "(a,1,1)\n" + "(c,6,1)\n" + "(c,11,1)\n" + "(b,5,3)\n" +
+ public static final String EXPECTED = "(a,1,1)\n" + "(c,6,1)\n" + "(c,11,1)\n" + "(b,1,3)\n" +
"(a,10,1)";
private SessionWindowingData() {
http://git-wip-us.apache.org/repos/asf/flink/blob/28a38bb7/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala
index 9054d95..d2d0a1d 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala
@@ -21,6 +21,8 @@ package org.apache.flink.streaming.api.scala
import org.apache.flink.api.common.functions.ReduceFunction
import org.apache.flink.api.common.typeinfo.TypeInformation
import org.apache.flink.streaming.api.datastream.{AllWindowedStream => JavaAllWStream}
+import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction.AggregationType
+import org.apache.flink.streaming.api.functions.aggregation.{ComparableAggregator, SumAggregator}
import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction
import org.apache.flink.streaming.api.windowing.evictors.Evictor
import org.apache.flink.streaming.api.windowing.triggers.Trigger
@@ -134,6 +136,96 @@ class AllWindowedStream[T, W <: Window](javaStream: JavaAllWStream[T, W]) {
javaStream.apply(clean(function), implicitly[TypeInformation[R]])
}
+ // ------------------------------------------------------------------------
+ // Aggregations on the keyed windows
+ // ------------------------------------------------------------------------
+
+ /**
+ * Applies an aggregation that that gives the maximum of the elements in the window at
+ * the given position.
+ */
+ def max(position: Int): DataStream[T] = aggregate(AggregationType.MAX, position)
+
+ /**
+ * Applies an aggregation that that gives the maximum of the elements in the window at
+ * the given field.
+ */
+ def max(field: String): DataStream[T] = aggregate(AggregationType.MAX, field)
+
+ /**
+ * Applies an aggregation that that gives the minimum of the elements in the window at
+ * the given position.
+ */
+ def min(position: Int): DataStream[T] = aggregate(AggregationType.MIN, position)
+
+ /**
+ * Applies an aggregation that that gives the minimum of the elements in the window at
+ * the given field.
+ */
+ def min(field: String): DataStream[T] = aggregate(AggregationType.MIN, field)
+
+ /**
+ * Applies an aggregation that sums the elements in the window at the given position.
+ */
+ def sum(position: Int): DataStream[T] = aggregate(AggregationType.SUM, position)
+
+ /**
+ * Applies an aggregation that sums the elements in the window at the given field.
+ */
+ def sum(field: String): DataStream[T] = aggregate(AggregationType.SUM, field)
+
+ /**
+ * Applies an aggregation that that gives the maximum element of the window by
+ * the given position. When equality, returns the first.
+ */
+ def maxBy(position: Int): DataStream[T] = aggregate(AggregationType.MAXBY,
+ position)
+
+ /**
+ * Applies an aggregation that that gives the maximum element of the window by
+ * the given field. When equality, returns the first.
+ */
+ def maxBy(field: String): DataStream[T] = aggregate(AggregationType.MAXBY,
+ field)
+
+ /**
+ * Applies an aggregation that that gives the minimum element of the window by
+ * the given position. When equality, returns the first.
+ */
+ def minBy(position: Int): DataStream[T] = aggregate(AggregationType.MINBY,
+ position)
+
+ /**
+ * Applies an aggregation that that gives the minimum element of the window by
+ * the given field. When equality, returns the first.
+ */
+ def minBy(field: String): DataStream[T] = aggregate(AggregationType.MINBY,
+ field)
+
+ private def aggregate(aggregationType: AggregationType, field: String): DataStream[T] = {
+ val position = fieldNames2Indices(getInputType(), Array(field))(0)
+ aggregate(aggregationType, position)
+ }
+
+ def aggregate(aggregationType: AggregationType, position: Int): DataStream[T] = {
+
+ val jStream = javaStream.asInstanceOf[JavaAllWStream[Product, W]]
+
+ val reducer = aggregationType match {
+ case AggregationType.SUM =>
+ new SumAggregator(position, jStream.getInputType, jStream.getExecutionEnvironment.getConfig)
+
+ case _ =>
+ new ComparableAggregator(
+ position,
+ jStream.getInputType,
+ aggregationType,
+ true,
+ jStream.getExecutionEnvironment.getConfig)
+ }
+
+ new DataStream[Product](jStream.reduce(reducer)).asInstanceOf[DataStream[T]]
+ }
// ------------------------------------------------------------------------
// Utilities
@@ -147,4 +239,8 @@ class AllWindowedStream[T, W <: Window](javaStream: JavaAllWStream[T, W]) {
new StreamExecutionEnvironment(javaStream.getExecutionEnvironment).scalaClean(f)
}
+ /**
+ * Gets the output type.
+ */
+ private def getInputType(): TypeInformation[T] = javaStream.getInputType
}
http://git-wip-us.apache.org/repos/asf/flink/blob/28a38bb7/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala
index 2d6806d..3963765 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala
@@ -21,6 +21,8 @@ package org.apache.flink.streaming.api.scala
import org.apache.flink.api.common.functions.ReduceFunction
import org.apache.flink.api.common.typeinfo.TypeInformation
import org.apache.flink.streaming.api.datastream.{WindowedStream => JavaWStream}
+import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction.AggregationType
+import org.apache.flink.streaming.api.functions.aggregation.{ComparableAggregator, SumAggregator}
import org.apache.flink.streaming.api.functions.windowing.WindowFunction
import org.apache.flink.streaming.api.windowing.evictors.Evictor
import org.apache.flink.streaming.api.windowing.triggers.Trigger
@@ -137,6 +139,96 @@ class WindowedStream[T, K, W <: Window](javaStream: JavaWStream[T, K, W]) {
javaStream.apply(clean(function), implicitly[TypeInformation[R]])
}
+ // ------------------------------------------------------------------------
+ // Aggregations on the keyed windows
+ // ------------------------------------------------------------------------
+
+ /**
+ * Applies an aggregation that that gives the maximum of the elements in the window at
+ * the given position.
+ */
+ def max(position: Int): DataStream[T] = aggregate(AggregationType.MAX, position)
+
+ /**
+ * Applies an aggregation that that gives the maximum of the elements in the window at
+ * the given field.
+ */
+ def max(field: String): DataStream[T] = aggregate(AggregationType.MAX, field)
+
+ /**
+ * Applies an aggregation that that gives the minimum of the elements in the window at
+ * the given position.
+ */
+ def min(position: Int): DataStream[T] = aggregate(AggregationType.MIN, position)
+
+ /**
+ * Applies an aggregation that that gives the minimum of the elements in the window at
+ * the given field.
+ */
+ def min(field: String): DataStream[T] = aggregate(AggregationType.MIN, field)
+
+ /**
+ * Applies an aggregation that sums the elements in the window at the given position.
+ */
+ def sum(position: Int): DataStream[T] = aggregate(AggregationType.SUM, position)
+
+ /**
+ * Applies an aggregation that sums the elements in the window at the given field.
+ */
+ def sum(field: String): DataStream[T] = aggregate(AggregationType.SUM, field)
+
+ /**
+ * Applies an aggregation that that gives the maximum element of the window by
+ * the given position. When equality, returns the first.
+ */
+ def maxBy(position: Int): DataStream[T] = aggregate(AggregationType.MAXBY,
+ position)
+
+ /**
+ * Applies an aggregation that that gives the maximum element of the window by
+ * the given field. When equality, returns the first.
+ */
+ def maxBy(field: String): DataStream[T] = aggregate(AggregationType.MAXBY,
+ field)
+
+ /**
+ * Applies an aggregation that that gives the minimum element of the window by
+ * the given position. When equality, returns the first.
+ */
+ def minBy(position: Int): DataStream[T] = aggregate(AggregationType.MINBY,
+ position)
+
+ /**
+ * Applies an aggregation that that gives the minimum element of the window by
+ * the given field. When equality, returns the first.
+ */
+ def minBy(field: String): DataStream[T] = aggregate(AggregationType.MINBY,
+ field)
+
+ private def aggregate(aggregationType: AggregationType, field: String): DataStream[T] = {
+ val position = fieldNames2Indices(getInputType(), Array(field))(0)
+ aggregate(aggregationType, position)
+ }
+
+ def aggregate(aggregationType: AggregationType, position: Int): DataStream[T] = {
+
+ val jStream = javaStream.asInstanceOf[JavaWStream[Product, K, W]]
+
+ val reducer = aggregationType match {
+ case AggregationType.SUM =>
+ new SumAggregator(position, jStream.getInputType, jStream.getExecutionEnvironment.getConfig)
+
+ case _ =>
+ new ComparableAggregator(
+ position,
+ jStream.getInputType,
+ aggregationType,
+ true,
+ jStream.getExecutionEnvironment.getConfig)
+ }
+
+ new DataStream[Product](jStream.reduce(reducer)).asInstanceOf[DataStream[T]]
+ }
// ------------------------------------------------------------------------
// Utilities
@@ -150,4 +242,8 @@ class WindowedStream[T, K, W <: Window](javaStream: JavaWStream[T, K, W]) {
new StreamExecutionEnvironment(javaStream.getExecutionEnvironment).scalaClean(f)
}
+ /**
+ * Gets the output type.
+ */
+ private def getInputType(): TypeInformation[T] = javaStream.getInputType
}
http://git-wip-us.apache.org/repos/asf/flink/blob/28a38bb7/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala
index c6bd87a..53aa1e2 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala
@@ -51,11 +51,14 @@ class StreamingScalaAPICompletenessTest extends ScalaAPICompletenessTestBase {
"org.apache.flink.streaming.api.datastream.ConnectedStreams.getType2",
"org.apache.flink.streaming.api.datastream.ConnectedStreams.addGeneralWindowCombine",
"org.apache.flink.streaming.api.datastream.ConnectedStreams.transform",
+
"org.apache.flink.streaming.api.datastream.WindowedDataStream.getType",
"org.apache.flink.streaming.api.datastream.WindowedDataStream.getExecutionConfig",
"org.apache.flink.streaming.api.datastream.WindowedStream.getExecutionEnvironment",
+ "org.apache.flink.streaming.api.datastream.WindowedStream.getInputType",
"org.apache.flink.streaming.api.datastream.AllWindowedStream.getExecutionEnvironment",
+ "org.apache.flink.streaming.api.datastream.AllWindowedStream.getInputType",
"org.apache.flink.streaming.api.datastream.KeyedStream.transform",
"org.apache.flink.streaming.api.datastream.KeyedStream.getKeySelector",
[2/8] flink git commit: [FLINK-2550] Rework interplay of Window
Assigners and TimeCharacteristic
Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala
index fa9c0a9..65f978c 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala
@@ -23,7 +23,7 @@ import java.util.concurrent.TimeUnit
import org.apache.flink.api.java.tuple.Tuple
import org.apache.flink.streaming.api.functions.windowing.WindowFunction
import org.apache.flink.streaming.api.transformations.OneInputTransformation
-import org.apache.flink.streaming.api.windowing.assigners.{TumblingProcessingTimeWindows, SlidingProcessingTimeWindows}
+import org.apache.flink.streaming.api.windowing.assigners.{TumblingTimeWindows, SlidingTimeWindows}
import org.apache.flink.streaming.api.windowing.evictors.{CountEvictor, TimeEvictor}
import org.apache.flink.streaming.api.windowing.time.Time
import org.apache.flink.streaming.api.windowing.triggers.{ProcessingTimeTrigger, CountTrigger}
@@ -52,7 +52,7 @@ class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
val window1 = source
.keyBy(0)
- .window(SlidingProcessingTimeWindows.of(
+ .window(SlidingTimeWindows.of(
Time.of(1, TimeUnit.SECONDS),
Time.of(100, TimeUnit.MILLISECONDS)))
.reduce(reducer)
@@ -66,7 +66,7 @@ class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
val window2 = source
.keyBy(0)
- .window(SlidingProcessingTimeWindows.of(
+ .window(SlidingTimeWindows.of(
Time.of(1, TimeUnit.SECONDS),
Time.of(100, TimeUnit.MILLISECONDS)))
.apply(new WindowFunction[(String, Int), (String, Int), Tuple, TimeWindow]() {
@@ -95,7 +95,7 @@ class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
val window1 = source
.keyBy(0)
- .window(SlidingProcessingTimeWindows.of(
+ .window(SlidingTimeWindows.of(
Time.of(1, TimeUnit.SECONDS),
Time.of(100, TimeUnit.MILLISECONDS)))
.trigger(CountTrigger.of(100))
@@ -109,14 +109,14 @@ class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
assertTrue(operator1.isInstanceOf[WindowOperator[_, _, _, _]])
val winOperator1 = operator1.asInstanceOf[WindowOperator[_, _, _, _]]
assertTrue(winOperator1.getTriggerTemplate.isInstanceOf[CountTrigger[_]])
- assertTrue(winOperator1.getWindowAssigner.isInstanceOf[SlidingProcessingTimeWindows])
+ assertTrue(winOperator1.getWindowAssigner.isInstanceOf[SlidingTimeWindows])
assertTrue(
winOperator1.getWindowBufferFactory.isInstanceOf[PreAggregatingHeapWindowBuffer.Factory[_]])
val window2 = source
.keyBy(0)
- .window(TumblingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
+ .window(TumblingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
.trigger(CountTrigger.of(100))
.apply(new WindowFunction[(String, Int), (String, Int), Tuple, TimeWindow]() {
def apply(
@@ -134,7 +134,7 @@ class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
assertTrue(operator2.isInstanceOf[WindowOperator[_, _, _, _]])
val winOperator2 = operator2.asInstanceOf[WindowOperator[_, _, _, _]]
assertTrue(winOperator2.getTriggerTemplate.isInstanceOf[CountTrigger[_]])
- assertTrue(winOperator2.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows])
+ assertTrue(winOperator2.getWindowAssigner.isInstanceOf[TumblingTimeWindows])
assertTrue(winOperator2.getWindowBufferFactory.isInstanceOf[HeapWindowBuffer.Factory[_]])
}
@@ -148,7 +148,7 @@ class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
val window1 = source
.keyBy(0)
- .window(SlidingProcessingTimeWindows.of(
+ .window(SlidingTimeWindows.of(
Time.of(1, TimeUnit.SECONDS),
Time.of(100, TimeUnit.MILLISECONDS)))
.evictor(TimeEvictor.of(Time.of(1, TimeUnit.SECONDS)))
@@ -163,13 +163,13 @@ class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
val winOperator1 = operator1.asInstanceOf[EvictingWindowOperator[_, _, _, _]]
assertTrue(winOperator1.getTriggerTemplate.isInstanceOf[ProcessingTimeTrigger])
assertTrue(winOperator1.getEvictor.isInstanceOf[TimeEvictor[_]])
- assertTrue(winOperator1.getWindowAssigner.isInstanceOf[SlidingProcessingTimeWindows])
+ assertTrue(winOperator1.getWindowAssigner.isInstanceOf[SlidingTimeWindows])
assertTrue(winOperator1.getWindowBufferFactory.isInstanceOf[HeapWindowBuffer.Factory[_]])
val window2 = source
.keyBy(0)
- .window(TumblingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
+ .window(TumblingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
.trigger(CountTrigger.of(100))
.evictor(CountEvictor.of(1000))
.apply(new WindowFunction[(String, Int), (String, Int), Tuple, TimeWindow]() {
@@ -189,7 +189,7 @@ class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
val winOperator2 = operator2.asInstanceOf[EvictingWindowOperator[_, _, _, _]]
assertTrue(winOperator2.getTriggerTemplate.isInstanceOf[CountTrigger[_]])
assertTrue(winOperator2.getEvictor.isInstanceOf[CountEvictor[_]])
- assertTrue(winOperator2.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows])
+ assertTrue(winOperator2.getWindowAssigner.isInstanceOf[TumblingTimeWindows])
assertTrue(winOperator2.getWindowBufferFactory.isInstanceOf[HeapWindowBuffer.Factory[_]])
}
}
[6/8] flink git commit: [FLINK-2550] Rename reduceWindow to reduce on
*WindowedStream, add Lambda Reduce
Posted by al...@apache.org.
[FLINK-2550] Rename reduceWindow to reduce on *WindowedStream, add Lambda Reduce
Lambda Reduce is the reduce method that takes a Scala Lambda function.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/0bac272c
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/0bac272c
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/0bac272c
Branch: refs/heads/master
Commit: 0bac272c8309f2e7567ba762076bd75eeb8ea83a
Parents: 8634dbb
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Tue Oct 6 17:36:33 2015 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Wed Oct 7 22:08:25 2015 +0200
----------------------------------------------------------------------
.../api/datastream/AllWindowedStream.java | 2 +-
.../api/datastream/WindowedStream.java | 2 +-
.../windowing/AllWindowTranslationTest.java | 6 ++--
.../windowing/TimeWindowTranslationTest.java | 4 +--
.../windowing/WindowTranslationTest.java | 6 ++--
.../GroupedProcessingTimeWindowExample.java | 2 +-
.../streaming/api/scala/AllWindowedStream.scala | 31 ++++++++++++++++++--
.../streaming/api/scala/WindowedStream.scala | 31 ++++++++++++++++++--
.../api/scala/AllWindowTranslationTest.scala | 6 ++--
.../api/scala/WindowTranslationTest.scala | 6 ++--
10 files changed, 75 insertions(+), 21 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/0bac272c/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
index 78ba8ad..0cc1854 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
@@ -120,7 +120,7 @@ public class AllWindowedStream<T, W extends Window> {
* @param function The reduce function.
* @return The data stream that is the result of applying the reduce function to the window.
*/
- public DataStream<T> reduceWindow(ReduceFunction<T> function) {
+ public DataStream<T> reduce(ReduceFunction<T> function) {
//clean the closure
function = input.getExecutionEnvironment().clean(function);
http://git-wip-us.apache.org/repos/asf/flink/blob/0bac272c/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
index 349651e..0ea9cad 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
@@ -131,7 +131,7 @@ public class WindowedStream<T, K, W extends Window> {
* @param function The reduce function.
* @return The data stream that is the result of applying the reduce function to the window.
*/
- public DataStream<T> reduceWindow(ReduceFunction<T> function) {
+ public DataStream<T> reduce(ReduceFunction<T> function) {
//clean the closure
function = input.getExecutionEnvironment().clean(function);
http://git-wip-us.apache.org/repos/asf/flink/blob/0bac272c/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java
index 767b40c..09a7149 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java
@@ -66,7 +66,7 @@ public class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase
DataStream<Tuple2<String, Integer>> window1 = source
.windowAll(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
- .reduceWindow(reducer);
+ .reduce(reducer);
OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
@@ -103,7 +103,7 @@ public class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase
DataStream<Tuple2<String, Integer>> window1 = source
.windowAll(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
.trigger(CountTrigger.of(100))
- .reduceWindow(reducer);
+ .reduce(reducer);
OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
@@ -149,7 +149,7 @@ public class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase
DataStream<Tuple2<String, Integer>> window1 = source
.windowAll(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
.evictor(CountEvictor.of(100))
- .reduceWindow(reducer);
+ .reduce(reducer);
OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
http://git-wip-us.apache.org/repos/asf/flink/blob/0bac272c/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TimeWindowTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TimeWindowTranslationTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TimeWindowTranslationTest.java
index 76d7bfe..76c6f20 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TimeWindowTranslationTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/TimeWindowTranslationTest.java
@@ -59,7 +59,7 @@ public class TimeWindowTranslationTest extends StreamingMultipleProgramsTestBase
DataStream<Tuple2<String, Integer>> window1 = source
.keyBy(0)
.timeWindow(Time.of(1000, TimeUnit.MILLISECONDS), Time.of(100, TimeUnit.MILLISECONDS))
- .reduceWindow(reducer);
+ .reduce(reducer);
OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
@@ -103,7 +103,7 @@ public class TimeWindowTranslationTest extends StreamingMultipleProgramsTestBase
DataStream<Tuple2<String, Integer>> window1 = source
.timeWindowAll(Time.of(1000, TimeUnit.MILLISECONDS),
Time.of(100, TimeUnit.MILLISECONDS))
- .reduceWindow(reducer);
+ .reduce(reducer);
OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
http://git-wip-us.apache.org/repos/asf/flink/blob/0bac272c/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java
index 9dc6687..5124add 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java
@@ -66,7 +66,7 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
.keyBy(0)
.window(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS),
Time.of(100, TimeUnit.MILLISECONDS)))
- .reduceWindow(reducer);
+ .reduce(reducer);
OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
@@ -105,7 +105,7 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
.keyBy(0)
.window(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
.trigger(CountTrigger.of(100))
- .reduceWindow(reducer);
+ .reduce(reducer);
OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
@@ -153,7 +153,7 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
.keyBy(0)
.window(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
.evictor(CountEvictor.of(100))
- .reduceWindow(reducer);
+ .reduce(reducer);
OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
http://git-wip-us.apache.org/repos/asf/flink/blob/0bac272c/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/GroupedProcessingTimeWindowExample.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/GroupedProcessingTimeWindowExample.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/GroupedProcessingTimeWindowExample.java
index 5d32b8e..982b73d 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/GroupedProcessingTimeWindowExample.java
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/windowing/GroupedProcessingTimeWindowExample.java
@@ -79,7 +79,7 @@ public class GroupedProcessingTimeWindowExample {
stream
.keyBy(0)
.timeWindow(Time.of(2500, MILLISECONDS), Time.of(500, MILLISECONDS))
- .reduceWindow(new SummingReducer())
+ .reduce(new SummingReducer())
// alternative: use a apply function which does not pre-aggregate
// .keyBy(new FirstFieldKeyExtractor<Tuple2<Long, Long>, Long>())
http://git-wip-us.apache.org/repos/asf/flink/blob/0bac272c/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala
index 4f36722..9054d95 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/AllWindowedStream.scala
@@ -73,6 +73,26 @@ class AllWindowedStream[T, W <: Window](javaStream: JavaAllWStream[T, W]) {
// ------------------------------------------------------------------------
// Operations on the keyed windows
// ------------------------------------------------------------------------
+
+ /**
+ * Applies a reduce function to the window. The window function is called for each evaluation
+ * of the window for each key individually. The output of the reduce function is interpreted
+ * as a regular non-windowed stream.
+ *
+ * This window will try and pre-aggregate data as much as the window policies permit. For example,
+ * tumbling time windows can perfectly pre-aggregate the data, meaning that only one element per
+ * key is stored. Sliding time windows will pre-aggregate on the granularity of the slide
+ * interval, so a few elements are stored per key (one per slide interval).
+ * Custom windows may not be able to pre-aggregate, or may need to store extra values in an
+ * aggregation tree.
+ *
+ * @param function The reduce function.
+ * @return The data stream that is the result of applying the reduce function to the window.
+ */
+ def reduce(function: ReduceFunction[T]): DataStream[T] = {
+ javaStream.reduce(clean(function))
+ }
+
/**
* Applies a reduce function to the window. The window function is called for each evaluation
* of the window for each key individually. The output of the reduce function is interpreted
@@ -88,8 +108,15 @@ class AllWindowedStream[T, W <: Window](javaStream: JavaAllWStream[T, W]) {
* @param function The reduce function.
* @return The data stream that is the result of applying the reduce function to the window.
*/
- def reduceWindow(function: ReduceFunction[T]): DataStream[T] = {
- javaStream.reduceWindow(clean(function))
+ def reduce(function: (T, T) => T): DataStream[T] = {
+ if (function == null) {
+ throw new NullPointerException("Reduce function must not be null.")
+ }
+ val cleanFun = clean(function)
+ val reducer = new ReduceFunction[T] {
+ def reduce(v1: T, v2: T) = { cleanFun(v1, v2) }
+ }
+ reduce(reducer)
}
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/0bac272c/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala
index a688846..2d6806d 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedStream.scala
@@ -76,6 +76,26 @@ class WindowedStream[T, K, W <: Window](javaStream: JavaWStream[T, K, W]) {
// ------------------------------------------------------------------------
// Operations on the keyed windows
// ------------------------------------------------------------------------
+
+ /**
+ * Applies a reduce function to the window. The window function is called for each evaluation
+ * of the window for each key individually. The output of the reduce function is interpreted
+ * as a regular non-windowed stream.
+ *
+ * This window will try and pre-aggregate data as much as the window policies permit. For example,
+ * tumbling time windows can perfectly pre-aggregate the data, meaning that only one element per
+ * key is stored. Sliding time windows will pre-aggregate on the granularity of the slide
+ * interval, so a few elements are stored per key (one per slide interval).
+ * Custom windows may not be able to pre-aggregate, or may need to store extra values in an
+ * aggregation tree.
+ *
+ * @param function The reduce function.
+ * @return The data stream that is the result of applying the reduce function to the window.
+ */
+ def reduce(function: ReduceFunction[T]): DataStream[T] = {
+ javaStream.reduce(clean(function))
+ }
+
/**
* Applies a reduce function to the window. The window function is called for each evaluation
* of the window for each key individually. The output of the reduce function is interpreted
@@ -91,8 +111,15 @@ class WindowedStream[T, K, W <: Window](javaStream: JavaWStream[T, K, W]) {
* @param function The reduce function.
* @return The data stream that is the result of applying the reduce function to the window.
*/
- def reduceWindow(function: ReduceFunction[T]): DataStream[T] = {
- javaStream.reduceWindow(clean(function))
+ def reduce(function: (T, T) => T): DataStream[T] = {
+ if (function == null) {
+ throw new NullPointerException("Reduce function must not be null.")
+ }
+ val cleanFun = clean(function)
+ val reducer = new ReduceFunction[T] {
+ def reduce(v1: T, v2: T) = { cleanFun(v1, v2) }
+ }
+ reduce(reducer)
}
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/0bac272c/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala
index 247256f..dece9f6 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala
@@ -58,7 +58,7 @@ class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
.windowAll(SlidingProcessingTimeWindows.of(
Time.of(1, TimeUnit.SECONDS),
Time.of(100, TimeUnit.MILLISECONDS)))
- .reduceWindow(reducer)
+ .reduce(reducer)
val transform1 = window1.getJavaStream.getTransformation
.asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
@@ -100,7 +100,7 @@ class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
Time.of(1, TimeUnit.SECONDS),
Time.of(100, TimeUnit.MILLISECONDS)))
.trigger(CountTrigger.of(100))
- .reduceWindow(reducer)
+ .reduce(reducer)
val transform1 = window1.getJavaStream.getTransformation
.asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
@@ -150,7 +150,7 @@ class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
Time.of(1, TimeUnit.SECONDS),
Time.of(100, TimeUnit.MILLISECONDS)))
.evictor(TimeEvictor.of(Time.of(1, TimeUnit.SECONDS)))
- .reduceWindow(reducer)
+ .reduce(reducer)
val transform1 = window1.getJavaStream.getTransformation
.asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
http://git-wip-us.apache.org/repos/asf/flink/blob/0bac272c/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala
index f1b05c6..fa9c0a9 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/WindowTranslationTest.scala
@@ -55,7 +55,7 @@ class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
.window(SlidingProcessingTimeWindows.of(
Time.of(1, TimeUnit.SECONDS),
Time.of(100, TimeUnit.MILLISECONDS)))
- .reduceWindow(reducer)
+ .reduce(reducer)
val transform1 = window1.getJavaStream.getTransformation
.asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
@@ -99,7 +99,7 @@ class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
Time.of(1, TimeUnit.SECONDS),
Time.of(100, TimeUnit.MILLISECONDS)))
.trigger(CountTrigger.of(100))
- .reduceWindow(reducer)
+ .reduce(reducer)
val transform1 = window1.getJavaStream.getTransformation
.asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
@@ -152,7 +152,7 @@ class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
Time.of(1, TimeUnit.SECONDS),
Time.of(100, TimeUnit.MILLISECONDS)))
.evictor(TimeEvictor.of(Time.of(1, TimeUnit.SECONDS)))
- .reduceWindow(reducer)
+ .reduce(reducer)
val transform1 = window1.getJavaStream.getTransformation
.asInstanceOf[OneInputTransformation[(String, Int), (String, Int)]]
[4/8] flink git commit: [FLINK-2819] Add Windowed Join/CoGroup
Operator Based on Tagged Union
Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java
index 5230e9b..8abf9d6 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java
@@ -19,31 +19,36 @@ package org.apache.flink.streaming.examples.join;
import org.apache.flink.api.common.functions.JoinFunction;
import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.tuple.Tuple3;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.TimestampExtractor;
import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.windowing.helper.Timestamp;
+import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows;
+import org.apache.flink.streaming.api.windowing.time.Time;
import java.util.Random;
+import java.util.concurrent.TimeUnit;
/**
* Example illustrating join over sliding windows of streams in Flink.
- * <p/>
+ *
* <p>
- * his example will join two streams with a sliding window. One which emits
- * grades and one which emits salaries of people.
- * </p>
- * <p/>
- * <p/>
+ * This example will join two streams with a sliding window. One which emits grades and one which
+ * emits salaries of people. The input format for both sources has an additional timestamp
+ * as field 0. This is used to to event-time windowing. Time timestamps must be
+ * monotonically increasing.
+ *
* This example shows how to:
* <ul>
- * <li>do windowed joins,
- * <li>use tuple data types,
- * <li>write a simple streaming program.
+ * <li>do windowed joins,
+ * <li>use tuple data types,
+ * <li>write a simple streaming program.
+ * </ul>
*/
public class WindowJoin {
@@ -51,9 +56,6 @@ public class WindowJoin {
// PROGRAM
// *************************************************************************
- private static DataStream<Tuple2<String, Integer>> grades;
- private static DataStream<Tuple2<String, Integer>> salaries;
-
public static void main(String[] args) throws Exception {
if (!parseParameters(args)) {
@@ -62,18 +64,25 @@ public class WindowJoin {
// obtain execution environment
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+ env.getConfig().enableTimestamps();
// connect to the data sources for grades and salaries
- setInputStreams(env);
+ Tuple2<DataStream<Tuple3<Long, String, Integer>>, DataStream<Tuple3<Long, String, Integer>>> input = getInputStreams(env);
+ DataStream<Tuple3<Long, String, Integer>> grades = input.f0;
+ DataStream<Tuple3<Long, String, Integer>> salaries = input.f1;
+
+ // extract the timestamps
+ grades = grades.extractTimestamp(new MyTimestampExtractor());
+ salaries = salaries.extractTimestamp(new MyTimestampExtractor());
// apply a temporal join over the two stream based on the names over one
// second windows
DataStream<Tuple3<String, Integer, Integer>> joinedStream = grades
.join(salaries)
- .onWindow(1, new MyTimestamp(0), new MyTimestamp(0))
- .where(0)
- .equalTo(0)
- .with(new MyJoinFunction());
+ .where(new NameKeySelector())
+ .equalTo(new NameKeySelector())
+ .window(TumblingTimeWindows.of(Time.of(5, TimeUnit.MILLISECONDS)))
+ .apply(new MyJoinFunction());
// emit result
if (fileOutput) {
@@ -98,24 +107,25 @@ public class WindowJoin {
/**
* Continuously emit tuples with random names and integers (grades).
*/
- public static class GradeSource implements SourceFunction<Tuple2<String, Integer>> {
+ public static class GradeSource implements SourceFunction<Tuple3<Long, String, Integer>> {
private static final long serialVersionUID = 1L;
private Random rand;
- private Tuple2<String, Integer> outTuple;
+ private Tuple3<Long, String, Integer> outTuple;
private volatile boolean isRunning = true;
private int counter;
public GradeSource() {
rand = new Random();
- outTuple = new Tuple2<String, Integer>();
+ outTuple = new Tuple3<>();
}
@Override
- public void run(SourceContext<Tuple2<String, Integer>> ctx) throws Exception {
+ public void run(SourceContext<Tuple3<Long, String, Integer>> ctx) throws Exception {
while (isRunning && counter < 100) {
- outTuple.f0 = names[rand.nextInt(names.length)];
- outTuple.f1 = rand.nextInt(GRADE_COUNT) + 1;
+ outTuple.f0 = System.currentTimeMillis();
+ outTuple.f1 = names[rand.nextInt(names.length)];
+ outTuple.f2 = rand.nextInt(GRADE_COUNT) + 1;
Thread.sleep(rand.nextInt(SLEEP_TIME) + 1);
counter++;
ctx.collect(outTuple);
@@ -131,27 +141,28 @@ public class WindowJoin {
/**
* Continuously emit tuples with random names and integers (salaries).
*/
- public static class SalarySource extends RichSourceFunction<Tuple2<String, Integer>> {
+ public static class SalarySource extends RichSourceFunction<Tuple3<Long, String, Integer>> {
private static final long serialVersionUID = 1L;
private transient Random rand;
- private transient Tuple2<String, Integer> outTuple;
+ private transient Tuple3<Long, String, Integer> outTuple;
private volatile boolean isRunning;
private int counter;
public void open(Configuration parameters) throws Exception {
super.open(parameters);
rand = new Random();
- outTuple = new Tuple2<String, Integer>();
+ outTuple = new Tuple3<Long, String, Integer>();
isRunning = true;
}
@Override
- public void run(SourceContext<Tuple2<String, Integer>> ctx) throws Exception {
+ public void run(SourceContext<Tuple3<Long, String, Integer>> ctx) throws Exception {
while (isRunning && counter < 100) {
- outTuple.f0 = names[rand.nextInt(names.length)];
- outTuple.f1 = rand.nextInt(SALARY_MAX) + 1;
+ outTuple.f0 = System.currentTimeMillis();
+ outTuple.f1 = names[rand.nextInt(names.length)];
+ outTuple.f2 = rand.nextInt(SALARY_MAX) + 1;
Thread.sleep(rand.nextInt(SLEEP_TIME) + 1);
counter++;
ctx.collect(outTuple);
@@ -164,7 +175,7 @@ public class WindowJoin {
}
}
- public static class MySourceMap extends RichMapFunction<String, Tuple2<String, Integer>> {
+ public static class MySourceMap extends RichMapFunction<String, Tuple3<Long, String, Integer>> {
private static final long serialVersionUID = 1L;
@@ -175,44 +186,55 @@ public class WindowJoin {
}
@Override
- public Tuple2<String, Integer> map(String line) throws Exception {
+ public Tuple3<Long, String, Integer> map(String line) throws Exception {
record = line.substring(1, line.length() - 1).split(",");
- return new Tuple2<String, Integer>(record[0], Integer.parseInt(record[1]));
+ return new Tuple3<>(Long.parseLong(record[0]), record[1], Integer.parseInt(record[2]));
}
}
public static class MyJoinFunction
implements
- JoinFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple3<String, Integer, Integer>> {
+ JoinFunction<Tuple3<Long, String, Integer>, Tuple3<Long, String, Integer>, Tuple3<String, Integer, Integer>> {
private static final long serialVersionUID = 1L;
- private Tuple3<String, Integer, Integer> joined = new Tuple3<String, Integer, Integer>();
+ private Tuple3<String, Integer, Integer> joined = new Tuple3<>();
@Override
- public Tuple3<String, Integer, Integer> join(Tuple2<String, Integer> first,
- Tuple2<String, Integer> second) throws Exception {
- joined.f0 = first.f0;
- joined.f1 = first.f1;
- joined.f2 = second.f1;
+ public Tuple3<String, Integer, Integer> join(Tuple3<Long, String, Integer> first,
+ Tuple3<Long, String, Integer> second) throws Exception {
+ joined.f0 = first.f1;
+ joined.f1 = first.f2;
+ joined.f2 = second.f2;
return joined;
}
}
- public static class MyTimestamp implements Timestamp<Tuple2<String, Integer>> {
-
+ private static class MyTimestampExtractor implements TimestampExtractor<Tuple3<Long, String, Integer>> {
private static final long serialVersionUID = 1L;
- private int counter;
+ @Override
+ public long extractTimestamp(Tuple3<Long, String, Integer> element, long currentTimestamp) {
+ return element.f0;
+ }
- public MyTimestamp(int starttime) {
- this.counter = starttime;
+ @Override
+ public long emitWatermark(Tuple3<Long, String, Integer> element, long currentTimestamp) {
+ return element.f0 - 1;
}
@Override
- public long getTimestamp(Tuple2<String, Integer> value) {
- counter += SLEEP_TIME;
- return counter;
+ public long getCurrentWatermark() {
+ return Long.MIN_VALUE;
+ }
+ }
+
+ private static class NameKeySelector implements KeySelector<Tuple3<Long, String, Integer>, String> {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public String getKey(Tuple3<Long, String, Integer> value) throws Exception {
+ return value.f1;
}
}
@@ -253,7 +275,12 @@ public class WindowJoin {
return true;
}
- private static void setInputStreams(StreamExecutionEnvironment env) {
+ private static Tuple2<DataStream<Tuple3<Long, String, Integer>>, DataStream<Tuple3<Long, String, Integer>>> getInputStreams(
+ StreamExecutionEnvironment env) {
+
+ DataStream<Tuple3<Long, String, Integer>> grades;
+ DataStream<Tuple3<Long, String, Integer>> salaries;
+
if (fileInput) {
grades = env.readTextFile(gradesPath).map(new MySourceMap());
salaries = env.readTextFile(salariesPath).map(new MySourceMap());
@@ -261,5 +288,8 @@ public class WindowJoin {
grades = env.addSource(new GradeSource());
salaries = env.addSource(new SalarySource());
}
+
+ return Tuple2.of(grades, salaries);
}
+
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/util/WindowJoinData.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/util/WindowJoinData.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/util/WindowJoinData.java
index 23d29b1..15c1280 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/util/WindowJoinData.java
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/util/WindowJoinData.java
@@ -19,42 +19,42 @@ package org.apache.flink.streaming.examples.join.util;
public class WindowJoinData {
- public static final String GRADES_INPUT = "(john,5)\n" + "(tom,3)\n" + "(alice,1)\n" + "(grace,5)\n" +
- "(john,4)\n" + "(bob,1)\n" + "(alice,2)\n" + "(alice,3)\n" + "(bob,5)\n" + "(alice,3)\n" + "(tom,5)\n" +
- "(john,2)\n" + "(john,1)\n" + "(grace,2)\n" + "(jerry,2)\n" + "(tom,4)\n" + "(bob,4)\n" + "(bob,2)\n" +
- "(tom,2)\n" + "(alice,5)\n" + "(grace,5)\n" + "(grace,1)\n" + "(alice,1)\n" + "(grace,3)\n" + "(tom,1)\n" +
- "(jerry,5)\n" + "(john,3)\n" + "(john,4)\n" + "(john,1)\n" + "(jerry,3)\n" + "(grace,3)\n" + "(bob,3)\n" +
- "(john,3)\n" + "(jerry,4)\n" + "(tom,5)\n" + "(tom,4)\n" + "(john,2)\n" + "(jerry,1)\n" + "(bob,1)\n" +
- "(john,5)\n" + "(grace,4)\n" + "(tom,5)\n" + "(john,4)\n" + "(tom,1)\n" + "(grace,1)\n" + "(john,2)\n" +
- "(jerry,3)\n" + "(jerry,5)\n" + "(tom,2)\n" + "(tom,2)\n" + "(alice,4)\n" + "(tom,4)\n" + "(jerry,4)\n" +
- "(john,3)\n" + "(grace,4)\n" + "(tom,3)\n" + "(jerry,4)\n" + "(john,5)\n" + "(john,4)\n" + "(jerry,1)\n" +
- "(john,5)\n" + "(alice,2)\n" + "(tom,1)\n" + "(alice,5)\n" + "(grace,4)\n" + "(bob,4)\n" + "(jerry,1)\n" +
- "(john,5)\n" + "(tom,4)\n" + "(tom,5)\n" + "(jerry,5)\n" + "(tom,1)\n" + "(grace,3)\n" + "(bob,5)\n" +
- "(john,1)\n" + "(alice,1)\n" + "(grace,3)\n" + "(grace,1)\n" + "(jerry,1)\n" + "(jerry,4)\n" +
- "(bob,4)\n" + "(alice,3)\n" + "(tom,5)\n" + "(alice,4)\n" + "(alice,4)\n" + "(grace,4)\n" + "(john,5)\n" +
- "(john,5)\n" + "(grace,4)\n" + "(tom,4)\n" + "(john,4)\n" + "(john,5)\n" + "(alice,5)\n" + "(jerry,5)\n" +
- "(john,3)\n" + "(tom,5)\n" + "(jerry,4)\n" + "(grace,4)\n" + "(john,3)\n" + "(bob,2)";
+ public static final String GRADES_INPUT = "(0,john,5)\n" + "(0,tom,3)\n" + "(0,alice,1)\n" + "(0,grace,5)\n" +
+ "(1,john,4)\n" + "(1,bob,1)\n" + "(1,alice,2)\n" + "(1,alice,3)\n" + "(1,bob,5)\n" + "(1,alice,3)\n" + "(1,tom,5)\n" +
+ "(2,john,2)\n" + "(2,john,1)\n" + "(2,grace,2)\n" + "(2,jerry,2)\n" + "(2,tom,4)\n" + "(2,bob,4)\n" + "(2,bob,2)\n" +
+ "(3, tom,2)\n" + "(3,alice,5)\n" + "(3,grace,5)\n" + "(3,grace,1)\n" + "(3,alice,1)\n" + "(3,grace,3)\n" + "(3,tom,1)\n" +
+ "(4,jerry,5)\n" + "(4,john,3)\n" + "(4,john,4)\n" + "(4,john,1)\n" + "(4,jerry,3)\n" + "(4,grace,3)\n" + "(4,bob,3)\n" +
+ "(5,john,3)\n" + "(5,jerry,4)\n" + "(5,tom,5)\n" + "(5,tom,4)\n" + "(5,john,2)\n" + "(5,jerry,1)\n" + "(5,bob,1)\n" +
+ "(6,john,5)\n" + "(6,grace,4)\n" + "(6,tom,5)\n" + "(6,john,4)\n" + "(6,tom,1)\n" + "(6,grace,1)\n" + "(6,john,2)\n" +
+ "(7,jerry,3)\n" + "(7,jerry,5)\n" + "(7,tom,2)\n" + "(7,tom,2)\n" + "(7,alice,4)\n" + "(7,tom,4)\n" + "(7,jerry,4)\n" +
+ "(8,john,3)\n" + "(8,grace,4)\n" + "(8,tom,3)\n" + "(8,jerry,4)\n" + "(8,john,5)\n" + "(8,john,4)\n" + "(8,jerry,1)\n" +
+ "(9,john,5)\n" + "(9,alice,2)\n" + "(9,tom,1)\n" + "(9,alice,5)\n" + "(9,grace,4)\n" + "(9,bob,4)\n" + "(9,jerry,1)\n" +
+ "(10,john,5)\n" + "(10,tom,4)\n" + "(10,tom,5)\n" + "(10,jerry,5)\n" + "(10,tom,1)\n" + "(10,grace,3)\n" + "(10,bob,5)\n" +
+ "(11,john,1)\n" + "(11,alice,1)\n" + "(11,grace,3)\n" + "(11,grace,1)\n" + "(11,jerry,1)\n" + "(11,jerry,4)\n" +
+ "(12,bob,4)\n" + "(12,alice,3)\n" + "(12,tom,5)\n" + "(12,alice,4)\n" + "(12,alice,4)\n" + "(12,grace,4)\n" + "(12,john,5)\n" +
+ "(13,john,5)\n" + "(13,grace,4)\n" + "(13,tom,4)\n" + "(13,john,4)\n" + "(13,john,5)\n" + "(13,alice,5)\n" + "(13,jerry,5)\n" +
+ "(14,john,3)\n" + "(14,tom,5)\n" + "(14,jerry,4)\n" + "(14,grace,4)\n" + "(14,john,3)\n" + "(14,bob,2)";
- public static final String SALARIES_INPUT = "(john,6469)\n" + "(jerry,6760)\n" + "(jerry,8069)\n" +
- "(tom,3662)\n" + "(grace,8427)\n" + "(john,9425)\n" + "(bob,9018)\n" + "(john,352)\n" + "(tom,3770)\n" +
- "(grace,7622)\n" + "(jerry,7441)\n" + "(alice,1468)\n" + "(bob,5472)\n" + "(grace,898)\n" +
- "(tom,3849)\n" + "(grace,1865)\n" + "(alice,5582)\n" + "(john,9511)\n" + "(alice,1541)\n" +
- "(john,2477)\n" + "(grace,3561)\n" + "(john,1670)\n" + "(grace,7290)\n" + "(grace,6565)\n" +
- "(tom,6179)\n" + "(tom,1601)\n" + "(john,2940)\n" + "(bob,4685)\n" + "(bob,710)\n" + "(bob,5936)\n" +
- "(jerry,1412)\n" + "(grace,6515)\n" + "(grace,3321)\n" + "(tom,8088)\n" + "(john,2876)\n" +
- "(bob,9896)\n" + "(grace,7368)\n" + "(grace,9749)\n" + "(bob,2048)\n" + "(alice,4782)\n" +
- "(alice,3375)\n" + "(tom,5841)\n" + "(bob,958)\n" + "(bob,5258)\n" + "(tom,3935)\n" + "(jerry,4394)\n" +
- "(alice,102)\n" + "(alice,4931)\n" + "(alice,5240)\n" + "(jerry,7951)\n" + "(john,5675)\n" +
- "(bob,609)\n" + "(alice,5997)\n" + "(jerry,9651)\n" + "(alice,1328)\n" + "(bob,1022)\n" +
- "(grace,2578)\n" + "(jerry,9704)\n" + "(tom,4476)\n" + "(grace,3784)\n" + "(alice,6144)\n" +
- "(bob,6213)\n" + "(alice,7525)\n" + "(jerry,2908)\n" + "(grace,8464)\n" + "(jerry,9920)\n" +
- "(bob,3720)\n" + "(bob,7612)\n" + "(alice,7211)\n" + "(jerry,6484)\n" + "(alice,1711)\n" +
- "(jerry,5994)\n" + "(grace,928)\n" + "(jerry,2492)\n" + "(grace,9080)\n" + "(tom,4330)\n" +
- "(bob,8302)\n" + "(john,4981)\n" + "(tom,1781)\n" + "(grace,1379)\n" + "(jerry,3700)\n" +
- "(jerry,3584)\n" + "(jerry,2038)\n" + "(jerry,3902)\n" + "(tom,1336)\n" + "(jerry,7500)\n" +
- "(tom,3648)\n" + "(alice,2533)\n" + "(tom,8685)\n" + "(bob,3968)\n" + "(tom,3241)\n" + "(bob,7461)\n" +
- "(jerry,2138)\n" + "(alice,7503)\n" + "(alice,6424)\n" + "(tom,140)\n" + "(john,9802)\n" +
- "(grace,2977)\n" + "(grace,889)\n" + "(john,1338)";
+ public static final String SALARIES_INPUT = "(0,john,6469)\n" + "(0,jerry,6760)\n" + "(0,jerry,8069)\n" +
+ "(1,tom,3662)\n" + "(1,grace,8427)\n" + "(1,john,9425)\n" + "(1,bob,9018)\n" + "(1,john,352)\n" + "(1,tom,3770)\n" +
+ "(2,grace,7622)\n" + "(2,jerry,7441)\n" + "(2,alice,1468)\n" + "(2,bob,5472)\n" + "(2,grace,898)\n" +
+ "(3,tom,3849)\n" + "(3,grace,1865)\n" + "(3,alice,5582)\n" + "(3,john,9511)\n" + "(3,alice,1541)\n" +
+ "(4,john,2477)\n" + "(4,grace,3561)\n" + "(4,john,1670)\n" + "(4,grace,7290)\n" + "(4,grace,6565)\n" +
+ "(5,tom,6179)\n" + "(5,tom,1601)\n" + "(5,john,2940)\n" + "(5,bob,4685)\n" + "(5,bob,710)\n" + "(5,bob,5936)\n" +
+ "(6,jerry,1412)\n" + "(6,grace,6515)\n" + "(6,grace,3321)\n" + "(6,tom,8088)\n" + "(6,john,2876)\n" +
+ "(7,bob,9896)\n" + "(7,grace,7368)\n" + "(7,grace,9749)\n" + "(7,bob,2048)\n" + "(7,alice,4782)\n" +
+ "(8,alice,3375)\n" + "(8,tom,5841)\n" + "(8,bob,958)\n" + "(8,bob,5258)\n" + "(8,tom,3935)\n" + "(8,jerry,4394)\n" +
+ "(9,alice,102)\n" + "(9,alice,4931)\n" + "(9,alice,5240)\n" + "(9,jerry,7951)\n" + "(9,john,5675)\n" +
+ "(10,bob,609)\n" + "(10,alice,5997)\n" + "(10,jerry,9651)\n" + "(10,alice,1328)\n" + "(10,bob,1022)\n" +
+ "(11,grace,2578)\n" + "(11,jerry,9704)\n" + "(11,tom,4476)\n" + "(11,grace,3784)\n" + "(11,alice,6144)\n" +
+ "(12,bob,6213)\n" + "(12,alice,7525)\n" + "(12,jerry,2908)\n" + "(12,grace,8464)\n" + "(12,jerry,9920)\n" +
+ "(13,bob,3720)\n" + "(13,bob,7612)\n" + "(13,alice,7211)\n" + "(13,jerry,6484)\n" + "(13,alice,1711)\n" +
+ "(14,jerry,5994)\n" + "(14,grace,928)\n" + "(14,jerry,2492)\n" + "(14,grace,9080)\n" + "(14,tom,4330)\n" +
+ "(15,bob,8302)\n" + "(15,john,4981)\n" + "(15,tom,1781)\n" + "(15,grace,1379)\n" + "(15,jerry,3700)\n" +
+ "(16,jerry,3584)\n" + "(16,jerry,2038)\n" + "(16,jerry,3902)\n" + "(16,tom,1336)\n" + "(16,jerry,7500)\n" +
+ "(17,tom,3648)\n" + "(17,alice,2533)\n" + "(17,tom,8685)\n" + "(17,bob,3968)\n" + "(17,tom,3241)\n" + "(17,bob,7461)\n" +
+ "(18,jerry,2138)\n" + "(18,alice,7503)\n" + "(18,alice,6424)\n" + "(18,tom,140)\n" + "(18,john,9802)\n" +
+ "(19,grace,2977)\n" + "(19,grace,889)\n" + "(19,john,1338)";
private WindowJoinData() {
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/join/WindowJoin.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/join/WindowJoin.scala b/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/join/WindowJoin.scala
index 239f1fa..225dab7 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/join/WindowJoin.scala
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/join/WindowJoin.scala
@@ -21,6 +21,8 @@ package org.apache.flink.streaming.scala.examples.join
import java.util.concurrent.TimeUnit
import org.apache.flink.streaming.api.scala._
+import org.apache.flink.streaming.api.windowing.assigners.SlidingTimeWindows
+import org.apache.flink.streaming.api.windowing.time.Time
import scala.Stream._
import scala.language.postfixOps
@@ -32,8 +34,8 @@ object WindowJoin {
// PROGRAM
// *************************************************************************
- case class Grade(name: String, grade: Int)
- case class Salary(name: String, salary: Int)
+ case class Grade(time: Long, name: String, grade: Int)
+ case class Salary(time: Long, name: String, salary: Int)
case class Person(name: String, grade: Int, salary: Int)
def main(args: Array[String]) {
@@ -43,6 +45,7 @@ object WindowJoin {
}
val env = StreamExecutionEnvironment.getExecutionEnvironment
+ env.getConfig.enableTimestamps()
//Create streams for grades and salaries by mapping the inputs to the corresponding objects
val grades = setGradesInput(env)
@@ -50,11 +53,11 @@ object WindowJoin {
//Join the two input streams by name on the last 2 seconds every second and create new
//Person objects containing both grade and salary
- val joined =
- grades.join(salaries).onWindow(2, TimeUnit.SECONDS)
- .every(1, TimeUnit.SECONDS)
- .where("name")
- .equalTo("name") { (g, s) => Person(g.name, g.grade, s.salary) }
+ val joined = grades.join(salaries)
+ .where(_.name)
+ .equalTo(_.name)
+ .window(SlidingTimeWindows.of(Time.of(2, TimeUnit.SECONDS), Time.of(1, TimeUnit.SECONDS)))
+ .apply { (g, s) => Person(g.name, g.grade, s.salary) }
if (fileOutput) {
joined.writeAsText(outputPath)
@@ -74,27 +77,27 @@ object WindowJoin {
val salaryMax = 10000
val sleepInterval = 100
- def gradeStream(): Stream[(String, Int)] = {
- def gradeMapper(names: Array[String])(x: Int): (String, Int) =
+ def gradeStream: Stream[(Long, String, Int)] = {
+ def gradeMapper(names: Array[String])(x: Int): (Long, String, Int) =
{
if (x % sleepInterval == 0) Thread.sleep(sleepInterval)
- (names(Random.nextInt(names.length)), Random.nextInt(gradeCount))
+ (System.currentTimeMillis(),names(Random.nextInt(names.length)),Random.nextInt(gradeCount))
}
range(1, 100).map(gradeMapper(names))
}
- def salaryStream(): Stream[(String, Int)] = {
- def salaryMapper(x: Int): (String, Int) =
+ def salaryStream: Stream[(Long, String, Int)] = {
+ def salaryMapper(x: Int): (Long, String, Int) =
{
if (x % sleepInterval == 0) Thread.sleep(sleepInterval)
- (names(Random.nextInt(names.length)), Random.nextInt(salaryMax))
+ (System.currentTimeMillis(), names(Random.nextInt(names.length)), Random.nextInt(salaryMax))
}
range(1, 100).map(salaryMapper)
}
- def parseMap(line : String): (String, Int) = {
+ def parseMap(line : String): (Long, String, Int) = {
val record = line.substring(1, line.length - 1).split(",")
- (record(0), record(1).toInt)
+ (record(0).toLong, record(1), record(2).toInt)
}
// *************************************************************************
@@ -130,23 +133,23 @@ object WindowJoin {
System.out.println(" Provide parameter to write to file.")
System.out.println(" Usage: WindowJoin <result path>")
}
- return true
+ true
}
private def setGradesInput(env: StreamExecutionEnvironment) : DataStream[Grade] = {
if (fileInput) {
- env.readTextFile(gradesPath).map(parseMap(_)).map(x => Grade(x._1, x._2))
+ env.readTextFile(gradesPath).map(parseMap _ ).map(x => Grade(x._1, x._2, x._3))
} else {
- env.fromCollection(gradeStream).map(x => Grade(x._1, x._2))
+ env.fromCollection(gradeStream).map(x => Grade(x._1, x._2, x._3))
}
}
private def setSalariesInput(env: StreamExecutionEnvironment) : DataStream[Salary] = {
if (fileInput) {
- env.readTextFile(salariesPath).map(parseMap(_)).map(x => Salary(x._1, x._2))
+ env.readTextFile(salariesPath).map(parseMap _).map(x => Salary(x._1, x._2, x._3))
}
else {
- env.fromCollection(salaryStream).map(x => Salary(x._1, x._2))
+ env.fromCollection(salaryStream).map(x => Salary(x._1, x._2, x._3))
}
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/join/WindowJoinITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/join/WindowJoinITCase.java b/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/join/WindowJoinITCase.java
index aae4b93..e657b67 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/join/WindowJoinITCase.java
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleJavaPrograms/join/WindowJoinITCase.java
@@ -1,51 +1,50 @@
-///*
-// * 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.
-// */
-//
-// TODO: reactivate once we have new join implementation
-//package org.apache.flink.streaming.test.exampleJavaPrograms.join;
-//
-//import org.apache.flink.streaming.examples.join.WindowJoin;
-//import org.apache.flink.streaming.examples.join.util.WindowJoinData;
-//import org.apache.flink.streaming.util.StreamingProgramTestBase;
-//
-//public class WindowJoinITCase extends StreamingProgramTestBase {
-//
-// protected String gradesPath;
-// protected String salariesPath;
-// protected String resultPath;
-//
-// @Override
-// protected void preSubmit() throws Exception {
-// gradesPath = createTempFile("gradesText.txt", WindowJoinData.GRADES_INPUT);
-// salariesPath = createTempFile("salariesText.txt", WindowJoinData.SALARIES_INPUT);
-// resultPath = getTempDirPath("result");
-// }
-//
-// @Override
-// protected void postSubmit() throws Exception {
-// // since the two sides of the join might have different speed
-// // the exact output can not be checked just whether it is well-formed
-// // checks that the result lines look like e.g. (bob, 2, 2015)
-// checkLinesAgainstRegexp(resultPath, "^\\([a-z]+,(\\d),(\\d)+\\)");
-// }
-//
-// @Override
-// protected void testProgram() throws Exception {
-// WindowJoin.main(new String[]{gradesPath, salariesPath, resultPath});
-// }
-//}
+/*
+ * 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.test.exampleJavaPrograms.join;
+
+import org.apache.flink.streaming.examples.join.WindowJoin;
+import org.apache.flink.streaming.examples.join.util.WindowJoinData;
+import org.apache.flink.streaming.util.StreamingProgramTestBase;
+
+public class WindowJoinITCase extends StreamingProgramTestBase {
+
+ protected String gradesPath;
+ protected String salariesPath;
+ protected String resultPath;
+
+ @Override
+ protected void preSubmit() throws Exception {
+ gradesPath = createTempFile("gradesText.txt", WindowJoinData.GRADES_INPUT);
+ salariesPath = createTempFile("salariesText.txt", WindowJoinData.SALARIES_INPUT);
+ resultPath = getTempDirPath("result");
+ }
+
+ @Override
+ protected void postSubmit() throws Exception {
+ // since the two sides of the join might have different speed
+ // the exact output can not be checked just whether it is well-formed
+ // checks that the result lines look like e.g. (bob, 2, 2015)
+ checkLinesAgainstRegexp(resultPath, "^\\([a-z]+,(\\d),(\\d)+\\)");
+ }
+
+ @Override
+ protected void testProgram() throws Exception {
+ WindowJoin.main(new String[]{gradesPath, salariesPath, resultPath});
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleScalaPrograms/join/WindowJoinITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleScalaPrograms/join/WindowJoinITCase.java b/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleScalaPrograms/join/WindowJoinITCase.java
index 0aa884f..08ce890 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleScalaPrograms/join/WindowJoinITCase.java
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/test/java/org/apache/flink/streaming/test/exampleScalaPrograms/join/WindowJoinITCase.java
@@ -1,51 +1,50 @@
-///*
-// * 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.
-// */
-//
-// TODO: reactivate once we have new join implementation
-//package org.apache.flink.streaming.test.exampleScalaPrograms.join;
-//
-//import org.apache.flink.streaming.scala.examples.join.WindowJoin;
-//import org.apache.flink.streaming.examples.join.util.WindowJoinData;
-//import org.apache.flink.streaming.util.StreamingProgramTestBase;
-//
-//public class WindowJoinITCase extends StreamingProgramTestBase {
-//
-// protected String gradesPath;
-// protected String salariesPath;
-// protected String resultPath;
-//
-// @Override
-// protected void preSubmit() throws Exception {
-// gradesPath = createTempFile("gradesText.txt", WindowJoinData.GRADES_INPUT);
-// salariesPath = createTempFile("salariesText.txt", WindowJoinData.SALARIES_INPUT);
-// resultPath = getTempDirPath("result");
-// }
-//
-// @Override
-// protected void postSubmit() throws Exception {
-// // since the two sides of the join might have different speed
-// // the exact output can not be checked just whether it is well-formed
-// // checks that the result lines look like e.g. Person(bob, 2, 2015)
-// checkLinesAgainstRegexp(resultPath, "^Person\\([a-z]+,(\\d),(\\d)+\\)");
-// }
-//
-// @Override
-// protected void testProgram() throws Exception {
-// WindowJoin.main(new String[]{gradesPath, salariesPath, resultPath});
-// }
-//}
+/*
+ * 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.test.exampleScalaPrograms.join;
+
+import org.apache.flink.streaming.scala.examples.join.WindowJoin;
+import org.apache.flink.streaming.examples.join.util.WindowJoinData;
+import org.apache.flink.streaming.util.StreamingProgramTestBase;
+
+public class WindowJoinITCase extends StreamingProgramTestBase {
+
+ protected String gradesPath;
+ protected String salariesPath;
+ protected String resultPath;
+
+ @Override
+ protected void preSubmit() throws Exception {
+ gradesPath = createTempFile("gradesText.txt", WindowJoinData.GRADES_INPUT);
+ salariesPath = createTempFile("salariesText.txt", WindowJoinData.SALARIES_INPUT);
+ resultPath = getTempDirPath("result");
+ }
+
+ @Override
+ protected void postSubmit() throws Exception {
+ // since the two sides of the join might have different speed
+ // the exact output can not be checked just whether it is well-formed
+ // checks that the result lines look like e.g. Person(bob, 2, 2015)
+ checkLinesAgainstRegexp(resultPath, "^Person\\([a-z]+,(\\d),(\\d)+\\)");
+ }
+
+ @Override
+ protected void testProgram() throws Exception {
+ WindowJoin.main(new String[]{gradesPath, salariesPath, resultPath});
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/CoGroupedStreams.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/CoGroupedStreams.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/CoGroupedStreams.scala
new file mode 100644
index 0000000..1b16e44
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/CoGroupedStreams.scala
@@ -0,0 +1,294 @@
+/*
+ * 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.scala
+
+import org.apache.flink.api.common.functions.CoGroupFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.functions.KeySelector
+import org.apache.flink.api.scala._
+import org.apache.flink.streaming.api.datastream.{CoGroupedStreams => JavaCoGroupedStreams}
+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.triggers.Trigger
+import org.apache.flink.streaming.api.windowing.windows.Window
+import org.apache.flink.util.Collector
+
+import scala.collection.JavaConverters._
+
+import scala.reflect.ClassTag
+
+/**
+ * `CoGroupedStreams` represents two [[DataStream]]s that have been co-grouped.
+ * A streaming co-group operation is evaluated over elements in a window.
+ *
+ * To finalize the co-group operation you also need to specify a [[KeySelector]] for
+ * both the first and second input and a [[WindowAssigner]]
+ *
+ * Note: Right now, the groups are being built in memory so you need to ensure that they don't
+ * get too big. Otherwise the JVM might crash.
+ *
+ * Example:
+ *
+ * {{{
+ * val one: DataStream[(String, Int)] = ...
+ * val two: DataStream[(String, Int)] = ...
+ *
+ * val result = one.coGroup(two)
+ * .where(new MyFirstKeySelector())
+ * .equalTo(new MyFirstKeySelector())
+ * .window(TumblingTimeWindows.of(Time.of(5, TimeUnit.SECONDS)))
+ * .apply(new MyCoGroupFunction())
+ * } }}}
+ */
+object CoGroupedStreams {
+
+ /**
+ * A co-group operation that does not yet have its [[KeySelector]]s defined.
+ *
+ * @tparam T1 Type of the elements from the first input
+ * @tparam T2 Type of the elements from the second input
+ */
+ class Unspecified[T1, T2](input1: DataStream[T1], input2: DataStream[T2]) {
+
+ /**
+ * Specifies a [[KeySelector]] for elements from the first input.
+ */
+ def where[KEY](keySelector: T1 => KEY): CoGroupedStreams.WithKey[T1, T2, KEY] = {
+ val cleanFun = clean(keySelector)
+ val javaSelector = new KeySelector[T1, KEY] {
+ def getKey(in: T1) = cleanFun(in)
+ }
+ new CoGroupedStreams.WithKey[T1, T2, KEY](input1, input2, javaSelector, null)
+ }
+
+ /**
+ * Specifies a [[KeySelector]] for elements from the second input.
+ */
+ def equalTo[KEY](keySelector: T2 => KEY): CoGroupedStreams.WithKey[T1, T2, KEY] = {
+ val cleanFun = clean(keySelector)
+ val javaSelector = new KeySelector[T2, KEY] {
+ def getKey(in: T2) = cleanFun(in)
+ }
+ new CoGroupedStreams.WithKey[T1, T2, KEY](input1, input2, null, javaSelector)
+ }
+
+ /**
+ * Returns a "closure-cleaned" version of the given function. Cleans only if closure cleaning
+ * is not disabled in the [[org.apache.flink.api.common.ExecutionConfig]].
+ */
+ private[flink] def clean[F <: AnyRef](f: F): F = {
+ new StreamExecutionEnvironment(input1.getJavaStream.getExecutionEnvironment).scalaClean(f)
+ }
+ }
+
+ /**
+ * A co-group operation that has [[KeySelector]]s defined for either both or
+ * one input.
+ *
+ * You need to specify a [[KeySelector]] for both inputs using [[where()]] and [[equalTo()]]
+ * before you can proceeed with specifying a [[WindowAssigner]] using [[window()]].
+ *
+ * @tparam T1 Type of the elements from the first input
+ * @tparam T2 Type of the elements from the second input
+ * @tparam KEY Type of the key. This must be the same for both inputs
+ */
+ class WithKey[T1, T2, KEY](
+ input1: DataStream[T1],
+ input2: DataStream[T2],
+ keySelector1: KeySelector[T1, KEY],
+ keySelector2: KeySelector[T2, KEY]) {
+
+ /**
+ * Specifies a [[KeySelector]] for elements from the first input.
+ */
+ def where(keySelector: T1 => KEY): CoGroupedStreams.WithKey[T1, T2, KEY] = {
+ val cleanFun = clean(keySelector)
+ val javaSelector = new KeySelector[T1, KEY] {
+ def getKey(in: T1) = cleanFun(in)
+ }
+ new CoGroupedStreams.WithKey[T1, T2, KEY](input1, input2, javaSelector, keySelector2)
+ }
+
+ /**
+ * Specifies a [[KeySelector]] for elements from the second input.
+ */
+ def equalTo(keySelector: T2 => KEY): CoGroupedStreams.WithKey[T1, T2, KEY] = {
+ val cleanFun = clean(keySelector)
+ val javaSelector = new KeySelector[T2, KEY] {
+ def getKey(in: T2) = cleanFun(in)
+ }
+ new CoGroupedStreams.WithKey[T1, T2, KEY](input1, input2, keySelector1, javaSelector)
+ }
+
+ /**
+ * Specifies the window on which the co-group operation works.
+ */
+ def window[W <: Window](
+ assigner: WindowAssigner[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], W])
+ : CoGroupedStreams.WithWindow[T1, T2, KEY, W] = {
+ if (keySelector1 == null || keySelector2 == null) {
+ throw new UnsupportedOperationException("You first need to specify KeySelectors for both" +
+ "inputs using where() and equalTo().")
+ }
+ new CoGroupedStreams.WithWindow[T1, T2, KEY, W](
+ input1,
+ input2,
+ keySelector1,
+ keySelector2,
+ clean(assigner),
+ null,
+ null)
+ }
+
+ /**
+ * Returns a "closure-cleaned" version of the given function. Cleans only if closure cleaning
+ * is not disabled in the [[org.apache.flink.api.common.ExecutionConfig]].
+ */
+ private[flink] def clean[F <: AnyRef](f: F): F = {
+ new StreamExecutionEnvironment(input1.getJavaStream.getExecutionEnvironment).scalaClean(f)
+ }
+ }
+
+ /**
+ * A co-group operation that has [[KeySelector]]s defined for both inputs as
+ * well as a [[WindowAssigner]].
+ *
+ * @tparam T1 Type of the elements from the first input
+ * @tparam T2 Type of the elements from the second input
+ * @tparam KEY Type of the key. This must be the same for both inputs
+ * @tparam W Type of { @link Window} on which the co-group operation works.
+ */
+ class WithWindow[T1, T2, KEY, W <: Window](
+ input1: DataStream[T1],
+ input2: DataStream[T2],
+ keySelector1: KeySelector[T1, KEY],
+ keySelector2: KeySelector[T2, KEY],
+ windowAssigner: WindowAssigner[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], W],
+ trigger: Trigger[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], _ >: W],
+ evictor: Evictor[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], _ >: W]) {
+
+
+ /**
+ * Sets the [[Trigger]] that should be used to trigger window emission.
+ */
+ def trigger(newTrigger: Trigger[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], _ >: W])
+ : CoGroupedStreams.WithWindow[T1, T2, KEY, W] = {
+ new WithWindow[T1, T2, KEY, W](
+ input1,
+ input2,
+ keySelector1,
+ keySelector2,
+ windowAssigner,
+ newTrigger,
+ evictor)
+ }
+
+ /**
+ * Sets the [[Evictor]] that should be used to evict elements from a window before emission.
+ *
+ * Note: When using an evictor window performance will degrade significantly, since
+ * pre-aggregation of window results cannot be used.
+ */
+ def evictor(newEvictor: Evictor[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], _ >: W])
+ : CoGroupedStreams.WithWindow[T1, T2, KEY, W] = {
+ new WithWindow[T1, T2, KEY, W](
+ input1,
+ input2,
+ keySelector1,
+ keySelector2,
+ windowAssigner,
+ trigger,
+ newEvictor)
+ }
+
+ /**
+ * Completes the co-group operation with the user function that is executed
+ * for windowed groups.
+ */
+ def apply[O: TypeInformation: ClassTag](
+ fun: (Iterator[T1], Iterator[T2]) => O): DataStream[O] = {
+ require(fun != null, "CoGroup function must not be null.")
+
+ val coGrouper = new CoGroupFunction[T1, T2, O] {
+ val cleanFun = clean(fun)
+ def coGroup(
+ left: java.lang.Iterable[T1],
+ right: java.lang.Iterable[T2], out: Collector[O]) = {
+ out.collect(cleanFun(left.iterator().asScala, right.iterator().asScala))
+ }
+ }
+ apply(coGrouper)
+ }
+
+ /**
+ * Completes the co-group operation with the user function that is executed
+ * for windowed groups.
+ */
+ def apply[O: TypeInformation: ClassTag](
+ fun: (Iterator[T1], Iterator[T2], Collector[O]) => Unit): DataStream[O] = {
+ require(fun != null, "CoGroup function must not be null.")
+
+ val coGrouper = new CoGroupFunction[T1, T2, O] {
+ val cleanFun = clean(fun)
+ def coGroup(
+ left: java.lang.Iterable[T1],
+ right: java.lang.Iterable[T2], out: Collector[O]) = {
+ cleanFun(left.iterator.asScala, right.iterator.asScala, out)
+ }
+ }
+ apply(coGrouper)
+ }
+
+ /**
+ * Completes the co-group operation with the user function that is executed
+ * for windowed groups.
+ */
+ def apply[T: TypeInformation](function: CoGroupFunction[T1, T2, T]): DataStream[T] = {
+
+ val coGroup = JavaCoGroupedStreams.createCoGroup(input1.getJavaStream, input2.getJavaStream)
+
+ coGroup
+ .where(keySelector1)
+ .equalTo(keySelector2)
+ .window(windowAssigner)
+ .trigger(trigger)
+ .evictor(evictor)
+ .apply(clean(function), implicitly[TypeInformation[T]])
+ }
+
+ /**
+ * Returns a "closure-cleaned" version of the given function. Cleans only if closure cleaning
+ * is not disabled in the [[org.apache.flink.api.common.ExecutionConfig]].
+ */
+ private[flink] def clean[F <: AnyRef](f: F): F = {
+ new StreamExecutionEnvironment(input1.getJavaStream.getExecutionEnvironment).scalaClean(f)
+ }
+ }
+
+
+ /**
+ * Creates a new co-group operation from the two given inputs.
+ */
+ def createCoGroup[T1, T2](input1: DataStream[T1], input2: DataStream[T2])
+ : CoGroupedStreams.Unspecified[T1, T2] = {
+ new CoGroupedStreams.Unspecified[T1, T2](input1, input2)
+ }
+
+}
+
http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
index 8aeacb4..7babc40 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
@@ -751,18 +751,20 @@ class DataStream[T](javaStream: JavaStream[T]) {
}
/**
- * Initiates a temporal Join transformation that joins the elements of two
- * data streams on key equality over a specified time window.
- *
- * This method returns a StreamJoinOperator on which the
- * .onWindow(..) should be called to define the
- * window, and then the .where(..) and .equalTo(..) methods can be used to defin
- * the join keys.</p> The user can also use the apply method of the returned JoinedStream
- * to use custom join function.
- *
+ * Creates a co-group operation. See [[CoGroupedStreams]] for an example of how the keys
+ * and window can be specified.
*/
- def join[R](stream: DataStream[R]): StreamJoinOperator[T, R] =
- new StreamJoinOperator[T, R](javaStream, stream.getJavaStream)
+ def coGroup[T2](otherStream: DataStream[T2]): CoGroupedStreams.Unspecified[T, T2] = {
+ CoGroupedStreams.createCoGroup(this, otherStream)
+ }
+
+ /**
+ * Creates a join operation. See [[JoinedStreams]] for an example of how the keys
+ * and window can be specified.
+ */
+ def join[T2](otherStream: DataStream[T2]): JoinedStreams.Unspecified[T, T2] = {
+ JoinedStreams.createJoin(this, otherStream)
+ }
/**
* Writes a DataStream to the standard output stream (stdout). For each
http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/JoinedStreams.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/JoinedStreams.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/JoinedStreams.scala
new file mode 100644
index 0000000..be059b8
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/JoinedStreams.scala
@@ -0,0 +1,303 @@
+/*
+ * 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.scala
+
+import org.apache.flink.api.common.functions.{FlatJoinFunction, JoinFunction}
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.functions.KeySelector
+import org.apache.flink.streaming.api.datastream.{JoinedStreams => JavaJoinedStreams, CoGroupedStreams => JavaCoGroupedStreams}
+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.triggers.Trigger
+import org.apache.flink.streaming.api.windowing.windows.Window
+import org.apache.flink.util.Collector
+
+import scala.reflect.ClassTag
+
+/**
+ * `JoinedStreams` represents two [[DataStream]]s that have been joined.
+ * A streaming join operation is evaluated over elements in a window.
+ *
+ * To finalize the join operation you also need to specify a [[KeySelector]] for
+ * both the first and second input and a [[WindowAssigner]]
+ *
+ * Note: Right now, the groups are being built in memory so you need to ensure that they don't
+ * get too big. Otherwise the JVM might crash.
+ *
+ * Example:
+ *
+ * {{{
+ * val one: DataStream[(String, Int)] = ...
+ * val two: DataStream[(String, Int)] = ...
+ *
+ * val result = one.join(two)
+ * .where {t => ... }
+ * .equal {t => ... }
+ * .window(TumblingTimeWindows.of(Time.of(5, TimeUnit.SECONDS)))
+ * .apply(new MyJoinFunction())
+ * } }}}
+ */
+object JoinedStreams {
+
+ /**
+ * A join operation that does not yet have its [[KeySelector]]s defined.
+ *
+ * @tparam T1 Type of the elements from the first input
+ * @tparam T2 Type of the elements from the second input
+ */
+ class Unspecified[T1, T2](input1: DataStream[T1], input2: DataStream[T2]) {
+
+ /**
+ * Specifies a [[KeySelector]] for elements from the first input.
+ */
+ def where[KEY](keySelector: T1 => KEY): JoinedStreams.WithKey[T1, T2, KEY] = {
+ val cleanFun = clean(keySelector)
+ val javaSelector = new KeySelector[T1, KEY] {
+ def getKey(in: T1) = cleanFun(in)
+ }
+ new JoinedStreams.WithKey[T1, T2, KEY](input1, input2, javaSelector, null)
+ }
+
+ /**
+ * Specifies a [[KeySelector]] for elements from the second input.
+ */
+ def equalTo[KEY](keySelector: T2 => KEY): JoinedStreams.WithKey[T1, T2, KEY] = {
+ val cleanFun = clean(keySelector)
+ val javaSelector = new KeySelector[T2, KEY] {
+ def getKey(in: T2) = cleanFun(in)
+ }
+ new JoinedStreams.WithKey[T1, T2, KEY](input1, input2, null, javaSelector)
+ }
+
+ /**
+ * Returns a "closure-cleaned" version of the given function. Cleans only if closure cleaning
+ * is not disabled in the [[org.apache.flink.api.common.ExecutionConfig]].
+ */
+ private[flink] def clean[F <: AnyRef](f: F): F = {
+ new StreamExecutionEnvironment(input1.getJavaStream.getExecutionEnvironment).scalaClean(f)
+ }
+ }
+
+ /**
+ * A join operation that has [[KeySelector]]s defined for either both or
+ * one input.
+ *
+ * You need to specify a [[KeySelector]] for both inputs using [[where()]] and [[equalTo()]]
+ * before you can proceeed with specifying a [[WindowAssigner]] using [[window()]].
+ *
+ * @tparam T1 Type of the elements from the first input
+ * @tparam T2 Type of the elements from the second input
+ * @tparam KEY Type of the key. This must be the same for both inputs
+ */
+ class WithKey[T1, T2, KEY](
+ input1: DataStream[T1],
+ input2: DataStream[T2],
+ keySelector1: KeySelector[T1, KEY],
+ keySelector2: KeySelector[T2, KEY]) {
+
+ /**
+ * Specifies a [[KeySelector]] for elements from the first input.
+ */
+ def where(keySelector: T1 => KEY): JoinedStreams.WithKey[T1, T2, KEY] = {
+ val cleanFun = clean(keySelector)
+ val javaSelector = new KeySelector[T1, KEY] {
+ def getKey(in: T1) = cleanFun(in)
+ }
+ new JoinedStreams.WithKey[T1, T2, KEY](input1, input2, javaSelector, keySelector2)
+ }
+
+ /**
+ * Specifies a [[KeySelector]] for elements from the second input.
+ */
+ def equalTo(keySelector: T2 => KEY): JoinedStreams.WithKey[T1, T2, KEY] = {
+ val cleanFun = clean(keySelector)
+ val javaSelector = new KeySelector[T2, KEY] {
+ def getKey(in: T2) = cleanFun(in)
+ }
+ new JoinedStreams.WithKey[T1, T2, KEY](input1, input2, keySelector1, javaSelector)
+ }
+
+ /**
+ * Specifies the window on which the join operation works.
+ */
+ def window[W <: Window](
+ assigner: WindowAssigner[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], W])
+ : JoinedStreams.WithWindow[T1, T2, KEY, W] = {
+ if (keySelector1 == null || keySelector2 == null) {
+ throw new UnsupportedOperationException("You first need to specify KeySelectors for both" +
+ "inputs using where() and equalTo().")
+ }
+ new JoinedStreams.WithWindow[T1, T2, KEY, W](
+ input1,
+ input2,
+ keySelector1,
+ keySelector2,
+ clean(assigner),
+ null,
+ null)
+ }
+
+ /**
+ * Returns a "closure-cleaned" version of the given function. Cleans only if closure cleaning
+ * is not disabled in the [[org.apache.flink.api.common.ExecutionConfig]].
+ */
+ private[flink] def clean[F <: AnyRef](f: F): F = {
+ new StreamExecutionEnvironment(input1.getJavaStream.getExecutionEnvironment).scalaClean(f)
+ }
+ }
+
+ /**
+ * A join operation that has [[KeySelector]]s defined for both inputs as
+ * well as a [[WindowAssigner]].
+ *
+ * @tparam T1 Type of the elements from the first input
+ * @tparam T2 Type of the elements from the second input
+ * @tparam KEY Type of the key. This must be the same for both inputs
+ * @tparam W Type of { @link Window} on which the join operation works.
+ */
+ class WithWindow[T1, T2, KEY, W <: Window](
+ input1: DataStream[T1],
+ input2: DataStream[T2],
+ keySelector1: KeySelector[T1, KEY],
+ keySelector2: KeySelector[T2, KEY],
+ windowAssigner: WindowAssigner[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], W],
+ trigger: Trigger[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], _ >: W],
+ evictor: Evictor[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], _ >: W]) {
+
+
+ /**
+ * Sets the [[Trigger]] that should be used to trigger window emission.
+ */
+ def trigger(newTrigger: Trigger[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], _ >: W])
+ : JoinedStreams.WithWindow[T1, T2, KEY, W] = {
+ new WithWindow[T1, T2, KEY, W](
+ input1,
+ input2,
+ keySelector1,
+ keySelector2,
+ windowAssigner,
+ newTrigger,
+ evictor)
+ }
+
+ /**
+ * Sets the [[Evictor]] that should be used to evict elements from a window before emission.
+ *
+ * Note: When using an evictor window performance will degrade significantly, since
+ * pre-aggregation of window results cannot be used.
+ */
+ def evictor(newEvictor: Evictor[_ >: JavaCoGroupedStreams.TaggedUnion[T1, T2], _ >: W])
+ : JoinedStreams.WithWindow[T1, T2, KEY, W] = {
+ new WithWindow[T1, T2, KEY, W](
+ input1,
+ input2,
+ keySelector1,
+ keySelector2,
+ windowAssigner,
+ trigger,
+ newEvictor)
+ }
+
+ /**
+ * Completes the join operation with the user function that is executed
+ * for windowed groups.
+ */
+ def apply[O: TypeInformation: ClassTag](fun: (T1, T2) => O): DataStream[O] = {
+ require(fun != null, "Join function must not be null.")
+
+ val joiner = new FlatJoinFunction[T1, T2, O] {
+ val cleanFun = clean(fun)
+ def join(left: T1, right: T2, out: Collector[O]) = {
+ out.collect(cleanFun(left, right))
+ }
+ }
+ apply(joiner)
+ }
+
+ /**
+ * Completes the join operation with the user function that is executed
+ * for windowed groups.
+ */
+
+ def apply[O: TypeInformation: ClassTag](fun: (T1, T2, Collector[O]) => Unit): DataStream[O] = {
+ require(fun != null, "Join function must not be null.")
+
+ val joiner = new FlatJoinFunction[T1, T2, O] {
+ val cleanFun = clean(fun)
+ def join(left: T1, right: T2, out: Collector[O]) = {
+ cleanFun(left, right, out)
+ }
+ }
+ apply(joiner)
+ }
+
+ /**
+ * Completes the join operation with the user function that is executed
+ * for windowed groups.
+ */
+ def apply[T: TypeInformation](function: JoinFunction[T1, T2, T]): DataStream[T] = {
+
+ val join = JavaJoinedStreams.createJoin(input1.getJavaStream, input2.getJavaStream)
+
+ join
+ .where(keySelector1)
+ .equalTo(keySelector2)
+ .window(windowAssigner)
+ .trigger(trigger)
+ .evictor(evictor)
+ .apply(clean(function), implicitly[TypeInformation[T]])
+ }
+
+ /**
+ * Completes the join operation with the user function that is executed
+ * for windowed groups.
+ */
+ def apply[T: TypeInformation](function: FlatJoinFunction[T1, T2, T]): DataStream[T] = {
+
+ val join = JavaJoinedStreams.createJoin(input1.getJavaStream, input2.getJavaStream)
+
+ join
+ .where(keySelector1)
+ .equalTo(keySelector2)
+ .window(windowAssigner)
+ .trigger(trigger)
+ .evictor(evictor)
+ .apply(clean(function), implicitly[TypeInformation[T]])
+ }
+
+ /**
+ * Returns a "closure-cleaned" version of the given function. Cleans only if closure cleaning
+ * is not disabled in the [[org.apache.flink.api.common.ExecutionConfig]].
+ */
+ private[flink] def clean[F <: AnyRef](f: F): F = {
+ new StreamExecutionEnvironment(input1.getJavaStream.getExecutionEnvironment).scalaClean(f)
+ }
+ }
+
+
+ /**
+ * Creates a new join operation from the two given inputs.
+ */
+ def createJoin[T1, T2](input1: DataStream[T1], input2: DataStream[T2])
+ : JoinedStreams.Unspecified[T1, T2] = {
+ new JoinedStreams.Unspecified[T1, T2](input1, input2)
+ }
+
+}
+
http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamJoinOperator.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamJoinOperator.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamJoinOperator.scala
deleted file mode 100644
index e0bbaf8..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamJoinOperator.scala
+++ /dev/null
@@ -1,203 +0,0 @@
-/*
- * 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.scala
-
-import java.util.concurrent.TimeUnit
-
-import org.apache.flink.api.common.functions.JoinFunction
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.functions.KeySelector
-import org.apache.flink.api.java.operators.Keys
-import org.apache.flink.streaming.api.datastream.temporal.TemporalWindow
-import org.apache.flink.streaming.api.datastream.{DataStream => JavaStream}
-import org.apache.flink.streaming.util.keys.KeySelectorUtil
-
-import scala.Array.canBuildFrom
-import scala.reflect.ClassTag
-
-class StreamJoinOperator[I1, I2](i1: JavaStream[I1], i2: JavaStream[I2]) extends
-TemporalOperator[I1, I2, StreamJoinOperator.JoinWindow[I1, I2]](i1, i2) {
-
- override def createNextWindowOperator() = {
- new StreamJoinOperator.JoinWindow[I1, I2](this)
- }
-}
-
-object StreamJoinOperator {
-
- class JoinWindow[I1, I2](private[flink] val op: StreamJoinOperator[I1, I2]) extends
- TemporalWindow[JoinWindow[I1, I2]] {
-
- private[flink] val type1 = op.input1.getType()
-
- /**
- * Continues a temporal Join transformation by defining
- * the fields in the first stream to be used as keys for the join.
- * The resulting incomplete join can be completed by JoinPredicate.equalTo()
- * to define the second key.
- */
- def where(fields: Int*) = {
- new JoinPredicate[I1, I2](op, KeySelectorUtil.getSelectorForKeys(
- new Keys.ExpressionKeys(fields.toArray, type1),
- type1,
- op.input1.getExecutionEnvironment.getConfig))
- }
-
- /**
- * Continues a temporal Join transformation by defining
- * the fields in the first stream to be used as keys for the join.
- * The resulting incomplete join can be completed by JoinPredicate.equalTo()
- * to define the second key.
- */
- def where(firstField: String, otherFields: String*) =
- new JoinPredicate[I1, I2](op, KeySelectorUtil.getSelectorForKeys(
- new Keys.ExpressionKeys(firstField +: otherFields.toArray, type1),
- type1,
- op.input1.getExecutionEnvironment.getConfig))
-
- /**
- * Continues a temporal Join transformation by defining
- * the keyselector function that will be used to extract keys from the first stream
- * for the join.
- * The resulting incomplete join can be completed by JoinPredicate.equalTo()
- * to define the second key.
- */
- def where[K: TypeInformation](fun: (I1) => K) = {
- val keyType = implicitly[TypeInformation[K]]
- val cleanFun = op.input1.clean(fun)
- val keyExtractor = new KeySelector[I1, K] {
- def getKey(in: I1) = cleanFun(in)
- }
- new JoinPredicate[I1, I2](op, keyExtractor)
- }
-
- override def every(length: Long, timeUnit: TimeUnit): JoinWindow[I1, I2] = {
- every(timeUnit.toMillis(length))
- }
-
- override def every(length: Long): JoinWindow[I1, I2] = {
- op.slideInterval = length
- this
- }
-
- }
-
- class JoinPredicate[I1, I2](private[flink] val op: StreamJoinOperator[I1, I2],
- private[flink] val keys1: KeySelector[I1, _]) {
- private[flink] var keys2: KeySelector[I2, _] = null
- private[flink] val type2 = op.input2.getType()
-
- /**
- * Creates a temporal join transformation by defining the second join key.
- * The returned transformation wrapes each joined element pair in a tuple2:
- * (first, second)
- * To define a custom wrapping, use JoinedStream.apply(...)
- */
- def equalTo(fields: Int*): JoinedStream[I1, I2] = {
- finish(KeySelectorUtil.getSelectorForKeys(
- new Keys.ExpressionKeys(fields.toArray, type2),
- type2,
- op.input1.getExecutionEnvironment.getConfig))
- }
-
- /**
- * Creates a temporal join transformation by defining the second join key.
- * The returned transformation wrapes each joined element pair in a tuple2:
- * (first, second)
- * To define a custom wrapping, use JoinedStream.apply(...)
- */
- def equalTo(firstField: String, otherFields: String*): JoinedStream[I1, I2] =
- finish(KeySelectorUtil.getSelectorForKeys(
- new Keys.ExpressionKeys(firstField +: otherFields.toArray, type2),
- type2,
- op.input1.getExecutionEnvironment.getConfig))
-
- /**
- * Creates a temporal join transformation by defining the second join key.
- * The returned transformation wrapes each joined element pair in a tuple2:
- * (first, second)
- * To define a custom wrapping, use JoinedStream.apply(...)
- */
- def equalTo[K: TypeInformation](fun: (I2) => K): JoinedStream[I1, I2] = {
- val keyType = implicitly[TypeInformation[K]]
- val cleanFun = op.input1.clean(fun)
- val keyExtractor = new KeySelector[I2, K] {
- def getKey(in: I2) = cleanFun(in)
- }
- finish(keyExtractor)
- }
-
- private def finish(keys2: KeySelector[I2, _]): JoinedStream[I1, I2] = {
- this.keys2 = keys2
- new JoinedStream[I1, I2](this, createJoinOperator())
- }
-
- private def createJoinOperator(): JavaStream[(I1, I2)] = {
-
-// val returnType = createTuple2TypeInformation[I1, I2](op.input1.getType, op.input2.getType)
-// op.input1.keyBy(keys1).connect(op.input2.keyBy(keys2))
-// .addGeneralWindowCombine(getJoinWindowFunction(this, (_, _)),
-// returnType, op.windowSize, op.slideInterval, op.timeStamp1, op.timeStamp2)
- null
- }
- }
-
- class JoinedStream[I1, I2](
- jp: JoinPredicate[I1, I2],
- javaStream: JavaStream[(I1, I2)]) extends DataStream[(I1, I2)](javaStream) {
-
- private val op = jp.op
-
- /**
- * Sets a wrapper for the joined elements. For each joined pair, the result of the
- * udf call will be emitted.
- */
- def apply[R: TypeInformation: ClassTag](fun: (I1, I2) => R): DataStream[R] = {
-
- val cleanFun = clean(getJoinWindowFunction(jp, fun))
-
-// op.input1.keyBy(jp.keys1).connect(op.input2.keyBy(jp.keys2))
-// .addGeneralWindowCombine[R](
-// cleanFun,
-// implicitly[TypeInformation[R]],
-// op.windowSize,
-// op.slideInterval,
-// op.timeStamp1,
-// op.timeStamp2)
- null
- }
- }
-
- private[flink] def getJoinWindowFunction[I1, I2, R](jp: JoinPredicate[I1, I2],
- joinFunction: (I1, I2) => R) = {
- require(joinFunction != null, "Join function must not be null.")
-
- val cleanFun = jp.op.input1.clean(joinFunction)
-
- val joinFun = new JoinFunction[I1, I2, R] {
- override def join(first: I1, second: I2): R = {
- cleanFun(first, second)
- }
- }
-
-// new JoinWindowFunction[I1, I2, R](jp.keys1, jp.keys2, joinFun)
- null
- }
-
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/TemporalOperator.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/TemporalOperator.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/TemporalOperator.scala
deleted file mode 100644
index 8357c4d..0000000
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/TemporalOperator.scala
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.scala
-
-import org.apache.flink.api.scala.ClosureCleaner
-import org.apache.flink.streaming.api.datastream.temporal.{ TemporalOperator => JTempOp }
-import org.apache.flink.streaming.api.datastream.{ DataStream => JavaStream }
-import org.apache.flink.streaming.api.datastream.temporal.TemporalWindow
-import org.apache.flink.streaming.api.windowing.helper.Timestamp
-
-abstract class TemporalOperator[I1, I2, OP <: TemporalWindow[OP]](
- i1: JavaStream[I1], i2: JavaStream[I2]) extends JTempOp[I1, I2, OP](i1, i2) {
-
- def onWindow(length: Long, ts1: I1 => Long, ts2: I2 => Long, startTime: Long = 0): OP = {
- val timeStamp1 = getTS(ts1)
- val timeStamp2 = getTS(ts2)
- onWindow(length, timeStamp1, timeStamp2, startTime)
- }
-
- def getTS[R](ts: R => Long): Timestamp[R] = {
- val cleanFun = clean(ts)
- new Timestamp[R] {
- def getTimestamp(in: R) = cleanFun(in)
- }
- }
-
- /**
- * Returns a "closure-cleaned" version of the given function. Cleans only if closure cleaning
- * is not disabled in the {@link org.apache.flink.api.common.ExecutionConfig}
- */
- private[flink] def clean[F <: AnyRef](f: F): F = {
- new StreamExecutionEnvironment(i1.getExecutionEnvironment).scalaClean(f)
- }
-
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/CoGroupJoinITCase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/CoGroupJoinITCase.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/CoGroupJoinITCase.scala
new file mode 100644
index 0000000..7232309
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/CoGroupJoinITCase.scala
@@ -0,0 +1,274 @@
+/*
+ * 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.scala
+
+import java.util.concurrent.TimeUnit
+
+import org.apache.flink.streaming.api.functions.TimestampExtractor
+import org.apache.flink.streaming.api.functions.sink.SinkFunction
+import org.apache.flink.streaming.api.functions.source.SourceFunction
+import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows
+import org.apache.flink.streaming.api.windowing.time.Time
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
+import org.junit.Test
+import org.junit.Assert._
+
+import scala.collection.mutable
+
+class CoGroupJoinITCase extends StreamingMultipleProgramsTestBase {
+
+ @Test
+ def testCoGroup(): Unit = {
+ CoGroupJoinITCase.testResults = mutable.MutableList()
+
+ val env = StreamExecutionEnvironment.getExecutionEnvironment
+ env.setParallelism(1)
+ env.getConfig.enableTimestamps
+
+ val source1 = env.addSource(new SourceFunction[(String, Int)]() {
+ def run(ctx: SourceFunction.SourceContext[(String, Int)]) {
+ ctx.collect(("a", 0))
+ ctx.collect(("a", 1))
+ ctx.collect(("a", 2))
+ ctx.collect(("b", 3))
+ ctx.collect(("b", 4))
+ ctx.collect(("b", 5))
+ ctx.collect(("a", 6))
+ ctx.collect(("a", 7))
+ ctx.collect(("a", 8))
+ }
+
+ def cancel() {
+ }
+ }).extractTimestamp(new CoGroupJoinITCase.Tuple2TimestampExtractor)
+
+ val source2 = env.addSource(new SourceFunction[(String, Int)]() {
+ def run(ctx: SourceFunction.SourceContext[(String, Int)]) {
+ ctx.collect(("a", 0))
+ ctx.collect(("a", 1))
+ ctx.collect(("b", 3))
+ ctx.collect(("c", 6))
+ ctx.collect(("c", 7))
+ ctx.collect(("c", 8))
+ }
+
+ def cancel() {
+ }
+ }).extractTimestamp(new CoGroupJoinITCase.Tuple2TimestampExtractor)
+
+ source1.coGroup(source2)
+ .where(_._1)
+ .equalTo(_._1)
+ .window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
+ .apply { (first: Iterator[(String, Int)], second: Iterator[(String, Int)]) =>
+ "F:" + first.mkString("") + " S:" + second.mkString("")
+ }
+ .addSink(new SinkFunction[String]() {
+ def invoke(value: String) {
+ CoGroupJoinITCase.testResults += value
+ }
+ })
+
+ env.execute("CoGroup Test")
+
+ val expectedResult = mutable.MutableList(
+ "F:(a,0)(a,1)(a,2) S:(a,0)(a,1)",
+ "F:(b,3)(b,4)(b,5) S:(b,3)",
+ "F:(a,6)(a,7)(a,8) S:",
+ "F: S:(c,6)(c,7)(c,8)")
+
+ assertEquals(expectedResult.sorted, CoGroupJoinITCase.testResults.sorted)
+ }
+
+ @Test
+ def testJoin(): Unit = {
+ CoGroupJoinITCase.testResults = mutable.MutableList()
+
+ val env = StreamExecutionEnvironment.getExecutionEnvironment
+ env.setParallelism(1)
+ env.getConfig.enableTimestamps
+
+ val source1 = env.addSource(new SourceFunction[(String, String, Int)]() {
+ def run(ctx: SourceFunction.SourceContext[(String, String, Int)]) {
+ ctx.collect(("a", "x", 0))
+ ctx.collect(("a", "y", 1))
+ ctx.collect(("a", "z", 2))
+
+ ctx.collect(("b", "u", 3))
+ ctx.collect(("b", "w", 5))
+
+ ctx.collect(("a", "i", 6))
+ ctx.collect(("a", "j", 7))
+ ctx.collect(("a", "k", 8))
+ }
+
+ def cancel() {
+ }
+ }).extractTimestamp(new CoGroupJoinITCase.Tuple3TimestampExtractor)
+
+ val source2 = env.addSource(new SourceFunction[(String, String, Int)]() {
+ def run(ctx: SourceFunction.SourceContext[(String, String, Int)]) {
+ ctx.collect(("a", "u", 0))
+ ctx.collect(("a", "w", 1))
+
+ ctx.collect(("b", "i", 3))
+ ctx.collect(("b", "k", 5))
+
+ ctx.collect(("a", "x", 6))
+ ctx.collect(("a", "z", 8))
+ }
+
+ def cancel() {
+ }
+ }).extractTimestamp(new CoGroupJoinITCase.Tuple3TimestampExtractor)
+
+ source1.join(source2)
+ .where(_._1)
+ .equalTo(_._1)
+ .window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
+ .apply( (l, r) => l.toString + ":" + r.toString)
+ .addSink(new SinkFunction[String]() {
+ def invoke(value: String) {
+ CoGroupJoinITCase.testResults += value
+ }
+ })
+
+ env.execute("Join Test")
+
+ val expectedResult = mutable.MutableList(
+ "(a,x,0):(a,u,0)",
+ "(a,x,0):(a,w,1)",
+ "(a,y,1):(a,u,0)",
+ "(a,y,1):(a,w,1)",
+ "(a,z,2):(a,u,0)",
+ "(a,z,2):(a,w,1)",
+ "(b,u,3):(b,i,3)",
+ "(b,u,3):(b,k,5)",
+ "(b,w,5):(b,i,3)",
+ "(b,w,5):(b,k,5)",
+ "(a,i,6):(a,x,6)",
+ "(a,i,6):(a,z,8)",
+ "(a,j,7):(a,x,6)",
+ "(a,j,7):(a,z,8)",
+ "(a,k,8):(a,x,6)",
+ "(a,k,8):(a,z,8)")
+
+ assertEquals(expectedResult.sorted, CoGroupJoinITCase.testResults.sorted)
+ }
+
+ @Test
+ def testSelfJoin(): Unit = {
+ CoGroupJoinITCase.testResults = mutable.MutableList()
+
+ val env = StreamExecutionEnvironment.getExecutionEnvironment
+ env.setParallelism(1)
+ env.getConfig.enableTimestamps
+
+ val source1 = env.addSource(new SourceFunction[(String, String, Int)]() {
+ def run(ctx: SourceFunction.SourceContext[(String, String, Int)]) {
+ ctx.collect(("a", "x", 0))
+ ctx.collect(("a", "y", 1))
+ ctx.collect(("a", "z", 2))
+
+ ctx.collect(("b", "u", 3))
+ ctx.collect(("b", "w", 5))
+
+ ctx.collect(("a", "i", 6))
+ ctx.collect(("a", "j", 7))
+ ctx.collect(("a", "k", 8))
+ }
+
+ def cancel() {
+ }
+ }).extractTimestamp(new CoGroupJoinITCase.Tuple3TimestampExtractor)
+
+ source1.join(source1)
+ .where(_._1)
+ .equalTo(_._1)
+ .window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
+ .apply( (l, r) => l.toString + ":" + r.toString)
+ .addSink(new SinkFunction[String]() {
+ def invoke(value: String) {
+ CoGroupJoinITCase.testResults += value
+ }
+ })
+
+ env.execute("Self-Join Test")
+
+ val expectedResult = mutable.MutableList(
+ "(a,x,0):(a,x,0)",
+ "(a,x,0):(a,y,1)",
+ "(a,x,0):(a,z,2)",
+ "(a,y,1):(a,x,0)",
+ "(a,y,1):(a,y,1)",
+ "(a,y,1):(a,z,2)",
+ "(a,z,2):(a,x,0)",
+ "(a,z,2):(a,y,1)",
+ "(a,z,2):(a,z,2)",
+ "(b,u,3):(b,u,3)",
+ "(b,u,3):(b,w,5)",
+ "(b,w,5):(b,u,3)",
+ "(b,w,5):(b,w,5)",
+ "(a,i,6):(a,i,6)",
+ "(a,i,6):(a,j,7)",
+ "(a,i,6):(a,k,8)",
+ "(a,j,7):(a,i,6)",
+ "(a,j,7):(a,j,7)",
+ "(a,j,7):(a,k,8)",
+ "(a,k,8):(a,i,6)",
+ "(a,k,8):(a,j,7)",
+ "(a,k,8):(a,k,8)")
+
+ assertEquals(expectedResult.sorted, CoGroupJoinITCase.testResults.sorted)
+ }
+
+}
+
+
+object CoGroupJoinITCase {
+ private var testResults: mutable.MutableList[String] = null
+
+ private class Tuple2TimestampExtractor extends TimestampExtractor[(String, Int)] {
+ def extractTimestamp(element: (String, Int), currentTimestamp: Long): Long = {
+ element._2
+ }
+
+ def emitWatermark(element: (String, Int), currentTimestamp: Long): Long = {
+ element._2 - 1
+ }
+
+ def getCurrentWatermark: Long = {
+ Long.MinValue
+ }
+ }
+
+ private class Tuple3TimestampExtractor extends TimestampExtractor[(String, String, Int)] {
+ def extractTimestamp(element: (String, String, Int), currentTimestamp: Long): Long = {
+ element._3
+ }
+
+ def emitWatermark(element: (String, String, Int), currentTimestamp: Long): Long = {
+ element._3 - 1
+ }
+
+ def getCurrentWatermark: Long = {
+ Long.MinValue
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/8634dbbe/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala
index d1fd233..c6bd87a 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingScalaAPICompletenessTest.scala
@@ -20,7 +20,7 @@ package org.apache.flink.streaming.api.scala
import java.lang.reflect.Method
import org.apache.flink.api.scala.completeness.ScalaAPICompletenessTestBase
-import org.apache.flink.streaming.api.datastream.{DataStream => JavaStream}
+import org.apache.flink.streaming.api.datastream.{DataStream => JavaStream, JoinedStreams}
import scala.language.existentials
@@ -138,14 +138,14 @@ class StreamingScalaAPICompletenessTest extends ScalaAPICompletenessTestBase {
classOf[KeyedStream[_, _]])
checkMethods(
- "StreamJoinOperator", "StreamJoinOperator",
- classOf[org.apache.flink.streaming.api.datastream.temporal.StreamJoinOperator[_,_]],
- classOf[StreamJoinOperator[_,_]])
+ "JoinedStreams.WithWindow", "JoinedStreams.WithWindow",
+ classOf[org.apache.flink.streaming.api.datastream.JoinedStreams.WithWindow[_,_,_,_]],
+ classOf[JoinedStreams.WithWindow[_,_,_,_]])
checkMethods(
- "TemporalOperator", "TemporalOperator",
- classOf[org.apache.flink.streaming.api.datastream.temporal.TemporalOperator[_,_,_]],
- classOf[TemporalOperator[_,_,_]])
+ "CoGroupedStreams.WithWindow", "CoGroupedStreams.WithWindow",
+ classOf[org.apache.flink.streaming.api.datastream.CoGroupedStreams.WithWindow[_,_,_,_]],
+ classOf[CoGroupedStreams.WithWindow[_,_,_,_]])
checkMethods(
"WindowedDataStream", "WindowedDataStream",
[7/8] flink git commit: Move CoGroupJoinITCase to windowing package
Posted by al...@apache.org.
Move CoGroupJoinITCase to windowing package
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/ce792b11
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/ce792b11
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/ce792b11
Branch: refs/heads/master
Commit: ce792b11a4a2b8d7b02a59dcadcc4c052ff5531a
Parents: ff367d6
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Wed Oct 7 16:18:24 2015 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Wed Oct 7 22:08:25 2015 +0200
----------------------------------------------------------------------
.../flink/streaming/api/CoGroupJoinITCase.java | 372 ------------------
.../operators/windowing/CoGroupJoinITCase.java | 373 +++++++++++++++++++
2 files changed, 373 insertions(+), 372 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/ce792b11/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoGroupJoinITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoGroupJoinITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoGroupJoinITCase.java
deleted file mode 100644
index 9ddd6eb..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoGroupJoinITCase.java
+++ /dev/null
@@ -1,372 +0,0 @@
-/*
-* 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;
-
-import com.google.common.collect.Lists;
-import org.apache.flink.api.common.functions.CoGroupFunction;
-import org.apache.flink.api.common.functions.JoinFunction;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.TimestampExtractor;
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows;
-import org.apache.flink.streaming.api.windowing.time.Time;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.apache.flink.util.Collector;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-public class CoGroupJoinITCase extends StreamingMultipleProgramsTestBase {
-
- private static List<String> testResults;
-
- @Test
- public void testCoGroup() throws Exception {
-
- testResults = Lists.newArrayList();
-
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
- env.setParallelism(1);
-
- DataStream<Tuple2<String, Integer>> source1 = env.addSource(new SourceFunction<Tuple2<String, Integer>>() {
- private static final long serialVersionUID = 1L;
-
- @Override
- public void run(SourceContext<Tuple2<String, Integer>> ctx) throws Exception {
- ctx.collect(Tuple2.of("a", 0));
- ctx.collect(Tuple2.of("a", 1));
- ctx.collect(Tuple2.of("a", 2));
-
- ctx.collect(Tuple2.of("b", 3));
- ctx.collect(Tuple2.of("b", 4));
- ctx.collect(Tuple2.of("b", 5));
-
- ctx.collect(Tuple2.of("a", 6));
- ctx.collect(Tuple2.of("a", 7));
- ctx.collect(Tuple2.of("a", 8));
- }
-
- @Override
- public void cancel() {
- }
- }).extractTimestamp(new Tuple2TimestampExtractor());
-
- DataStream<Tuple2<String, Integer>> source2 = env.addSource(new SourceFunction<Tuple2<String, Integer>>() {
- private static final long serialVersionUID = 1L;
-
- @Override
- public void run(SourceContext<Tuple2<String, Integer>> ctx) throws Exception {
- ctx.collect(Tuple2.of("a", 0));
- ctx.collect(Tuple2.of("a", 1));
-
- ctx.collect(Tuple2.of("b", 3));
-
- ctx.collect(Tuple2.of("c", 6));
- ctx.collect(Tuple2.of("c", 7));
- ctx.collect(Tuple2.of("c", 8));
- }
-
- @Override
- public void cancel() {
- }
- }).extractTimestamp(new Tuple2TimestampExtractor());
-
-
- source1.coGroup(source2)
- .where(new Tuple2KeyExtractor())
- .equalTo(new Tuple2KeyExtractor())
- .window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
- .apply(new CoGroupFunction<Tuple2<String,Integer>, Tuple2<String,Integer>, String>() {
- @Override
- public void coGroup(Iterable<Tuple2<String, Integer>> first,
- Iterable<Tuple2<String, Integer>> second,
- Collector<String> out) throws Exception {
- StringBuilder result = new StringBuilder();
- result.append("F:");
- for (Tuple2<String, Integer> t: first) {
- result.append(t.toString());
- }
- result.append(" S:");
- for (Tuple2<String, Integer> t: second) {
- result.append(t.toString());
- }
- out.collect(result.toString());
- }
- })
- .addSink(new SinkFunction<String>() {
- @Override
- public void invoke(String value) throws Exception {
- testResults.add(value);
- }
- });
-
- env.execute("CoGroup Test");
-
- List<String> expectedResult = Lists.newArrayList(
- "F:(a,0)(a,1)(a,2) S:(a,0)(a,1)",
- "F:(b,3)(b,4)(b,5) S:(b,3)",
- "F:(a,6)(a,7)(a,8) S:",
- "F: S:(c,6)(c,7)(c,8)");
-
- Collections.sort(expectedResult);
- Collections.sort(testResults);
-
- Assert.assertEquals(expectedResult, testResults);
- }
-
- @Test
- public void testJoin() throws Exception {
-
- testResults = Lists.newArrayList();
-
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
- env.setParallelism(1);
-
- DataStream<Tuple3<String, String, Integer>> source1 = env.addSource(new SourceFunction<Tuple3<String, String, Integer>>() {
- private static final long serialVersionUID = 1L;
-
- @Override
- public void run(SourceContext<Tuple3<String, String, Integer>> ctx) throws Exception {
- ctx.collect(Tuple3.of("a", "x", 0));
- ctx.collect(Tuple3.of("a", "y", 1));
- ctx.collect(Tuple3.of("a", "z", 2));
-
- ctx.collect(Tuple3.of("b", "u", 3));
- ctx.collect(Tuple3.of("b", "w", 5));
-
- ctx.collect(Tuple3.of("a", "i", 6));
- ctx.collect(Tuple3.of("a", "j", 7));
- ctx.collect(Tuple3.of("a", "k", 8));
- }
-
- @Override
- public void cancel() {
- }
- }).extractTimestamp(new Tuple3TimestampExtractor());
-
- DataStream<Tuple3<String, String, Integer>> source2 = env.addSource(new SourceFunction<Tuple3<String, String, Integer>>() {
- private static final long serialVersionUID = 1L;
-
- @Override
- public void run(SourceContext<Tuple3<String, String, Integer>> ctx) throws Exception {
- ctx.collect(Tuple3.of("a", "u", 0));
- ctx.collect(Tuple3.of("a", "w", 1));
-
- ctx.collect(Tuple3.of("b", "i", 3));
- ctx.collect(Tuple3.of("b", "k", 5));
-
- ctx.collect(Tuple3.of("a", "x", 6));
- ctx.collect(Tuple3.of("a", "z", 8));
- }
-
- @Override
- public void cancel() {
- }
- }).extractTimestamp(new Tuple3TimestampExtractor());
-
-
- source1.join(source2)
- .where(new Tuple3KeyExtractor())
- .equalTo(new Tuple3KeyExtractor())
- .window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
- .apply(new JoinFunction<Tuple3<String, String, Integer>, Tuple3<String, String, Integer>, String>() {
- @Override
- public String join(Tuple3<String, String, Integer> first, Tuple3<String, String, Integer> second) throws Exception {
- return first + ":" + second;
- }
- })
- .addSink(new SinkFunction<String>() {
- @Override
- public void invoke(String value) throws Exception {
- testResults.add(value);
- }
- });
-
- env.execute("Join Test");
-
- List<String> expectedResult = Lists.newArrayList(
- "(a,x,0):(a,u,0)",
- "(a,x,0):(a,w,1)",
- "(a,y,1):(a,u,0)",
- "(a,y,1):(a,w,1)",
- "(a,z,2):(a,u,0)",
- "(a,z,2):(a,w,1)",
- "(b,u,3):(b,i,3)",
- "(b,u,3):(b,k,5)",
- "(b,w,5):(b,i,3)",
- "(b,w,5):(b,k,5)",
- "(a,i,6):(a,x,6)",
- "(a,i,6):(a,z,8)",
- "(a,j,7):(a,x,6)",
- "(a,j,7):(a,z,8)",
- "(a,k,8):(a,x,6)",
- "(a,k,8):(a,z,8)");
-
- Collections.sort(expectedResult);
- Collections.sort(testResults);
-
- Assert.assertEquals(expectedResult, testResults);
- }
-
- @Test
- public void testSelfJoin() throws Exception {
-
- testResults = Lists.newArrayList();
-
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
- env.setParallelism(1);
-
- DataStream<Tuple3<String, String, Integer>> source1 = env.addSource(new SourceFunction<Tuple3<String, String, Integer>>() {
- private static final long serialVersionUID = 1L;
-
- @Override
- public void run(SourceContext<Tuple3<String, String, Integer>> ctx) throws Exception {
- ctx.collect(Tuple3.of("a", "x", 0));
- ctx.collect(Tuple3.of("a", "y", 1));
- ctx.collect(Tuple3.of("a", "z", 2));
-
- ctx.collect(Tuple3.of("b", "u", 3));
- ctx.collect(Tuple3.of("b", "w", 5));
-
- ctx.collect(Tuple3.of("a", "i", 6));
- ctx.collect(Tuple3.of("a", "j", 7));
- ctx.collect(Tuple3.of("a", "k", 8));
- }
-
- @Override
- public void cancel() {
- }
- }).extractTimestamp(new Tuple3TimestampExtractor());
-
- source1.join(source1)
- .where(new Tuple3KeyExtractor())
- .equalTo(new Tuple3KeyExtractor())
- .window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
- .apply(new JoinFunction<Tuple3<String, String, Integer>, Tuple3<String, String, Integer>, String>() {
- @Override
- public String join(Tuple3<String, String, Integer> first, Tuple3<String, String, Integer> second) throws Exception {
- return first + ":" + second;
- }
- })
- .addSink(new SinkFunction<String>() {
- @Override
- public void invoke(String value) throws Exception {
- testResults.add(value);
- }
- });
-
- env.execute("Self-Join Test");
-
- List<String> expectedResult = Lists.newArrayList(
- "(a,x,0):(a,x,0)",
- "(a,x,0):(a,y,1)",
- "(a,x,0):(a,z,2)",
- "(a,y,1):(a,x,0)",
- "(a,y,1):(a,y,1)",
- "(a,y,1):(a,z,2)",
- "(a,z,2):(a,x,0)",
- "(a,z,2):(a,y,1)",
- "(a,z,2):(a,z,2)",
- "(b,u,3):(b,u,3)",
- "(b,u,3):(b,w,5)",
- "(b,w,5):(b,u,3)",
- "(b,w,5):(b,w,5)",
- "(a,i,6):(a,i,6)",
- "(a,i,6):(a,j,7)",
- "(a,i,6):(a,k,8)",
- "(a,j,7):(a,i,6)",
- "(a,j,7):(a,j,7)",
- "(a,j,7):(a,k,8)",
- "(a,k,8):(a,i,6)",
- "(a,k,8):(a,j,7)",
- "(a,k,8):(a,k,8)");
-
- Collections.sort(expectedResult);
- Collections.sort(testResults);
-
- Assert.assertEquals(expectedResult, testResults);
- }
-
- private static class Tuple2TimestampExtractor implements TimestampExtractor<Tuple2<String, Integer>> {
- private static final long serialVersionUID = 1L;
-
- @Override
- public long extractTimestamp(Tuple2<String, Integer> element, long currentTimestamp) {
- return element.f1;
- }
-
- @Override
- public long emitWatermark(Tuple2<String, Integer> element, long currentTimestamp) {
- return element.f1 - 1;
- }
-
- @Override
- public long getCurrentWatermark() {
- return Long.MIN_VALUE;
- }
- }
-
- private static class Tuple3TimestampExtractor implements TimestampExtractor<Tuple3<String, String, Integer>> {
- private static final long serialVersionUID = 1L;
-
- @Override
- public long extractTimestamp(Tuple3<String, String, Integer> element, long currentTimestamp) {
- return element.f2;
- }
-
- @Override
- public long emitWatermark(Tuple3<String, String, Integer> element, long currentTimestamp) {
- return element.f2 - 1;
- }
-
- @Override
- public long getCurrentWatermark() {
- return Long.MIN_VALUE;
- }
- }
-
- private static class Tuple2KeyExtractor implements KeySelector<Tuple2<String,Integer>, String> {
- private static final long serialVersionUID = 1L;
-
- @Override
- public String getKey(Tuple2<String, Integer> value) throws Exception {
- return value.f0;
- }
- }
-
- private static class Tuple3KeyExtractor implements KeySelector<Tuple3<String, String, Integer>, String> {
- private static final long serialVersionUID = 1L;
-
- @Override
- public String getKey(Tuple3<String, String, Integer> value) throws Exception {
- return value.f0;
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/ce792b11/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/CoGroupJoinITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/CoGroupJoinITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/CoGroupJoinITCase.java
new file mode 100644
index 0000000..bb79e5e
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/CoGroupJoinITCase.java
@@ -0,0 +1,373 @@
+/*
+* 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.runtime.operators.windowing;
+
+import com.google.common.collect.Lists;
+import org.apache.flink.api.common.functions.CoGroupFunction;
+import org.apache.flink.api.common.functions.JoinFunction;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.TimestampExtractor;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
+import org.apache.flink.util.Collector;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+public class CoGroupJoinITCase extends StreamingMultipleProgramsTestBase {
+
+ private static List<String> testResults;
+
+ @Test
+ public void testCoGroup() throws Exception {
+
+ testResults = Lists.newArrayList();
+
+ StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+ env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+ env.setParallelism(1);
+
+ DataStream<Tuple2<String, Integer>> source1 = env.addSource(new SourceFunction<Tuple2<String, Integer>>() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public void run(SourceContext<Tuple2<String, Integer>> ctx) throws Exception {
+ ctx.collect(Tuple2.of("a", 0));
+ ctx.collect(Tuple2.of("a", 1));
+ ctx.collect(Tuple2.of("a", 2));
+
+ ctx.collect(Tuple2.of("b", 3));
+ ctx.collect(Tuple2.of("b", 4));
+ ctx.collect(Tuple2.of("b", 5));
+
+ ctx.collect(Tuple2.of("a", 6));
+ ctx.collect(Tuple2.of("a", 7));
+ ctx.collect(Tuple2.of("a", 8));
+ }
+
+ @Override
+ public void cancel() {
+ }
+ }).extractTimestamp(new Tuple2TimestampExtractor());
+
+ DataStream<Tuple2<String, Integer>> source2 = env.addSource(new SourceFunction<Tuple2<String, Integer>>() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public void run(SourceContext<Tuple2<String, Integer>> ctx) throws Exception {
+ ctx.collect(Tuple2.of("a", 0));
+ ctx.collect(Tuple2.of("a", 1));
+
+ ctx.collect(Tuple2.of("b", 3));
+
+ ctx.collect(Tuple2.of("c", 6));
+ ctx.collect(Tuple2.of("c", 7));
+ ctx.collect(Tuple2.of("c", 8));
+ }
+
+ @Override
+ public void cancel() {
+ }
+ }).extractTimestamp(new Tuple2TimestampExtractor());
+
+
+ source1.coGroup(source2)
+ .where(new Tuple2KeyExtractor())
+ .equalTo(new Tuple2KeyExtractor())
+ .window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
+ .apply(new CoGroupFunction<Tuple2<String,Integer>, Tuple2<String,Integer>, String>() {
+ @Override
+ public void coGroup(Iterable<Tuple2<String, Integer>> first,
+ Iterable<Tuple2<String, Integer>> second,
+ Collector<String> out) throws Exception {
+ StringBuilder result = new StringBuilder();
+ result.append("F:");
+ for (Tuple2<String, Integer> t: first) {
+ result.append(t.toString());
+ }
+ result.append(" S:");
+ for (Tuple2<String, Integer> t: second) {
+ result.append(t.toString());
+ }
+ out.collect(result.toString());
+ }
+ })
+ .addSink(new SinkFunction<String>() {
+ @Override
+ public void invoke(String value) throws Exception {
+ testResults.add(value);
+ }
+ });
+
+ env.execute("CoGroup Test");
+
+ List<String> expectedResult = Lists.newArrayList(
+ "F:(a,0)(a,1)(a,2) S:(a,0)(a,1)",
+ "F:(b,3)(b,4)(b,5) S:(b,3)",
+ "F:(a,6)(a,7)(a,8) S:",
+ "F: S:(c,6)(c,7)(c,8)");
+
+ Collections.sort(expectedResult);
+ Collections.sort(testResults);
+
+ Assert.assertEquals(expectedResult, testResults);
+ }
+
+ @Test
+ public void testJoin() throws Exception {
+
+ testResults = Lists.newArrayList();
+
+ StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+ env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+ env.setParallelism(1);
+
+ DataStream<Tuple3<String, String, Integer>> source1 = env.addSource(new SourceFunction<Tuple3<String, String, Integer>>() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public void run(SourceContext<Tuple3<String, String, Integer>> ctx) throws Exception {
+ ctx.collect(Tuple3.of("a", "x", 0));
+ ctx.collect(Tuple3.of("a", "y", 1));
+ ctx.collect(Tuple3.of("a", "z", 2));
+
+ ctx.collect(Tuple3.of("b", "u", 3));
+ ctx.collect(Tuple3.of("b", "w", 5));
+
+ ctx.collect(Tuple3.of("a", "i", 6));
+ ctx.collect(Tuple3.of("a", "j", 7));
+ ctx.collect(Tuple3.of("a", "k", 8));
+ }
+
+ @Override
+ public void cancel() {
+ }
+ }).extractTimestamp(new Tuple3TimestampExtractor());
+
+ DataStream<Tuple3<String, String, Integer>> source2 = env.addSource(new SourceFunction<Tuple3<String, String, Integer>>() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public void run(SourceContext<Tuple3<String, String, Integer>> ctx) throws Exception {
+ ctx.collect(Tuple3.of("a", "u", 0));
+ ctx.collect(Tuple3.of("a", "w", 1));
+
+ ctx.collect(Tuple3.of("b", "i", 3));
+ ctx.collect(Tuple3.of("b", "k", 5));
+
+ ctx.collect(Tuple3.of("a", "x", 6));
+ ctx.collect(Tuple3.of("a", "z", 8));
+ }
+
+ @Override
+ public void cancel() {
+ }
+ }).extractTimestamp(new Tuple3TimestampExtractor());
+
+
+ source1.join(source2)
+ .where(new Tuple3KeyExtractor())
+ .equalTo(new Tuple3KeyExtractor())
+ .window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
+ .apply(new JoinFunction<Tuple3<String, String, Integer>, Tuple3<String, String, Integer>, String>() {
+ @Override
+ public String join(Tuple3<String, String, Integer> first, Tuple3<String, String, Integer> second) throws Exception {
+ return first + ":" + second;
+ }
+ })
+ .addSink(new SinkFunction<String>() {
+ @Override
+ public void invoke(String value) throws Exception {
+ testResults.add(value);
+ }
+ });
+
+ env.execute("Join Test");
+
+ List<String> expectedResult = Lists.newArrayList(
+ "(a,x,0):(a,u,0)",
+ "(a,x,0):(a,w,1)",
+ "(a,y,1):(a,u,0)",
+ "(a,y,1):(a,w,1)",
+ "(a,z,2):(a,u,0)",
+ "(a,z,2):(a,w,1)",
+ "(b,u,3):(b,i,3)",
+ "(b,u,3):(b,k,5)",
+ "(b,w,5):(b,i,3)",
+ "(b,w,5):(b,k,5)",
+ "(a,i,6):(a,x,6)",
+ "(a,i,6):(a,z,8)",
+ "(a,j,7):(a,x,6)",
+ "(a,j,7):(a,z,8)",
+ "(a,k,8):(a,x,6)",
+ "(a,k,8):(a,z,8)");
+
+ Collections.sort(expectedResult);
+ Collections.sort(testResults);
+
+ Assert.assertEquals(expectedResult, testResults);
+ }
+
+ @Test
+ public void testSelfJoin() throws Exception {
+
+ testResults = Lists.newArrayList();
+
+ StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+ env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+ env.setParallelism(1);
+
+ DataStream<Tuple3<String, String, Integer>> source1 = env.addSource(new SourceFunction<Tuple3<String, String, Integer>>() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public void run(SourceContext<Tuple3<String, String, Integer>> ctx) throws Exception {
+ ctx.collect(Tuple3.of("a", "x", 0));
+ ctx.collect(Tuple3.of("a", "y", 1));
+ ctx.collect(Tuple3.of("a", "z", 2));
+
+ ctx.collect(Tuple3.of("b", "u", 3));
+ ctx.collect(Tuple3.of("b", "w", 5));
+
+ ctx.collect(Tuple3.of("a", "i", 6));
+ ctx.collect(Tuple3.of("a", "j", 7));
+ ctx.collect(Tuple3.of("a", "k", 8));
+ }
+
+ @Override
+ public void cancel() {
+ }
+ }).extractTimestamp(new Tuple3TimestampExtractor());
+
+ source1.join(source1)
+ .where(new Tuple3KeyExtractor())
+ .equalTo(new Tuple3KeyExtractor())
+ .window(TumblingTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS)))
+ .apply(new JoinFunction<Tuple3<String, String, Integer>, Tuple3<String, String, Integer>, String>() {
+ @Override
+ public String join(Tuple3<String, String, Integer> first, Tuple3<String, String, Integer> second) throws Exception {
+ return first + ":" + second;
+ }
+ })
+ .addSink(new SinkFunction<String>() {
+ @Override
+ public void invoke(String value) throws Exception {
+ testResults.add(value);
+ }
+ });
+
+ env.execute("Self-Join Test");
+
+ List<String> expectedResult = Lists.newArrayList(
+ "(a,x,0):(a,x,0)",
+ "(a,x,0):(a,y,1)",
+ "(a,x,0):(a,z,2)",
+ "(a,y,1):(a,x,0)",
+ "(a,y,1):(a,y,1)",
+ "(a,y,1):(a,z,2)",
+ "(a,z,2):(a,x,0)",
+ "(a,z,2):(a,y,1)",
+ "(a,z,2):(a,z,2)",
+ "(b,u,3):(b,u,3)",
+ "(b,u,3):(b,w,5)",
+ "(b,w,5):(b,u,3)",
+ "(b,w,5):(b,w,5)",
+ "(a,i,6):(a,i,6)",
+ "(a,i,6):(a,j,7)",
+ "(a,i,6):(a,k,8)",
+ "(a,j,7):(a,i,6)",
+ "(a,j,7):(a,j,7)",
+ "(a,j,7):(a,k,8)",
+ "(a,k,8):(a,i,6)",
+ "(a,k,8):(a,j,7)",
+ "(a,k,8):(a,k,8)");
+
+ Collections.sort(expectedResult);
+ Collections.sort(testResults);
+
+ Assert.assertEquals(expectedResult, testResults);
+ }
+
+ private static class Tuple2TimestampExtractor implements TimestampExtractor<Tuple2<String, Integer>> {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public long extractTimestamp(Tuple2<String, Integer> element, long currentTimestamp) {
+ return element.f1;
+ }
+
+ @Override
+ public long emitWatermark(Tuple2<String, Integer> element, long currentTimestamp) {
+ return element.f1 - 1;
+ }
+
+ @Override
+ public long getCurrentWatermark() {
+ return Long.MIN_VALUE;
+ }
+ }
+
+ private static class Tuple3TimestampExtractor implements TimestampExtractor<Tuple3<String, String, Integer>> {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public long extractTimestamp(Tuple3<String, String, Integer> element, long currentTimestamp) {
+ return element.f2;
+ }
+
+ @Override
+ public long emitWatermark(Tuple3<String, String, Integer> element, long currentTimestamp) {
+ return element.f2 - 1;
+ }
+
+ @Override
+ public long getCurrentWatermark() {
+ return Long.MIN_VALUE;
+ }
+ }
+
+ private static class Tuple2KeyExtractor implements KeySelector<Tuple2<String,Integer>, String> {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public String getKey(Tuple2<String, Integer> value) throws Exception {
+ return value.f0;
+ }
+ }
+
+ private static class Tuple3KeyExtractor implements KeySelector<Tuple3<String, String, Integer>, String> {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public String getKey(Tuple3<String, String, Integer> value) throws Exception {
+ return value.f0;
+ }
+ }
+
+}
[3/8] flink git commit: [FLINK-2550] Rework interplay of Window
Assigners and TimeCharacteristic
Posted by al...@apache.org.
[FLINK-2550] Rework interplay of Window Assigners and TimeCharacteristic
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/ff367d6e
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/ff367d6e
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/ff367d6e
Branch: refs/heads/master
Commit: ff367d6ea728a7c5bc334f34591a4d79e573972f
Parents: 28a38bb
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Tue Oct 6 18:19:56 2015 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Wed Oct 7 22:08:25 2015 +0200
----------------------------------------------------------------------
.../api/datastream/AllWindowedStream.java | 17 +--
.../streaming/api/datastream/DataStream.java | 20 +---
.../streaming/api/datastream/KeyedStream.java | 20 +---
.../api/datastream/WindowedStream.java | 29 ++---
.../environment/StreamExecutionEnvironment.java | 12 +++
.../api/windowing/assigners/GlobalWindows.java | 3 +-
.../assigners/SlidingProcessingTimeWindows.java | 106 -------------------
.../windowing/assigners/SlidingTimeWindows.java | 11 +-
.../TumblingProcessingTimeWindows.java | 81 --------------
.../assigners/TumblingTimeWindows.java | 11 +-
.../api/windowing/assigners/WindowAssigner.java | 3 +-
.../operators/windowing/WindowOperator.java | 5 +
.../flink/streaming/api/CoGroupJoinITCase.java | 6 +-
.../windowing/AllWindowTranslationTest.java | 60 +++++++----
.../windowing/WindowTranslationTest.java | 89 +++++++++++++---
.../streaming/examples/join/WindowJoin.java | 3 +-
.../scala/examples/join/WindowJoin.scala | 3 +-
.../flink/streaming/api/scala/DataStream.scala | 37 +------
.../flink/streaming/api/scala/KeyedStream.scala | 37 +------
.../api/scala/StreamExecutionEnvironment.scala | 5 +
.../api/scala/AllWindowTranslationTest.scala | 24 +++--
.../streaming/api/scala/CoGroupJoinITCase.scala | 9 +-
.../api/scala/WindowTranslationTest.scala | 22 ++--
23 files changed, 233 insertions(+), 380 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
index 89c4857..a8d7654 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java
@@ -24,6 +24,7 @@ import org.apache.flink.api.common.functions.ReduceFunction;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.Utils;
import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.streaming.api.TimeCharacteristic;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction;
import org.apache.flink.streaming.api.functions.aggregation.ComparableAggregator;
@@ -80,7 +81,7 @@ public class AllWindowedStream<T, W extends Window> {
WindowAssigner<? super T, W> windowAssigner) {
this.input = input;
this.windowAssigner = windowAssigner;
- this.trigger = windowAssigner.getDefaultTrigger();
+ this.trigger = windowAssigner.getDefaultTrigger(input.getExecutionEnvironment());
}
/**
@@ -139,12 +140,14 @@ public class AllWindowedStream<T, W extends Window> {
OneInputStreamOperator<T, T> operator;
+ boolean setProcessingTime = input.getExecutionEnvironment().getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime;
+
if (evictor != null) {
operator = new EvictingNonKeyedWindowOperator<>(windowAssigner,
new HeapWindowBuffer.Factory<T>(),
new ReduceAllWindowFunction<W, T>(function),
trigger,
- evictor);
+ evictor).enableSetProcessingTime(setProcessingTime);
} else {
// we need to copy because we need our own instance of the pre aggregator
@@ -154,7 +157,7 @@ public class AllWindowedStream<T, W extends Window> {
operator = new NonKeyedWindowOperator<>(windowAssigner,
new PreAggregatingHeapWindowBuffer.Factory<>(functionCopy),
new ReduceAllWindowFunction<W, T>(function),
- trigger);
+ trigger).enableSetProcessingTime(setProcessingTime);
}
return input.transform(opName, input.getType(), operator).setParallelism(1);
@@ -205,20 +208,22 @@ public class AllWindowedStream<T, W extends Window> {
String opName = "TriggerWindow(" + windowAssigner + ", " + trigger + ", " + udfName + ")";
- OneInputStreamOperator<T, R> operator;
+ NonKeyedWindowOperator<T, R, W> operator;
+
+ boolean setProcessingTime = input.getExecutionEnvironment().getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime;
if (evictor != null) {
operator = new EvictingNonKeyedWindowOperator<>(windowAssigner,
new HeapWindowBuffer.Factory<T>(),
function,
trigger,
- evictor);
+ evictor).enableSetProcessingTime(setProcessingTime);
} else {
operator = new NonKeyedWindowOperator<>(windowAssigner,
new HeapWindowBuffer.Factory<T>(),
function,
- trigger);
+ trigger).enableSetProcessingTime(setProcessingTime);
}
return input.transform(opName, resultType, operator).setParallelism(1);
http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
index 0be1d56..ee8b3d2 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
@@ -59,9 +59,7 @@ import org.apache.flink.streaming.api.transformations.OneInputTransformation;
import org.apache.flink.streaming.api.transformations.PartitionTransformation;
import org.apache.flink.streaming.api.transformations.StreamTransformation;
import org.apache.flink.streaming.api.transformations.UnionTransformation;
-import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows;
import org.apache.flink.streaming.api.windowing.assigners.SlidingTimeWindows;
-import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows;
import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
import org.apache.flink.streaming.api.windowing.helper.Count;
@@ -72,7 +70,6 @@ import org.apache.flink.streaming.api.windowing.helper.WindowingHelper;
import org.apache.flink.streaming.api.windowing.policy.EvictionPolicy;
import org.apache.flink.streaming.api.windowing.policy.TriggerPolicy;
import org.apache.flink.streaming.api.windowing.time.AbstractTime;
-import org.apache.flink.streaming.api.windowing.time.EventTime;
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.ExtractTimestampsOperator;
@@ -726,13 +723,7 @@ public class DataStream<T> {
* @param size The size of the window.
*/
public AllWindowedStream<T, TimeWindow> timeWindowAll(AbstractTime size) {
- AbstractTime actualSize = size.makeSpecificBasedOnTimeCharacteristic(environment.getStreamTimeCharacteristic());
-
- if (actualSize instanceof EventTime) {
- return windowAll(TumblingTimeWindows.of(actualSize));
- } else {
- return windowAll(TumblingProcessingTimeWindows.of(actualSize));
- }
+ return windowAll(TumblingTimeWindows.of(size));
}
/**
@@ -747,14 +738,7 @@ public class DataStream<T> {
* @param size The size of the window.
*/
public AllWindowedStream<T, TimeWindow> timeWindowAll(AbstractTime size, AbstractTime slide) {
- AbstractTime actualSize = size.makeSpecificBasedOnTimeCharacteristic(environment.getStreamTimeCharacteristic());
- AbstractTime actualSlide = slide.makeSpecificBasedOnTimeCharacteristic(environment.getStreamTimeCharacteristic());
-
- if (actualSize instanceof EventTime) {
- return windowAll(SlidingTimeWindows.of(size, slide));
- } else {
- return windowAll(SlidingProcessingTimeWindows.of(actualSize, actualSlide));
- }
+ return windowAll(SlidingTimeWindows.of(size, slide));
}
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java
index edb7981..2e6d7d6 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java
@@ -32,13 +32,10 @@ import org.apache.flink.streaming.api.operators.StreamGroupedFold;
import org.apache.flink.streaming.api.operators.StreamGroupedReduce;
import org.apache.flink.streaming.api.transformations.OneInputTransformation;
import org.apache.flink.streaming.api.transformations.PartitionTransformation;
-import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows;
import org.apache.flink.streaming.api.windowing.assigners.SlidingTimeWindows;
-import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows;
import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
import org.apache.flink.streaming.api.windowing.time.AbstractTime;
-import org.apache.flink.streaming.api.windowing.time.EventTime;
import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
import org.apache.flink.streaming.api.windowing.windows.Window;
import org.apache.flink.streaming.runtime.partitioner.HashPartitioner;
@@ -122,13 +119,7 @@ public class KeyedStream<T, KEY> extends DataStream<T> {
* @param size The size of the window.
*/
public WindowedStream<T, KEY, TimeWindow> timeWindow(AbstractTime size) {
- AbstractTime actualSize = size.makeSpecificBasedOnTimeCharacteristic(environment.getStreamTimeCharacteristic());
-
- if (actualSize instanceof EventTime) {
- return window(TumblingTimeWindows.of(actualSize));
- } else {
- return window(TumblingProcessingTimeWindows.of(actualSize));
- }
+ return window(TumblingTimeWindows.of(size));
}
/**
@@ -143,14 +134,7 @@ public class KeyedStream<T, KEY> extends DataStream<T> {
* @param size The size of the window.
*/
public WindowedStream<T, KEY, TimeWindow> timeWindow(AbstractTime size, AbstractTime slide) {
- AbstractTime actualSize = size.makeSpecificBasedOnTimeCharacteristic(environment.getStreamTimeCharacteristic());
- AbstractTime actualSlide = slide.makeSpecificBasedOnTimeCharacteristic(environment.getStreamTimeCharacteristic());
-
- if (actualSize instanceof EventTime) {
- return window(SlidingTimeWindows.of(actualSize, actualSlide));
- } else {
- return window(SlidingProcessingTimeWindows.of(actualSize, actualSlide));
- }
+ return window(SlidingTimeWindows.of(size, slide));
}
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
index 1273b42..99f7d06 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
@@ -25,6 +25,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.Utils;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.streaming.api.TimeCharacteristic;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction;
import org.apache.flink.streaming.api.functions.aggregation.ComparableAggregator;
@@ -32,8 +33,8 @@ import org.apache.flink.streaming.api.functions.aggregation.SumAggregator;
import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
import org.apache.flink.streaming.api.functions.windowing.ReduceWindowFunction;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows;
-import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
+import org.apache.flink.streaming.api.windowing.assigners.SlidingTimeWindows;
+import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows;
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.triggers.ProcessingTimeTrigger;
@@ -91,7 +92,7 @@ public class WindowedStream<T, K, W extends Window> {
WindowAssigner<? super T, W> windowAssigner) {
this.input = input;
this.windowAssigner = windowAssigner;
- this.trigger = windowAssigner.getDefaultTrigger();
+ this.trigger = windowAssigner.getDefaultTrigger(input.getExecutionEnvironment());
}
/**
@@ -151,13 +152,15 @@ public class WindowedStream<T, K, W extends Window> {
OneInputStreamOperator<T, T> operator;
+ boolean setProcessingTime = input.getExecutionEnvironment().getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime;
+
if (evictor != null) {
operator = new EvictingWindowOperator<>(windowAssigner,
keySel,
new HeapWindowBuffer.Factory<T>(),
new ReduceWindowFunction<K, W, T>(function),
trigger,
- evictor);
+ evictor).enableSetProcessingTime(setProcessingTime);
} else {
// we need to copy because we need our own instance of the pre aggregator
@@ -168,7 +171,7 @@ public class WindowedStream<T, K, W extends Window> {
keySel,
new PreAggregatingHeapWindowBuffer.Factory<>(functionCopy),
new ReduceWindowFunction<K, W, T>(function),
- trigger);
+ trigger).enableSetProcessingTime(setProcessingTime);
}
return input.transform(opName, input.getType(), operator);
@@ -222,7 +225,9 @@ public class WindowedStream<T, K, W extends Window> {
String opName = "TriggerWindow(" + windowAssigner + ", " + trigger + ", " + udfName + ")";
KeySelector<T, K> keySel = input.getKeySelector();
- OneInputStreamOperator<T, R> operator;
+ WindowOperator<K, T, R, W> operator;
+
+ boolean setProcessingTime = input.getExecutionEnvironment().getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime;
if (evictor != null) {
operator = new EvictingWindowOperator<>(windowAssigner,
@@ -230,14 +235,14 @@ public class WindowedStream<T, K, W extends Window> {
new HeapWindowBuffer.Factory<T>(),
function,
trigger,
- evictor);
+ evictor).enableSetProcessingTime(setProcessingTime);
} else {
operator = new WindowOperator<>(windowAssigner,
keySel,
new HeapWindowBuffer.Factory<T>(),
function,
- trigger);
+ trigger).enableSetProcessingTime(setProcessingTime);;
}
return input.transform(opName, resultType, operator);
@@ -450,8 +455,8 @@ public class WindowedStream<T, K, W extends Window> {
TypeInformation<R> resultType,
String functionName) {
- if (windowAssigner instanceof SlidingProcessingTimeWindows && trigger instanceof ProcessingTimeTrigger && evictor == null) {
- SlidingProcessingTimeWindows timeWindows = (SlidingProcessingTimeWindows) windowAssigner;
+ if (windowAssigner instanceof SlidingTimeWindows && trigger instanceof ProcessingTimeTrigger && evictor == null) {
+ SlidingTimeWindows timeWindows = (SlidingTimeWindows) windowAssigner;
final long windowLength = timeWindows.getSize();
final long windowSlide = timeWindows.getSlide();
@@ -475,8 +480,8 @@ public class WindowedStream<T, K, W extends Window> {
wf, input.getKeySelector(), windowLength, windowSlide);
return input.transform(opName, resultType, op);
}
- } else if (windowAssigner instanceof TumblingProcessingTimeWindows && trigger instanceof ProcessingTimeTrigger && evictor == null) {
- TumblingProcessingTimeWindows timeWindows = (TumblingProcessingTimeWindows) windowAssigner;
+ } else if (windowAssigner instanceof TumblingTimeWindows && trigger instanceof ProcessingTimeTrigger && evictor == null) {
+ TumblingTimeWindows timeWindows = (TumblingTimeWindows) windowAssigner;
final long windowLength = timeWindows.getSize();
final long windowSlide = timeWindows.getSize();
http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
index c2e2880..cc96217 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
@@ -541,11 +541,23 @@ public abstract class StreamExecutionEnvironment {
/**
* Sets the time characteristic for all streams create from this environment, e.g., processing
* time, event time, or ingestion time.
+ *
+ * <p>
+ * If you set the characteristic to IngestionTime of EventTime this will set a default
+ * watermark update interval of 200 ms. If this is not applicable for your application
+ * you should change it using {@link ExecutionConfig#setAutoWatermarkInterval(long)}.
*
* @param characteristic The time characteristic.
*/
public void setStreamTimeCharacteristic(TimeCharacteristic characteristic) {
this.timeCharacteristic = Objects.requireNonNull(characteristic);
+ if (characteristic == TimeCharacteristic.ProcessingTime) {
+ getConfig().disableTimestamps();
+ getConfig().setAutoWatermarkInterval(0);
+ } else {
+ getConfig().enableTimestamps();
+ getConfig().setAutoWatermarkInterval(200);
+ }
}
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/GlobalWindows.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/GlobalWindows.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/GlobalWindows.java
index 52c8f55..dbeb5ce 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/GlobalWindows.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/GlobalWindows.java
@@ -17,6 +17,7 @@
*/
package org.apache.flink.streaming.api.windowing.assigners;
+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.GlobalWindow;
@@ -42,7 +43,7 @@ public class GlobalWindows extends WindowAssigner<Object, GlobalWindow> {
}
@Override
- public Trigger<Object, GlobalWindow> getDefaultTrigger() {
+ public Trigger<Object, GlobalWindow> getDefaultTrigger(StreamExecutionEnvironment env) {
return null;
}
http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingProcessingTimeWindows.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingProcessingTimeWindows.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingProcessingTimeWindows.java
deleted file mode 100644
index 65d7641..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingProcessingTimeWindows.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/**
- * 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 com.google.common.collect.Lists;
-import org.apache.flink.streaming.api.windowing.time.AbstractTime;
-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 java.io.IOException;
-import java.io.ObjectInputStream;
-import java.util.Collection;
-import java.util.List;
-
-/**
- * A {@link WindowAssigner} that windows elements into sliding, time-based windows. The windowing
- * is based on system time. Windows can possibly overlap.
- *
- * <p>
- * For example, in order to window into windows of 1 minute, every 10 seconds:
- * <pre> {@code
- * DataStream<Tuple2<String, Integer>> in = ...;
- * KeyedStream<String, Tuple2<String, Integer>> keyed = in.keyBy(...);
- * WindowedStream<Tuple2<String, Integer>, String, TimeWindows> windowed =
- * keyed.window(SlidingProcessingTimeWindows.of(Time.of(1, MINUTES), Time.of(10, SECONDS));
- * } </pre>
- */
-public class SlidingProcessingTimeWindows extends WindowAssigner<Object, TimeWindow> {
- private static final long serialVersionUID = 1L;
-
- private final long size;
-
- private final long slide;
-
- private transient List<TimeWindow> result;
-
- private SlidingProcessingTimeWindows(long size, long slide) {
- this.size = size;
- this.slide = slide;
- this.result = Lists.newArrayListWithCapacity((int) (size / slide));
- }
-
- private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
- in.defaultReadObject();
- this.result = Lists.newArrayListWithCapacity((int) (size / slide));
- }
-
- @Override
- public Collection<TimeWindow> assignWindows(Object element, long timestamp) {
- result.clear();
- long time = System.currentTimeMillis();
- long lastStart = time - time % slide;
- for (long start = lastStart;
- start > time - size;
- start -= slide) {
- result.add(new TimeWindow(start, size));
- }
- return result;
- }
-
- public long getSize() {
- return size;
- }
-
- public long getSlide() {
- return slide;
- }
-
- @Override
- public Trigger<Object, TimeWindow> getDefaultTrigger() {
- return ProcessingTimeTrigger.create();
- }
-
- @Override
- public String toString() {
- return "SlidingProcessingTimeWindows(" + size + ", " + slide + ")";
- }
-
- /**
- * Creates a new {@code SlidingProcessingTimeWindows} {@link WindowAssigner} that assigns
- * elements to sliding time windows based on the current processing time.
- *
- * @param size The size of the generated windows.
- * @param slide The slide interval of the generated windows.
- * @return The time policy.
- */
- public static SlidingProcessingTimeWindows of(AbstractTime size, AbstractTime slide) {
- return new SlidingProcessingTimeWindows(size.toMilliseconds(), slide.toMilliseconds());
- }
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingTimeWindows.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingTimeWindows.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingTimeWindows.java
index 52ae356..6036dfb 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingTimeWindows.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/SlidingTimeWindows.java
@@ -17,7 +17,10 @@
*/
package org.apache.flink.streaming.api.windowing.assigners;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.windowing.time.AbstractTime;
+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.triggers.WatermarkTrigger;
import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
@@ -72,8 +75,12 @@ public class SlidingTimeWindows extends WindowAssigner<Object, TimeWindow> {
}
@Override
- public Trigger<Object, TimeWindow> getDefaultTrigger() {
- return WatermarkTrigger.create();
+ public Trigger<Object, TimeWindow> getDefaultTrigger(StreamExecutionEnvironment env) {
+ if (env.getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime) {
+ return ProcessingTimeTrigger.create();
+ } else {
+ return WatermarkTrigger.create();
+ }
}
@Override
http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingProcessingTimeWindows.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingProcessingTimeWindows.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingProcessingTimeWindows.java
deleted file mode 100644
index 41f6362..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingProcessingTimeWindows.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/**
- * 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.AbstractTime;
-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 java.util.Collection;
-import java.util.Collections;
-
-/**
- * A {@link WindowAssigner} that windows elements into time-based windows. The windowing is
- * based on system time. Windows cannot overlap.
- *
- * <p>
- * For example, in order to window into windows of 1 minute, every 10 seconds:
- * <pre> {@code
- * DataStream<Tuple2<String, Integer>> in = ...;
- * KeyedStream<String, Tuple2<String, Integer>> keyed = in.keyBy(...);
- * WindowedStream<Tuple2<String, Integer>, String, TimeWindows> windowed =
- * keyed.window(TumblingProcessingTimeWindows.of(Time.of(1, MINUTES), Time.of(10, SECONDS));
- * } </pre>
- */
-public class TumblingProcessingTimeWindows extends WindowAssigner<Object, TimeWindow> {
- private static final long serialVersionUID = 1L;
-
- private long size;
-
- private TumblingProcessingTimeWindows(long size) {
- this.size = size;
- }
-
- @Override
- public Collection<TimeWindow> assignWindows(Object element, long timestamp) {
- long time = System.currentTimeMillis();
- long start = time - (time % size);
- return Collections.singletonList(new TimeWindow(start, size));
- }
-
- public long getSize() {
- return size;
- }
-
- @Override
- public Trigger<Object, TimeWindow> getDefaultTrigger() {
- return ProcessingTimeTrigger.create();
- }
-
- @Override
- public String toString() {
- return "TumblingProcessingTimeWindows(" + size + ")";
- }
-
- /**
- * Creates a new {@code TumblingProcessingTimeWindows} {@link WindowAssigner} that assigns
- * elements to time windows based on the current processing time.
- *
- * @param size The size of the generated windows.
- * @return The time policy.
- */
- public static TumblingProcessingTimeWindows of(AbstractTime size) {
- return new TumblingProcessingTimeWindows(size.toMilliseconds());
- }
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingTimeWindows.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingTimeWindows.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingTimeWindows.java
index b6022b3..d57dc33 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingTimeWindows.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/TumblingTimeWindows.java
@@ -17,7 +17,10 @@
*/
package org.apache.flink.streaming.api.windowing.assigners;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.windowing.time.AbstractTime;
+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.triggers.WatermarkTrigger;
import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
@@ -58,8 +61,12 @@ public class TumblingTimeWindows extends WindowAssigner<Object, TimeWindow> {
}
@Override
- public Trigger<Object, TimeWindow> getDefaultTrigger() {
- return WatermarkTrigger.create();
+ public Trigger<Object, TimeWindow> getDefaultTrigger(StreamExecutionEnvironment env) {
+ if (env.getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime) {
+ return ProcessingTimeTrigger.create();
+ } else {
+ return WatermarkTrigger.create();
+ }
}
@Override
http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/WindowAssigner.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/WindowAssigner.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/WindowAssigner.java
index 20fe365..d0b1ed0 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/WindowAssigner.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/assigners/WindowAssigner.java
@@ -17,6 +17,7 @@
*/
package org.apache.flink.streaming.api.windowing.assigners;
+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.Window;
import java.io.Serializable;
@@ -50,5 +51,5 @@ public abstract class WindowAssigner<T, W extends Window> implements Serializabl
/**
* Returns the default trigger associated with this {@code WindowAssigner}.
*/
- public abstract Trigger<T, W> getDefaultTrigger();
+ public abstract Trigger<T, W> getDefaultTrigger(StreamExecutionEnvironment env);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
index 548afb3..368b8fa 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
@@ -347,6 +347,11 @@ public class WindowOperator<K, IN, OUT, W extends Window>
// ------------------------------------------------------------------------
@VisibleForTesting
+ public boolean isSetProcessingTime() {
+ return setProcessingTime;
+ }
+
+ @VisibleForTesting
public Trigger<? super IN, ? super W> getTriggerTemplate() {
return triggerTemplate;
}
http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoGroupJoinITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoGroupJoinITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoGroupJoinITCase.java
index c06a608..9ddd6eb 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoGroupJoinITCase.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/CoGroupJoinITCase.java
@@ -49,8 +49,8 @@ public class CoGroupJoinITCase extends StreamingMultipleProgramsTestBase {
testResults = Lists.newArrayList();
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+ env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
env.setParallelism(1);
- env.getConfig().enableTimestamps();
DataStream<Tuple2<String, Integer>> source1 = env.addSource(new SourceFunction<Tuple2<String, Integer>>() {
private static final long serialVersionUID = 1L;
@@ -144,8 +144,8 @@ public class CoGroupJoinITCase extends StreamingMultipleProgramsTestBase {
testResults = Lists.newArrayList();
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+ env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
env.setParallelism(1);
- env.getConfig().enableTimestamps();
DataStream<Tuple3<String, String, Integer>> source1 = env.addSource(new SourceFunction<Tuple3<String, String, Integer>>() {
private static final long serialVersionUID = 1L;
@@ -239,8 +239,8 @@ public class CoGroupJoinITCase extends StreamingMultipleProgramsTestBase {
testResults = Lists.newArrayList();
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+ env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
env.setParallelism(1);
- env.getConfig().enableTimestamps();
DataStream<Tuple3<String, String, Integer>> source1 = env.addSource(new SourceFunction<Tuple3<String, String, Integer>>() {
private static final long serialVersionUID = 1L;
http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java
index 09a7149..4fa16ac 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AllWindowTranslationTest.java
@@ -19,25 +19,25 @@ package org.apache.flink.streaming.runtime.operators.windowing;
import org.apache.flink.api.common.functions.RichReduceFunction;
import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.api.TimeCharacteristic;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.transformations.OneInputTransformation;
-import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows;
-import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
+import org.apache.flink.streaming.api.windowing.assigners.SlidingTimeWindows;
+import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows;
import org.apache.flink.streaming.api.windowing.evictors.CountEvictor;
import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor;
import org.apache.flink.streaming.api.windowing.time.Time;
import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
-import org.apache.flink.streaming.api.windowing.triggers.ProcessingTimeTrigger;
+import org.apache.flink.streaming.api.windowing.triggers.WatermarkTrigger;
import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
import org.apache.flink.streaming.runtime.operators.windowing.buffers.HeapWindowBuffer;
import org.apache.flink.streaming.runtime.operators.windowing.buffers.PreAggregatingHeapWindowBuffer;
import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
import org.apache.flink.util.Collector;
import org.junit.Assert;
-import org.junit.Ignore;
import org.junit.Test;
import java.util.concurrent.TimeUnit;
@@ -50,30 +50,33 @@ import java.util.concurrent.TimeUnit;
public class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
/**
- * These tests ensure that the fast aligned time windows operator is used if the
- * conditions are right.
- *
- * TODO: update once fast aligned time windows operator is in
+ * These tests ensure that the correct trigger is set when using event-time windows.
*/
- @Ignore
@Test
- public void testFastTimeWindows() throws Exception {
+ @SuppressWarnings("rawtypes")
+ public void testEventTime() throws Exception {
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
+ env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime);
DummyReducer reducer = new DummyReducer();
DataStream<Tuple2<String, Integer>> window1 = source
- .windowAll(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
+ .windowAll(SlidingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
.reduce(reducer);
OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
- Assert.assertTrue(operator1 instanceof AggregatingProcessingTimeWindowOperator);
+ Assert.assertTrue(operator1 instanceof NonKeyedWindowOperator);
+ NonKeyedWindowOperator winOperator1 = (NonKeyedWindowOperator) operator1;
+ Assert.assertFalse(winOperator1.isSetProcessingTime());
+ Assert.assertTrue(winOperator1.getTriggerTemplate() instanceof WatermarkTrigger);
+ Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingTimeWindows);
+ Assert.assertTrue(winOperator1.getWindowBufferFactory() instanceof PreAggregatingHeapWindowBuffer.Factory);
DataStream<Tuple2<String, Integer>> window2 = source
- .windowAll(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
+ .windowAll(TumblingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
.apply(new AllWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow>() {
private static final long serialVersionUID = 1L;
@@ -88,20 +91,26 @@ public class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase
OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform2 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window2.getTransformation();
OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator2 = transform2.getOperator();
- Assert.assertTrue(operator2 instanceof AccumulatingProcessingTimeWindowOperator);
+ Assert.assertTrue(operator2 instanceof NonKeyedWindowOperator);
+ NonKeyedWindowOperator winOperator2 = (NonKeyedWindowOperator) operator2;
+ Assert.assertFalse(winOperator2.isSetProcessingTime());
+ Assert.assertTrue(winOperator2.getTriggerTemplate() instanceof WatermarkTrigger);
+ Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingTimeWindows);
+ Assert.assertTrue(winOperator2.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
}
@Test
@SuppressWarnings("rawtypes")
public void testNonEvicting() throws Exception {
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+ env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime);
DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
DummyReducer reducer = new DummyReducer();
DataStream<Tuple2<String, Integer>> window1 = source
- .windowAll(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
+ .windowAll(SlidingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
.trigger(CountTrigger.of(100))
.reduce(reducer);
@@ -109,12 +118,13 @@ public class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase
OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
Assert.assertTrue(operator1 instanceof NonKeyedWindowOperator);
NonKeyedWindowOperator winOperator1 = (NonKeyedWindowOperator) operator1;
+ Assert.assertTrue(winOperator1.isSetProcessingTime());
Assert.assertTrue(winOperator1.getTriggerTemplate() instanceof CountTrigger);
- Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingProcessingTimeWindows);
+ Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingTimeWindows);
Assert.assertTrue(winOperator1.getWindowBufferFactory() instanceof PreAggregatingHeapWindowBuffer.Factory);
DataStream<Tuple2<String, Integer>> window2 = source
- .windowAll(TumblingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
+ .windowAll(TumblingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
.trigger(CountTrigger.of(100))
.apply(new AllWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow>() {
private static final long serialVersionUID = 1L;
@@ -132,8 +142,9 @@ public class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase
OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator2 = transform2.getOperator();
Assert.assertTrue(operator2 instanceof NonKeyedWindowOperator);
NonKeyedWindowOperator winOperator2 = (NonKeyedWindowOperator) operator2;
+ Assert.assertTrue(winOperator1.isSetProcessingTime());
Assert.assertTrue(winOperator2.getTriggerTemplate() instanceof CountTrigger);
- Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingProcessingTimeWindows);
+ Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingTimeWindows);
Assert.assertTrue(winOperator2.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
}
@@ -141,13 +152,14 @@ public class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase
@SuppressWarnings("rawtypes")
public void testEvicting() throws Exception {
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+ env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime);
DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
DummyReducer reducer = new DummyReducer();
DataStream<Tuple2<String, Integer>> window1 = source
- .windowAll(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
+ .windowAll(SlidingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
.evictor(CountEvictor.of(100))
.reduce(reducer);
@@ -155,13 +167,14 @@ public class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase
OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
Assert.assertTrue(operator1 instanceof EvictingNonKeyedWindowOperator);
EvictingNonKeyedWindowOperator winOperator1 = (EvictingNonKeyedWindowOperator) operator1;
- Assert.assertTrue(winOperator1.getTriggerTemplate() instanceof ProcessingTimeTrigger);
- Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingProcessingTimeWindows);
+ Assert.assertFalse(winOperator1.isSetProcessingTime());
+ Assert.assertTrue(winOperator1.getTriggerTemplate() instanceof WatermarkTrigger);
+ Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingTimeWindows);
Assert.assertTrue(winOperator1.getEvictor() instanceof CountEvictor);
Assert.assertTrue(winOperator1.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
DataStream<Tuple2<String, Integer>> window2 = source
- .windowAll(TumblingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
+ .windowAll(TumblingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
.trigger(CountTrigger.of(100))
.evictor(TimeEvictor.of(Time.of(100, TimeUnit.MILLISECONDS)))
.apply(new AllWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow>() {
@@ -180,8 +193,9 @@ public class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase
OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator2 = transform2.getOperator();
Assert.assertTrue(operator2 instanceof EvictingNonKeyedWindowOperator);
EvictingNonKeyedWindowOperator winOperator2 = (EvictingNonKeyedWindowOperator) operator2;
+ Assert.assertFalse(winOperator2.isSetProcessingTime());
Assert.assertTrue(winOperator2.getTriggerTemplate() instanceof CountTrigger);
- Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingProcessingTimeWindows);
+ Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingTimeWindows);
Assert.assertTrue(winOperator2.getEvictor() instanceof TimeEvictor);
Assert.assertTrue(winOperator2.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java
index 5124add..10fe734 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowTranslationTest.java
@@ -20,19 +20,20 @@ package org.apache.flink.streaming.runtime.operators.windowing;
import org.apache.flink.api.common.functions.RichReduceFunction;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.api.TimeCharacteristic;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.WindowedStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.transformations.OneInputTransformation;
-import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows;
-import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
+import org.apache.flink.streaming.api.windowing.assigners.SlidingTimeWindows;
+import org.apache.flink.streaming.api.windowing.assigners.TumblingTimeWindows;
import org.apache.flink.streaming.api.windowing.evictors.CountEvictor;
import org.apache.flink.streaming.api.windowing.evictors.TimeEvictor;
import org.apache.flink.streaming.api.windowing.time.Time;
import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
-import org.apache.flink.streaming.api.windowing.triggers.ProcessingTimeTrigger;
+import org.apache.flink.streaming.api.windowing.triggers.WatermarkTrigger;
import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
import org.apache.flink.streaming.runtime.operators.windowing.buffers.HeapWindowBuffer;
import org.apache.flink.streaming.runtime.operators.windowing.buffers.PreAggregatingHeapWindowBuffer;
@@ -59,13 +60,13 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
+ env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime);
DummyReducer reducer = new DummyReducer();
DataStream<Tuple2<String, Integer>> window1 = source
.keyBy(0)
- .window(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS),
- Time.of(100, TimeUnit.MILLISECONDS)))
+ .window(SlidingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
.reduce(reducer);
OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
@@ -74,7 +75,7 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
DataStream<Tuple2<String, Integer>> window2 = source
.keyBy(0)
- .window(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
+ .window(SlidingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
.apply(new WindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple, TimeWindow>() {
private static final long serialVersionUID = 1L;
@@ -92,10 +93,63 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
Assert.assertTrue(operator2 instanceof AccumulatingProcessingTimeWindowOperator);
}
+ /**
+ * These tests ensure that the correct trigger is set when using event-time windows.
+ */
+ @Test
+ @SuppressWarnings("rawtypes")
+ public void testEventTime() throws Exception {
+ StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+ DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
+ env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime);
+
+ DummyReducer reducer = new DummyReducer();
+
+ DataStream<Tuple2<String, Integer>> window1 = source
+ .keyBy(0)
+ .window(SlidingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
+ .reduce(reducer);
+
+ OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation();
+ OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
+ Assert.assertTrue(operator1 instanceof WindowOperator);
+ WindowOperator winOperator1 = (WindowOperator) operator1;
+ Assert.assertFalse(winOperator1.isSetProcessingTime());
+ Assert.assertTrue(winOperator1.getTriggerTemplate() instanceof WatermarkTrigger);
+ Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingTimeWindows);
+ Assert.assertTrue(winOperator1.getWindowBufferFactory() instanceof PreAggregatingHeapWindowBuffer.Factory);
+
+ DataStream<Tuple2<String, Integer>> window2 = source
+ .keyBy(0)
+ .window(TumblingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
+ .apply(new WindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple, TimeWindow>() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public void apply(Tuple tuple,
+ TimeWindow window,
+ Iterable<Tuple2<String, Integer>> values,
+ Collector<Tuple2<String, Integer>> out) throws Exception {
+
+ }
+ });
+
+ OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform2 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window2.getTransformation();
+ OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator2 = transform2.getOperator();
+ Assert.assertTrue(operator2 instanceof WindowOperator);
+ WindowOperator winOperator2 = (WindowOperator) operator2;
+ Assert.assertFalse(winOperator2.isSetProcessingTime());
+ Assert.assertTrue(winOperator2.getTriggerTemplate() instanceof WatermarkTrigger);
+ Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingTimeWindows);
+ Assert.assertTrue(winOperator2.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
+ }
+
@Test
@SuppressWarnings("rawtypes")
public void testNonEvicting() throws Exception {
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+ env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime);
DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
@@ -103,7 +157,7 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
DataStream<Tuple2<String, Integer>> window1 = source
.keyBy(0)
- .window(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
+ .window(SlidingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
.trigger(CountTrigger.of(100))
.reduce(reducer);
@@ -111,13 +165,14 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
Assert.assertTrue(operator1 instanceof WindowOperator);
WindowOperator winOperator1 = (WindowOperator) operator1;
+ Assert.assertTrue(winOperator1.isSetProcessingTime());
Assert.assertTrue(winOperator1.getTriggerTemplate() instanceof CountTrigger);
- Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingProcessingTimeWindows);
+ Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingTimeWindows);
Assert.assertTrue(winOperator1.getWindowBufferFactory() instanceof PreAggregatingHeapWindowBuffer.Factory);
DataStream<Tuple2<String, Integer>> window2 = source
.keyBy(0)
- .window(TumblingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
+ .window(TumblingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
.trigger(CountTrigger.of(100))
.apply(new WindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple, TimeWindow>() {
private static final long serialVersionUID = 1L;
@@ -135,8 +190,9 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator2 = transform2.getOperator();
Assert.assertTrue(operator2 instanceof WindowOperator);
WindowOperator winOperator2 = (WindowOperator) operator2;
+ Assert.assertTrue(winOperator2.isSetProcessingTime());
Assert.assertTrue(winOperator2.getTriggerTemplate() instanceof CountTrigger);
- Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingProcessingTimeWindows);
+ Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingTimeWindows);
Assert.assertTrue(winOperator2.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
}
@@ -144,6 +200,7 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
@SuppressWarnings("rawtypes")
public void testEvicting() throws Exception {
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+ env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime);
DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2));
@@ -151,7 +208,7 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
DataStream<Tuple2<String, Integer>> window1 = source
.keyBy(0)
- .window(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
+ .window(SlidingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS)))
.evictor(CountEvictor.of(100))
.reduce(reducer);
@@ -159,14 +216,15 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator();
Assert.assertTrue(operator1 instanceof EvictingWindowOperator);
EvictingWindowOperator winOperator1 = (EvictingWindowOperator) operator1;
- Assert.assertTrue(winOperator1.getTriggerTemplate() instanceof ProcessingTimeTrigger);
- Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingProcessingTimeWindows);
+ Assert.assertFalse(winOperator1.isSetProcessingTime());
+ Assert.assertTrue(winOperator1.getTriggerTemplate() instanceof WatermarkTrigger);
+ Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingTimeWindows);
Assert.assertTrue(winOperator1.getEvictor() instanceof CountEvictor);
Assert.assertTrue(winOperator1.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
DataStream<Tuple2<String, Integer>> window2 = source
.keyBy(0)
- .window(TumblingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
+ .window(TumblingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
.trigger(CountTrigger.of(100))
.evictor(TimeEvictor.of(Time.of(100, TimeUnit.MILLISECONDS)))
.apply(new WindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple, TimeWindow>() {
@@ -185,8 +243,9 @@ public class WindowTranslationTest extends StreamingMultipleProgramsTestBase {
OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator2 = transform2.getOperator();
Assert.assertTrue(operator2 instanceof EvictingWindowOperator);
EvictingWindowOperator winOperator2 = (EvictingWindowOperator) operator2;
+ Assert.assertFalse(winOperator2.isSetProcessingTime());
Assert.assertTrue(winOperator2.getTriggerTemplate() instanceof CountTrigger);
- Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingProcessingTimeWindows);
+ Assert.assertTrue(winOperator2.getWindowAssigner() instanceof TumblingTimeWindows);
Assert.assertTrue(winOperator2.getEvictor() instanceof TimeEvictor);
Assert.assertTrue(winOperator2.getWindowBufferFactory() instanceof HeapWindowBuffer.Factory);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java
index 8abf9d6..5915a7a 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java
@@ -23,6 +23,7 @@ import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.tuple.Tuple3;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.TimeCharacteristic;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.TimestampExtractor;
@@ -64,7 +65,7 @@ public class WindowJoin {
// obtain execution environment
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.getConfig().enableTimestamps();
+ env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
// connect to the data sources for grades and salaries
Tuple2<DataStream<Tuple3<Long, String, Integer>>, DataStream<Tuple3<Long, String, Integer>>> input = getInputStreams(env);
http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/join/WindowJoin.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/join/WindowJoin.scala b/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/join/WindowJoin.scala
index 225dab7..42484e8 100644
--- a/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/join/WindowJoin.scala
+++ b/flink-staging/flink-streaming/flink-streaming-examples/src/main/scala/org/apache/flink/streaming/scala/examples/join/WindowJoin.scala
@@ -20,6 +20,7 @@ package org.apache.flink.streaming.scala.examples.join
import java.util.concurrent.TimeUnit
+import org.apache.flink.streaming.api.TimeCharacteristic
import org.apache.flink.streaming.api.scala._
import org.apache.flink.streaming.api.windowing.assigners.SlidingTimeWindows
import org.apache.flink.streaming.api.windowing.time.Time
@@ -45,7 +46,7 @@ object WindowJoin {
}
val env = StreamExecutionEnvironment.getExecutionEnvironment
- env.getConfig.enableTimestamps()
+ env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
//Create streams for grades and salaries by mapping the inputs to the corresponding objects
val grades = setGradesInput(env)
http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
index 7babc40..fb4d75d 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
@@ -34,7 +34,7 @@ import org.apache.flink.streaming.api.scala.function.StatefulFunction
import org.apache.flink.streaming.api.windowing.assigners._
import org.apache.flink.streaming.api.windowing.helper.WindowingHelper
import org.apache.flink.streaming.api.windowing.policy.{EvictionPolicy, TriggerPolicy}
-import org.apache.flink.streaming.api.windowing.time.{AbstractTime, EventTime, ProcessingTime}
+import org.apache.flink.streaming.api.windowing.time.AbstractTime
import org.apache.flink.streaming.api.windowing.windows.{TimeWindow, Window}
import org.apache.flink.streaming.util.serialization.SerializationSchema
import org.apache.flink.util.Collector
@@ -624,20 +624,8 @@ class DataStream[T](javaStream: JavaStream[T]) {
* @param size The size of the window.
*/
def timeWindowAll(size: AbstractTime): AllWindowedStream[T, TimeWindow] = {
- val env = new StreamExecutionEnvironment(javaStream.getExecutionEnvironment)
- val actualSize = size.makeSpecificBasedOnTimeCharacteristic(env.getStreamTimeCharacteristic)
-
- actualSize match {
- case t: EventTime =>
- val assigner = TumblingTimeWindows.of(actualSize)
- .asInstanceOf[WindowAssigner[T, TimeWindow]]
- windowAll(assigner)
- case t: ProcessingTime =>
- val assigner = TumblingProcessingTimeWindows.of(actualSize)
- .asInstanceOf[WindowAssigner[T, TimeWindow]]
- windowAll(assigner)
- case _ => throw new RuntimeException("Invalid time: " + actualSize)
- }
+ val assigner = TumblingTimeWindows.of(size).asInstanceOf[WindowAssigner[T, TimeWindow]]
+ windowAll(assigner)
}
/**
@@ -651,23 +639,8 @@ class DataStream[T](javaStream: JavaStream[T]) {
* @param size The size of the window.
*/
def timeWindowAll(size: AbstractTime, slide: AbstractTime): AllWindowedStream[T, TimeWindow] = {
- val env = new StreamExecutionEnvironment(javaStream.getExecutionEnvironment)
- val actualSize = size.makeSpecificBasedOnTimeCharacteristic(env.getStreamTimeCharacteristic)
- val actualSlide = slide.makeSpecificBasedOnTimeCharacteristic(env.getStreamTimeCharacteristic)
-
- actualSize match {
- case t: EventTime =>
- val assigner = SlidingTimeWindows.of(
- actualSize,
- actualSlide).asInstanceOf[WindowAssigner[T, TimeWindow]]
- windowAll(assigner)
- case t: ProcessingTime =>
- val assigner = SlidingProcessingTimeWindows.of(
- actualSize,
- actualSlide).asInstanceOf[WindowAssigner[T, TimeWindow]]
- windowAll(assigner)
- case _ => throw new RuntimeException("Invalid time: " + actualSize)
- }
+ val assigner = SlidingTimeWindows.of(size, slide).asInstanceOf[WindowAssigner[T, TimeWindow]]
+ windowAll(assigner)
}
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala
index 0ce36aa..c605bb1 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala
@@ -24,7 +24,7 @@ import org.apache.flink.streaming.api.functions.aggregation.SumAggregator
import org.apache.flink.streaming.api.functions.aggregation.ComparableAggregator
import org.apache.flink.streaming.api.operators.StreamGroupedReduce
import org.apache.flink.streaming.api.windowing.assigners._
-import org.apache.flink.streaming.api.windowing.time.{ProcessingTime, EventTime, AbstractTime}
+import org.apache.flink.streaming.api.windowing.time.AbstractTime
import org.apache.flink.streaming.api.windowing.windows.{Window, TimeWindow}
import scala.reflect.ClassTag
import org.apache.flink.api.common.typeinfo.TypeInformation
@@ -50,20 +50,8 @@ class KeyedStream[T, K](javaStream: KeyedJavaStream[T, K]) extends DataStream[T]
* @param size The size of the window.
*/
def timeWindow(size: AbstractTime): WindowedStream[T, K, TimeWindow] = {
- val env = new StreamExecutionEnvironment(javaStream.getExecutionEnvironment)
- val actualSize = size.makeSpecificBasedOnTimeCharacteristic(env.getStreamTimeCharacteristic)
-
- actualSize match {
- case t: EventTime =>
- val assigner = TumblingTimeWindows.of(actualSize)
- .asInstanceOf[WindowAssigner[T, TimeWindow]]
- window(assigner)
- case t: ProcessingTime =>
- val assigner = TumblingProcessingTimeWindows.of(actualSize)
- .asInstanceOf[WindowAssigner[T, TimeWindow]]
- window(assigner)
- case _ => throw new RuntimeException("Invalid time: " + actualSize)
- }
+ val assigner = TumblingTimeWindows.of(size).asInstanceOf[WindowAssigner[T, TimeWindow]]
+ window(assigner)
}
/**
@@ -78,23 +66,8 @@ class KeyedStream[T, K](javaStream: KeyedJavaStream[T, K]) extends DataStream[T]
* @param size The size of the window.
*/
def timeWindow(size: AbstractTime, slide: AbstractTime): WindowedStream[T, K, TimeWindow] = {
- val env = new StreamExecutionEnvironment(javaStream.getExecutionEnvironment)
- val actualSize = size.makeSpecificBasedOnTimeCharacteristic(env.getStreamTimeCharacteristic)
- val actualSlide = slide.makeSpecificBasedOnTimeCharacteristic(env.getStreamTimeCharacteristic)
-
- actualSize match {
- case t: EventTime =>
- val assigner = SlidingTimeWindows.of(
- actualSize,
- actualSlide).asInstanceOf[WindowAssigner[T, TimeWindow]]
- window(assigner)
- case t: ProcessingTime =>
- val assigner = SlidingProcessingTimeWindows.of(
- actualSize,
- actualSlide).asInstanceOf[WindowAssigner[T, TimeWindow]]
- window(assigner)
- case _ => throw new RuntimeException("Invalid time: " + actualSize)
- }
+ val assigner = SlidingTimeWindows.of(size, slide).asInstanceOf[WindowAssigner[T, TimeWindow]]
+ window(assigner)
}
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
index 7492e48..f767aba 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
@@ -299,6 +299,11 @@ class StreamExecutionEnvironment(javaEnv: JavaEnv) {
* Sets the time characteristic for all streams create from this environment, e.g., processing
* time, event time, or ingestion time.
*
+ * If you set the characteristic to IngestionTime of EventTime this will set a default
+ * watermark update interval of 200 ms. If this is not applicable for your application
+ * you should change it using
+ * [[org.apache.flink.api.common.ExecutionConfig#setAutoWatermarkInterval(long)]]
+ *
* @param characteristic The time characteristic.
*/
def setStreamTimeCharacteristic(characteristic: TimeCharacteristic) : Unit = {
http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala
index dece9f6..99fcd07 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AllWindowTranslationTest.scala
@@ -22,9 +22,10 @@ package org.apache.flink.streaming.api.scala
import java.util.concurrent.TimeUnit
import org.apache.flink.api.common.functions.RichReduceFunction
+import org.apache.flink.streaming.api.TimeCharacteristic
import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction
import org.apache.flink.streaming.api.transformations.OneInputTransformation
-import org.apache.flink.streaming.api.windowing.assigners.{TumblingProcessingTimeWindows, SlidingProcessingTimeWindows}
+import org.apache.flink.streaming.api.windowing.assigners.{TumblingTimeWindows, SlidingTimeWindows}
import org.apache.flink.streaming.api.windowing.evictors.{CountEvictor, TimeEvictor}
import org.apache.flink.streaming.api.windowing.time.Time
import org.apache.flink.streaming.api.windowing.triggers.{ProcessingTimeTrigger, CountTrigger}
@@ -49,13 +50,14 @@ class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
@Test
def testFastTimeWindows(): Unit = {
val env = StreamExecutionEnvironment.getExecutionEnvironment
+ env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime)
val source = env.fromElements(("hello", 1), ("hello", 2))
val reducer = new DummyReducer
val window1 = source
- .windowAll(SlidingProcessingTimeWindows.of(
+ .windowAll(SlidingTimeWindows.of(
Time.of(1, TimeUnit.SECONDS),
Time.of(100, TimeUnit.MILLISECONDS)))
.reduce(reducer)
@@ -69,7 +71,7 @@ class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
val window2 = source
.keyBy(0)
- .windowAll(SlidingProcessingTimeWindows.of(
+ .windowAll(SlidingTimeWindows.of(
Time.of(1, TimeUnit.SECONDS),
Time.of(100, TimeUnit.MILLISECONDS)))
.apply(new AllWindowFunction[(String, Int), (String, Int), TimeWindow]() {
@@ -96,7 +98,7 @@ class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
val reducer = new DummyReducer
val window1 = source
- .windowAll(SlidingProcessingTimeWindows.of(
+ .windowAll(SlidingTimeWindows.of(
Time.of(1, TimeUnit.SECONDS),
Time.of(100, TimeUnit.MILLISECONDS)))
.trigger(CountTrigger.of(100))
@@ -110,13 +112,13 @@ class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
assertTrue(operator1.isInstanceOf[NonKeyedWindowOperator[_, _, _]])
val winOperator1 = operator1.asInstanceOf[NonKeyedWindowOperator[_, _, _]]
assertTrue(winOperator1.getTriggerTemplate.isInstanceOf[CountTrigger[_]])
- assertTrue(winOperator1.getWindowAssigner.isInstanceOf[SlidingProcessingTimeWindows])
+ assertTrue(winOperator1.getWindowAssigner.isInstanceOf[SlidingTimeWindows])
assertTrue(
winOperator1.getWindowBufferFactory.isInstanceOf[PreAggregatingHeapWindowBuffer.Factory[_]])
val window2 = source
- .windowAll(TumblingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
+ .windowAll(TumblingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
.trigger(CountTrigger.of(100))
.apply(new AllWindowFunction[(String, Int), (String, Int), TimeWindow]() {
def apply(
@@ -133,7 +135,7 @@ class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
assertTrue(operator2.isInstanceOf[NonKeyedWindowOperator[_, _, _]])
val winOperator2 = operator2.asInstanceOf[NonKeyedWindowOperator[_, _, _]]
assertTrue(winOperator2.getTriggerTemplate.isInstanceOf[CountTrigger[_]])
- assertTrue(winOperator2.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows])
+ assertTrue(winOperator2.getWindowAssigner.isInstanceOf[TumblingTimeWindows])
assertTrue(winOperator2.getWindowBufferFactory.isInstanceOf[HeapWindowBuffer.Factory[_]])
}
@@ -146,7 +148,7 @@ class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
val reducer = new DummyReducer
val window1 = source
- .windowAll(SlidingProcessingTimeWindows.of(
+ .windowAll(SlidingTimeWindows.of(
Time.of(1, TimeUnit.SECONDS),
Time.of(100, TimeUnit.MILLISECONDS)))
.evictor(TimeEvictor.of(Time.of(1, TimeUnit.SECONDS)))
@@ -161,12 +163,12 @@ class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
val winOperator1 = operator1.asInstanceOf[EvictingNonKeyedWindowOperator[_, _, _]]
assertTrue(winOperator1.getTriggerTemplate.isInstanceOf[ProcessingTimeTrigger])
assertTrue(winOperator1.getEvictor.isInstanceOf[TimeEvictor[_]])
- assertTrue(winOperator1.getWindowAssigner.isInstanceOf[SlidingProcessingTimeWindows])
+ assertTrue(winOperator1.getWindowAssigner.isInstanceOf[SlidingTimeWindows])
assertTrue(winOperator1.getWindowBufferFactory.isInstanceOf[HeapWindowBuffer.Factory[_]])
val window2 = source
- .windowAll(TumblingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
+ .windowAll(TumblingTimeWindows.of(Time.of(1, TimeUnit.SECONDS)))
.trigger(CountTrigger.of(100))
.evictor(CountEvictor.of(1000))
.apply(new AllWindowFunction[(String, Int), (String, Int), TimeWindow]() {
@@ -185,7 +187,7 @@ class AllWindowTranslationTest extends StreamingMultipleProgramsTestBase {
val winOperator2 = operator2.asInstanceOf[EvictingNonKeyedWindowOperator[_, _, _]]
assertTrue(winOperator2.getTriggerTemplate.isInstanceOf[CountTrigger[_]])
assertTrue(winOperator2.getEvictor.isInstanceOf[CountEvictor[_]])
- assertTrue(winOperator2.getWindowAssigner.isInstanceOf[TumblingProcessingTimeWindows])
+ assertTrue(winOperator2.getWindowAssigner.isInstanceOf[TumblingTimeWindows])
assertTrue(winOperator2.getWindowBufferFactory.isInstanceOf[HeapWindowBuffer.Factory[_]])
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/ff367d6e/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/CoGroupJoinITCase.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/CoGroupJoinITCase.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/CoGroupJoinITCase.scala
index 7232309..3f6e10f 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/CoGroupJoinITCase.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/CoGroupJoinITCase.scala
@@ -20,6 +20,7 @@ package org.apache.flink.streaming.api.scala
import java.util.concurrent.TimeUnit
+import org.apache.flink.streaming.api.TimeCharacteristic
import org.apache.flink.streaming.api.functions.TimestampExtractor
import org.apache.flink.streaming.api.functions.sink.SinkFunction
import org.apache.flink.streaming.api.functions.source.SourceFunction
@@ -38,9 +39,9 @@ class CoGroupJoinITCase extends StreamingMultipleProgramsTestBase {
CoGroupJoinITCase.testResults = mutable.MutableList()
val env = StreamExecutionEnvironment.getExecutionEnvironment
+ env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
env.setParallelism(1)
- env.getConfig.enableTimestamps
-
+
val source1 = env.addSource(new SourceFunction[(String, Int)]() {
def run(ctx: SourceFunction.SourceContext[(String, Int)]) {
ctx.collect(("a", 0))
@@ -101,8 +102,8 @@ class CoGroupJoinITCase extends StreamingMultipleProgramsTestBase {
CoGroupJoinITCase.testResults = mutable.MutableList()
val env = StreamExecutionEnvironment.getExecutionEnvironment
+ env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
env.setParallelism(1)
- env.getConfig.enableTimestamps
val source1 = env.addSource(new SourceFunction[(String, String, Int)]() {
def run(ctx: SourceFunction.SourceContext[(String, String, Int)]) {
@@ -177,8 +178,8 @@ class CoGroupJoinITCase extends StreamingMultipleProgramsTestBase {
CoGroupJoinITCase.testResults = mutable.MutableList()
val env = StreamExecutionEnvironment.getExecutionEnvironment
+ env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
env.setParallelism(1)
- env.getConfig.enableTimestamps
val source1 = env.addSource(new SourceFunction[(String, String, Int)]() {
def run(ctx: SourceFunction.SourceContext[(String, String, Int)]) {