提交 ec82d973 编写于 作者: G Gyula Fora 提交者: mbalassi

[streaming] CoWindow operator rework + stream iteration example added

上级 97a7322f
......@@ -21,7 +21,6 @@ import java.io.Serializable;
import org.apache.commons.lang3.SerializationException;
import org.apache.commons.lang3.SerializationUtils;
import org.apache.flink.api.common.functions.CoGroupFunction;
import org.apache.flink.api.common.functions.Function;
import org.apache.flink.api.common.functions.RichFlatMapFunction;
import org.apache.flink.api.common.typeinfo.TypeInformation;
......@@ -30,14 +29,15 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.function.co.CoFlatMapFunction;
import org.apache.flink.streaming.api.function.co.CoMapFunction;
import org.apache.flink.streaming.api.function.co.CoReduceFunction;
import org.apache.flink.streaming.api.function.co.CoWindowFunction;
import org.apache.flink.streaming.api.function.co.RichCoMapFunction;
import org.apache.flink.streaming.api.function.co.RichCoReduceFunction;
import org.apache.flink.streaming.api.invokable.operator.co.CoFlatMapInvokable;
import org.apache.flink.streaming.api.invokable.operator.co.CoGroupInvokable;
import org.apache.flink.streaming.api.invokable.operator.co.CoGroupedReduceInvokable;
import org.apache.flink.streaming.api.invokable.operator.co.CoInvokable;
import org.apache.flink.streaming.api.invokable.operator.co.CoMapInvokable;
import org.apache.flink.streaming.api.invokable.operator.co.CoReduceInvokable;
import org.apache.flink.streaming.api.invokable.operator.co.CoWindowInvokable;
import org.apache.flink.streaming.api.invokable.util.DefaultTimeStamp;
import org.apache.flink.streaming.api.invokable.util.TimeStamp;
import org.apache.flink.streaming.util.serialization.FunctionTypeWrapper;
......@@ -167,11 +167,12 @@ public class ConnectedDataStream<IN1, IN2> {
/**
* Applies a CoFlatMap transformation on a {@link ConnectedDataStream} and
* maps the output to a common type. The transformation calls a
* {@link CoFlatMapFunction#flatMap1} for each element of the first input and
* {@link CoFlatMapFunction#flatMap2} for each element of the second input. Each
* CoFlatMapFunction call returns any number of elements including none. The
* user can also extend {@link RichFlatMapFunction} to gain access to other
* features provided by the {@link RichFuntion} interface.
* {@link CoFlatMapFunction#flatMap1} for each element of the first input
* and {@link CoFlatMapFunction#flatMap2} for each element of the second
* input. Each CoFlatMapFunction call returns any number of elements
* including none. The user can also extend {@link RichFlatMapFunction} to
* gain access to other features provided by the {@link RichFuntion}
* interface.
*
* @param coFlatMapper
* The CoFlatMapFunction used to jointly transform the two input
......@@ -226,13 +227,13 @@ public class ConnectedDataStream<IN1, IN2> {
}
/**
* Applies a CoGroup transformation on the connected DataStreams. The
* transformation calls the {@link CoGroupFunction#coGroupache} method for
* for time aligned windows of the two data streams. System time is used as
* Applies a CoWindow transformation on the connected DataStreams. The
* transformation calls the {@link CoWindowFunction#coWindow} method for for
* time aligned windows of the two data streams. System time is used as
* default to compute windows.
*
* @param coGroupFunction
* The {@link CoGroupFunction} that will be applied for the time
* @param coWindowFunction
* The {@link CoWindowFunction} that will be applied for the time
* windows.
* @param windowSize
* Size of the windows that will be aligned for both streams in
......@@ -243,20 +244,20 @@ public class ConnectedDataStream<IN1, IN2> {
*
* @return The transformed {@link DataStream}.
*/
public <OUT> SingleOutputStreamOperator<OUT, ?> windowReduceGroup(
CoGroupFunction<IN1, IN2, OUT> coGroupFunction, long windowSize, long slideInterval) {
return windowReduceGroup(coGroupFunction, windowSize, slideInterval,
public <OUT> SingleOutputStreamOperator<OUT, ?> windowReduce(
CoWindowFunction<IN1, IN2, OUT> coWindowFunction, long windowSize, long slideInterval) {
return windowReduce(coWindowFunction, windowSize, slideInterval,
new DefaultTimeStamp<IN1>(), new DefaultTimeStamp<IN2>());
}
/**
* Applies a CoGroup transformation on the connected DataStreams. The
* transformation calls the {@link CoGroupFunction#coGroupache} method for
* for time aligned windows of the two data streams. The user can implement
* Applies a CoWindow transformation on the connected DataStreams. The
* transformation calls the {@link CoWindowFunction#coWindow} method for
* time aligned windows of the two data streams. The user can implement
* their own time stamps or use the system time by default.
*
* @param coGroupFunction
* The {@link CoGroupFunction} that will be applied for the time
* @param coWindowFunction
* The {@link CoWindowFunction} that will be applied for the time
* windows.
* @param windowSize
* Size of the windows that will be aligned for both streams. If
......@@ -272,8 +273,8 @@ public class ConnectedDataStream<IN1, IN2> {
* User defined time stamps for the second input.
* @return The transformed {@link DataStream}.
*/
public <OUT> SingleOutputStreamOperator<OUT, ?> windowReduceGroup(
CoGroupFunction<IN1, IN2, OUT> coGroupFunction, long windowSize, long slideInterval,
public <OUT> SingleOutputStreamOperator<OUT, ?> windowReduce(
CoWindowFunction<IN1, IN2, OUT> coWindowFunction, long windowSize, long slideInterval,
TimeStamp<IN1> timestamp1, TimeStamp<IN2> timestamp2) {
if (windowSize < 1) {
......@@ -283,15 +284,15 @@ public class ConnectedDataStream<IN1, IN2> {
throw new IllegalArgumentException("Slide interval must be positive");
}
FunctionTypeWrapper<IN1> in1TypeWrapper = new FunctionTypeWrapper<IN1>(coGroupFunction,
CoGroupFunction.class, 0);
FunctionTypeWrapper<IN2> in2TypeWrapper = new FunctionTypeWrapper<IN2>(coGroupFunction,
CoGroupFunction.class, 1);
FunctionTypeWrapper<OUT> outTypeWrapper = new FunctionTypeWrapper<OUT>(coGroupFunction,
CoGroupFunction.class, 2);
FunctionTypeWrapper<IN1> in1TypeWrapper = new FunctionTypeWrapper<IN1>(coWindowFunction,
CoWindowFunction.class, 0);
FunctionTypeWrapper<IN2> in2TypeWrapper = new FunctionTypeWrapper<IN2>(coWindowFunction,
CoWindowFunction.class, 1);
FunctionTypeWrapper<OUT> outTypeWrapper = new FunctionTypeWrapper<OUT>(coWindowFunction,
CoWindowFunction.class, 2);
return addCoFunction("coWindowReduce", coGroupFunction, in1TypeWrapper, in2TypeWrapper,
outTypeWrapper, new CoGroupInvokable<IN1, IN2, OUT>(coGroupFunction, windowSize,
return addCoFunction("coWindowReduce", coWindowFunction, in1TypeWrapper, in2TypeWrapper,
outTypeWrapper, new CoWindowInvokable<IN1, IN2, OUT>(coWindowFunction, windowSize,
slideInterval, timestamp1, timestamp2));
}
......
/*
* 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.function.co;
import java.io.Serializable;
import java.util.List;
import org.apache.flink.api.common.functions.Function;
import org.apache.flink.util.Collector;
public interface CoWindowFunction<IN1, IN2, O> extends Function, Serializable {
public void coWindow(List<IN1> first, List<IN2> second, Collector<O> out) throws Exception;
}
/*
* 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.function.co;
import java.util.List;
import org.apache.flink.api.common.functions.AbstractRichFunction;
import org.apache.flink.util.Collector;
public abstract class RichCoWindowFunction<IN1, IN2, O> extends AbstractRichFunction implements
CoWindowFunction<IN1, IN2, O> {
private static final long serialVersionUID = 1L;
@Override
public abstract void coWindow(List<IN1> first, List<IN2> second, Collector<O> out)
throws Exception;
}
......@@ -216,6 +216,7 @@ public class BatchReduceInvokable<OUT> extends StreamInvokable<OUT, OUT> {
@Override
public void open(Configuration config) throws Exception{
super.open(config);
serializer = inSerializer.getObjectSerializer();
this.batch = new StreamBatch();
}
......
......@@ -21,7 +21,6 @@ import java.io.Serializable;
import java.util.Iterator;
import org.apache.commons.math.util.MathUtils;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.function.co.CoReduceFunction;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
......@@ -46,8 +45,6 @@ public class CoBatchReduceInvokable<IN1, IN2, OUT> extends CoInvokable<IN1, IN2,
protected StreamBatch<IN2> batch2;
protected StreamBatch<IN1> currentBatch1;
protected StreamBatch<IN2> currentBatch2;
protected TypeSerializer<IN1> serializer1;
protected TypeSerializer<IN2> serializer2;
public CoBatchReduceInvokable(CoReduceFunction<IN1, IN2, OUT> coReducer, long batchSize1,
long batchSize2, long slideSize1, long slideSize2) {
......@@ -173,8 +170,6 @@ public class CoBatchReduceInvokable<IN1, IN2, OUT> extends CoInvokable<IN1, IN2,
super.open(config);
this.batch1 = new StreamBatch<IN1>(batchSize1, slideSize1);
this.batch2 = new StreamBatch<IN2>(batchSize2, slideSize2);
this.serializer1 = srSerializer1.getObjectSerializer();
this.serializer2 = srSerializer2.getObjectSerializer();
}
public void reduceToBuffer1(StreamRecord<IN1> next, StreamBatch<IN1> streamBatch)
......
......@@ -18,6 +18,7 @@
package org.apache.flink.streaming.api.invokable.operator.co;
import org.apache.flink.api.common.functions.Function;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.streaming.api.invokable.StreamInvokable;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.apache.flink.streaming.api.streamrecord.StreamRecordSerializer;
......@@ -41,6 +42,8 @@ public abstract class CoInvokable<IN1, IN2, OUT> extends StreamInvokable<IN1, OU
protected StreamRecord<IN2> reuse2;
protected StreamRecordSerializer<IN1> srSerializer1;
protected StreamRecordSerializer<IN2> srSerializer2;
protected TypeSerializer<IN1> serializer1;
protected TypeSerializer<IN2> serializer2;
public void initialize(Collector<OUT> collector,
CoReaderIterator<StreamRecord<IN1>, StreamRecord<IN2>> recordIterator,
......@@ -55,6 +58,8 @@ public abstract class CoInvokable<IN1, IN2, OUT> extends StreamInvokable<IN1, OU
this.srSerializer1 = serializer1;
this.srSerializer2 = serializer2;
this.isMutable = isMutable;
this.serializer1 = srSerializer1.getObjectSerializer();
this.serializer2 = srSerializer2.getObjectSerializer();
}
protected void resetReuseAll() {
......
......@@ -18,17 +18,19 @@
package org.apache.flink.streaming.api.invokable.operator.co;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.List;
import org.apache.commons.math.util.MathUtils;
import org.apache.flink.api.common.functions.CoGroupFunction;
import org.apache.flink.streaming.api.function.co.CoWindowFunction;
import org.apache.flink.streaming.api.invokable.util.TimeStamp;
import org.apache.flink.streaming.api.streamrecord.StreamRecord;
import org.apache.flink.streaming.state.CircularFifoList;
public class CoGroupInvokable<IN1, IN2, OUT> extends CoInvokable<IN1, IN2, OUT> {
public class CoWindowInvokable<IN1, IN2, OUT> extends CoInvokable<IN1, IN2, OUT> {
private static final long serialVersionUID = 1L;
protected CoGroupFunction<IN1, IN2, OUT> coGroupFunction;
protected CoWindowFunction<IN1, IN2, OUT> coWindowFunction;
protected long windowSize;
protected long slideSize;
protected CircularFifoList<StreamRecord<IN1>> circularList1;
......@@ -41,10 +43,10 @@ public class CoGroupInvokable<IN1, IN2, OUT> extends CoInvokable<IN1, IN2, OUT>
protected long startTime;
protected long nextRecordTime;
public CoGroupInvokable(CoGroupFunction<IN1, IN2, OUT> coGroupFunction, long windowSize,
public CoWindowInvokable(CoWindowFunction<IN1, IN2, OUT> coWindowFunction, long windowSize,
long slideInterval, TimeStamp<IN1> timeStamp1, TimeStamp<IN2> timeStamp2) {
super(coGroupFunction);
this.coGroupFunction = coGroupFunction;
super(coWindowFunction);
this.coWindowFunction = coWindowFunction;
this.windowSize = windowSize;
this.slideSize = slideInterval;
this.circularList1 = new CircularFifoList<StreamRecord<IN1>>();
......@@ -73,8 +75,19 @@ public class CoGroupInvokable<IN1, IN2, OUT> extends CoInvokable<IN1, IN2, OUT>
@Override
protected void callUserFunction() throws Exception {
if(!window.circularList1.isEmpty() || !window.circularList2.isEmpty()){
coGroupFunction.coGroup(window.getIterable1(), window.getIterable2(), collector);
List<IN1> first = new ArrayList<IN1>();
List<IN2> second = new ArrayList<IN2>();
for (IN1 element : window.circularList1.getElements()) {
first.add(serializer1.copy(element));
}
for (IN2 element : window.circularList2.getElements()) {
second.add(serializer2.copy(element));
}
if (!window.circularList1.isEmpty() || !window.circularList2.isEmpty()) {
coWindowFunction.coWindow(first, second, collector);
}
}
......@@ -167,10 +180,11 @@ public class CoGroupInvokable<IN1, IN2, OUT> extends CoInvokable<IN1, IN2, OUT>
}
@Override
public void close() {
public void close() throws Exception {
if (!window.miniBatchEnd()) {
callUserFunctionAndLogException();
}
super.close();
}
@Override
......@@ -178,7 +192,7 @@ public class CoGroupInvokable<IN1, IN2, OUT> extends CoInvokable<IN1, IN2, OUT>
}
@Override
protected void callUserFunction2() throws Exception {
protected void callUserFunction2() throws Exception {
}
}
......@@ -20,6 +20,7 @@ package org.apache.flink.streaming.state;
import java.io.Serializable;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Queue;
/**
......@@ -73,6 +74,11 @@ public class CircularFifoList<T> implements Serializable {
}
}
@SuppressWarnings("unchecked")
public List<T> getElements(){
return (List<T>) queue;
}
public Iterator<T> getIterator() {
return queue.iterator();
......
......@@ -24,24 +24,24 @@ import java.util.HashSet;
import java.util.List;
import java.util.Set;
import org.apache.flink.api.common.functions.CoGroupFunction;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.streaming.api.invokable.operator.co.CoGroupInvokable;
import org.apache.flink.streaming.api.function.co.CoWindowFunction;
import org.apache.flink.streaming.api.invokable.operator.co.CoWindowInvokable;
import org.apache.flink.streaming.api.invokable.util.TimeStamp;
import org.apache.flink.streaming.util.MockCoInvokable;
import org.apache.flink.util.Collector;
import org.junit.Test;
public class CoWindowGroupReduceTest{
public class CoWindowTest {
public static final class MyCoGroup1 implements CoGroupFunction<Integer, Integer, Integer> {
public static final class MyCoGroup1 implements CoWindowFunction<Integer, Integer, Integer> {
private static final long serialVersionUID = 1L;
@SuppressWarnings("unused")
@Override
public void coGroup(Iterable<Integer> first, Iterable<Integer> second,
Collector<Integer> out) throws Exception {
public void coWindow(List<Integer> first, List<Integer> second, Collector<Integer> out)
throws Exception {
Integer count1 = 0;
for (Integer i : first) {
count1++;
......@@ -58,13 +58,13 @@ public class CoWindowGroupReduceTest{
}
public static final class MyCoGroup2 implements
CoGroupFunction<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>, Integer> {
CoWindowFunction<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>, Integer> {
private static final long serialVersionUID = 1L;
@Override
public void coGroup(Iterable<Tuple2<Integer, Integer>> first,
Iterable<Tuple2<Integer, Integer>> second, Collector<Integer> out) throws Exception {
public void coWindow(List<Tuple2<Integer, Integer>> first,
List<Tuple2<Integer, Integer>> second, Collector<Integer> out) throws Exception {
Set<Integer> firstElements = new HashSet<Integer>();
for (Tuple2<Integer, Integer> value : first) {
......@@ -115,7 +115,7 @@ public class CoWindowGroupReduceTest{
@Test
public void coWindowGroupReduceTest2() throws Exception {
CoGroupInvokable<Integer, Integer, Integer> invokable1 = new CoGroupInvokable<Integer, Integer, Integer>(
CoWindowInvokable<Integer, Integer, Integer> invokable1 = new CoWindowInvokable<Integer, Integer, Integer>(
new MyCoGroup1(), 2, 1, new MyTS1(), new MyTS1());
// Windowsize 2, slide 1
......@@ -150,39 +150,39 @@ public class CoWindowGroupReduceTest{
List<Integer> actual1 = MockCoInvokable.createAndExecute(invokable1, input11, input12);
assertEquals(expected1, actual1);
CoGroupInvokable<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>, Integer> invokable2 = new CoGroupInvokable<Tuple2<Integer,Integer>, Tuple2<Integer,Integer>, Integer>(new MyCoGroup2(), 2, 3, new MyTS2(), new MyTS2());
//WindowSize 2, slide 3
//1,2|4,5|7,8|
List<Tuple2<Integer,Integer>> input21 = new ArrayList<Tuple2<Integer,Integer>>();
input21.add(new Tuple2<Integer, Integer>(1,1));
input21.add(new Tuple2<Integer, Integer>(1,2));
input21.add(new Tuple2<Integer, Integer>(2,3));
input21.add(new Tuple2<Integer, Integer>(3,4));
input21.add(new Tuple2<Integer, Integer>(3,5));
input21.add(new Tuple2<Integer, Integer>(4,6));
input21.add(new Tuple2<Integer, Integer>(4,7));
input21.add(new Tuple2<Integer, Integer>(5,8));
List<Tuple2<Integer,Integer>> input22 = new ArrayList<Tuple2<Integer,Integer>>();
input22.add(new Tuple2<Integer, Integer>(1,1));
input22.add(new Tuple2<Integer, Integer>(2,0));
input22.add(new Tuple2<Integer, Integer>(2,2));
input22.add(new Tuple2<Integer, Integer>(3,9));
input22.add(new Tuple2<Integer, Integer>(3,4));
input22.add(new Tuple2<Integer, Integer>(4,10));
input22.add(new Tuple2<Integer, Integer>(5,8));
input22.add(new Tuple2<Integer, Integer>(5,7));
CoWindowInvokable<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>, Integer> invokable2 = new CoWindowInvokable<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>, Integer>(
new MyCoGroup2(), 2, 3, new MyTS2(), new MyTS2());
// WindowSize 2, slide 3
// 1,2|4,5|7,8|
List<Tuple2<Integer, Integer>> input21 = new ArrayList<Tuple2<Integer, Integer>>();
input21.add(new Tuple2<Integer, Integer>(1, 1));
input21.add(new Tuple2<Integer, Integer>(1, 2));
input21.add(new Tuple2<Integer, Integer>(2, 3));
input21.add(new Tuple2<Integer, Integer>(3, 4));
input21.add(new Tuple2<Integer, Integer>(3, 5));
input21.add(new Tuple2<Integer, Integer>(4, 6));
input21.add(new Tuple2<Integer, Integer>(4, 7));
input21.add(new Tuple2<Integer, Integer>(5, 8));
List<Tuple2<Integer, Integer>> input22 = new ArrayList<Tuple2<Integer, Integer>>();
input22.add(new Tuple2<Integer, Integer>(1, 1));
input22.add(new Tuple2<Integer, Integer>(2, 0));
input22.add(new Tuple2<Integer, Integer>(2, 2));
input22.add(new Tuple2<Integer, Integer>(3, 9));
input22.add(new Tuple2<Integer, Integer>(3, 4));
input22.add(new Tuple2<Integer, Integer>(4, 10));
input22.add(new Tuple2<Integer, Integer>(5, 8));
input22.add(new Tuple2<Integer, Integer>(5, 7));
List<Integer> expected2 = new ArrayList<Integer>();
expected2.add(1);
expected2.add(2);
expected2.add(8);
expected2.add(7);
List<Integer> actual2 = MockCoInvokable.createAndExecute(invokable2, input21, input22);
assertEquals(expected2, actual2);
}
......
/*
* 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.examples.iteration;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.Random;
import org.apache.flink.api.common.functions.MapFunction;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.streaming.api.collector.OutputSelector;
import org.apache.flink.streaming.api.datastream.IterativeDataStream;
import org.apache.flink.streaming.api.datastream.SplitDataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
public class IterateExample {
public static class Step implements
MapFunction<Tuple2<Double, Integer>, Tuple2<Double, Integer>> {
private static final long serialVersionUID = 1L;
Random rnd;
public Step() {
rnd = new Random();
}
@Override
public Tuple2<Double, Integer> map(Tuple2<Double, Integer> value) throws Exception {
return new Tuple2<Double, Integer>(value.f0 + rnd.nextDouble(), value.f1 + 1);
}
}
public static class MySelector extends OutputSelector<Tuple2<Double, Integer>> {
private static final long serialVersionUID = 1L;
@Override
public void select(Tuple2<Double, Integer> value, Collection<String> outputs) {
if (value.f0 > 100) {
outputs.add("output");
} else {
outputs.add("iterate");
}
}
}
public static void main(String[] args) throws Exception {
List<Tuple2<Double, Integer>> input = new ArrayList<Tuple2<Double, Integer>>();
for (int i = 0; i < 100; i++) {
input.add(new Tuple2<Double, Integer>(0., 0));
}
StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(2)
.setBufferTimeout(1);
IterativeDataStream<Tuple2<Double, Integer>> it = env.fromCollection(input).iterate()
.setMaxWaitTime(3000);
SplitDataStream<Tuple2<Double,Integer>> step = it.map(new Step()).shuffle().setParallelism(2).split(new MySelector());
it.closeWith(step.select("iterate"));
step.select("output").project(1).types(Integer.class).print();
env.execute();
}
}
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册