Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
doujutun3207
flink
提交
5e464871
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,发现更多精彩内容 >>
提交
5e464871
编写于
1月 07, 2019
作者:
D
Dawid Wysakowicz
提交者:
Dawid Wysakowicz
1月 09, 2019
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
[hotfix][cep] Change contract of cep TimerContext#timestamp to never
return null
上级
9742ef7a
变更
5
隐藏空白更改
内联
并排
Showing
5 changed file
with
28 addition
and
30 deletion
+28
-30
flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/context/TimerContext.java
.../main/java/org/apache/flink/cep/context/TimerContext.java
+3
-3
flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/functions/adaptors/PatternTimeoutFlatSelectAdapter.java
...p/functions/adaptors/PatternTimeoutFlatSelectAdapter.java
+1
-2
flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/functions/adaptors/PatternTimeoutSelectAdapter.java
...k/cep/functions/adaptors/PatternTimeoutSelectAdapter.java
+1
-2
flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/CepOperator.java
.../main/java/org/apache/flink/cep/operator/CepOperator.java
+4
-7
flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CepProcessFunctionContextTest.java
...che/flink/cep/operator/CepProcessFunctionContextTest.java
+19
-16
未找到文件。
flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/context/TimerContext.java
浏览文件 @
5e464871
...
...
@@ -32,10 +32,10 @@ public interface TimerContext {
/**
* Timestamp of the element currently being processed.
*
* <p>
This might be {@code null}, for example if the time characteristic of your program
*
is set to {@link org.apache.flink.streaming.api.TimeCharacteristic#ProcessingTime}
.
* <p>
In case of {@link org.apache.flink.streaming.api.TimeCharacteristic#ProcessingTime} this will be set to the
*
time when event entered the cep operator
.
*/
L
ong
timestamp
();
l
ong
timestamp
();
/** Returns the current processing time. */
long
currentProcessingTime
();
...
...
flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/functions/adaptors/PatternTimeoutFlatSelectAdapter.java
浏览文件 @
5e464871
...
...
@@ -78,8 +78,7 @@ public class PatternTimeoutFlatSelectAdapter<IN, OUT, T>
Map
<
String
,
List
<
IN
>>
match
,
Context
ctx
)
throws
Exception
{
sideCollector
.
setCtx
(
ctx
);
long
timestamp
=
ctx
.
timestamp
()
!=
null
?
ctx
.
timestamp
()
:
ctx
.
currentProcessingTime
();
flatTimeoutFunction
.
timeout
(
match
,
timestamp
,
sideCollector
);
flatTimeoutFunction
.
timeout
(
match
,
ctx
.
timestamp
(),
sideCollector
);
}
/**
...
...
flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/functions/adaptors/PatternTimeoutSelectAdapter.java
浏览文件 @
5e464871
...
...
@@ -71,8 +71,7 @@ public class PatternTimeoutSelectAdapter<IN, OUT, T>
final
Map
<
String
,
List
<
IN
>>
match
,
final
Context
ctx
)
throws
Exception
{
final
long
resultTimestamp
=
ctx
.
timestamp
()
!=
null
?
ctx
.
timestamp
()
:
ctx
.
currentProcessingTime
();
final
T
timedOutPatternResult
=
timeoutFunction
.
timeout
(
match
,
resultTimestamp
);
final
T
timedOutPatternResult
=
timeoutFunction
.
timeout
(
match
,
ctx
.
timestamp
());
ctx
.
output
(
timedOutPartialMatchesTag
,
timedOutPatternResult
);
}
...
...
flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/CepOperator.java
浏览文件 @
5e464871
...
...
@@ -459,8 +459,9 @@ public class CepOperator<IN, KEY, OUT>
private
void
setTimestamp
(
long
timestamp
)
{
if
(!
isProcessingTime
)
{
collector
.
setAbsoluteTimestamp
(
timestamp
);
context
.
setTimestamp
(
timestamp
);
}
context
.
setTimestamp
(
timestamp
);
}
/**
...
...
@@ -493,12 +494,8 @@ public class CepOperator<IN, KEY, OUT>
}
@Override
public
Long
timestamp
()
{
if
(
isProcessingTime
)
{
return
null
;
}
else
{
return
timestamp
;
}
public
long
timestamp
()
{
return
timestamp
;
}
@Override
...
...
flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CepProcessFunctionContextTest.java
浏览文件 @
5e464871
...
...
@@ -45,8 +45,8 @@ import static org.apache.flink.cep.utils.OutputAsserter.assertOutput;
*/
public
class
CepProcessFunctionContextTest
extends
TestLogger
{
private
static
final
boolean
PROCESSING_TIME
=
fals
e
;
private
static
final
boolean
EVENT_TIME
=
tru
e
;
private
static
final
boolean
PROCESSING_TIME
=
tru
e
;
private
static
final
boolean
EVENT_TIME
=
fals
e
;
@Test
public
void
testTimestampPassingInEventTime
()
throws
Exception
{
...
...
@@ -56,7 +56,7 @@ public class CepProcessFunctionContextTest extends TestLogger {
createCepOperator
(
extractTimestampAndNames
(
1
),
new
NFAForwardingFactory
(),
PROCESSING
_TIME
)))
{
EVENT
_TIME
)))
{
harness
.
open
();
// events out of order to test if internal sorting does not mess up the timestamps
...
...
@@ -81,15 +81,18 @@ public class CepProcessFunctionContextTest extends TestLogger {
createCepOperator
(
extractTimestampAndNames
(
1
),
new
NFAForwardingFactory
(),
EVENT
_TIME
)))
{
PROCESSING
_TIME
)))
{
harness
.
open
();
harness
.
setProcessingTime
(
1
);
harness
.
processElement
(
event
().
withName
(
"A"
).
withTimestamp
(
5
).
asStreamRecord
());
harness
.
setProcessingTime
(
2
);
harness
.
processElement
(
event
().
withName
(
"B"
).
withTimestamp
(
3
).
asStreamRecord
());
harness
.
setProcessingTime
(
3
);
assertOutput
(
harness
.
getOutput
())
.
nextElementEquals
(
"
(NO_TIMESTAMP)
:A"
)
.
nextElementEquals
(
"
(NO_TIMESTAMP)
:B"
)
.
nextElementEquals
(
"
1
:A"
)
.
nextElementEquals
(
"
2
:B"
)
.
hasNoMoreElements
();
}
}
...
...
@@ -102,7 +105,7 @@ public class CepProcessFunctionContextTest extends TestLogger {
createCepOperator
(
extractCurrentProcessingTimeAndNames
(
1
),
new
NFAForwardingFactory
(),
EVENT
_TIME
)))
{
PROCESSING
_TIME
)))
{
harness
.
open
();
harness
.
setProcessingTime
(
15
);
...
...
@@ -125,7 +128,7 @@ public class CepProcessFunctionContextTest extends TestLogger {
createCepOperator
(
extractCurrentProcessingTimeAndNames
(
1
),
new
NFAForwardingFactory
(),
PROCESSING
_TIME
)))
{
EVENT
_TIME
)))
{
harness
.
open
();
harness
.
setProcessingTime
(
10
);
...
...
@@ -150,7 +153,7 @@ public class CepProcessFunctionContextTest extends TestLogger {
createCepOperator
(
extractTimestampAndNames
(
2
,
timedOut
),
new
NFATimingOutFactory
(),
PROCESSING
_TIME
)))
{
EVENT
_TIME
)))
{
harness
.
open
();
// events out of order to test if internal sorting does not mess up the timestamps
...
...
@@ -181,7 +184,7 @@ public class CepProcessFunctionContextTest extends TestLogger {
createCepOperator
(
extractTimestampAndNames
(
2
,
timedOut
),
new
NFATimingOutFactory
(),
EVENT
_TIME
)))
{
PROCESSING
_TIME
)))
{
harness
.
open
();
harness
.
setProcessingTime
(
3
);
...
...
@@ -192,11 +195,11 @@ public class CepProcessFunctionContextTest extends TestLogger {
harness
.
processElement
(
event
().
withName
(
"B"
).
withTimestamp
(
20
).
asStreamRecord
());
assertOutput
(
harness
.
getOutput
())
.
nextElementEquals
(
"
(NO_TIMESTAMP)
:A:C"
)
.
nextElementEquals
(
"
5
:A:C"
)
.
hasNoMoreElements
();
assertOutput
(
harness
.
getSideOutput
(
timedOut
))
.
nextElementEquals
(
"
(NO_TIMESTAMP)
:C"
)
.
nextElementEquals
(
"
15
:C"
)
.
hasNoMoreElements
();
}
}
...
...
@@ -211,7 +214,7 @@ public class CepProcessFunctionContextTest extends TestLogger {
createCepOperator
(
extractCurrentProcessingTimeAndNames
(
2
,
sideOutputTag
),
new
NFATimingOutFactory
(),
PROCESSING
_TIME
)))
{
EVENT
_TIME
)))
{
harness
.
open
();
// events out of order to test if internal sorting does not mess up the timestamps
...
...
@@ -243,7 +246,7 @@ public class CepProcessFunctionContextTest extends TestLogger {
createCepOperator
(
extractCurrentProcessingTimeAndNames
(
2
,
sideOutputTag
),
new
NFATimingOutFactory
(),
EVENT
_TIME
)))
{
PROCESSING
_TIME
)))
{
harness
.
open
();
harness
.
setProcessingTime
(
3
);
...
...
@@ -290,7 +293,7 @@ public class CepProcessFunctionContextTest extends TestLogger {
*/
private
static
PatternProcessFunction
<
Event
,
String
>
extractTimestampAndNames
(
int
stateNumber
)
{
return
new
AccessContextWithNames
(
stateNumber
,
context
->
context
.
timestamp
()
!=
null
?
String
.
valueOf
(
context
.
timestamp
())
:
NO_TIMESTAMP
);
context
->
String
.
valueOf
(
context
.
timestamp
())
);
}
/**
...
...
@@ -310,7 +313,7 @@ public class CepProcessFunctionContextTest extends TestLogger {
OutputTag
<
String
>
timedOutTag
)
{
return
new
AccessContextWithNamesWithTimedOut
(
stateNumber
,
timedOutTag
,
context
->
context
.
timestamp
()
!=
null
?
String
.
valueOf
(
context
.
timestamp
())
:
NO_TIMESTAMP
);
context
->
String
.
valueOf
(
context
.
timestamp
())
);
}
/**
...
...
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录