Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
doujutun3207
flink
提交
f828657d
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,发现更多精彩内容 >>
提交
f828657d
编写于
10月 21, 2016
作者:
A
Aljoscha Krettek
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
[FLINK-4460] Add side outputs for ProcessFunction
上级
d4069159
变更
5
隐藏空白更改
内联
并排
Showing
5 changed file
with
329 addition
and
6 deletion
+329
-6
flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/sideoutput/SideOutputExample.java
...link/streaming/examples/sideoutput/SideOutputExample.java
+151
-0
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/ProcessFunction.java
...apache/flink/streaming/api/functions/ProcessFunction.java
+9
-0
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/KeyedProcessOperator.java
...e/flink/streaming/api/operators/KeyedProcessOperator.java
+17
-6
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ProcessOperator.java
...apache/flink/streaming/api/operators/ProcessOperator.java
+6
-0
flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SideOutputITCase.java
...apache/flink/test/streaming/runtime/SideOutputITCase.java
+146
-0
未找到文件。
flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/sideoutput/SideOutputExample.java
0 → 100644
浏览文件 @
f828657d
/*
* 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.sideoutput
;
import
org.apache.flink.api.common.functions.MapFunction
;
import
org.apache.flink.api.java.functions.KeySelector
;
import
org.apache.flink.streaming.api.TimeCharacteristic
;
import
org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator
;
import
org.apache.flink.streaming.api.functions.ProcessFunction
;
import
org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows
;
import
org.apache.flink.streaming.api.windowing.time.Time
;
import
org.apache.flink.util.OutputTag
;
import
org.apache.flink.api.java.tuple.Tuple2
;
import
org.apache.flink.api.java.utils.ParameterTool
;
import
org.apache.flink.examples.java.wordcount.util.WordCountData
;
import
org.apache.flink.streaming.api.datastream.DataStream
;
import
org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
;
import
org.apache.flink.util.Collector
;
/**
* An example that illustrates the use of side outputs.
*
* <p>This is a modified version of {@link org.apache.flink.streaming.examples.windowing.WindowWordCount}
* that has a filter in the tokenizer and only emits some words for counting
* while emitting the other words to a side output.
*/
public
class
SideOutputExample
{
/**
* We need to create an {@link OutputTag} so that we can reference it when emitting
* data to a side output and also to retrieve the side output stream from an operation.
*/
static
final
OutputTag
<
String
>
rejectedWordsTag
=
new
OutputTag
<
String
>(
"rejected"
)
{};
public
static
void
main
(
String
[]
args
)
throws
Exception
{
// Checking input parameters
final
ParameterTool
params
=
ParameterTool
.
fromArgs
(
args
);
// set up the execution environment
final
StreamExecutionEnvironment
env
=
StreamExecutionEnvironment
.
getExecutionEnvironment
();
env
.
setStreamTimeCharacteristic
(
TimeCharacteristic
.
IngestionTime
);
// make parameters available in the web interface
env
.
getConfig
().
setGlobalJobParameters
(
params
);
// get input data
DataStream
<
String
>
text
;
if
(
params
.
has
(
"input"
))
{
// read the text file from given input path
text
=
env
.
readTextFile
(
params
.
get
(
"input"
));
}
else
{
System
.
out
.
println
(
"Executing WordCount example with default input data set."
);
System
.
out
.
println
(
"Use --input to specify file input."
);
// get default test text data
text
=
env
.
fromElements
(
WordCountData
.
WORDS
);
}
SingleOutputStreamOperator
<
Tuple2
<
String
,
Integer
>>
tokenized
=
text
.
keyBy
(
new
KeySelector
<
String
,
Integer
>()
{
private
static
final
long
serialVersionUID
=
1L
;
@Override
public
Integer
getKey
(
String
value
)
throws
Exception
{
return
0
;
}
})
.
process
(
new
Tokenizer
());
DataStream
<
String
>
rejectedWords
=
tokenized
.
getSideOutput
(
rejectedWordsTag
)
.
map
(
new
MapFunction
<
String
,
String
>()
{
private
static
final
long
serialVersionUID
=
1L
;
@Override
public
String
map
(
String
value
)
throws
Exception
{
return
"rejected: "
+
value
;
}
});
DataStream
<
Tuple2
<
String
,
Integer
>>
counts
=
tokenized
.
keyBy
(
0
)
.
window
(
TumblingEventTimeWindows
.
of
(
Time
.
seconds
(
5
)))
// group by the tuple field "0" and sum up tuple field "1"
.
sum
(
1
);
// emit result
if
(
params
.
has
(
"output"
))
{
counts
.
writeAsText
(
params
.
get
(
"output"
));
rejectedWords
.
writeAsText
(
params
.
get
(
"rejected-words-output"
));
}
else
{
System
.
out
.
println
(
"Printing result to stdout. Use --output to specify output path."
);
counts
.
print
();
rejectedWords
.
print
();
}
// execute program
env
.
execute
(
"Streaming WordCount SideOutput"
);
}
// *************************************************************************
// USER FUNCTIONS
// *************************************************************************
/**
* Implements the string tokenizer that splits sentences into words as a
* user-defined FlatMapFunction. The function takes a line (String) and
* splits it into multiple pairs in the form of "(word,1)" ({@code Tuple2<String,
* Integer>}).
*
* <p>This rejects words that are longer than 5 characters long.
*/
public
static
final
class
Tokenizer
extends
ProcessFunction
<
String
,
Tuple2
<
String
,
Integer
>>
{
private
static
final
long
serialVersionUID
=
1L
;
@Override
public
void
processElement
(
String
value
,
Context
ctx
,
Collector
<
Tuple2
<
String
,
Integer
>>
out
)
throws
Exception
{
// normalize and split the line
String
[]
tokens
=
value
.
toLowerCase
().
split
(
"\\W+"
);
// emit the pairs
for
(
String
token
:
tokens
)
{
if
(
token
.
length
()
>
5
)
{
ctx
.
output
(
rejectedWordsTag
,
token
);
}
else
if
(
token
.
length
()
>
0
)
{
out
.
collect
(
new
Tuple2
<>(
token
,
1
));
}
}
}
}
}
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/ProcessFunction.java
浏览文件 @
f828657d
...
...
@@ -23,6 +23,7 @@ import org.apache.flink.api.common.functions.AbstractRichFunction;
import
org.apache.flink.streaming.api.TimeDomain
;
import
org.apache.flink.streaming.api.TimerService
;
import
org.apache.flink.util.Collector
;
import
org.apache.flink.util.OutputTag
;
/**
* A function that processes elements of a stream.
...
...
@@ -101,6 +102,14 @@ public abstract class ProcessFunction<I, O> extends AbstractRichFunction {
* A {@link TimerService} for querying time and registering timers.
*/
public
abstract
TimerService
timerService
();
/**
* 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/operators/KeyedProcessOperator.java
浏览文件 @
f828657d
...
...
@@ -25,6 +25,7 @@ import org.apache.flink.streaming.api.TimeDomain;
import
org.apache.flink.streaming.api.TimerService
;
import
org.apache.flink.streaming.api.functions.ProcessFunction
;
import
org.apache.flink.streaming.runtime.streamrecord.StreamRecord
;
import
org.apache.flink.util.OutputTag
;
import
static
org
.
apache
.
flink
.
util
.
Preconditions
.
checkNotNull
;
import
static
org
.
apache
.
flink
.
util
.
Preconditions
.
checkState
;
...
...
@@ -38,9 +39,9 @@ public class KeyedProcessOperator<K, IN, OUT>
private
transient
TimestampedCollector
<
OUT
>
collector
;
private
transient
ContextImpl
<
IN
,
OUT
>
context
;
private
transient
ContextImpl
context
;
private
transient
OnTimerContextImpl
<
IN
,
OUT
>
onTimerContext
;
private
transient
OnTimerContextImpl
onTimerContext
;
public
KeyedProcessOperator
(
ProcessFunction
<
IN
,
OUT
>
function
)
{
super
(
function
);
...
...
@@ -58,8 +59,8 @@ public class KeyedProcessOperator<K, IN, OUT>
TimerService
timerService
=
new
SimpleTimerService
(
internalTimerService
);
context
=
new
ContextImpl
<>
(
userFunction
,
timerService
);
onTimerContext
=
new
OnTimerContextImpl
<>
(
userFunction
,
timerService
);
context
=
new
ContextImpl
(
userFunction
,
timerService
);
onTimerContext
=
new
OnTimerContextImpl
(
userFunction
,
timerService
);
}
@Override
...
...
@@ -90,7 +91,7 @@ public class KeyedProcessOperator<K, IN, OUT>
context
.
element
=
null
;
}
private
static
class
ContextImpl
<
IN
,
OUT
>
extends
ProcessFunction
<
IN
,
OUT
>.
Context
{
private
class
ContextImpl
extends
ProcessFunction
<
IN
,
OUT
>.
Context
{
private
final
TimerService
timerService
;
...
...
@@ -112,13 +113,18 @@ public class KeyedProcessOperator<K, IN, OUT>
}
}
@Override
public
<
X
>
void
output
(
OutputTag
<
X
>
outputTag
,
X
value
)
{
output
.
collect
(
outputTag
,
new
StreamRecord
<>(
value
,
element
.
getTimestamp
()));
}
@Override
public
TimerService
timerService
()
{
return
timerService
;
}
}
private
static
class
OnTimerContextImpl
<
IN
,
OUT
>
extends
ProcessFunction
<
IN
,
OUT
>.
OnTimerContext
{
private
class
OnTimerContextImpl
extends
ProcessFunction
<
IN
,
OUT
>.
OnTimerContext
{
private
final
TimerService
timerService
;
...
...
@@ -143,6 +149,11 @@ public class KeyedProcessOperator<K, IN, OUT>
return
timer
.
getTimestamp
();
}
@Override
public
<
X
>
void
output
(
OutputTag
<
X
>
outputTag
,
X
value
)
{
output
.
collect
(
outputTag
,
new
StreamRecord
<>(
value
,
timer
.
getTimestamp
()));
}
@Override
public
TimerService
timerService
()
{
return
timerService
;
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ProcessOperator.java
浏览文件 @
f828657d
...
...
@@ -23,6 +23,7 @@ import org.apache.flink.streaming.api.functions.ProcessFunction;
import
org.apache.flink.streaming.api.watermark.Watermark
;
import
org.apache.flink.streaming.runtime.streamrecord.StreamRecord
;
import
org.apache.flink.streaming.runtime.tasks.ProcessingTimeService
;
import
org.apache.flink.util.OutputTag
;
import
static
org
.
apache
.
flink
.
util
.
Preconditions
.
checkState
;
...
...
@@ -92,6 +93,11 @@ public class ProcessOperator<IN, OUT>
}
}
@Override
public
<
X
>
void
output
(
OutputTag
<
X
>
outputTag
,
X
value
)
{
output
.
collect
(
outputTag
,
new
StreamRecord
<>(
value
,
element
.
getTimestamp
()));
}
@Override
public
long
currentProcessingTime
()
{
return
processingTimeService
.
getCurrentProcessingTime
();
...
...
flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SideOutputITCase.java
浏览文件 @
f828657d
...
...
@@ -18,6 +18,7 @@
package
org.apache.flink.test.streaming.runtime
;
import
org.apache.flink.api.common.functions.FlatMapFunction
;
import
org.apache.flink.streaming.api.functions.ProcessFunction
;
import
org.apache.flink.util.OutputTag
;
import
org.apache.flink.api.java.functions.KeySelector
;
import
org.apache.flink.streaming.api.TimeCharacteristic
;
...
...
@@ -57,6 +58,151 @@ public class SideOutputITCase extends StreamingMultipleProgramsTestBase {
elements
.
add
(
4
);
}
private
final
static
OutputTag
<
String
>
sideOutputTag
=
new
OutputTag
<
String
>(
"side"
){};
private
final
static
OutputTag
<
String
>
otherSideOutputTag
=
new
OutputTag
<
String
>(
"other-side"
){};
/**
* Test ProcessFunction side output.
*/
@Test
public
void
testProcessFunctionSideOutput
()
throws
Exception
{
TestListResultSink
<
String
>
sideOutputResultSink
=
new
TestListResultSink
<>();
TestListResultSink
<
Integer
>
resultSink
=
new
TestListResultSink
<>();
StreamExecutionEnvironment
see
=
StreamExecutionEnvironment
.
getExecutionEnvironment
();
see
.
setParallelism
(
3
);
DataStream
<
Integer
>
dataStream
=
see
.
fromCollection
(
elements
);
SingleOutputStreamOperator
<
Integer
>
passThroughtStream
=
dataStream
.
process
(
new
ProcessFunction
<
Integer
,
Integer
>()
{
private
static
final
long
serialVersionUID
=
1L
;
@Override
public
void
processElement
(
Integer
value
,
Context
ctx
,
Collector
<
Integer
>
out
)
throws
Exception
{
out
.
collect
(
value
);
ctx
.
output
(
sideOutputTag
,
"sideout-"
+
String
.
valueOf
(
value
));
}
});
passThroughtStream
.
getSideOutput
(
sideOutputTag
).
addSink
(
sideOutputResultSink
);
passThroughtStream
.
addSink
(
resultSink
);
see
.
execute
();
assertEquals
(
Arrays
.
asList
(
"sideout-1"
,
"sideout-2"
,
"sideout-3"
,
"sideout-4"
,
"sideout-5"
),
sideOutputResultSink
.
getSortedResult
());
assertEquals
(
Arrays
.
asList
(
1
,
2
,
3
,
4
,
5
),
resultSink
.
getSortedResult
());
}
/**
* Test keyed ProcessFunction side output.
*/
@Test
public
void
testKeyedProcessFunctionSideOutput
()
throws
Exception
{
TestListResultSink
<
String
>
sideOutputResultSink
=
new
TestListResultSink
<>();
TestListResultSink
<
Integer
>
resultSink
=
new
TestListResultSink
<>();
StreamExecutionEnvironment
see
=
StreamExecutionEnvironment
.
getExecutionEnvironment
();
see
.
setParallelism
(
3
);
DataStream
<
Integer
>
dataStream
=
see
.
fromCollection
(
elements
);
SingleOutputStreamOperator
<
Integer
>
passThroughtStream
=
dataStream
.
keyBy
(
new
KeySelector
<
Integer
,
Integer
>()
{
private
static
final
long
serialVersionUID
=
1L
;
@Override
public
Integer
getKey
(
Integer
value
)
throws
Exception
{
return
value
;
}
})
.
process
(
new
ProcessFunction
<
Integer
,
Integer
>()
{
private
static
final
long
serialVersionUID
=
1L
;
@Override
public
void
processElement
(
Integer
value
,
Context
ctx
,
Collector
<
Integer
>
out
)
throws
Exception
{
out
.
collect
(
value
);
ctx
.
output
(
sideOutputTag
,
"sideout-"
+
String
.
valueOf
(
value
));
}
});
passThroughtStream
.
getSideOutput
(
sideOutputTag
).
addSink
(
sideOutputResultSink
);
passThroughtStream
.
addSink
(
resultSink
);
see
.
execute
();
assertEquals
(
Arrays
.
asList
(
"sideout-1"
,
"sideout-2"
,
"sideout-3"
,
"sideout-4"
,
"sideout-5"
),
sideOutputResultSink
.
getSortedResult
());
assertEquals
(
Arrays
.
asList
(
1
,
2
,
3
,
4
,
5
),
resultSink
.
getSortedResult
());
}
/**
* Test ProcessFunction side outputs with wrong {@code OutputTag}.
*/
@Test
public
void
testProcessFunctionSideOutputWithWrongTag
()
throws
Exception
{
TestListResultSink
<
String
>
sideOutputResultSink
=
new
TestListResultSink
<>();
StreamExecutionEnvironment
see
=
StreamExecutionEnvironment
.
getExecutionEnvironment
();
see
.
setParallelism
(
3
);
DataStream
<
Integer
>
dataStream
=
see
.
fromCollection
(
elements
);
dataStream
.
process
(
new
ProcessFunction
<
Integer
,
Integer
>()
{
private
static
final
long
serialVersionUID
=
1L
;
@Override
public
void
processElement
(
Integer
value
,
Context
ctx
,
Collector
<
Integer
>
out
)
throws
Exception
{
out
.
collect
(
value
);
ctx
.
output
(
otherSideOutputTag
,
"sideout-"
+
String
.
valueOf
(
value
));
}
}).
getSideOutput
(
sideOutputTag
).
addSink
(
sideOutputResultSink
);
see
.
execute
();
assertEquals
(
Arrays
.
asList
(),
sideOutputResultSink
.
getSortedResult
());
}
/**
* Test keyed ProcessFunction side outputs with wrong {@code OutputTag}.
*/
@Test
public
void
testKeyedProcessFunctionSideOutputWithWrongTag
()
throws
Exception
{
TestListResultSink
<
String
>
sideOutputResultSink
=
new
TestListResultSink
<>();
StreamExecutionEnvironment
see
=
StreamExecutionEnvironment
.
getExecutionEnvironment
();
see
.
setParallelism
(
3
);
DataStream
<
Integer
>
dataStream
=
see
.
fromCollection
(
elements
);
dataStream
.
keyBy
(
new
KeySelector
<
Integer
,
Integer
>()
{
private
static
final
long
serialVersionUID
=
1L
;
@Override
public
Integer
getKey
(
Integer
value
)
throws
Exception
{
return
value
;
}
})
.
process
(
new
ProcessFunction
<
Integer
,
Integer
>()
{
private
static
final
long
serialVersionUID
=
1L
;
@Override
public
void
processElement
(
Integer
value
,
Context
ctx
,
Collector
<
Integer
>
out
)
throws
Exception
{
out
.
collect
(
value
);
ctx
.
output
(
otherSideOutputTag
,
"sideout-"
+
String
.
valueOf
(
value
));
}
}).
getSideOutput
(
sideOutputTag
).
addSink
(
sideOutputResultSink
);
see
.
execute
();
assertEquals
(
Arrays
.
asList
(),
sideOutputResultSink
.
getSortedResult
());
}
private
static
class
TestWatermarkAssigner
implements
AssignerWithPunctuatedWatermarks
<
Integer
>
{
private
static
final
long
serialVersionUID
=
1L
;
...
...
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录