Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
doujutun3207
flink
提交
b474d287
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,发现更多精彩内容 >>
未验证
提交
b474d287
编写于
12月 24, 2020
作者:
A
Aljoscha Krettek
提交者:
Xintong Song
12月 25, 2020
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
[FLINK-20764] In BatchGroupedReduceOperator, fix timer logic
This closes #14489
上级
4e5448cc
变更
2
显示空白变更内容
内联
并排
Showing
2 changed file
with
130 addition
and
2 deletion
+130
-2
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/BatchGroupedReduceOperator.java
...k/streaming/api/operators/BatchGroupedReduceOperator.java
+6
-2
flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/BatchGroupedReduceOperatorTest.java
...reaming/api/operators/BatchGroupedReduceOperatorTest.java
+124
-0
未找到文件。
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/BatchGroupedReduceOperator.java
浏览文件 @
b474d287
...
...
@@ -69,9 +69,13 @@ public class BatchGroupedReduceOperator<IN, KEY>
IN
value
=
element
.
getValue
();
IN
currentValue
=
values
.
value
();
if
(
currentValue
!=
null
)
{
value
=
userFunction
.
reduce
(
currentValue
,
value
);
if
(
currentValue
==
null
)
{
// register a timer for emitting the result at the end when this is the
// first input for this key
timerService
.
registerEventTimeTimer
(
VoidNamespace
.
INSTANCE
,
Long
.
MAX_VALUE
);
}
else
{
// otherwise, reduce things
value
=
userFunction
.
reduce
(
currentValue
,
value
);
}
values
.
update
(
value
);
}
...
...
flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/BatchGroupedReduceOperatorTest.java
0 → 100644
浏览文件 @
b474d287
/*
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.api.operators
;
import
org.apache.flink.api.common.functions.ReduceFunction
;
import
org.apache.flink.api.common.typeinfo.BasicTypeInfo
;
import
org.apache.flink.api.common.typeutils.base.StringSerializer
;
import
org.apache.flink.streaming.api.watermark.Watermark
;
import
org.apache.flink.streaming.runtime.streamrecord.StreamRecord
;
import
org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness
;
import
org.apache.flink.util.TestLogger
;
import
org.junit.Rule
;
import
org.junit.Test
;
import
org.junit.rules.ExpectedException
;
import
java.util.ArrayDeque
;
import
static
org
.
hamcrest
.
Matchers
.
contains
;
import
static
org
.
hamcrest
.
Matchers
.
empty
;
import
static
org
.
junit
.
Assert
.
assertThat
;
/**
* Tests {@link BatchGroupedReduceOperator}.
*/
public
class
BatchGroupedReduceOperatorTest
extends
TestLogger
{
@Rule
public
ExpectedException
expectedException
=
ExpectedException
.
none
();
@Test
public
void
noIncrementalResults
()
throws
Exception
{
KeyedOneInputStreamOperatorTestHarness
<
String
,
String
,
String
>
testHarness
=
createTestHarness
();
testHarness
.
processElement
(
new
StreamRecord
<>(
"hello"
));
testHarness
.
processElement
(
new
StreamRecord
<>(
"hello"
));
testHarness
.
processElement
(
new
StreamRecord
<>(
"ciao"
));
testHarness
.
processElement
(
new
StreamRecord
<>(
"ciao"
));
assertThat
(
testHarness
.
getOutput
(),
empty
());
}
@Test
public
void
resultsOnMaxWatermark
()
throws
Exception
{
KeyedOneInputStreamOperatorTestHarness
<
String
,
String
,
String
>
testHarness
=
createTestHarness
();
testHarness
.
processElement
(
new
StreamRecord
<>(
"hello"
));
testHarness
.
processElement
(
new
StreamRecord
<>(
"hello"
));
testHarness
.
processElement
(
new
StreamRecord
<>(
"ciao"
));
testHarness
.
processElement
(
new
StreamRecord
<>(
"ciao"
));
testHarness
.
processElement
(
new
StreamRecord
<>(
"ciao"
));
testHarness
.
processWatermark
(
Long
.
MAX_VALUE
);
ArrayDeque
<
Object
>
expectedOutput
=
new
ArrayDeque
<>();
expectedOutput
.
add
(
new
StreamRecord
<>(
"hellohello"
,
Long
.
MAX_VALUE
));
expectedOutput
.
add
(
new
StreamRecord
<>(
"ciaociaociao"
,
Long
.
MAX_VALUE
));
expectedOutput
.
add
(
new
Watermark
(
Long
.
MAX_VALUE
));
assertThat
(
testHarness
.
getOutput
(),
contains
(
expectedOutput
.
toArray
()));
}
@Test
public
void
resultForSingleInput
()
throws
Exception
{
KeyedOneInputStreamOperatorTestHarness
<
String
,
String
,
String
>
testHarness
=
createTestHarness
();
testHarness
.
processElement
(
new
StreamRecord
<>(
"hello"
));
testHarness
.
processElement
(
new
StreamRecord
<>(
"ciao"
));
testHarness
.
processWatermark
(
Long
.
MAX_VALUE
);
ArrayDeque
<
Object
>
expectedOutput
=
new
ArrayDeque
<>();
expectedOutput
.
add
(
new
StreamRecord
<>(
"hello"
,
Long
.
MAX_VALUE
));
expectedOutput
.
add
(
new
StreamRecord
<>(
"ciao"
,
Long
.
MAX_VALUE
));
expectedOutput
.
add
(
new
Watermark
(
Long
.
MAX_VALUE
));
assertThat
(
testHarness
.
getOutput
(),
contains
(
expectedOutput
.
toArray
()));
}
private
KeyedOneInputStreamOperatorTestHarness
<
String
,
String
,
String
>
createTestHarness
()
throws
Exception
{
BatchGroupedReduceOperator
<
String
,
Object
>
operator
=
new
BatchGroupedReduceOperator
<>(
new
Concatenator
(),
StringSerializer
.
INSTANCE
);
KeyedOneInputStreamOperatorTestHarness
<
String
,
String
,
String
>
testHarness
=
new
KeyedOneInputStreamOperatorTestHarness
<>(
operator
,
in
->
in
,
BasicTypeInfo
.
STRING_TYPE_INFO
);
testHarness
.
setup
();
testHarness
.
open
();
return
testHarness
;
}
static
class
Concatenator
implements
ReduceFunction
<
String
>
{
@Override
public
String
reduce
(
String
value1
,
String
value2
)
throws
Exception
{
return
value1
+
value2
;
}
}
}
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录