Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
doujutun3207
flink
提交
456d0aba
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,发现更多精彩内容 >>
提交
456d0aba
编写于
2月 02, 2016
作者:
A
Aljoscha Krettek
提交者:
Stephan Ewen
2月 03, 2016
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
[FLINK-3200] Fix Triggers by introducing clear() method to clean up state/triggers
上级
e4d05f72
变更
12
隐藏空白更改
内联
并排
Showing
12 changed file
with
157 addition
and
38 deletion
+157
-38
flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java
.../flink/streaming/examples/windowing/SessionWindowing.java
+16
-3
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/GlobalWindows.java
...link/streaming/api/windowing/assigners/GlobalWindows.java
+3
-0
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousEventTimeTrigger.java
...ng/api/windowing/triggers/ContinuousEventTimeTrigger.java
+3
-0
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousProcessingTimeTrigger.java
...i/windowing/triggers/ContinuousProcessingTimeTrigger.java
+3
-0
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/CountTrigger.java
.../flink/streaming/api/windowing/triggers/CountTrigger.java
+5
-0
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/DeltaTrigger.java
.../flink/streaming/api/windowing/triggers/DeltaTrigger.java
+5
-0
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/EventTimeTrigger.java
...nk/streaming/api/windowing/triggers/EventTimeTrigger.java
+5
-0
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ProcessingTimeTrigger.java
...reaming/api/windowing/triggers/ProcessingTimeTrigger.java
+5
-0
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/PurgingTrigger.java
...link/streaming/api/windowing/triggers/PurgingTrigger.java
+5
-0
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/Trigger.java
...pache/flink/streaming/api/windowing/triggers/Trigger.java
+17
-0
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperator.java
...g/runtime/operators/windowing/NonKeyedWindowOperator.java
+25
-0
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
...streaming/runtime/operators/windowing/WindowOperator.java
+65
-35
未找到文件。
flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java
浏览文件 @
456d0aba
...
...
@@ -45,7 +45,7 @@ public class SessionWindowing {
final
StreamExecutionEnvironment
env
=
StreamExecutionEnvironment
.
getExecutionEnvironment
();
env
.
setStreamTimeCharacteristic
(
TimeCharacteristic
.
EventTime
);
env
.
setParallelism
(
2
);
env
.
setParallelism
(
1
);
final
List
<
Tuple3
<
String
,
Long
,
Integer
>>
input
=
new
ArrayList
<>();
...
...
@@ -103,7 +103,7 @@ public class SessionWindowing {
private
final
Long
sessionTimeout
;
private
final
ValueStateDescriptor
<
Long
>
stateDesc
=
new
ValueStateDescriptor
<>(
"last-seen"
,
1L
,
private
final
ValueStateDescriptor
<
Long
>
stateDesc
=
new
ValueStateDescriptor
<>(
"last-seen"
,
-
1L
,
BasicTypeInfo
.
LONG_TYPE_INFO
.
createSerializer
(
new
ExecutionConfig
()));
...
...
@@ -120,12 +120,15 @@ public class SessionWindowing {
Long
timeSinceLastEvent
=
timestamp
-
lastSeen
;
ctx
.
deleteEventTimeTimer
(
lastSeen
+
sessionTimeout
);
// Update the last seen event time
lastSeenState
.
update
(
timestamp
);
ctx
.
registerEventTimeTimer
(
timestamp
+
sessionTimeout
);
if
(
timeSinceLastEvent
>
sessionTimeout
)
{
if
(
lastSeen
!=
-
1
&&
timeSinceLastEvent
>
sessionTimeout
)
{
System
.
out
.
println
(
"FIRING ON ELEMENT: "
+
element
+
" ts: "
+
timestamp
+
" last "
+
lastSeen
);
return
TriggerResult
.
FIRE_AND_PURGE
;
}
else
{
return
TriggerResult
.
CONTINUE
;
...
...
@@ -138,6 +141,7 @@ public class SessionWindowing {
Long
lastSeen
=
lastSeenState
.
value
();
if
(
time
-
lastSeen
>=
sessionTimeout
)
{
System
.
out
.
println
(
"CTX: "
+
ctx
+
" Firing Time "
+
time
+
" last seen "
+
lastSeen
);
return
TriggerResult
.
FIRE_AND_PURGE
;
}
return
TriggerResult
.
CONTINUE
;
...
...
@@ -147,6 +151,15 @@ public class SessionWindowing {
public
TriggerResult
onProcessingTime
(
long
time
,
GlobalWindow
window
,
TriggerContext
ctx
)
throws
Exception
{
return
TriggerResult
.
CONTINUE
;
}
@Override
public
void
clear
(
GlobalWindow
window
,
TriggerContext
ctx
)
throws
Exception
{
ValueState
<
Long
>
lastSeenState
=
ctx
.
getPartitionedState
(
stateDesc
);
if
(
lastSeenState
.
value
()
!=
-
1
)
{
ctx
.
deleteEventTimeTimer
(
lastSeenState
.
value
()
+
sessionTimeout
);
}
lastSeenState
.
clear
();
}
}
// *************************************************************************
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/GlobalWindows.java
浏览文件 @
456d0aba
...
...
@@ -87,6 +87,9 @@ public class GlobalWindows extends WindowAssigner<Object, GlobalWindow> {
public
TriggerResult
onProcessingTime
(
long
time
,
GlobalWindow
window
,
TriggerContext
ctx
)
{
return
TriggerResult
.
CONTINUE
;
}
@Override
public
void
clear
(
GlobalWindow
window
,
TriggerContext
ctx
)
throws
Exception
{}
}
@Override
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousEventTimeTrigger.java
浏览文件 @
456d0aba
...
...
@@ -73,6 +73,9 @@ public class ContinuousEventTimeTrigger<W extends Window> implements Trigger<Obj
return
TriggerResult
.
CONTINUE
;
}
@Override
public
void
clear
(
W
window
,
TriggerContext
ctx
)
throws
Exception
{}
@Override
public
String
toString
()
{
return
"ContinuousProcessingTimeTrigger("
+
interval
+
")"
;
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousProcessingTimeTrigger.java
浏览文件 @
456d0aba
...
...
@@ -90,6 +90,9 @@ public class ContinuousProcessingTimeTrigger<W extends Window> implements Trigge
return
TriggerResult
.
CONTINUE
;
}
@Override
public
void
clear
(
W
window
,
TriggerContext
ctx
)
throws
Exception
{}
@VisibleForTesting
public
long
getInterval
()
{
return
interval
;
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/CountTrigger.java
浏览文件 @
456d0aba
...
...
@@ -65,6 +65,11 @@ public class CountTrigger<W extends Window> implements Trigger<Object, W> {
return
TriggerResult
.
CONTINUE
;
}
@Override
public
void
clear
(
W
window
,
TriggerContext
ctx
)
throws
Exception
{
ctx
.
getPartitionedState
(
stateDesc
).
clear
();
}
@Override
public
String
toString
()
{
return
"CountTrigger("
+
maxCount
+
")"
;
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/DeltaTrigger.java
浏览文件 @
456d0aba
...
...
@@ -71,6 +71,11 @@ public class DeltaTrigger<T, W extends Window> implements Trigger<T, W> {
return
TriggerResult
.
CONTINUE
;
}
@Override
public
void
clear
(
W
window
,
TriggerContext
ctx
)
throws
Exception
{
ctx
.
getPartitionedState
(
stateDesc
).
clear
();
}
@Override
public
String
toString
()
{
return
"DeltaTrigger("
+
deltaFunction
+
", "
+
threshold
+
")"
;
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/EventTimeTrigger.java
浏览文件 @
456d0aba
...
...
@@ -46,6 +46,11 @@ public class EventTimeTrigger implements Trigger<Object, TimeWindow> {
return
TriggerResult
.
CONTINUE
;
}
@Override
public
void
clear
(
TimeWindow
window
,
TriggerContext
ctx
)
throws
Exception
{
ctx
.
deleteEventTimeTimer
(
window
.
maxTimestamp
());
}
@Override
public
String
toString
()
{
return
"EventTimeTrigger()"
;
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ProcessingTimeTrigger.java
浏览文件 @
456d0aba
...
...
@@ -44,6 +44,11 @@ public class ProcessingTimeTrigger implements Trigger<Object, TimeWindow> {
return
TriggerResult
.
FIRE_AND_PURGE
;
}
@Override
public
void
clear
(
TimeWindow
window
,
TriggerContext
ctx
)
throws
Exception
{
ctx
.
deleteProcessingTimeTimer
(
window
.
maxTimestamp
());
}
@Override
public
String
toString
()
{
return
"ProcessingTimeTrigger()"
;
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/PurgingTrigger.java
浏览文件 @
456d0aba
...
...
@@ -78,6 +78,11 @@ public class PurgingTrigger<T, W extends Window> implements Trigger<T, W> {
}
}
@Override
public
void
clear
(
W
window
,
TriggerContext
ctx
)
throws
Exception
{
nestedTrigger
.
clear
(
window
,
ctx
);
}
@Override
public
String
toString
()
{
return
"PurgingTrigger("
+
nestedTrigger
.
toString
()
+
")"
;
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/Trigger.java
浏览文件 @
456d0aba
...
...
@@ -73,6 +73,13 @@ public interface Trigger<T, W extends Window> extends Serializable {
*/
TriggerResult
onEventTime
(
long
time
,
W
window
,
TriggerContext
ctx
)
throws
Exception
;
/**
* Clears any state that the trigger might still hold for the given window. This is called
* when a window is purged. Timers set using {@link TriggerContext#registerEventTimeTimer(long)}
* and {@link TriggerContext#registerProcessingTimeTimer(long)} should be deleted here as
* well as state acquired using {@link TriggerContext#getPartitionedState(StateDescriptor)}.
*/
void
clear
(
W
window
,
TriggerContext
ctx
)
throws
Exception
;
/**
* Result type for trigger methods. This determines what happens with the window.
...
...
@@ -150,6 +157,16 @@ public interface Trigger<T, W extends Window> extends Serializable {
*/
void
registerEventTimeTimer
(
long
time
);
/**
* Delete the processing time trigger for the given time.
*/
void
deleteProcessingTimeTimer
(
long
time
);
/**
* Delete the event-time trigger for the given time.
*/
void
deleteEventTimeTimer
(
long
time
);
/**
* Retrieves an {@link State} object that can be used to interact with
* fault-tolerant state that is scoped to the window and key of the current
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperator.java
浏览文件 @
456d0aba
...
...
@@ -284,6 +284,10 @@ public class NonKeyedWindowOperator<IN, OUT, W extends Window>
if
(
triggerResult
.
isFire
())
{
emitWindow
(
context
);
}
if
(
triggerResult
.
isPurge
())
{
context
.
clear
();
}
}
@Override
...
...
@@ -516,6 +520,23 @@ public class NonKeyedWindowOperator<IN, OUT, W extends Window>
triggers
.
add
(
this
);
}
@Override
public
void
deleteProcessingTimeTimer
(
long
time
)
{
Set
<
Context
>
triggers
=
processingTimeTimers
.
get
(
time
);
if
(
triggers
!=
null
)
{
triggers
.
remove
(
this
);
}
}
@Override
public
void
deleteEventTimeTimer
(
long
time
)
{
Set
<
Context
>
triggers
=
watermarkTimers
.
get
(
time
);
if
(
triggers
!=
null
)
{
triggers
.
remove
(
this
);
}
}
public
Trigger
.
TriggerResult
onElement
(
StreamRecord
<
IN
>
element
)
throws
Exception
{
Trigger
.
TriggerResult
onElementResult
=
trigger
.
onElement
(
element
.
getValue
(),
element
.
getTimestamp
(),
window
,
this
);
if
(
watermarkTimer
>
0
&&
watermarkTimer
<=
currentWatermark
)
{
...
...
@@ -553,6 +574,10 @@ public class NonKeyedWindowOperator<IN, OUT, W extends Window>
return
Trigger
.
TriggerResult
.
CONTINUE
;
}
}
public
void
clear
()
throws
Exception
{
trigger
.
clear
(
window
,
this
);
}
}
/**
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
浏览文件 @
456d0aba
/**
* 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.
*/
* 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.annotations.VisibleForTesting
;
...
...
@@ -85,8 +85,8 @@ import static java.util.Objects.requireNonNull;
* @param <W> The type of {@code Window} that the {@code WindowAssigner} assigns.
*/
public
class
WindowOperator
<
K
,
IN
,
ACC
,
OUT
,
W
extends
Window
>
extends
AbstractUdfStreamOperator
<
OUT
,
WindowFunction
<
ACC
,
OUT
,
K
,
W
>>
implements
OneInputStreamOperator
<
IN
,
OUT
>,
Triggerable
,
InputTypeConfigurable
{
extends
AbstractUdfStreamOperator
<
OUT
,
WindowFunction
<
ACC
,
OUT
,
K
,
W
>>
implements
OneInputStreamOperator
<
IN
,
OUT
>,
Triggerable
,
InputTypeConfigurable
{
private
static
final
long
serialVersionUID
=
1L
;
...
...
@@ -164,12 +164,12 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
* Creates a new {@code WindowOperator} based on the given policies and user functions.
*/
public
WindowOperator
(
WindowAssigner
<?
super
IN
,
W
>
windowAssigner
,
TypeSerializer
<
W
>
windowSerializer
,
KeySelector
<
IN
,
K
>
keySelector
,
TypeSerializer
<
K
>
keySerializer
,
StateDescriptor
<?
extends
MergingState
<
IN
,
ACC
>>
windowStateDescriptor
,
WindowFunction
<
ACC
,
OUT
,
K
,
W
>
windowFunction
,
Trigger
<?
super
IN
,
?
super
W
>
trigger
)
{
TypeSerializer
<
W
>
windowSerializer
,
KeySelector
<
IN
,
K
>
keySelector
,
TypeSerializer
<
K
>
keySerializer
,
StateDescriptor
<?
extends
MergingState
<
IN
,
ACC
>>
windowStateDescriptor
,
WindowFunction
<
ACC
,
OUT
,
K
,
W
>
windowFunction
,
Trigger
<?
super
IN
,
?
super
W
>
trigger
)
{
super
(
windowFunction
);
...
...
@@ -258,8 +258,6 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
if
(
triggerResult
.
isFire
())
{
timestampedCollector
.
setTimestamp
(
window
.
maxTimestamp
());
setKeyContext
(
key
);
MergingState
<
IN
,
ACC
>
windowState
=
getPartitionedState
(
window
,
windowSerializer
,
windowStateDescriptor
);
...
...
@@ -269,12 +267,13 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
if
(
triggerResult
.
isPurge
())
{
windowState
.
clear
();
context
.
clear
();
}
}
else
if
(
triggerResult
.
isPurge
())
{
setKeyContext
(
key
);
MergingState
<
IN
,
ACC
>
windowState
=
getPartitionedState
(
window
,
windowSerializer
,
windowStateDescriptor
);
windowState
.
clear
();
context
.
clear
();
}
}
...
...
@@ -293,7 +292,8 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
context
.
key
=
timer
.
key
;
context
.
window
=
timer
.
window
;
Trigger
.
TriggerResult
triggerResult
=
context
.
onEventTime
(
mark
.
getTimestamp
());
setKeyContext
(
timer
.
key
);
Trigger
.
TriggerResult
triggerResult
=
context
.
onEventTime
(
timer
.
timestamp
);
processTriggerResult
(
triggerResult
,
context
.
key
,
context
.
window
);
}
else
{
fire
=
false
;
...
...
@@ -319,7 +319,8 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
context
.
key
=
timer
.
key
;
context
.
window
=
timer
.
window
;
Trigger
.
TriggerResult
triggerResult
=
context
.
onProcessingTime
(
time
);
setKeyContext
(
timer
.
key
);
Trigger
.
TriggerResult
triggerResult
=
context
.
onProcessingTime
(
timer
.
timestamp
);
processTriggerResult
(
triggerResult
,
context
.
key
,
context
.
window
);
}
else
{
fire
=
false
;
...
...
@@ -410,6 +411,23 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
}
}
@Override
public
void
deleteProcessingTimeTimer
(
long
time
)
{
Timer
<
K
,
W
>
timer
=
new
Timer
<>(
time
,
key
,
window
);
if
(
processingTimeTimers
.
remove
(
timer
))
{
processingTimeTimersQueue
.
remove
(
timer
);
}
}
@Override
public
void
deleteEventTimeTimer
(
long
time
)
{
Timer
<
K
,
W
>
timer
=
new
Timer
<>(
time
,
key
,
window
);
if
(
watermarkTimers
.
remove
(
timer
))
{
watermarkTimersQueue
.
remove
(
timer
);
}
}
public
Trigger
.
TriggerResult
onElement
(
StreamRecord
<
IN
>
element
)
throws
Exception
{
return
trigger
.
onElement
(
element
.
getValue
(),
element
.
getTimestamp
(),
window
,
this
);
}
...
...
@@ -421,6 +439,18 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
public
Trigger
.
TriggerResult
onEventTime
(
long
time
)
throws
Exception
{
return
trigger
.
onEventTime
(
time
,
window
,
this
);
}
public
void
clear
()
throws
Exception
{
trigger
.
clear
(
window
,
this
);
}
@Override
public
String
toString
()
{
return
"Context{"
+
"key="
+
key
+
", window="
+
window
+
'}'
;
}
}
/**
...
...
@@ -454,8 +484,8 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
Timer
<?,
?>
timer
=
(
Timer
<?,
?>)
o
;
return
timestamp
==
timer
.
timestamp
&&
key
.
equals
(
timer
.
key
)
&&
window
.
equals
(
timer
.
window
);
&&
key
.
equals
(
timer
.
key
)
&&
window
.
equals
(
timer
.
window
);
}
...
...
@@ -470,10 +500,10 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
@Override
public
String
toString
()
{
return
"Timer{"
+
"timestamp="
+
timestamp
+
", key="
+
key
+
", window="
+
window
+
'}'
;
"timestamp="
+
timestamp
+
", key="
+
key
+
", window="
+
window
+
'}'
;
}
}
...
...
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录