Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
doujutun3207
flink
提交
2558ae51
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,发现更多精彩内容 >>
提交
2558ae51
编写于
12月 07, 2017
作者:
N
Nico Kruber
提交者:
Stefan Richter
1月 08, 2018
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
[FLINK-8221][network-benchmarks] Define latency network benchmarks in Flink project
This closes #5255.
上级
c8161911
变更
3
隐藏空白更改
内联
并排
Showing
3 changed file
with
165 addition
and
0 deletion
+165
-0
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkBenchmarkEnvironment.java
...ntime/io/benchmark/StreamNetworkBenchmarkEnvironment.java
+40
-0
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkPointToPointBenchmark.java
...time/io/benchmark/StreamNetworkPointToPointBenchmark.java
+87
-0
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkPointToPointBenchmarkTest.java
.../io/benchmark/StreamNetworkPointToPointBenchmarkTest.java
+38
-0
未找到文件。
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkBenchmarkEnvironment.java
0 → 100644
浏览文件 @
2558ae51
/*
* 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.io.benchmark
;
import
org.apache.flink.core.io.IOReadableWritable
;
import
org.apache.flink.runtime.io.network.api.writer.RecordWriter
;
import
org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter
;
import
org.apache.flink.runtime.io.network.api.writer.RoundRobinChannelSelector
;
import
org.apache.flink.runtime.io.network.benchmark.NetworkBenchmarkEnvironment
;
import
org.apache.flink.streaming.runtime.io.StreamRecordWriter
;
/**
* Context for stream network benchmarks executed by the external
* <a href="https://github.com/dataArtisans/flink-benchmarks">flink-benchmarks</a> project.
*/
public
class
StreamNetworkBenchmarkEnvironment
<
T
extends
IOReadableWritable
>
extends
NetworkBenchmarkEnvironment
<
T
>
{
public
RecordWriter
<
T
>
createStreamRecordWriter
(
int
partitionIndex
,
long
flushTimeout
)
throws
Exception
{
ResultPartitionWriter
sender
=
createResultPartition
(
jobId
,
partitionIds
[
partitionIndex
],
senderEnv
,
channels
);
return
new
StreamRecordWriter
<>(
sender
,
new
RoundRobinChannelSelector
<>(),
flushTimeout
);
}
}
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkPointToPointBenchmark.java
0 → 100644
浏览文件 @
2558ae51
/*
* 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.io.benchmark
;
import
org.apache.flink.runtime.io.network.api.writer.RecordWriter
;
import
org.apache.flink.runtime.io.network.benchmark.ReceiverThread
;
import
org.apache.flink.types.LongValue
;
import
java.util.concurrent.CompletableFuture
;
import
java.util.concurrent.TimeUnit
;
/**
* Streaming point-to-point latency network benchmarks executed by the external
* <a href="https://github.com/dataArtisans/flink-benchmarks">flink-benchmarks</a> project.
*/
public
class
StreamNetworkPointToPointBenchmark
{
private
static
final
long
RECEIVER_TIMEOUT
=
2000
;
private
StreamNetworkBenchmarkEnvironment
<
LongValue
>
environment
;
private
ReceiverThread
receiver
;
private
RecordWriter
<
LongValue
>
recordWriter
;
/**
* Executes the latency benchmark with the given number of records.
*
* @param records
* records to pass through the network stack
* @param flushAfterLastEmit
* whether to flush the {@link RecordWriter} after the last record
*/
public
void
executeBenchmark
(
long
records
,
boolean
flushAfterLastEmit
)
throws
Exception
{
final
LongValue
value
=
new
LongValue
();
value
.
setValue
(
0
);
CompletableFuture
<?>
recordsReceived
=
receiver
.
setExpectedRecord
(
records
);
for
(
int
i
=
1
;
i
<
records
;
i
++)
{
recordWriter
.
emit
(
value
);
}
value
.
setValue
(
records
);
recordWriter
.
broadcastEmit
(
value
);
if
(
flushAfterLastEmit
)
{
recordWriter
.
flush
();
}
recordsReceived
.
get
(
RECEIVER_TIMEOUT
,
TimeUnit
.
MILLISECONDS
);
}
/**
* Initializes the throughput benchmark with the given parameters.
*
* @param flushTimeout
* output flushing interval of the
* {@link org.apache.flink.streaming.runtime.io.StreamRecordWriter}'s output flusher thread
*/
public
void
setUp
(
long
flushTimeout
)
throws
Exception
{
environment
=
new
StreamNetworkBenchmarkEnvironment
<>();
environment
.
setUp
(
1
,
1
);
receiver
=
environment
.
createReceiver
();
recordWriter
=
environment
.
createStreamRecordWriter
(
0
,
flushTimeout
);
}
/**
* Shuts down a benchmark previously set up via {@link #setUp}.
*/
public
void
tearDown
()
{
environment
.
tearDown
();
receiver
.
shutdown
();
}
}
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkPointToPointBenchmarkTest.java
0 → 100644
浏览文件 @
2558ae51
/*
* 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.io.benchmark
;
import
org.junit.Test
;
/**
* Tests for {@link StreamNetworkPointToPointBenchmark}.
*/
public
class
StreamNetworkPointToPointBenchmarkTest
{
@Test
public
void
test
()
throws
Exception
{
StreamNetworkPointToPointBenchmark
benchmark
=
new
StreamNetworkPointToPointBenchmark
();
benchmark
.
setUp
(
10
);
try
{
benchmark
.
executeBenchmark
(
100
,
false
);
}
finally
{
benchmark
.
tearDown
();
}
}
}
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录