Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
doujutun3207
flink
提交
39682c45
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,发现更多精彩内容 >>
提交
39682c45
编写于
9月 27, 2017
作者:
B
Bowen Li
提交者:
Aljoscha Krettek
10月 12, 2017
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
[FLINK-7660] Support sideOutput in ProcessAllWindowFunction
上级
89de78c7
变更
10
隐藏空白更改
内联
并排
Showing
10 changed file
with
121 addition
and
13 deletion
+121
-13
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldApplyProcessAllWindowFunction.java
...unctions/windowing/FoldApplyProcessAllWindowFunction.java
+2
-5
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/InternalProcessApplyAllWindowContext.java
...tions/windowing/InternalProcessApplyAllWindowContext.java
+9
-4
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ProcessAllWindowFunction.java
...ing/api/functions/windowing/ProcessAllWindowFunction.java
+9
-0
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceApplyProcessAllWindowFunction.java
...ctions/windowing/ReduceApplyProcessAllWindowFunction.java
+2
-4
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
...streaming/runtime/operators/windowing/WindowOperator.java
+1
-0
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalProcessAllWindowContext.java
.../windowing/functions/InternalProcessAllWindowContext.java
+6
-0
flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/ProcessAllWindowFunction.scala
...reaming/api/scala/function/ProcessAllWindowFunction.scala
+6
-0
flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/util/ScalaProcessWindowFunctionWrapper.scala
...ala/function/util/ScalaProcessWindowFunctionWrapper.scala
+4
-0
flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/SideOutputITCase.scala
...g/apache/flink/streaming/api/scala/SideOutputITCase.scala
+45
-0
flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SideOutputITCase.java
...apache/flink/test/streaming/runtime/SideOutputITCase.java
+37
-0
未找到文件。
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/FoldApplyProcessAllWindowFunction.java
浏览文件 @
39682c45
...
...
@@ -106,8 +106,7 @@ public class FoldApplyProcessAllWindowFunction<W extends Window, T, ACC, R>
}
this
.
ctx
.
window
=
context
.
window
();
this
.
ctx
.
windowState
=
context
.
windowState
();
this
.
ctx
.
globalState
=
context
.
globalState
();
this
.
ctx
.
context
=
context
;
windowFunction
.
process
(
ctx
,
Collections
.
singletonList
(
result
),
out
);
}
...
...
@@ -115,8 +114,7 @@ public class FoldApplyProcessAllWindowFunction<W extends Window, T, ACC, R>
@Override
public
void
clear
(
final
Context
context
)
throws
Exception
{
this
.
ctx
.
window
=
context
.
window
();
this
.
ctx
.
windowState
=
context
.
windowState
();
this
.
ctx
.
globalState
=
context
.
globalState
();
this
.
ctx
.
context
=
context
;
windowFunction
.
clear
(
ctx
);
}
...
...
@@ -136,5 +134,4 @@ public class FoldApplyProcessAllWindowFunction<W extends Window, T, ACC, R>
serializedInitialValue
=
baos
.
toByteArray
();
}
}
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/InternalProcessApplyAllWindowContext.java
浏览文件 @
39682c45
...
...
@@ -21,6 +21,7 @@ package org.apache.flink.streaming.api.functions.windowing;
import
org.apache.flink.annotation.Internal
;
import
org.apache.flink.api.common.state.KeyedStateStore
;
import
org.apache.flink.streaming.api.windowing.windows.Window
;
import
org.apache.flink.util.OutputTag
;
/**
* Internal reusable context wrapper.
...
...
@@ -34,8 +35,7 @@ public class InternalProcessApplyAllWindowContext<IN, OUT, W extends Window>
extends
ProcessAllWindowFunction
<
IN
,
OUT
,
W
>.
Context
{
W
window
;
KeyedStateStore
windowState
;
KeyedStateStore
globalState
;
ProcessAllWindowFunction
.
Context
context
;
InternalProcessApplyAllWindowContext
(
ProcessAllWindowFunction
<
IN
,
OUT
,
W
>
function
)
{
function
.
super
();
...
...
@@ -48,11 +48,16 @@ public class InternalProcessApplyAllWindowContext<IN, OUT, W extends Window>
@Override
public
KeyedStateStore
windowState
()
{
return
windowState
;
return
context
.
windowState
()
;
}
@Override
public
KeyedStateStore
globalState
()
{
return
globalState
;
return
context
.
globalState
();
}
@Override
public
<
X
>
void
output
(
OutputTag
<
X
>
outputTag
,
X
value
)
{
context
.
output
(
outputTag
,
value
);
}
}
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ProcessAllWindowFunction.java
浏览文件 @
39682c45
...
...
@@ -23,6 +23,7 @@ import org.apache.flink.api.common.functions.AbstractRichFunction;
import
org.apache.flink.api.common.state.KeyedStateStore
;
import
org.apache.flink.streaming.api.windowing.windows.Window
;
import
org.apache.flink.util.Collector
;
import
org.apache.flink.util.OutputTag
;
/**
* Base abstract class for functions that are evaluated over non-keyed windows using a context
...
...
@@ -77,5 +78,13 @@ public abstract class ProcessAllWindowFunction<IN, OUT, W extends Window> extend
* State accessor for per-key global state.
*/
public
abstract
KeyedStateStore
globalState
();
/**
* Emits a record to the side output identified by the {@link OutputTag}.
*
* @param outputTag the {@code OutputTag} that identifies the side output to emit to.
* @param value The record to emit.
*/
public
abstract
<
X
>
void
output
(
OutputTag
<
X
>
outputTag
,
X
value
);
}
}
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceApplyProcessAllWindowFunction.java
浏览文件 @
39682c45
...
...
@@ -60,8 +60,7 @@ public class ReduceApplyProcessAllWindowFunction<W extends Window, T, R> extends
}
this
.
ctx
.
window
=
context
.
window
();
this
.
ctx
.
windowState
=
context
.
windowState
();
this
.
ctx
.
globalState
=
context
.
globalState
();
this
.
ctx
.
context
=
context
;
windowFunction
.
process
(
ctx
,
Collections
.
singletonList
(
curr
),
out
);
}
...
...
@@ -69,8 +68,7 @@ public class ReduceApplyProcessAllWindowFunction<W extends Window, T, R> extends
@Override
public
void
clear
(
final
Context
context
)
throws
Exception
{
this
.
ctx
.
window
=
context
.
window
();
this
.
ctx
.
windowState
=
context
.
windowState
();
this
.
ctx
.
globalState
=
context
.
globalState
();
this
.
ctx
.
context
=
context
;
windowFunction
.
clear
(
ctx
);
}
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
浏览文件 @
39682c45
...
...
@@ -775,6 +775,7 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
return
WindowOperator
.
this
.
getKeyedStateStore
();
}
@Override
public
<
X
>
void
output
(
OutputTag
<
X
>
outputTag
,
X
value
)
{
if
(
outputTag
==
null
)
{
throw
new
IllegalArgumentException
(
"OutputTag must not be null."
);
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalProcessAllWindowContext.java
浏览文件 @
39682c45
...
...
@@ -22,6 +22,7 @@ import org.apache.flink.annotation.Internal;
import
org.apache.flink.api.common.state.KeyedStateStore
;
import
org.apache.flink.streaming.api.functions.windowing.ProcessAllWindowFunction
;
import
org.apache.flink.streaming.api.windowing.windows.Window
;
import
org.apache.flink.util.OutputTag
;
/**
* Internal reusable context wrapper.
...
...
@@ -55,4 +56,9 @@ public class InternalProcessAllWindowContext<IN, OUT, W extends Window>
public
KeyedStateStore
globalState
()
{
return
internalContext
.
globalState
();
}
@Override
public
<
X
>
void
output
(
OutputTag
<
X
>
outputTag
,
X
value
)
{
internalContext
.
output
(
outputTag
,
value
);
}
}
flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/ProcessAllWindowFunction.scala
浏览文件 @
39682c45
...
...
@@ -21,6 +21,7 @@ package org.apache.flink.streaming.api.scala.function
import
org.apache.flink.annotation.PublicEvolving
import
org.apache.flink.api.common.functions.AbstractRichFunction
import
org.apache.flink.api.common.state.KeyedStateStore
import
org.apache.flink.streaming.api.scala.OutputTag
import
org.apache.flink.streaming.api.windowing.windows.Window
import
org.apache.flink.util.Collector
...
...
@@ -73,6 +74,11 @@ abstract class ProcessAllWindowFunction[IN, OUT, W <: Window]
* State accessor for per-key global state.
*/
def
globalState
:
KeyedStateStore
/**
* Emits a record to the side output identified by the [[OutputTag]].
*/
def
output
[
X
](
outputTag
:
OutputTag
[
X
],
value
:
X
)
}
}
flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/util/ScalaProcessWindowFunctionWrapper.scala
浏览文件 @
39682c45
...
...
@@ -127,6 +127,8 @@ final class ScalaProcessAllWindowFunctionWrapper[IN, OUT, W <: Window](
override
def
windowState
=
context
.
windowState
()
override
def
globalState
=
context
.
globalState
()
override
def
output
[
X
](
outputTag
:
OutputTag
[
X
],
value
:
X
)
=
context
.
output
(
outputTag
,
value
)
}
func
.
process
(
ctx
,
elements
.
asScala
,
out
)
}
...
...
@@ -138,6 +140,8 @@ final class ScalaProcessAllWindowFunctionWrapper[IN, OUT, W <: Window](
override
def
windowState
=
context
.
windowState
()
override
def
globalState
=
context
.
globalState
()
override
def
output
[
X
](
outputTag
:
OutputTag
[
X
],
value
:
X
)
=
context
.
output
(
outputTag
,
value
)
}
func
.
clear
(
ctx
)
}
...
...
flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/SideOutputITCase.scala
浏览文件 @
39682c45
...
...
@@ -280,6 +280,51 @@ class SideOutputITCase extends StreamingMultipleProgramsTestBase {
assertEquals
(
util
.
Arrays
.
asList
(
"sideout-1"
,
"sideout-2"
,
"sideout-5"
),
sideOutputResultSink
.
getResult
)
}
/**
* Test ProcessAllWindowFunction side output.
*/
@Test
def
testProcessAllWindowFunctionSideOutput
()
{
val
resultSink
=
new
TestListResultSink
[
String
]
val
sideOutputResultSink
=
new
TestListResultSink
[
String
]
val
env
=
StreamExecutionEnvironment
.
getExecutionEnvironment
env
.
setParallelism
(
1
)
env
.
setStreamTimeCharacteristic
(
TimeCharacteristic
.
EventTime
)
val
dataStream
=
env
.
fromElements
((
"1"
,
1
),
(
"2"
,
2
),
(
"5"
,
5
),
(
"3"
,
3
),
(
"4"
,
4
))
val
sideOutputTag
=
OutputTag
[
String
](
"side"
)
val
windowOperator
=
dataStream
.
assignTimestampsAndWatermarks
(
new
TestAssigner
)
.
windowAll
(
TumblingEventTimeWindows
.
of
(
Time
.
milliseconds
(
1
)))
.
process
(
new
ProcessAllWindowFunction
[(
String
,
Int
)
,
String
,
TimeWindow
]
{
override
def
process
(
context
:
Context
,
elements
:
Iterable
[(
String
,
Int
)],
out
:
Collector
[
String
])
:
Unit
=
{
for
(
in
<-
elements
)
{
out
.
collect
(
in
.
_1
)
context
.
output
(
sideOutputTag
,
"sideout-"
+
in
.
_1
)
}
}
})
windowOperator
.
getSideOutput
(
sideOutputTag
)
.
addSink
(
sideOutputResultSink
)
windowOperator
.
addSink
(
resultSink
)
env
.
execute
()
assertEquals
(
util
.
Arrays
.
asList
(
"1"
,
"2"
,
"5"
),
resultSink
.
getResult
)
assertEquals
(
util
.
Arrays
.
asList
(
"sideout-1"
,
"sideout-2"
,
"sideout-5"
),
sideOutputResultSink
.
getResult
)
}
}
class
TestAssigner
extends
AssignerWithPunctuatedWatermarks
[(
String
,
Int
)]
{
...
...
flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SideOutputITCase.java
浏览文件 @
39682c45
...
...
@@ -29,6 +29,7 @@ import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks
import
org.apache.flink.streaming.api.functions.ProcessFunction
;
import
org.apache.flink.streaming.api.functions.source.SourceFunction
;
import
org.apache.flink.streaming.api.functions.windowing.AllWindowFunction
;
import
org.apache.flink.streaming.api.functions.windowing.ProcessAllWindowFunction
;
import
org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction
;
import
org.apache.flink.streaming.api.functions.windowing.WindowFunction
;
import
org.apache.flink.streaming.api.operators.AbstractStreamOperator
;
...
...
@@ -582,4 +583,40 @@ public class SideOutputITCase extends StreamingMultipleProgramsTestBase implemen
assertEquals
(
Arrays
.
asList
(
"sideout-1"
,
"sideout-2"
,
"sideout-5"
),
sideOutputResultSink
.
getSortedResult
());
assertEquals
(
Arrays
.
asList
(
1
,
2
,
5
),
resultSink
.
getSortedResult
());
}
@Test
public
void
testProcessAllWindowFunctionSideOutput
()
throws
Exception
{
TestListResultSink
<
Integer
>
resultSink
=
new
TestListResultSink
<>();
TestListResultSink
<
String
>
sideOutputResultSink
=
new
TestListResultSink
<>();
StreamExecutionEnvironment
see
=
StreamExecutionEnvironment
.
getExecutionEnvironment
();
see
.
setParallelism
(
1
);
see
.
setStreamTimeCharacteristic
(
TimeCharacteristic
.
EventTime
);
DataStream
<
Integer
>
dataStream
=
see
.
fromCollection
(
elements
);
OutputTag
<
String
>
sideOutputTag
=
new
OutputTag
<
String
>(
"side"
){};
SingleOutputStreamOperator
<
Integer
>
windowOperator
=
dataStream
.
assignTimestampsAndWatermarks
(
new
TestWatermarkAssigner
())
.
timeWindowAll
(
Time
.
milliseconds
(
1
),
Time
.
milliseconds
(
1
))
.
process
(
new
ProcessAllWindowFunction
<
Integer
,
Integer
,
TimeWindow
>()
{
private
static
final
long
serialVersionUID
=
1L
;
@Override
public
void
process
(
Context
context
,
Iterable
<
Integer
>
elements
,
Collector
<
Integer
>
out
)
throws
Exception
{
for
(
Integer
e
:
elements
)
{
out
.
collect
(
e
);
context
.
output
(
sideOutputTag
,
"sideout-"
+
String
.
valueOf
(
e
));
}
}
});
windowOperator
.
getSideOutput
(
sideOutputTag
).
addSink
(
sideOutputResultSink
);
windowOperator
.
addSink
(
resultSink
);
see
.
execute
();
assertEquals
(
Arrays
.
asList
(
"sideout-1"
,
"sideout-2"
,
"sideout-5"
),
sideOutputResultSink
.
getSortedResult
());
assertEquals
(
Arrays
.
asList
(
1
,
2
,
5
),
resultSink
.
getSortedResult
());
}
}
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录