Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
doujutun3207
flink
提交
ec82d973
F
flink
项目概览
doujutun3207
/
flink
与 Fork 源项目一致
从无法访问的项目Fork
通知
24
Star
0
Fork
0
代码
文件
提交
分支
Tags
贡献者
分支图
Diff
Issue
0
列表
看板
标记
里程碑
合并请求
0
Wiki
0
Wiki
分析
仓库
DevOps
项目成员
Pages
F
flink
项目概览
项目概览
详情
发布
仓库
仓库
文件
提交
分支
标签
贡献者
分支图
比较
Issue
0
Issue
0
列表
看板
标记
里程碑
合并请求
0
合并请求
0
Pages
分析
分析
仓库分析
DevOps
Wiki
0
Wiki
成员
成员
收起侧边栏
关闭侧边栏
动态
分支图
创建新Issue
提交
Issue看板
体验新版 GitCode,发现更多精彩内容 >>
提交
ec82d973
编写于
10月 07, 2014
作者:
G
Gyula Fora
提交者:
mbalassi
10月 08, 2014
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
[streaming] CoWindow operator rework + stream iteration example added
上级
97a7322f
变更
10
隐藏空白更改
内联
并排
Showing
10 changed file
with
259 addition
and
83 deletion
+259
-83
flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
...e/flink/streaming/api/datastream/ConnectedDataStream.java
+31
-30
flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/CoWindowFunction.java
...che/flink/streaming/api/function/co/CoWindowFunction.java
+30
-0
flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/RichCoWindowFunction.java
...flink/streaming/api/function/co/RichCoWindowFunction.java
+34
-0
flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
...treaming/api/invokable/operator/BatchReduceInvokable.java
+1
-0
flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoBatchReduceInvokable.java
...ing/api/invokable/operator/co/CoBatchReduceInvokable.java
+0
-5
flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java
...link/streaming/api/invokable/operator/co/CoInvokable.java
+5
-0
flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoWindowInvokable.java
...treaming/api/invokable/operator/co/CoWindowInvokable.java
+24
-10
flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/CircularFifoList.java
...va/org/apache/flink/streaming/state/CircularFifoList.java
+6
-0
flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoWindowTest.java
.../flink/streaming/api/invokable/operator/CoWindowTest.java
+38
-38
flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java
...he/flink/streaming/examples/iteration/IterateExample.java
+90
-0
未找到文件。
flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedDataStream.java
浏览文件 @
ec82d973
...
...
@@ -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 Co
Group
transformation on the connected DataStreams. The
* transformation calls the {@link Co
GroupFunction#coGroupache} method
for
*
for
time aligned windows of the two data streams. System time is used as
* Applies a Co
Window
transformation on the connected DataStreams. The
* transformation calls the {@link Co
WindowFunction#coWindow} method for
for
* time aligned windows of the two data streams. System time is used as
* default to compute windows.
*
* @param co
Group
Function
* The {@link Co
Group
Function} that will be applied for the time
* @param co
Window
Function
* The {@link Co
Window
Function} 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
,
?>
windowReduce
Group
(
Co
GroupFunction
<
IN1
,
IN2
,
OUT
>
coGroup
Function
,
long
windowSize
,
long
slideInterval
)
{
return
windowReduce
Group
(
coGroup
Function
,
windowSize
,
slideInterval
,
public
<
OUT
>
SingleOutputStreamOperator
<
OUT
,
?>
windowReduce
(
Co
WindowFunction
<
IN1
,
IN2
,
OUT
>
coWindow
Function
,
long
windowSize
,
long
slideInterval
)
{
return
windowReduce
(
coWindow
Function
,
windowSize
,
slideInterval
,
new
DefaultTimeStamp
<
IN1
>(),
new
DefaultTimeStamp
<
IN2
>());
}
/**
* Applies a Co
Group
transformation on the connected DataStreams. The
* transformation calls the {@link Co
GroupFunction#coGroupache
} method for
*
for
time aligned windows of the two data streams. The user can implement
* Applies a Co
Window
transformation on the connected DataStreams. The
* transformation calls the {@link Co
WindowFunction#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 co
Group
Function
* The {@link Co
Group
Function} that will be applied for the time
* @param co
Window
Function
* The {@link Co
Window
Function} 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
,
?>
windowReduce
Group
(
Co
GroupFunction
<
IN1
,
IN2
,
OUT
>
coGroup
Function
,
long
windowSize
,
long
slideInterval
,
public
<
OUT
>
SingleOutputStreamOperator
<
OUT
,
?>
windowReduce
(
Co
WindowFunction
<
IN1
,
IN2
,
OUT
>
coWindow
Function
,
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
>(
co
Group
Function
,
Co
Group
Function
.
class
,
0
);
FunctionTypeWrapper
<
IN2
>
in2TypeWrapper
=
new
FunctionTypeWrapper
<
IN2
>(
co
Group
Function
,
Co
Group
Function
.
class
,
1
);
FunctionTypeWrapper
<
OUT
>
outTypeWrapper
=
new
FunctionTypeWrapper
<
OUT
>(
co
Group
Function
,
Co
Group
Function
.
class
,
2
);
FunctionTypeWrapper
<
IN1
>
in1TypeWrapper
=
new
FunctionTypeWrapper
<
IN1
>(
co
Window
Function
,
Co
Window
Function
.
class
,
0
);
FunctionTypeWrapper
<
IN2
>
in2TypeWrapper
=
new
FunctionTypeWrapper
<
IN2
>(
co
Window
Function
,
Co
Window
Function
.
class
,
1
);
FunctionTypeWrapper
<
OUT
>
outTypeWrapper
=
new
FunctionTypeWrapper
<
OUT
>(
co
Window
Function
,
Co
Window
Function
.
class
,
2
);
return
addCoFunction
(
"coWindowReduce"
,
co
Group
Function
,
in1TypeWrapper
,
in2TypeWrapper
,
outTypeWrapper
,
new
Co
GroupInvokable
<
IN1
,
IN2
,
OUT
>(
coGroup
Function
,
windowSize
,
return
addCoFunction
(
"coWindowReduce"
,
co
Window
Function
,
in1TypeWrapper
,
in2TypeWrapper
,
outTypeWrapper
,
new
Co
WindowInvokable
<
IN1
,
IN2
,
OUT
>(
coWindow
Function
,
windowSize
,
slideInterval
,
timestamp1
,
timestamp2
));
}
...
...
flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/CoWindowFunction.java
0 → 100644
浏览文件 @
ec82d973
/*
* 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
;
}
flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/co/RichCoWindowFunction.java
0 → 100644
浏览文件 @
ec82d973
/*
* 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
;
}
flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/BatchReduceInvokable.java
浏览文件 @
ec82d973
...
...
@@ -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
();
}
...
...
flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoBatchReduceInvokable.java
浏览文件 @
ec82d973
...
...
@@ -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
)
...
...
flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java
浏览文件 @
ec82d973
...
...
@@ -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
()
{
...
...
flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/Co
Group
Invokable.java
→
flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/Co
Window
Invokable.java
浏览文件 @
ec82d973
...
...
@@ -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.CoGroup
Function
;
import
org.apache.flink.
streaming.api.function.co.CoWindow
Function
;
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
Co
Group
Invokable
<
IN1
,
IN2
,
OUT
>
extends
CoInvokable
<
IN1
,
IN2
,
OUT
>
{
public
class
Co
Window
Invokable
<
IN1
,
IN2
,
OUT
>
extends
CoInvokable
<
IN1
,
IN2
,
OUT
>
{
private
static
final
long
serialVersionUID
=
1L
;
protected
Co
GroupFunction
<
IN1
,
IN2
,
OUT
>
coGroup
Function
;
protected
Co
WindowFunction
<
IN1
,
IN2
,
OUT
>
coWindow
Function
;
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
Co
GroupInvokable
(
CoGroupFunction
<
IN1
,
IN2
,
OUT
>
coGroup
Function
,
long
windowSize
,
public
Co
WindowInvokable
(
CoWindowFunction
<
IN1
,
IN2
,
OUT
>
coWindow
Function
,
long
windowSize
,
long
slideInterval
,
TimeStamp
<
IN1
>
timeStamp1
,
TimeStamp
<
IN2
>
timeStamp2
)
{
super
(
co
Group
Function
);
this
.
co
GroupFunction
=
coGroup
Function
;
super
(
co
Window
Function
);
this
.
co
WindowFunction
=
coWindow
Function
;
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
{
}
}
flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/state/CircularFifoList.java
浏览文件 @
ec82d973
...
...
@@ -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
();
...
...
flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoWindow
GroupReduce
Test.java
→
flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/invokable/operator/CoWindowTest.java
浏览文件 @
ec82d973
...
...
@@ -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
CoWindow
GroupReduceTest
{
public
class
CoWindow
Test
{
public
static
final
class
MyCoGroup1
implements
Co
Group
Function
<
Integer
,
Integer
,
Integer
>
{
public
static
final
class
MyCoGroup1
implements
Co
Window
Function
<
Integer
,
Integer
,
Integer
>
{
private
static
final
long
serialVersionUID
=
1L
;
@SuppressWarnings
(
"unused"
)
@Override
public
void
co
Group
(
Iterable
<
Integer
>
first
,
Iterable
<
Integer
>
second
,
Collector
<
Integer
>
out
)
throws
Exception
{
public
void
co
Window
(
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
Co
Group
Function
<
Tuple2
<
Integer
,
Integer
>,
Tuple2
<
Integer
,
Integer
>,
Integer
>
{
Co
Window
Function
<
Tuple2
<
Integer
,
Integer
>,
Tuple2
<
Integer
,
Integer
>,
Integer
>
{
private
static
final
long
serialVersionUID
=
1L
;
@Override
public
void
co
Group
(
Iterable
<
Tuple2
<
Integer
,
Integer
>>
first
,
Iterable
<
Tuple2
<
Integer
,
Integer
>>
second
,
Collector
<
Integer
>
out
)
throws
Exception
{
public
void
co
Window
(
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
{
Co
GroupInvokable
<
Integer
,
Integer
,
Integer
>
invokable1
=
new
CoGroup
Invokable
<
Integer
,
Integer
,
Integer
>(
Co
WindowInvokable
<
Integer
,
Integer
,
Integer
>
invokable1
=
new
CoWindow
Invokable
<
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
);
Co
GroupInvokable
<
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
));
Co
WindowInvokable
<
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
);
}
...
...
flink-addons/flink-streaming/flink-streaming-examples/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java
0 → 100644
浏览文件 @
ec82d973
/*
* 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.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录