Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
doujutun3207
flink
提交
12bf7c1a
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,发现更多精彩内容 >>
提交
12bf7c1a
编写于
7月 18, 2016
作者:
K
kl0u
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
[FLINK-4207] WindowOperator becomes very slow with allowed lateness
上级
884d3e2a
变更
16
显示空白变更内容
内联
并排
Showing
16 changed file
with
580 addition
and
102 deletion
+580
-102
flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBFoldingState.java
...he/flink/contrib/streaming/state/RocksDBFoldingState.java
+1
-1
flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBListState.java
...pache/flink/contrib/streaming/state/RocksDBListState.java
+1
-2
flink-core/src/main/java/org/apache/flink/api/common/state/AppendingState.java
...ava/org/apache/flink/api/common/state/AppendingState.java
+8
-2
flink-runtime/src/main/java/org/apache/flink/runtime/state/GenericListState.java
...java/org/apache/flink/runtime/state/GenericListState.java
+1
-6
flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsFoldingState.java
...apache/flink/runtime/state/filesystem/FsFoldingState.java
+2
-5
flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsListState.java
...rg/apache/flink/runtime/state/filesystem/FsListState.java
+2
-11
flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemFoldingState.java
...rg/apache/flink/runtime/state/memory/MemFoldingState.java
+2
-5
flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemListState.java
...a/org/apache/flink/runtime/state/memory/MemListState.java
+2
-11
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java
.../org/apache/flink/runtime/state/StateBackendTestBase.java
+4
-4
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/PurgingTrigger.java
...link/streaming/api/windowing/triggers/PurgingTrigger.java
+1
-1
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java
...g/runtime/operators/windowing/EvictingWindowOperator.java
+56
-19
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/MergingWindowSet.java
...reaming/runtime/operators/windowing/MergingWindowSet.java
+6
-8
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
...streaming/runtime/operators/windowing/WindowOperator.java
+55
-19
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/MergingWindowSetTest.java
...ing/runtime/operators/windowing/MergingWindowSetTest.java
+1
-6
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
...aming/runtime/operators/windowing/WindowOperatorTest.java
+438
-0
flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/SessionWindowITCase.java
...nk/test/windowing/sessionwindows/SessionWindowITCase.java
+0
-2
未找到文件。
flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBFoldingState.java
浏览文件 @
12bf7c1a
...
@@ -93,7 +93,7 @@ public class RocksDBFoldingState<K, N, T, ACC>
...
@@ -93,7 +93,7 @@ public class RocksDBFoldingState<K, N, T, ACC>
byte
[]
key
=
baos
.
toByteArray
();
byte
[]
key
=
baos
.
toByteArray
();
byte
[]
valueBytes
=
backend
.
db
.
get
(
columnFamily
,
key
);
byte
[]
valueBytes
=
backend
.
db
.
get
(
columnFamily
,
key
);
if
(
valueBytes
==
null
)
{
if
(
valueBytes
==
null
)
{
return
stateDesc
.
getDefaultValue
()
;
return
null
;
}
}
return
valueSerializer
.
deserialize
(
new
DataInputViewStreamWrapper
(
new
ByteArrayInputStream
(
valueBytes
)));
return
valueSerializer
.
deserialize
(
new
DataInputViewStreamWrapper
(
new
ByteArrayInputStream
(
valueBytes
)));
}
catch
(
IOException
|
RocksDBException
e
)
{
}
catch
(
IOException
|
RocksDBException
e
)
{
...
...
flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBListState.java
浏览文件 @
12bf7c1a
...
@@ -32,7 +32,6 @@ import java.io.ByteArrayInputStream;
...
@@ -32,7 +32,6 @@ import java.io.ByteArrayInputStream;
import
java.io.ByteArrayOutputStream
;
import
java.io.ByteArrayOutputStream
;
import
java.io.IOException
;
import
java.io.IOException
;
import
java.util.ArrayList
;
import
java.util.ArrayList
;
import
java.util.Collections
;
import
java.util.List
;
import
java.util.List
;
import
static
java
.
util
.
Objects
.
requireNonNull
;
import
static
java
.
util
.
Objects
.
requireNonNull
;
...
@@ -94,7 +93,7 @@ public class RocksDBListState<K, N, V>
...
@@ -94,7 +93,7 @@ public class RocksDBListState<K, N, V>
byte
[]
valueBytes
=
backend
.
db
.
get
(
columnFamily
,
key
);
byte
[]
valueBytes
=
backend
.
db
.
get
(
columnFamily
,
key
);
if
(
valueBytes
==
null
)
{
if
(
valueBytes
==
null
)
{
return
Collections
.
emptyList
()
;
return
null
;
}
}
ByteArrayInputStream
bais
=
new
ByteArrayInputStream
(
valueBytes
);
ByteArrayInputStream
bais
=
new
ByteArrayInputStream
(
valueBytes
);
...
...
flink-core/src/main/java/org/apache/flink/api/common/state/AppendingState.java
浏览文件 @
12bf7c1a
...
@@ -47,7 +47,13 @@ public interface AppendingState<IN, OUT> extends State {
...
@@ -47,7 +47,13 @@ public interface AppendingState<IN, OUT> extends State {
* depends on the current operator input, as the operator maintains an
* depends on the current operator input, as the operator maintains an
* independent state for each partition.
* independent state for each partition.
*
*
* @return The operator state value corresponding to the current input.
* <p>
* <b>NOTE TO IMPLEMENTERS:</b> if the state is empty, then this method
* should return {@code null}.
* </p>
*
* @return The operator state value corresponding to the current input or {@code null}
* if the state is empty.
*
*
* @throws Exception Thrown if the system cannot access the state.
* @throws Exception Thrown if the system cannot access the state.
*/
*/
...
...
flink-runtime/src/main/java/org/apache/flink/runtime/state/GenericListState.java
浏览文件 @
12bf7c1a
...
@@ -25,7 +25,6 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
...
@@ -25,7 +25,6 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
import
java.io.IOException
;
import
java.io.IOException
;
import
java.util.ArrayList
;
import
java.util.ArrayList
;
import
java.util.Collections
;
/**
/**
* Generic implementation of {@link ListState} based on a wrapped {@link ValueState}.
* Generic implementation of {@link ListState} based on a wrapped {@link ValueState}.
...
@@ -82,11 +81,7 @@ public class GenericListState<K, N, T, Backend extends AbstractStateBackend, W e
...
@@ -82,11 +81,7 @@ public class GenericListState<K, N, T, Backend extends AbstractStateBackend, W e
@Override
@Override
public
Iterable
<
T
>
get
()
throws
Exception
{
public
Iterable
<
T
>
get
()
throws
Exception
{
ArrayList
<
T
>
result
=
wrappedState
.
value
();
return
wrappedState
.
value
();
if
(
result
==
null
)
{
return
Collections
.
emptyList
();
}
return
result
;
}
}
@Override
@Override
...
...
flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsFoldingState.java
浏览文件 @
12bf7c1a
...
@@ -88,11 +88,8 @@ public class FsFoldingState<K, N, T, ACC>
...
@@ -88,11 +88,8 @@ public class FsFoldingState<K, N, T, ACC>
if
(
currentNSState
==
null
)
{
if
(
currentNSState
==
null
)
{
currentNSState
=
state
.
get
(
currentNamespace
);
currentNSState
=
state
.
get
(
currentNamespace
);
}
}
if
(
currentNSState
!=
null
)
{
return
currentNSState
!=
null
?
ACC
value
=
currentNSState
.
get
(
currentKey
);
currentNSState
.
get
(
currentKey
)
:
null
;
return
value
!=
null
?
value
:
stateDesc
.
getDefaultValue
();
}
return
stateDesc
.
getDefaultValue
();
}
}
@Override
@Override
...
...
flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsListState.java
浏览文件 @
12bf7c1a
...
@@ -27,9 +27,7 @@ import org.apache.flink.runtime.state.KvState;
...
@@ -27,9 +27,7 @@ import org.apache.flink.runtime.state.KvState;
import
org.apache.flink.runtime.state.KvStateSnapshot
;
import
org.apache.flink.runtime.state.KvStateSnapshot
;
import
java.util.ArrayList
;
import
java.util.ArrayList
;
import
java.util.Collections
;
import
java.util.HashMap
;
import
java.util.HashMap
;
import
java.util.List
;
import
java.util.Map
;
import
java.util.Map
;
/**
/**
...
@@ -85,15 +83,8 @@ public class FsListState<K, N, V>
...
@@ -85,15 +83,8 @@ public class FsListState<K, N, V>
if
(
currentNSState
==
null
)
{
if
(
currentNSState
==
null
)
{
currentNSState
=
state
.
get
(
currentNamespace
);
currentNSState
=
state
.
get
(
currentNamespace
);
}
}
if
(
currentNSState
!=
null
)
{
return
currentNSState
!=
null
?
List
<
V
>
result
=
currentNSState
.
get
(
currentKey
);
currentNSState
.
get
(
currentKey
)
:
null
;
if
(
result
==
null
)
{
return
Collections
.
emptyList
();
}
else
{
return
result
;
}
}
return
Collections
.
emptyList
();
}
}
@Override
@Override
...
...
flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemFoldingState.java
浏览文件 @
12bf7c1a
...
@@ -64,11 +64,8 @@ public class MemFoldingState<K, N, T, ACC>
...
@@ -64,11 +64,8 @@ public class MemFoldingState<K, N, T, ACC>
if
(
currentNSState
==
null
)
{
if
(
currentNSState
==
null
)
{
currentNSState
=
state
.
get
(
currentNamespace
);
currentNSState
=
state
.
get
(
currentNamespace
);
}
}
if
(
currentNSState
!=
null
)
{
return
currentNSState
!=
null
?
ACC
value
=
currentNSState
.
get
(
currentKey
);
currentNSState
.
get
(
currentKey
)
:
null
;
return
value
!=
null
?
value
:
stateDesc
.
getDefaultValue
();
}
return
stateDesc
.
getDefaultValue
();
}
}
@Override
@Override
...
...
flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemListState.java
浏览文件 @
12bf7c1a
...
@@ -26,9 +26,7 @@ import org.apache.flink.runtime.state.KvState;
...
@@ -26,9 +26,7 @@ import org.apache.flink.runtime.state.KvState;
import
org.apache.flink.runtime.state.KvStateSnapshot
;
import
org.apache.flink.runtime.state.KvStateSnapshot
;
import
java.util.ArrayList
;
import
java.util.ArrayList
;
import
java.util.Collections
;
import
java.util.HashMap
;
import
java.util.HashMap
;
import
java.util.List
;
import
java.util.Map
;
import
java.util.Map
;
/**
/**
...
@@ -56,15 +54,8 @@ public class MemListState<K, N, V>
...
@@ -56,15 +54,8 @@ public class MemListState<K, N, V>
if
(
currentNSState
==
null
)
{
if
(
currentNSState
==
null
)
{
currentNSState
=
state
.
get
(
currentNamespace
);
currentNSState
=
state
.
get
(
currentNamespace
);
}
}
if
(
currentNSState
!=
null
)
{
return
currentNSState
!=
null
?
List
<
V
>
result
=
currentNSState
.
get
(
currentKey
);
currentNSState
.
get
(
currentKey
)
:
null
;
if
(
result
==
null
)
{
return
Collections
.
emptyList
();
}
else
{
return
result
;
}
}
return
Collections
.
emptyList
();
}
}
@Override
@Override
...
...
flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java
浏览文件 @
12bf7c1a
...
@@ -243,10 +243,10 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> {
...
@@ -243,10 +243,10 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> {
Joiner
joiner
=
Joiner
.
on
(
","
);
Joiner
joiner
=
Joiner
.
on
(
","
);
// some modifications to the state
// some modifications to the state
backend
.
setCurrentKey
(
1
);
backend
.
setCurrentKey
(
1
);
assertEquals
(
""
,
joiner
.
join
(
state
.
get
()
));
assertEquals
(
null
,
state
.
get
(
));
state
.
add
(
"1"
);
state
.
add
(
"1"
);
backend
.
setCurrentKey
(
2
);
backend
.
setCurrentKey
(
2
);
assertEquals
(
""
,
joiner
.
join
(
state
.
get
()
));
assertEquals
(
null
,
state
.
get
(
));
state
.
add
(
"2"
);
state
.
add
(
"2"
);
backend
.
setCurrentKey
(
1
);
backend
.
setCurrentKey
(
1
);
assertEquals
(
"1"
,
joiner
.
join
(
state
.
get
()));
assertEquals
(
"1"
,
joiner
.
join
(
state
.
get
()));
...
@@ -438,10 +438,10 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> {
...
@@ -438,10 +438,10 @@ public abstract class StateBackendTestBase<B extends AbstractStateBackend> {
// some modifications to the state
// some modifications to the state
backend
.
setCurrentKey
(
1
);
backend
.
setCurrentKey
(
1
);
assertEquals
(
"Fold-Initial:"
,
state
.
get
());
assertEquals
(
null
,
state
.
get
());
state
.
add
(
1
);
state
.
add
(
1
);
backend
.
setCurrentKey
(
2
);
backend
.
setCurrentKey
(
2
);
assertEquals
(
"Fold-Initial:"
,
state
.
get
());
assertEquals
(
null
,
state
.
get
());
state
.
add
(
2
);
state
.
add
(
2
);
backend
.
setCurrentKey
(
1
);
backend
.
setCurrentKey
(
1
);
assertEquals
(
"Fold-Initial:,1"
,
state
.
get
());
assertEquals
(
"Fold-Initial:,1"
,
state
.
get
());
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/PurgingTrigger.java
浏览文件 @
12bf7c1a
...
@@ -72,7 +72,7 @@ public class PurgingTrigger<T, W extends Window> extends Trigger<T, W> {
...
@@ -72,7 +72,7 @@ public class PurgingTrigger<T, W extends Window> extends Trigger<T, W> {
@Override
@Override
public
TriggerResult
onMerge
(
W
window
,
OnMergeContext
ctx
)
throws
Exception
{
public
TriggerResult
onMerge
(
W
window
,
OnMergeContext
ctx
)
throws
Exception
{
TriggerResult
triggerResult
=
nestedTrigger
.
onMerge
(
window
,
ctx
);
TriggerResult
triggerResult
=
nestedTrigger
.
onMerge
(
window
,
ctx
);
return
triggerResult
.
is
Purg
e
()
?
TriggerResult
.
FIRE_AND_PURGE
:
triggerResult
;
return
triggerResult
.
is
Fir
e
()
?
TriggerResult
.
FIRE_AND_PURGE
:
triggerResult
;
}
}
@Override
@Override
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java
浏览文件 @
12bf7c1a
...
@@ -132,12 +132,14 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
...
@@ -132,12 +132,14 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
// check if the window is already inactive
// check if the window is already inactive
if
(
isLate
(
actualWindow
))
{
if
(
isLate
(
actualWindow
))
{
LOG
.
info
(
"Dropped element "
+
element
+
" for window "
+
actualWindow
+
" due to lateness."
);
mergingWindows
.
retireWindow
(
actualWindow
);
mergingWindows
.
retireWindow
(
actualWindow
);
continue
;
continue
;
}
}
W
stateWindow
=
mergingWindows
.
getStateWindow
(
actualWindow
);
W
stateWindow
=
mergingWindows
.
getStateWindow
(
actualWindow
);
if
(
stateWindow
==
null
)
{
throw
new
IllegalStateException
(
"Window "
+
window
+
" is not in in-flight window set."
);
}
ListState
<
StreamRecord
<
IN
>>
windowState
=
getPartitionedState
(
ListState
<
StreamRecord
<
IN
>>
windowState
=
getPartitionedState
(
stateWindow
,
windowSerializer
,
windowStateDescriptor
);
stateWindow
,
windowSerializer
,
windowStateDescriptor
);
windowState
.
add
(
element
);
windowState
.
add
(
element
);
...
@@ -149,7 +151,15 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
...
@@ -149,7 +151,15 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
// on the (possibly merged) window
// on the (possibly merged) window
TriggerResult
triggerResult
=
context
.
onElement
(
element
);
TriggerResult
triggerResult
=
context
.
onElement
(
element
);
TriggerResult
combinedTriggerResult
=
TriggerResult
.
merge
(
triggerResult
,
mergeTriggerResult
.
f0
);
TriggerResult
combinedTriggerResult
=
TriggerResult
.
merge
(
triggerResult
,
mergeTriggerResult
.
f0
);
fireOrContinue
(
combinedTriggerResult
,
actualWindow
,
windowState
);
if
(
combinedTriggerResult
.
isFire
())
{
Iterable
<
StreamRecord
<
IN
>>
contents
=
windowState
.
get
();
if
(
contents
==
null
)
{
// if we have no state, there is nothing to do
continue
;
}
fire
(
actualWindow
,
contents
);
}
if
(
combinedTriggerResult
.
isPurge
())
{
if
(
combinedTriggerResult
.
isPurge
())
{
cleanup
(
actualWindow
,
windowState
,
mergingWindows
);
cleanup
(
actualWindow
,
windowState
,
mergingWindows
);
...
@@ -163,7 +173,6 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
...
@@ -163,7 +173,6 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
// check if the window is already inactive
// check if the window is already inactive
if
(
isLate
(
window
))
{
if
(
isLate
(
window
))
{
LOG
.
info
(
"Dropped element "
+
element
+
" for window "
+
window
+
" due to lateness."
);
continue
;
continue
;
}
}
...
@@ -175,7 +184,15 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
...
@@ -175,7 +184,15 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
context
.
window
=
window
;
context
.
window
=
window
;
TriggerResult
triggerResult
=
context
.
onElement
(
element
);
TriggerResult
triggerResult
=
context
.
onElement
(
element
);
fireOrContinue
(
triggerResult
,
window
,
windowState
);
if
(
triggerResult
.
isFire
())
{
Iterable
<
StreamRecord
<
IN
>>
contents
=
windowState
.
get
();
if
(
contents
==
null
)
{
// if we have no state, there is nothing to do
continue
;
}
fire
(
window
,
contents
);
}
if
(
triggerResult
.
isPurge
())
{
if
(
triggerResult
.
isPurge
())
{
cleanup
(
window
,
windowState
,
null
);
cleanup
(
window
,
windowState
,
null
);
...
@@ -207,16 +224,30 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
...
@@ -207,16 +224,30 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
if
(
windowAssigner
instanceof
MergingWindowAssigner
)
{
if
(
windowAssigner
instanceof
MergingWindowAssigner
)
{
mergingWindows
=
getMergingWindowSet
();
mergingWindows
=
getMergingWindowSet
();
W
stateWindow
=
mergingWindows
.
getStateWindow
(
context
.
window
);
W
stateWindow
=
mergingWindows
.
getStateWindow
(
context
.
window
);
if
(
stateWindow
==
null
)
{
// then the window is already purged and this is a cleanup
// timer set due to allowed lateness that has nothing to clean,
// so it is safe to just ignore
continue
;
}
windowState
=
getPartitionedState
(
stateWindow
,
windowSerializer
,
windowStateDescriptor
);
windowState
=
getPartitionedState
(
stateWindow
,
windowSerializer
,
windowStateDescriptor
);
}
else
{
}
else
{
windowState
=
getPartitionedState
(
context
.
window
,
windowSerializer
,
windowStateDescriptor
);
windowState
=
getPartitionedState
(
context
.
window
,
windowSerializer
,
windowStateDescriptor
);
}
}
Iterable
<
StreamRecord
<
IN
>>
contents
=
windowState
.
get
();
if
(
contents
==
null
)
{
// if we have no state, there is nothing to do
continue
;
}
TriggerResult
triggerResult
=
context
.
onEventTime
(
timer
.
timestamp
);
TriggerResult
triggerResult
=
context
.
onEventTime
(
timer
.
timestamp
);
fireOrContinue
(
triggerResult
,
context
.
window
,
windowState
);
if
(
triggerResult
.
isFire
())
{
fire
(
context
.
window
,
contents
);
}
if
(
triggerResult
.
isPurge
()
||
(
windowAssigner
.
isEventTime
()
&&
isCleanupTime
(
timer
.
window
,
timer
.
timestamp
)))
{
if
(
triggerResult
.
isPurge
()
||
(
windowAssigner
.
isEventTime
()
&&
isCleanupTime
(
context
.
window
,
timer
.
timestamp
)))
{
cleanup
(
timer
.
window
,
windowState
,
mergingWindows
);
cleanup
(
context
.
window
,
windowState
,
mergingWindows
);
}
}
}
else
{
}
else
{
...
@@ -255,16 +286,30 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
...
@@ -255,16 +286,30 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
if
(
windowAssigner
instanceof
MergingWindowAssigner
)
{
if
(
windowAssigner
instanceof
MergingWindowAssigner
)
{
mergingWindows
=
getMergingWindowSet
();
mergingWindows
=
getMergingWindowSet
();
W
stateWindow
=
mergingWindows
.
getStateWindow
(
context
.
window
);
W
stateWindow
=
mergingWindows
.
getStateWindow
(
context
.
window
);
if
(
stateWindow
==
null
)
{
// then the window is already purged and this is a cleanup
// timer set due to allowed lateness that has nothing to clean,
// so it is safe to just ignore
continue
;
}
windowState
=
getPartitionedState
(
stateWindow
,
windowSerializer
,
windowStateDescriptor
);
windowState
=
getPartitionedState
(
stateWindow
,
windowSerializer
,
windowStateDescriptor
);
}
else
{
}
else
{
windowState
=
getPartitionedState
(
context
.
window
,
windowSerializer
,
windowStateDescriptor
);
windowState
=
getPartitionedState
(
context
.
window
,
windowSerializer
,
windowStateDescriptor
);
}
}
Iterable
<
StreamRecord
<
IN
>>
contents
=
windowState
.
get
();
if
(
contents
==
null
)
{
// if we have no state, there is nothing to do
continue
;
}
TriggerResult
triggerResult
=
context
.
onProcessingTime
(
timer
.
timestamp
);
TriggerResult
triggerResult
=
context
.
onProcessingTime
(
timer
.
timestamp
);
fireOrContinue
(
triggerResult
,
context
.
window
,
windowState
);
if
(
triggerResult
.
isFire
())
{
fire
(
context
.
window
,
contents
);
}
if
(
triggerResult
.
isPurge
()
||
(!
windowAssigner
.
isEventTime
()
&&
isCleanupTime
(
timer
.
window
,
timer
.
timestamp
)))
{
if
(
triggerResult
.
isPurge
()
||
(!
windowAssigner
.
isEventTime
()
&&
isCleanupTime
(
context
.
window
,
timer
.
timestamp
)))
{
cleanup
(
timer
.
window
,
windowState
,
mergingWindows
);
cleanup
(
context
.
window
,
windowState
,
mergingWindows
);
}
}
}
else
{
}
else
{
...
@@ -273,15 +318,8 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
...
@@ -273,15 +318,8 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
}
while
(
fire
);
}
while
(
fire
);
}
}
private
void
fireOrContinue
(
TriggerResult
triggerResult
,
private
void
fire
(
W
window
,
Iterable
<
StreamRecord
<
IN
>>
contents
)
throws
Exception
{
W
window
,
ListState
<
StreamRecord
<
IN
>>
windowState
)
throws
Exception
{
if
(!
triggerResult
.
isFire
())
{
return
;
}
timestampedCollector
.
setAbsoluteTimestamp
(
window
.
maxTimestamp
());
timestampedCollector
.
setAbsoluteTimestamp
(
window
.
maxTimestamp
());
Iterable
<
StreamRecord
<
IN
>>
contents
=
windowState
.
get
();
// Work around type system restrictions...
// Work around type system restrictions...
int
toEvict
=
evictor
.
evict
((
Iterable
)
contents
,
Iterables
.
size
(
contents
),
context
.
window
);
int
toEvict
=
evictor
.
evict
((
Iterable
)
contents
,
Iterables
.
size
(
contents
),
context
.
window
);
...
@@ -307,7 +345,6 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
...
@@ -307,7 +345,6 @@ public class EvictingWindowOperator<K, IN, OUT, W extends Window> extends Window
mergingWindows
.
retireWindow
(
window
);
mergingWindows
.
retireWindow
(
window
);
}
}
context
.
clear
();
context
.
clear
();
deleteCleanupTimer
(
window
);
}
}
// ------------------------------------------------------------------------
// ------------------------------------------------------------------------
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/MergingWindowSet.java
浏览文件 @
12bf7c1a
...
@@ -80,10 +80,13 @@ public class MergingWindowSet<W extends Window> {
...
@@ -80,10 +80,13 @@ public class MergingWindowSet<W extends Window> {
this
.
windowAssigner
=
windowAssigner
;
this
.
windowAssigner
=
windowAssigner
;
windows
=
new
HashMap
<>();
windows
=
new
HashMap
<>();
for
(
Tuple2
<
W
,
W
>
window:
state
.
get
())
{
Iterable
<
Tuple2
<
W
,
W
>>
windowState
=
state
.
get
();
if
(
windowState
!=
null
)
{
for
(
Tuple2
<
W
,
W
>
window:
windowState
)
{
windows
.
put
(
window
.
f0
,
window
.
f1
);
windows
.
put
(
window
.
f0
,
window
.
f1
);
}
}
}
}
}
public
void
persist
(
ListState
<
Tuple2
<
W
,
W
>>
state
)
throws
Exception
{
public
void
persist
(
ListState
<
Tuple2
<
W
,
W
>>
state
)
throws
Exception
{
for
(
Map
.
Entry
<
W
,
W
>
window:
windows
.
entrySet
())
{
for
(
Map
.
Entry
<
W
,
W
>
window:
windows
.
entrySet
())
{
...
@@ -100,12 +103,7 @@ public class MergingWindowSet<W extends Window> {
...
@@ -100,12 +103,7 @@ public class MergingWindowSet<W extends Window> {
* @param window The window for which to get the state window.
* @param window The window for which to get the state window.
*/
*/
public
W
getStateWindow
(
W
window
)
{
public
W
getStateWindow
(
W
window
)
{
W
result
=
windows
.
get
(
window
);
return
windows
.
get
(
window
);
if
(
result
==
null
)
{
throw
new
IllegalStateException
(
"Window "
+
window
+
" is not in in-flight window set."
);
}
return
result
;
}
}
/**
/**
...
...
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
浏览文件 @
12bf7c1a
...
@@ -334,12 +334,15 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
...
@@ -334,12 +334,15 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
// drop if the window is already late
// drop if the window is already late
if
(
isLate
(
actualWindow
))
{
if
(
isLate
(
actualWindow
))
{
LOG
.
info
(
"Dropped element "
+
element
+
" for window "
+
actualWindow
+
" due to lateness."
);
mergingWindows
.
retireWindow
(
actualWindow
);
mergingWindows
.
retireWindow
(
actualWindow
);
continue
;
continue
;
}
}
W
stateWindow
=
mergingWindows
.
getStateWindow
(
actualWindow
);
W
stateWindow
=
mergingWindows
.
getStateWindow
(
actualWindow
);
if
(
stateWindow
==
null
)
{
throw
new
IllegalStateException
(
"Window "
+
window
+
" is not in in-flight window set."
);
}
AppendingState
<
IN
,
ACC
>
windowState
=
getPartitionedState
(
AppendingState
<
IN
,
ACC
>
windowState
=
getPartitionedState
(
stateWindow
,
windowSerializer
,
windowStateDescriptor
);
stateWindow
,
windowSerializer
,
windowStateDescriptor
);
windowState
.
add
(
element
.
getValue
());
windowState
.
add
(
element
.
getValue
());
...
@@ -351,7 +354,14 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
...
@@ -351,7 +354,14 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
// on the (possibly merged) window
// on the (possibly merged) window
TriggerResult
triggerResult
=
context
.
onElement
(
element
);
TriggerResult
triggerResult
=
context
.
onElement
(
element
);
TriggerResult
combinedTriggerResult
=
TriggerResult
.
merge
(
triggerResult
,
mergeTriggerResult
.
f0
);
TriggerResult
combinedTriggerResult
=
TriggerResult
.
merge
(
triggerResult
,
mergeTriggerResult
.
f0
);
fireOrContinue
(
combinedTriggerResult
,
actualWindow
,
windowState
);
if
(
combinedTriggerResult
.
isFire
())
{
ACC
contents
=
windowState
.
get
();
if
(
contents
==
null
)
{
continue
;
}
fire
(
actualWindow
,
contents
);
}
if
(
combinedTriggerResult
.
isPurge
())
{
if
(
combinedTriggerResult
.
isPurge
())
{
cleanup
(
actualWindow
,
windowState
,
mergingWindows
);
cleanup
(
actualWindow
,
windowState
,
mergingWindows
);
...
@@ -364,7 +374,6 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
...
@@ -364,7 +374,6 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
// drop if the window is already late
// drop if the window is already late
if
(
isLate
(
window
))
{
if
(
isLate
(
window
))
{
LOG
.
info
(
"Dropped element "
+
element
+
" for window "
+
window
+
" due to lateness."
);
continue
;
continue
;
}
}
...
@@ -376,7 +385,14 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
...
@@ -376,7 +385,14 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
context
.
window
=
window
;
context
.
window
=
window
;
TriggerResult
triggerResult
=
context
.
onElement
(
element
);
TriggerResult
triggerResult
=
context
.
onElement
(
element
);
fireOrContinue
(
triggerResult
,
window
,
windowState
);
if
(
triggerResult
.
isFire
())
{
ACC
contents
=
windowState
.
get
();
if
(
contents
==
null
)
{
continue
;
}
fire
(
window
,
contents
);
}
if
(
triggerResult
.
isPurge
())
{
if
(
triggerResult
.
isPurge
())
{
cleanup
(
window
,
windowState
,
null
);
cleanup
(
window
,
windowState
,
null
);
...
@@ -408,16 +424,30 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
...
@@ -408,16 +424,30 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
if
(
windowAssigner
instanceof
MergingWindowAssigner
)
{
if
(
windowAssigner
instanceof
MergingWindowAssigner
)
{
mergingWindows
=
getMergingWindowSet
();
mergingWindows
=
getMergingWindowSet
();
W
stateWindow
=
mergingWindows
.
getStateWindow
(
context
.
window
);
W
stateWindow
=
mergingWindows
.
getStateWindow
(
context
.
window
);
if
(
stateWindow
==
null
)
{
// then the window is already purged and this is a cleanup
// timer set due to allowed lateness that has nothing to clean,
// so it is safe to just ignore
continue
;
}
windowState
=
getPartitionedState
(
stateWindow
,
windowSerializer
,
windowStateDescriptor
);
windowState
=
getPartitionedState
(
stateWindow
,
windowSerializer
,
windowStateDescriptor
);
}
else
{
}
else
{
windowState
=
getPartitionedState
(
context
.
window
,
windowSerializer
,
windowStateDescriptor
);
windowState
=
getPartitionedState
(
context
.
window
,
windowSerializer
,
windowStateDescriptor
);
}
}
ACC
contents
=
windowState
.
get
();
if
(
contents
==
null
)
{
// if we have no state, there is nothing to do
continue
;
}
TriggerResult
triggerResult
=
context
.
onEventTime
(
timer
.
timestamp
);
TriggerResult
triggerResult
=
context
.
onEventTime
(
timer
.
timestamp
);
fireOrContinue
(
triggerResult
,
context
.
window
,
windowState
);
if
(
triggerResult
.
isFire
())
{
fire
(
context
.
window
,
contents
);
}
if
(
triggerResult
.
isPurge
()
||
(
windowAssigner
.
isEventTime
()
&&
isCleanupTime
(
timer
.
window
,
timer
.
timestamp
)))
{
if
(
triggerResult
.
isPurge
()
||
(
windowAssigner
.
isEventTime
()
&&
isCleanupTime
(
context
.
window
,
timer
.
timestamp
)))
{
cleanup
(
timer
.
window
,
windowState
,
mergingWindows
);
cleanup
(
context
.
window
,
windowState
,
mergingWindows
);
}
}
}
else
{
}
else
{
...
@@ -456,16 +486,30 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
...
@@ -456,16 +486,30 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
if
(
windowAssigner
instanceof
MergingWindowAssigner
)
{
if
(
windowAssigner
instanceof
MergingWindowAssigner
)
{
mergingWindows
=
getMergingWindowSet
();
mergingWindows
=
getMergingWindowSet
();
W
stateWindow
=
mergingWindows
.
getStateWindow
(
context
.
window
);
W
stateWindow
=
mergingWindows
.
getStateWindow
(
context
.
window
);
if
(
stateWindow
==
null
)
{
// then the window is already purged and this is a cleanup
// timer set due to allowed lateness that has nothing to clean,
// so it is safe to just ignore
continue
;
}
windowState
=
getPartitionedState
(
stateWindow
,
windowSerializer
,
windowStateDescriptor
);
windowState
=
getPartitionedState
(
stateWindow
,
windowSerializer
,
windowStateDescriptor
);
}
else
{
}
else
{
windowState
=
getPartitionedState
(
context
.
window
,
windowSerializer
,
windowStateDescriptor
);
windowState
=
getPartitionedState
(
context
.
window
,
windowSerializer
,
windowStateDescriptor
);
}
}
ACC
contents
=
windowState
.
get
();
if
(
contents
==
null
)
{
// if we have no state, there is nothing to do
continue
;
}
TriggerResult
triggerResult
=
context
.
onProcessingTime
(
timer
.
timestamp
);
TriggerResult
triggerResult
=
context
.
onProcessingTime
(
timer
.
timestamp
);
fireOrContinue
(
triggerResult
,
context
.
window
,
windowState
);
if
(
triggerResult
.
isFire
())
{
fire
(
context
.
window
,
contents
);
}
if
(
triggerResult
.
isPurge
()
||
(!
windowAssigner
.
isEventTime
()
&&
isCleanupTime
(
timer
.
window
,
timer
.
timestamp
)))
{
if
(
triggerResult
.
isPurge
()
||
(!
windowAssigner
.
isEventTime
()
&&
isCleanupTime
(
context
.
window
,
timer
.
timestamp
)))
{
cleanup
(
timer
.
window
,
windowState
,
mergingWindows
);
cleanup
(
context
.
window
,
windowState
,
mergingWindows
);
}
}
}
else
{
}
else
{
...
@@ -487,7 +531,6 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
...
@@ -487,7 +531,6 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
mergingWindows
.
retireWindow
(
window
);
mergingWindows
.
retireWindow
(
window
);
}
}
context
.
clear
();
context
.
clear
();
deleteCleanupTimer
(
window
);
}
}
/**
/**
...
@@ -495,15 +538,8 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
...
@@ -495,15 +538,8 @@ public class WindowOperator<K, IN, ACC, OUT, W extends Window>
* The caller must ensure that the correct key is set in the state backend and the context object.
* The caller must ensure that the correct key is set in the state backend and the context object.
*/
*/
@SuppressWarnings
(
"unchecked"
)
@SuppressWarnings
(
"unchecked"
)
private
void
fireOrContinue
(
TriggerResult
triggerResult
,
private
void
fire
(
W
window
,
ACC
contents
)
throws
Exception
{
W
window
,
AppendingState
<
IN
,
ACC
>
windowState
)
throws
Exception
{
if
(!
triggerResult
.
isFire
())
{
return
;
}
timestampedCollector
.
setAbsoluteTimestamp
(
window
.
maxTimestamp
());
timestampedCollector
.
setAbsoluteTimestamp
(
window
.
maxTimestamp
());
ACC
contents
=
windowState
.
get
();
userFunction
.
apply
(
context
.
key
,
context
.
window
,
contents
,
timestampedCollector
);
userFunction
.
apply
(
context
.
key
,
context
.
window
,
contents
,
timestampedCollector
);
}
}
...
...
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/MergingWindowSetTest.java
浏览文件 @
12bf7c1a
...
@@ -133,12 +133,7 @@ public class MergingWindowSetTest {
...
@@ -133,12 +133,7 @@ public class MergingWindowSetTest {
// retire the first batch of windows
// retire the first batch of windows
windowSet
.
retireWindow
(
new
TimeWindow
(
0
,
6
));
windowSet
.
retireWindow
(
new
TimeWindow
(
0
,
6
));
try
{
assertTrue
(
windowSet
.
getStateWindow
(
new
TimeWindow
(
0
,
6
))
==
null
);
windowSet
.
getStateWindow
(
new
TimeWindow
(
0
,
6
));
fail
(
"Expected exception"
);
}
catch
(
IllegalStateException
e
)
{
//ignore
}
assertTrue
(
windowSet
.
getStateWindow
(
new
TimeWindow
(
10
,
15
)).
equals
(
new
TimeWindow
(
11
,
14
)));
assertTrue
(
windowSet
.
getStateWindow
(
new
TimeWindow
(
10
,
15
)).
equals
(
new
TimeWindow
(
11
,
14
)));
}
}
...
...
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java
浏览文件 @
12bf7c1a
...
@@ -17,9 +17,12 @@
...
@@ -17,9 +17,12 @@
*/
*/
package
org.apache.flink.streaming.runtime.operators.windowing
;
package
org.apache.flink.streaming.runtime.operators.windowing
;
import
com.google.common.base.Joiner
;
import
com.google.common.collect.Iterables
;
import
com.google.common.collect.Iterables
;
import
org.apache.flink.api.common.ExecutionConfig
;
import
org.apache.flink.api.common.ExecutionConfig
;
import
org.apache.flink.api.common.functions.FoldFunction
;
import
org.apache.flink.api.common.functions.ReduceFunction
;
import
org.apache.flink.api.common.functions.ReduceFunction
;
import
org.apache.flink.api.common.state.FoldingStateDescriptor
;
import
org.apache.flink.api.common.state.ListStateDescriptor
;
import
org.apache.flink.api.common.state.ListStateDescriptor
;
import
org.apache.flink.api.common.state.ReducingStateDescriptor
;
import
org.apache.flink.api.common.state.ReducingStateDescriptor
;
import
org.apache.flink.api.common.typeinfo.BasicTypeInfo
;
import
org.apache.flink.api.common.typeinfo.BasicTypeInfo
;
...
@@ -51,6 +54,8 @@ import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
...
@@ -51,6 +54,8 @@ import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
import
org.apache.flink.streaming.api.windowing.triggers.EventTimeTrigger
;
import
org.apache.flink.streaming.api.windowing.triggers.EventTimeTrigger
;
import
org.apache.flink.streaming.api.windowing.triggers.ProcessingTimeTrigger
;
import
org.apache.flink.streaming.api.windowing.triggers.ProcessingTimeTrigger
;
import
org.apache.flink.streaming.api.windowing.triggers.PurgingTrigger
;
import
org.apache.flink.streaming.api.windowing.triggers.PurgingTrigger
;
import
org.apache.flink.streaming.api.windowing.triggers.Trigger
;
import
org.apache.flink.streaming.api.windowing.triggers.TriggerResult
;
import
org.apache.flink.streaming.api.windowing.windows.GlobalWindow
;
import
org.apache.flink.streaming.api.windowing.windows.GlobalWindow
;
import
org.apache.flink.streaming.api.windowing.windows.TimeWindow
;
import
org.apache.flink.streaming.api.windowing.windows.TimeWindow
;
import
org.apache.flink.streaming.api.windowing.windows.Window
;
import
org.apache.flink.streaming.api.windowing.windows.Window
;
...
@@ -1952,10 +1957,379 @@ public class WindowOperatorTest {
...
@@ -1952,10 +1957,379 @@ public class WindowOperatorTest {
testHarness
.
close
();
testHarness
.
close
();
}
}
@Test
public
void
testCleanupTimerWithEmptyListStateForTumblingWindows2
()
throws
Exception
{
final
int
WINDOW_SIZE
=
2
;
final
long
LATENESS
=
100
;
TypeInformation
<
Tuple2
<
String
,
Integer
>>
inputType
=
TypeInfoParser
.
parse
(
"Tuple2<String, Integer>"
);
ListStateDescriptor
<
Tuple2
<
String
,
Integer
>>
windowStateDesc
=
new
ListStateDescriptor
<>(
"window-contents"
,
inputType
.
createSerializer
(
new
ExecutionConfig
()));
WindowOperator
<
String
,
Tuple2
<
String
,
Integer
>,
Iterable
<
Tuple2
<
String
,
Integer
>>,
String
,
TimeWindow
>
operator
=
new
WindowOperator
<>(
TumblingEventTimeWindows
.
of
(
Time
.
of
(
WINDOW_SIZE
,
TimeUnit
.
SECONDS
)),
new
TimeWindow
.
Serializer
(),
new
TupleKeySelector
(),
BasicTypeInfo
.
STRING_TYPE_INFO
.
createSerializer
(
new
ExecutionConfig
()),
windowStateDesc
,
new
InternalIterableWindowFunction
<>(
new
PassThroughFunction2
()),
new
EventTimeTriggerAccumGC
(
LATENESS
),
LATENESS
);
OneInputStreamOperatorTestHarness
<
Tuple2
<
String
,
Integer
>,
String
>
testHarness
=
new
OneInputStreamOperatorTestHarness
<>(
operator
);
testHarness
.
configureForKeyedStream
(
new
TupleKeySelector
(),
BasicTypeInfo
.
STRING_TYPE_INFO
);
operator
.
setInputType
(
inputType
,
new
ExecutionConfig
());
testHarness
.
open
();
ConcurrentLinkedQueue
<
Object
>
expected
=
new
ConcurrentLinkedQueue
<>();
// normal element
testHarness
.
processElement
(
new
StreamRecord
<>(
new
Tuple2
<>(
"key2"
,
1
),
1000
));
testHarness
.
processWatermark
(
new
Watermark
(
1599
));
testHarness
.
processWatermark
(
new
Watermark
(
1999
));
testHarness
.
processWatermark
(
new
Watermark
(
2100
));
testHarness
.
processWatermark
(
new
Watermark
(
5000
));
expected
.
add
(
new
Watermark
(
1599
));
expected
.
add
(
new
StreamRecord
<>(
"GOT: (key2,1)"
,
1999
));
expected
.
add
(
new
Watermark
(
1999
));
// here it fires and purges
expected
.
add
(
new
Watermark
(
2100
));
// here is the cleanup timer
expected
.
add
(
new
Watermark
(
5000
));
TestHarnessUtil
.
assertOutputEqualsSorted
(
"Output was not correct."
,
expected
,
testHarness
.
getOutput
(),
new
Tuple2ResultSortComparator
());
testHarness
.
close
();
}
private
class
PassThroughFunction2
implements
WindowFunction
<
Tuple2
<
String
,
Integer
>,
String
,
String
,
TimeWindow
>
{
private
static
final
long
serialVersionUID
=
1L
;
@Override
public
void
apply
(
String
k
,
TimeWindow
window
,
Iterable
<
Tuple2
<
String
,
Integer
>>
input
,
Collector
<
String
>
out
)
throws
Exception
{
out
.
collect
(
"GOT: "
+
Joiner
.
on
(
","
).
join
(
input
));
}
}
@Test
public
void
testCleanupTimerWithEmptyListStateForTumblingWindows
()
throws
Exception
{
final
int
WINDOW_SIZE
=
2
;
final
long
LATENESS
=
1
;
TypeInformation
<
Tuple2
<
String
,
Integer
>>
inputType
=
TypeInfoParser
.
parse
(
"Tuple2<String, Integer>"
);
ListStateDescriptor
<
Tuple2
<
String
,
Integer
>>
windowStateDesc
=
new
ListStateDescriptor
<>(
"window-contents"
,
inputType
.
createSerializer
(
new
ExecutionConfig
()));
WindowOperator
<
String
,
Tuple2
<
String
,
Integer
>,
Iterable
<
Tuple2
<
String
,
Integer
>>,
Tuple2
<
String
,
Integer
>,
TimeWindow
>
operator
=
new
WindowOperator
<>(
TumblingEventTimeWindows
.
of
(
Time
.
of
(
WINDOW_SIZE
,
TimeUnit
.
SECONDS
)),
new
TimeWindow
.
Serializer
(),
new
TupleKeySelector
(),
BasicTypeInfo
.
STRING_TYPE_INFO
.
createSerializer
(
new
ExecutionConfig
()),
windowStateDesc
,
new
InternalIterableWindowFunction
<>(
new
PassThroughFunction
()),
EventTimeTrigger
.
create
(),
LATENESS
);
OneInputStreamOperatorTestHarness
<
Tuple2
<
String
,
Integer
>,
Tuple2
<
String
,
Integer
>>
testHarness
=
new
OneInputStreamOperatorTestHarness
<>(
operator
);
testHarness
.
configureForKeyedStream
(
new
TupleKeySelector
(),
BasicTypeInfo
.
STRING_TYPE_INFO
);
operator
.
setInputType
(
inputType
,
new
ExecutionConfig
());
testHarness
.
open
();
ConcurrentLinkedQueue
<
Object
>
expected
=
new
ConcurrentLinkedQueue
<>();
// normal element
testHarness
.
processElement
(
new
StreamRecord
<>(
new
Tuple2
<>(
"key2"
,
1
),
1000
));
testHarness
.
processWatermark
(
new
Watermark
(
1599
));
testHarness
.
processWatermark
(
new
Watermark
(
1999
));
testHarness
.
processWatermark
(
new
Watermark
(
2000
));
testHarness
.
processWatermark
(
new
Watermark
(
5000
));
expected
.
add
(
new
Watermark
(
1599
));
expected
.
add
(
new
StreamRecord
<>(
new
Tuple2
<>(
"key2"
,
1
),
1999
));
expected
.
add
(
new
Watermark
(
1999
));
// here it fires and purges
expected
.
add
(
new
Watermark
(
2000
));
// here is the cleanup timer
expected
.
add
(
new
Watermark
(
5000
));
TestHarnessUtil
.
assertOutputEqualsSorted
(
"Output was not correct."
,
expected
,
testHarness
.
getOutput
(),
new
Tuple2ResultSortComparator
());
testHarness
.
close
();
}
@Test
public
void
testCleanupTimerWithEmptyReduceStateForTumblingWindows
()
throws
Exception
{
final
int
WINDOW_SIZE
=
2
;
final
long
LATENESS
=
1
;
TypeInformation
<
Tuple2
<
String
,
Integer
>>
inputType
=
TypeInfoParser
.
parse
(
"Tuple2<String, Integer>"
);
ReducingStateDescriptor
<
Tuple2
<
String
,
Integer
>>
stateDesc
=
new
ReducingStateDescriptor
<>(
"window-contents"
,
new
SumReducer
(),
inputType
.
createSerializer
(
new
ExecutionConfig
()));
WindowOperator
<
String
,
Tuple2
<
String
,
Integer
>,
Tuple2
<
String
,
Integer
>,
Tuple2
<
String
,
Integer
>,
TimeWindow
>
operator
=
new
WindowOperator
<>(
TumblingEventTimeWindows
.
of
(
Time
.
of
(
WINDOW_SIZE
,
TimeUnit
.
SECONDS
)),
new
TimeWindow
.
Serializer
(),
new
TupleKeySelector
(),
BasicTypeInfo
.
STRING_TYPE_INFO
.
createSerializer
(
new
ExecutionConfig
()),
stateDesc
,
new
InternalSingleValueWindowFunction
<>(
new
PassThroughWindowFunction
<
String
,
TimeWindow
,
Tuple2
<
String
,
Integer
>>()),
EventTimeTrigger
.
create
(),
LATENESS
);
OneInputStreamOperatorTestHarness
<
Tuple2
<
String
,
Integer
>,
Tuple2
<
String
,
Integer
>>
testHarness
=
new
OneInputStreamOperatorTestHarness
<>(
operator
);
testHarness
.
configureForKeyedStream
(
new
TupleKeySelector
(),
BasicTypeInfo
.
STRING_TYPE_INFO
);
operator
.
setInputType
(
inputType
,
new
ExecutionConfig
());
testHarness
.
open
();
ConcurrentLinkedQueue
<
Object
>
expected
=
new
ConcurrentLinkedQueue
<>();
// normal element
testHarness
.
processElement
(
new
StreamRecord
<>(
new
Tuple2
<>(
"key2"
,
1
),
1000
));
testHarness
.
processWatermark
(
new
Watermark
(
1599
));
testHarness
.
processWatermark
(
new
Watermark
(
1999
));
testHarness
.
processWatermark
(
new
Watermark
(
2000
));
testHarness
.
processWatermark
(
new
Watermark
(
5000
));
expected
.
add
(
new
Watermark
(
1599
));
expected
.
add
(
new
StreamRecord
<>(
new
Tuple2
<>(
"key2"
,
1
),
1999
));
expected
.
add
(
new
Watermark
(
1999
));
// here it fires and purges
expected
.
add
(
new
Watermark
(
2000
));
// here is the cleanup timer
expected
.
add
(
new
Watermark
(
5000
));
TestHarnessUtil
.
assertOutputEqualsSorted
(
"Output was not correct."
,
expected
,
testHarness
.
getOutput
(),
new
Tuple2ResultSortComparator
());
testHarness
.
close
();
}
@Test
public
void
testCleanupTimerWithEmptyFoldingStateForTumblingWindows
()
throws
Exception
{
final
int
WINDOW_SIZE
=
2
;
final
long
LATENESS
=
1
;
TypeInformation
<
Tuple2
<
String
,
Integer
>>
inputType
=
TypeInfoParser
.
parse
(
"Tuple2<String, Integer>"
);
FoldingStateDescriptor
<
Tuple2
<
String
,
Integer
>,
Tuple2
<
String
,
Integer
>>
windowStateDesc
=
new
FoldingStateDescriptor
<>(
"window-contents"
,
new
Tuple2
<>((
String
)
null
,
0
),
new
FoldFunction
<
Tuple2
<
String
,
Integer
>,
Tuple2
<
String
,
Integer
>>()
{
@Override
public
Tuple2
<
String
,
Integer
>
fold
(
Tuple2
<
String
,
Integer
>
accumulator
,
Tuple2
<
String
,
Integer
>
value
)
throws
Exception
{
return
new
Tuple2
<>(
value
.
f0
,
accumulator
.
f1
+
value
.
f1
);
}
},
inputType
);
WindowOperator
<
String
,
Tuple2
<
String
,
Integer
>,
Tuple2
<
String
,
Integer
>,
Tuple2
<
String
,
Integer
>,
TimeWindow
>
operator
=
new
WindowOperator
<>(
TumblingEventTimeWindows
.
of
(
Time
.
of
(
WINDOW_SIZE
,
TimeUnit
.
SECONDS
)),
new
TimeWindow
.
Serializer
(),
new
TupleKeySelector
(),
BasicTypeInfo
.
STRING_TYPE_INFO
.
createSerializer
(
new
ExecutionConfig
()),
windowStateDesc
,
new
InternalSingleValueWindowFunction
<>(
new
PassThroughFunction
()),
EventTimeTrigger
.
create
(),
LATENESS
);
OneInputStreamOperatorTestHarness
<
Tuple2
<
String
,
Integer
>,
Tuple2
<
String
,
Integer
>>
testHarness
=
new
OneInputStreamOperatorTestHarness
<>(
operator
);
testHarness
.
configureForKeyedStream
(
new
TupleKeySelector
(),
BasicTypeInfo
.
STRING_TYPE_INFO
);
operator
.
setInputType
(
inputType
,
new
ExecutionConfig
());
testHarness
.
open
();
ConcurrentLinkedQueue
<
Object
>
expected
=
new
ConcurrentLinkedQueue
<>();
// normal element
testHarness
.
processElement
(
new
StreamRecord
<>(
new
Tuple2
<>(
"key2"
,
1
),
1000
));
testHarness
.
processWatermark
(
new
Watermark
(
1599
));
testHarness
.
processWatermark
(
new
Watermark
(
1999
));
testHarness
.
processWatermark
(
new
Watermark
(
2000
));
testHarness
.
processWatermark
(
new
Watermark
(
5000
));
expected
.
add
(
new
Watermark
(
1599
));
expected
.
add
(
new
StreamRecord
<>(
new
Tuple2
<>(
"key2"
,
1
),
1999
));
expected
.
add
(
new
Watermark
(
1999
));
// here it fires and purges
expected
.
add
(
new
Watermark
(
2000
));
// here is the cleanup timer
expected
.
add
(
new
Watermark
(
5000
));
TestHarnessUtil
.
assertOutputEqualsSorted
(
"Output was not correct."
,
expected
,
testHarness
.
getOutput
(),
new
Tuple2ResultSortComparator
());
testHarness
.
close
();
}
@Test
public
void
testCleanupTimerWithEmptyListStateForSessionWindows
()
throws
Exception
{
final
int
GAP_SIZE
=
3
;
final
long
LATENESS
=
10
;
TypeInformation
<
Tuple2
<
String
,
Integer
>>
inputType
=
TypeInfoParser
.
parse
(
"Tuple2<String, Integer>"
);
ListStateDescriptor
<
Tuple2
<
String
,
Integer
>>
windowStateDesc
=
new
ListStateDescriptor
<>(
"window-contents"
,
inputType
.
createSerializer
(
new
ExecutionConfig
()));
WindowOperator
<
String
,
Tuple2
<
String
,
Integer
>,
Iterable
<
Tuple2
<
String
,
Integer
>>,
Tuple2
<
String
,
Integer
>,
TimeWindow
>
operator
=
new
WindowOperator
<>(
EventTimeSessionWindows
.
withGap
(
Time
.
seconds
(
GAP_SIZE
)),
new
TimeWindow
.
Serializer
(),
new
TupleKeySelector
(),
BasicTypeInfo
.
STRING_TYPE_INFO
.
createSerializer
(
new
ExecutionConfig
()),
windowStateDesc
,
new
InternalIterableWindowFunction
<>(
new
PassThroughFunction
()),
EventTimeTrigger
.
create
(),
LATENESS
);
OneInputStreamOperatorTestHarness
<
Tuple2
<
String
,
Integer
>,
Tuple2
<
String
,
Integer
>>
testHarness
=
new
OneInputStreamOperatorTestHarness
<>(
operator
);
testHarness
.
configureForKeyedStream
(
new
TupleKeySelector
(),
BasicTypeInfo
.
STRING_TYPE_INFO
);
operator
.
setInputType
(
inputType
,
new
ExecutionConfig
());
testHarness
.
open
();
ConcurrentLinkedQueue
<
Object
>
expected
=
new
ConcurrentLinkedQueue
<>();
testHarness
.
processElement
(
new
StreamRecord
<>(
new
Tuple2
<>(
"key2"
,
1
),
1000
));
testHarness
.
processWatermark
(
new
Watermark
(
4998
));
expected
.
add
(
new
StreamRecord
<>(
new
Tuple2
<>(
"key2"
,
1
),
3999
));
expected
.
add
(
new
Watermark
(
4998
));
testHarness
.
processWatermark
(
new
Watermark
(
14600
));
expected
.
add
(
new
Watermark
(
14600
));
ConcurrentLinkedQueue
<
Object
>
actual
=
testHarness
.
getOutput
();
TestHarnessUtil
.
assertOutputEqualsSorted
(
"Output was not correct."
,
expected
,
actual
,
new
Tuple2ResultSortComparator
());
testHarness
.
close
();
}
@Test
public
void
testCleanupTimerWithEmptyReduceStateForSessionWindows
()
throws
Exception
{
final
int
GAP_SIZE
=
3
;
final
long
LATENESS
=
10
;
TypeInformation
<
Tuple2
<
String
,
Integer
>>
inputType
=
TypeInfoParser
.
parse
(
"Tuple2<String, Integer>"
);
ReducingStateDescriptor
<
Tuple2
<
String
,
Integer
>>
stateDesc
=
new
ReducingStateDescriptor
<>(
"window-contents"
,
new
SumReducer
(),
inputType
.
createSerializer
(
new
ExecutionConfig
()));
WindowOperator
<
String
,
Tuple2
<
String
,
Integer
>,
Tuple2
<
String
,
Integer
>,
Tuple3
<
String
,
Long
,
Long
>,
TimeWindow
>
operator
=
new
WindowOperator
<>(
EventTimeSessionWindows
.
withGap
(
Time
.
seconds
(
GAP_SIZE
)),
new
TimeWindow
.
Serializer
(),
new
TupleKeySelector
(),
BasicTypeInfo
.
STRING_TYPE_INFO
.
createSerializer
(
new
ExecutionConfig
()),
stateDesc
,
new
InternalSingleValueWindowFunction
<>(
new
ReducedSessionWindowFunction
()),
EventTimeTrigger
.
create
(),
LATENESS
);
operator
.
setInputType
(
TypeInfoParser
.<
Tuple2
<
String
,
Integer
>>
parse
(
"Tuple2<String, Integer>"
),
new
ExecutionConfig
());
OneInputStreamOperatorTestHarness
<
Tuple2
<
String
,
Integer
>,
Tuple3
<
String
,
Long
,
Long
>>
testHarness
=
new
OneInputStreamOperatorTestHarness
<>(
operator
);
testHarness
.
configureForKeyedStream
(
new
TupleKeySelector
(),
BasicTypeInfo
.
STRING_TYPE_INFO
);
testHarness
.
open
();
ConcurrentLinkedQueue
<
Object
>
expected
=
new
ConcurrentLinkedQueue
<>();
testHarness
.
processElement
(
new
StreamRecord
<>(
new
Tuple2
<>(
"key2"
,
1
),
1000
));
testHarness
.
processWatermark
(
new
Watermark
(
4998
));
expected
.
add
(
new
StreamRecord
<>(
new
Tuple3
<>(
"key2-1"
,
1000L
,
4000L
),
3999
));
expected
.
add
(
new
Watermark
(
4998
));
testHarness
.
processWatermark
(
new
Watermark
(
14600
));
expected
.
add
(
new
Watermark
(
14600
));
ConcurrentLinkedQueue
<
Object
>
actual
=
testHarness
.
getOutput
();
TestHarnessUtil
.
assertOutputEqualsSorted
(
"Output was not correct."
,
expected
,
actual
,
new
Tuple2ResultSortComparator
());
testHarness
.
close
();
}
@Test
public
void
testCleanupTimerWithEmptyFoldingStateForSessionWindows
()
throws
Exception
{
final
int
GAP_SIZE
=
3
;
final
long
LATENESS
=
10
;
TypeInformation
<
Tuple2
<
String
,
Integer
>>
inputType
=
TypeInfoParser
.
parse
(
"Tuple2<String, Integer>"
);
FoldingStateDescriptor
<
Tuple2
<
String
,
Integer
>,
Tuple2
<
String
,
Integer
>>
windowStateDesc
=
new
FoldingStateDescriptor
<>(
"window-contents"
,
new
Tuple2
<>((
String
)
null
,
0
),
new
FoldFunction
<
Tuple2
<
String
,
Integer
>,
Tuple2
<
String
,
Integer
>>()
{
@Override
public
Tuple2
<
String
,
Integer
>
fold
(
Tuple2
<
String
,
Integer
>
accumulator
,
Tuple2
<
String
,
Integer
>
value
)
throws
Exception
{
return
new
Tuple2
<>(
value
.
f0
,
accumulator
.
f1
+
value
.
f1
);
}
},
inputType
);
WindowOperator
<
String
,
Tuple2
<
String
,
Integer
>,
Tuple2
<
String
,
Integer
>,
Tuple2
<
String
,
Integer
>,
TimeWindow
>
operator
=
new
WindowOperator
<>(
EventTimeSessionWindows
.
withGap
(
Time
.
seconds
(
GAP_SIZE
)),
new
TimeWindow
.
Serializer
(),
new
TupleKeySelector
(),
BasicTypeInfo
.
STRING_TYPE_INFO
.
createSerializer
(
new
ExecutionConfig
()),
windowStateDesc
,
new
InternalSingleValueWindowFunction
<>(
new
PassThroughFunction
()),
EventTimeTrigger
.
create
(),
LATENESS
);
OneInputStreamOperatorTestHarness
<
Tuple2
<
String
,
Integer
>,
Tuple2
<
String
,
Integer
>>
testHarness
=
new
OneInputStreamOperatorTestHarness
<>(
operator
);
testHarness
.
configureForKeyedStream
(
new
TupleKeySelector
(),
BasicTypeInfo
.
STRING_TYPE_INFO
);
operator
.
setInputType
(
inputType
,
new
ExecutionConfig
());
testHarness
.
open
();
ConcurrentLinkedQueue
<
Object
>
expected
=
new
ConcurrentLinkedQueue
<>();
testHarness
.
processElement
(
new
StreamRecord
<>(
new
Tuple2
<>(
"key2"
,
1
),
1000
));
testHarness
.
processWatermark
(
new
Watermark
(
4998
));
expected
.
add
(
new
StreamRecord
<>(
new
Tuple2
<>(
"key2"
,
1
),
3999
));
expected
.
add
(
new
Watermark
(
4998
));
testHarness
.
processWatermark
(
new
Watermark
(
14600
));
expected
.
add
(
new
Watermark
(
14600
));
ConcurrentLinkedQueue
<
Object
>
actual
=
testHarness
.
getOutput
();
TestHarnessUtil
.
assertOutputEqualsSorted
(
"Output was not correct."
,
expected
,
actual
,
new
Tuple2ResultSortComparator
());
testHarness
.
close
();
}
// ------------------------------------------------------------------------
// ------------------------------------------------------------------------
// UDFs
// UDFs
// ------------------------------------------------------------------------
// ------------------------------------------------------------------------
private
class
PassThroughFunction
implements
WindowFunction
<
Tuple2
<
String
,
Integer
>,
Tuple2
<
String
,
Integer
>,
String
,
TimeWindow
>
{
private
static
final
long
serialVersionUID
=
1L
;
@Override
public
void
apply
(
String
k
,
TimeWindow
window
,
Iterable
<
Tuple2
<
String
,
Integer
>>
input
,
Collector
<
Tuple2
<
String
,
Integer
>>
out
)
throws
Exception
{
for
(
Tuple2
<
String
,
Integer
>
in:
input
)
{
out
.
collect
(
in
);
}
}
}
public
static
class
SumReducer
implements
ReduceFunction
<
Tuple2
<
String
,
Integer
>>
{
public
static
class
SumReducer
implements
ReduceFunction
<
Tuple2
<
String
,
Integer
>>
{
private
static
final
long
serialVersionUID
=
1L
;
private
static
final
long
serialVersionUID
=
1L
;
@Override
@Override
...
@@ -2112,4 +2486,68 @@ public class WindowOperatorTest {
...
@@ -2112,4 +2486,68 @@ public class WindowOperatorTest {
return
Collections
.
singletonList
(
new
TimeWindow
(
timestamp
,
timestamp
+
sessionTimeout
));
return
Collections
.
singletonList
(
new
TimeWindow
(
timestamp
,
timestamp
+
sessionTimeout
));
}
}
}
}
/**
* A trigger that fires at the end of the window but does not
* purge the state of the fired window. This is to test the state
* garbage collection mechanism.
*/
public
class
EventTimeTriggerAccumGC
extends
Trigger
<
Object
,
TimeWindow
>
{
private
static
final
long
serialVersionUID
=
1L
;
private
long
cleanupTime
;
private
EventTimeTriggerAccumGC
()
{
cleanupTime
=
0L
;
}
public
EventTimeTriggerAccumGC
(
long
cleanupTime
)
{
this
.
cleanupTime
=
cleanupTime
;
}
@Override
public
TriggerResult
onElement
(
Object
element
,
long
timestamp
,
TimeWindow
window
,
TriggerContext
ctx
)
throws
Exception
{
if
(
window
.
maxTimestamp
()
<=
ctx
.
getCurrentWatermark
())
{
// if the watermark is already past the window fire immediately
return
TriggerResult
.
FIRE
;
}
else
{
ctx
.
registerEventTimeTimer
(
window
.
maxTimestamp
());
return
TriggerResult
.
CONTINUE
;
}
}
@Override
public
TriggerResult
onEventTime
(
long
time
,
TimeWindow
window
,
TriggerContext
ctx
)
{
return
time
==
window
.
maxTimestamp
()
||
time
==
window
.
maxTimestamp
()
+
cleanupTime
?
TriggerResult
.
FIRE_AND_PURGE
:
TriggerResult
.
CONTINUE
;
}
@Override
public
TriggerResult
onProcessingTime
(
long
time
,
TimeWindow
window
,
TriggerContext
ctx
)
throws
Exception
{
return
TriggerResult
.
CONTINUE
;
}
@Override
public
void
clear
(
TimeWindow
window
,
TriggerContext
ctx
)
throws
Exception
{
ctx
.
deleteEventTimeTimer
(
window
.
maxTimestamp
());
}
@Override
public
boolean
canMerge
()
{
return
true
;
}
@Override
public
TriggerResult
onMerge
(
TimeWindow
window
,
OnMergeContext
ctx
)
{
ctx
.
registerEventTimeTimer
(
window
.
maxTimestamp
());
return
TriggerResult
.
CONTINUE
;
}
@Override
public
String
toString
()
{
return
"EventTimeTrigger()"
;
}
}
}
}
flink-tests/src/test/java/org/apache/flink/test/windowing/sessionwindows/SessionWindowITCase.java
浏览文件 @
12bf7c1a
...
@@ -22,9 +22,7 @@ import org.apache.flink.api.common.JobExecutionResult;
...
@@ -22,9 +22,7 @@ import org.apache.flink.api.common.JobExecutionResult;
import
org.apache.flink.api.java.tuple.Tuple
;
import
org.apache.flink.api.java.tuple.Tuple
;
import
org.apache.flink.streaming.api.TimeCharacteristic
;
import
org.apache.flink.streaming.api.TimeCharacteristic
;
import
org.apache.flink.streaming.api.datastream.WindowedStream
;
import
org.apache.flink.streaming.api.datastream.WindowedStream
;
import
org.apache.flink.streaming.api.environment.LocalStreamEnvironment
;
import
org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
;
import
org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
;
import
org.apache.flink.streaming.api.functions.source.ParallelSourceFunction
;
import
org.apache.flink.streaming.api.functions.source.SourceFunction
;
import
org.apache.flink.streaming.api.functions.source.SourceFunction
;
import
org.apache.flink.streaming.api.functions.windowing.RichWindowFunction
;
import
org.apache.flink.streaming.api.functions.windowing.RichWindowFunction
;
import
org.apache.flink.streaming.api.functions.windowing.WindowFunction
;
import
org.apache.flink.streaming.api.functions.windowing.WindowFunction
;
...
...
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录