You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by pn...@apache.org on 2018/09/17 11:55:50 UTC
[flink] branch master updated: [hotfix][benchmarks] Add network
broadcast benchmark
This is an automated email from the ASF dual-hosted git repository.
pnowojski pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git
The following commit(s) were added to refs/heads/master by this push:
new 809ad35 [hotfix][benchmarks] Add network broadcast benchmark
809ad35 is described below
commit 809ad3585237a8ca406ae1a0907ea6e3a3ba4f76
Author: Piotr Nowojski <pi...@gmail.com>
AuthorDate: Fri Sep 14 14:03:14 2018 +0200
[hotfix][benchmarks] Add network broadcast benchmark
---
.../io/benchmark/LongRecordWriterThread.java | 13 +++++-
.../StreamNetworkBenchmarkEnvironment.java | 4 ++
.../StreamNetworkBroadcastThroughputBenchmark.java | 52 ++++++++++++++++++++++
...eamNetworkBroadcastThroughputBenchmarkTest.java | 29 ++++++++++++
.../StreamNetworkPointToPointBenchmark.java | 2 +-
.../StreamNetworkThroughputBenchmark.java | 21 ++++++---
.../StreamNetworkThroughputBenchmarkTest.java | 12 +++--
7 files changed, 121 insertions(+), 12 deletions(-)
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/LongRecordWriterThread.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/LongRecordWriterThread.java
index 05ae276..ba3294b 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/LongRecordWriterThread.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/LongRecordWriterThread.java
@@ -35,6 +35,7 @@ import static org.apache.flink.util.Preconditions.checkState;
*/
public class LongRecordWriterThread extends CheckedThread {
private final StreamRecordWriter<LongValue> recordWriter;
+ private final boolean broadcastMode;
/**
* Future to wait on a definition of the number of records to send.
@@ -43,8 +44,11 @@ public class LongRecordWriterThread extends CheckedThread {
private volatile boolean running = true;
- public LongRecordWriterThread(StreamRecordWriter<LongValue> recordWriter) {
+ public LongRecordWriterThread(
+ StreamRecordWriter<LongValue> recordWriter,
+ boolean broadcastMode) {
this.recordWriter = checkNotNull(recordWriter);
+ this.broadcastMode = broadcastMode;
}
public synchronized void shutdown() {
@@ -89,7 +93,12 @@ public class LongRecordWriterThread extends CheckedThread {
LongValue value = new LongValue(0);
for (int i = 1; i < records; i++) {
- recordWriter.emit(value);
+ if (broadcastMode) {
+ recordWriter.broadcastEmit(value);
+ }
+ else {
+ recordWriter.emit(value);
+ }
}
value.setValue(records);
recordWriter.broadcastEmit(value);
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkBenchmarkEnvironment.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkBenchmarkEnvironment.java
index bfaed43..1b769c8 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkBenchmarkEnvironment.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkBenchmarkEnvironment.java
@@ -87,6 +87,7 @@ public class StreamNetworkBenchmarkEnvironment<T extends IOReadableWritable> {
protected IOManager ioManager;
protected int channels;
+ protected boolean broadcastMode = false;
protected boolean localMode = false;
protected ResultPartitionID[] partitionIds;
@@ -100,6 +101,7 @@ public class StreamNetworkBenchmarkEnvironment<T extends IOReadableWritable> {
setUp(
writers,
channels,
+ false,
localMode,
senderBufferPoolSize,
receiverBufferPoolSize,
@@ -123,10 +125,12 @@ public class StreamNetworkBenchmarkEnvironment<T extends IOReadableWritable> {
public void setUp(
int writers,
int channels,
+ boolean broadcastMode,
boolean localMode,
int senderBufferPoolSize,
int receiverBufferPoolSize,
Configuration config) throws Exception {
+ this.broadcastMode = broadcastMode;
this.localMode = localMode;
this.channels = channels;
this.partitionIds = new ResultPartitionID[writers];
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkBroadcastThroughputBenchmark.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkBroadcastThroughputBenchmark.java
new file mode 100644
index 0000000..f4854d6
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkBroadcastThroughputBenchmark.java
@@ -0,0 +1,52 @@
+/*
+ * 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.configuration.Configuration;
+
+/**
+ * Network throughput benchmarks executed by the external
+ * <a href="https://github.com/dataArtisans/flink-benchmarks">flink-benchmarks</a> project.
+ */
+public class StreamNetworkBroadcastThroughputBenchmark extends StreamNetworkThroughputBenchmark {
+
+ /**
+ * Same as {@link StreamNetworkThroughputBenchmark#setUp(int, int, int, boolean, int, int)}
+ * but also setups broadcast mode.
+ */
+ @Override
+ public void setUp(
+ int recordWriters,
+ int channels,
+ int flushTimeout,
+ boolean localMode,
+ int senderBufferPoolSize,
+ int receiverBufferPoolSize) throws Exception {
+ setUp(
+ recordWriters,
+ channels,
+ flushTimeout,
+ true,
+ localMode,
+ senderBufferPoolSize,
+ receiverBufferPoolSize,
+ new Configuration()
+ );
+ }
+}
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkBroadcastThroughputBenchmarkTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkBroadcastThroughputBenchmarkTest.java
new file mode 100644
index 0000000..acee9fe
--- /dev/null
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkBroadcastThroughputBenchmarkTest.java
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+/**
+ * Tests for various network benchmarks based on {@link StreamNetworkBroadcastThroughputBenchmark}.
+ */
+public class StreamNetworkBroadcastThroughputBenchmarkTest extends StreamNetworkThroughputBenchmarkTest {
+ @Override
+ protected StreamNetworkThroughputBenchmark createBenchmark() {
+ return new StreamNetworkBroadcastThroughputBenchmark();
+ }
+}
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkPointToPointBenchmark.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkPointToPointBenchmark.java
index a8d18e4..bf6fdc4 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkPointToPointBenchmark.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkPointToPointBenchmark.java
@@ -75,7 +75,7 @@ public class StreamNetworkPointToPointBenchmark {
*/
public void setUp(long flushTimeout, Configuration config) throws Exception {
environment = new StreamNetworkBenchmarkEnvironment<>();
- environment.setUp(1, 1, false, -1, -1, config);
+ environment.setUp(1, 1, false, false, -1, -1, config);
receiver = environment.createReceiver();
recordWriter = environment.createRecordWriter(0, flushTimeout);
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkThroughputBenchmark.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkThroughputBenchmark.java
index 28d7f35..0586f54 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkThroughputBenchmark.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkThroughputBenchmark.java
@@ -29,9 +29,9 @@ import java.util.concurrent.TimeUnit;
* <a href="https://github.com/dataArtisans/flink-benchmarks">flink-benchmarks</a> project.
*/
public class StreamNetworkThroughputBenchmark {
- private StreamNetworkBenchmarkEnvironment<LongValue> environment;
- private ReceiverThread receiver;
- private LongRecordWriterThread[] writerThreads;
+ protected StreamNetworkBenchmarkEnvironment<LongValue> environment;
+ protected ReceiverThread receiver;
+ protected LongRecordWriterThread[] writerThreads;
public void executeBenchmark(long records) throws Exception {
executeBenchmark(records, Long.MAX_VALUE);
@@ -75,6 +75,7 @@ public class StreamNetworkThroughputBenchmark {
recordWriters,
channels,
flushTimeout,
+ false,
localMode,
senderBufferPoolSize,
receiverBufferPoolSize,
@@ -95,16 +96,26 @@ public class StreamNetworkThroughputBenchmark {
int recordWriters,
int channels,
int flushTimeout,
+ boolean broadcastMode,
boolean localMode,
int senderBufferPoolSize,
int receiverBufferPoolSize,
Configuration config) throws Exception {
environment = new StreamNetworkBenchmarkEnvironment<>();
- environment.setUp(recordWriters, channels, localMode, senderBufferPoolSize, receiverBufferPoolSize, config);
+ environment.setUp(
+ recordWriters,
+ channels,
+ broadcastMode,
+ localMode,
+ senderBufferPoolSize,
+ receiverBufferPoolSize,
+ config);
receiver = environment.createReceiver();
writerThreads = new LongRecordWriterThread[recordWriters];
for (int writer = 0; writer < recordWriters; writer++) {
- writerThreads[writer] = new LongRecordWriterThread(environment.createRecordWriter(writer, flushTimeout));
+ writerThreads[writer] = new LongRecordWriterThread(
+ environment.createRecordWriter(writer, flushTimeout),
+ broadcastMode);
writerThreads[writer].start();
}
}
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkThroughputBenchmarkTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkThroughputBenchmarkTest.java
index dac8ee2..97675db 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkThroughputBenchmarkTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkThroughputBenchmarkTest.java
@@ -33,9 +33,13 @@ public class StreamNetworkThroughputBenchmarkTest {
@Rule
public ExpectedException expectedException = ExpectedException.none();
+ protected StreamNetworkThroughputBenchmark createBenchmark() {
+ return new StreamNetworkThroughputBenchmark();
+ }
+
@Test
public void pointToPointBenchmark() throws Exception {
- StreamNetworkThroughputBenchmark benchmark = new StreamNetworkThroughputBenchmark();
+ StreamNetworkThroughputBenchmark benchmark = createBenchmark();
benchmark.setUp(1, 1, 100);
try {
benchmark.executeBenchmark(1_000);
@@ -107,7 +111,7 @@ public class StreamNetworkThroughputBenchmarkTest {
@Test
public void pointToMultiPointBenchmark() throws Exception {
- StreamNetworkThroughputBenchmark benchmark = new StreamNetworkThroughputBenchmark();
+ StreamNetworkThroughputBenchmark benchmark = createBenchmark();
benchmark.setUp(1, 100, 100);
try {
benchmark.executeBenchmark(1_000);
@@ -119,7 +123,7 @@ public class StreamNetworkThroughputBenchmarkTest {
@Test
public void multiPointToPointBenchmark() throws Exception {
- StreamNetworkThroughputBenchmark benchmark = new StreamNetworkThroughputBenchmark();
+ StreamNetworkThroughputBenchmark benchmark = createBenchmark();
benchmark.setUp(4, 1, 100);
try {
benchmark.executeBenchmark(1_000);
@@ -131,7 +135,7 @@ public class StreamNetworkThroughputBenchmarkTest {
@Test
public void multiPointToMultiPointBenchmark() throws Exception {
- StreamNetworkThroughputBenchmark benchmark = new StreamNetworkThroughputBenchmark();
+ StreamNetworkThroughputBenchmark benchmark = createBenchmark();
benchmark.setUp(4, 100, 100);
try {
benchmark.executeBenchmark(1_000);