Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Add some microbenchmarks for Coders and DoFnReflector #553

Closed
wants to merge 8 commits into from
Closed
Show file tree
Hide file tree
Changes from 6 commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
42 changes: 42 additions & 0 deletions sdks/java/microbenchmarks/README.md
@@ -0,0 +1,42 @@
<!--
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.
-->

# Microbenchmarks for parts of the Beam SDK

To run benchmarks:

1. Run `mvn install` in the top directory to install the SDK.

2. Build the benchmark package:

cd microbenchmarks
mvn package

3. run benchmark harness:

java -jar target/microbenchmarks.jar
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do you get legitimate results from an invocation such as mvn exec:java -DmainClass=<whatever> or is there something wrong with that? Just to see if this might be a one liner.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think so, based on reading on Stackoverflow. My goal here is to get the microbenchmarks added for datapoints in other PRs. Let's look into how we can get JMH benchmarks into some database separately, possibly via mvn, etc.


4. (alternate to step 3)
to run just a subset of benchmarks, pass a regular expression that
matches the benchmarks you want to run (this can match against the class
name, or the method name). E.g., to run any benchmarks with
"DoFnReflector" in the name:

java -jar target/microbenchmarks.jar ".*DoFnReflector.*"

110 changes: 110 additions & 0 deletions sdks/java/microbenchmarks/pom.xml
@@ -0,0 +1,110 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
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.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">

<modelVersion>4.0.0</modelVersion>

<parent>
<groupId>org.apache.beam</groupId>
<artifactId>beam-sdks-java-parent</artifactId>
<version>0.2.0-incubating-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath>
</parent>

<artifactId>beam-sdks-java-io-microbenchmarks</artifactId>
<name>Apache Beam :: SDKs :: Java :: Microbenchamkrs</name>
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

typo in "Microbenchmarks"

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Or typo in "Microbenchambers"

<description>Microbenchmarks for components in the Beam Java SDK.</description>
<packaging>jar</packaging>

<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
</plugin>

<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-source-plugin</artifactId>
</plugin>

<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-jar-plugin</artifactId>
</plugin>

<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-checkstyle-plugin</artifactId>
</plugin>

<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-shade-plugin</artifactId>
<executions>
<execution>
<phase>package</phase>
<goals>
<goal>shade</goal>
</goals>
<configuration>
<finalName>microbenchmarks</finalName>
<transformers>
<transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
<mainClass>org.openjdk.jmh.Main</mainClass>
</transformer>
</transformers>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>

<dependencies>
<dependency>
<groupId>org.apache.beam</groupId>
<artifactId>beam-sdks-java-core</artifactId>
</dependency>

<dependency>
<groupId>joda-time</groupId>
<artifactId>joda-time</artifactId>
</dependency>

<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-jdk14</artifactId>
<!-- When loaded at runtime this will wire up slf4j to the JUL backend -->
<scope>runtime</scope>
</dependency>

<dependency>
<groupId>org.openjdk.jmh</groupId>
<artifactId>jmh-core</artifactId>
<version>1.0.1</version>
</dependency>

<dependency>
<groupId>org.openjdk.jmh</groupId>
<artifactId>jmh-generator-annprocess</artifactId>
<version>1.6.1</version>
<scope>provided</scope>
</dependency>
</dependencies>
</project>
@@ -0,0 +1,118 @@
/*
* 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.beam.sdk.coders;

import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.Fork;
import org.openjdk.jmh.annotations.Param;
import org.openjdk.jmh.annotations.Scope;
import org.openjdk.jmh.annotations.Setup;
import org.openjdk.jmh.annotations.State;
import org.openjdk.jmh.annotations.Warmup;

import java.io.IOException;
import java.util.Arrays;

/**
* Benchmarks for AvroCoder.
*/
@State(Scope.Benchmark)
@Fork(1)
@Warmup(iterations = 5)
public class AvroCoderBenchmark {

@DefaultCoder(AvroCoder.class)
private static class Pojo {
public String text;
public int count;

// Empty constructor required for Avro decoding.
@SuppressWarnings("unused")
public Pojo() {
}

public Pojo(String text, int count) {
this.text = text;
this.count = count;
}

// auto-generated
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}

Pojo pojo = (Pojo) o;

if (count != pojo.count) {
return false;
}
if (text != null
? !text.equals(pojo.text)
: pojo.text != null) {
return false;
}

return true;
}

@Override
public int hashCode() {
return 0;
}

@Override
public String toString() {
return "Pojo{"
+ "text='" + text + '\''
+ ", count=" + count
+ '}';
}
}

AvroCoder<Pojo> coder = AvroCoder.of(Pojo.class);

@Param({"true", "false"})
boolean isWholeStream;

Pojo shortPojo;
Pojo longPojo;

@Setup
public void setUp() {
shortPojo = new Pojo("hello world", 42);

char[] bytes60k = new char[60 * 1024];
Arrays.fill(bytes60k, 'a');
longPojo = new Pojo(new String(bytes60k), 42);
}

@Benchmark
public Pojo codeShortPojo() throws IOException {
return CoderBenchmarking.testCoder(coder, isWholeStream, shortPojo);
}

@Benchmark
public Pojo codeLongPojo() throws Exception {
return CoderBenchmarking.testCoder(coder, isWholeStream, longPojo);
}
}
@@ -0,0 +1,64 @@
/*
* 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.beam.sdk.coders;

import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.Fork;
import org.openjdk.jmh.annotations.Param;
import org.openjdk.jmh.annotations.Scope;
import org.openjdk.jmh.annotations.Setup;
import org.openjdk.jmh.annotations.State;
import org.openjdk.jmh.annotations.Warmup;

import java.io.IOException;
import java.util.Arrays;

/**
* Benchmarks for ByteArrayCoder.
*/
@State(Scope.Benchmark)
@Fork(1)
@Warmup(iterations = 5)
public class ByteArrayCoderBenchmark {

ByteArrayCoder coder = ByteArrayCoder.of();

@Param({"true", "false"})
boolean isWholeStream;

byte[] shortArray;
byte[] longArray;

@Setup
public void setUp() {
shortArray = new byte[10];
Arrays.fill(shortArray, (byte) 47);
longArray = new byte[60 * 1024];
Arrays.fill(longArray, (byte) 47);
}

@Benchmark
public byte[] codeShortArray() throws IOException {
return CoderBenchmarking.testCoder(coder, isWholeStream, shortArray);
}

@Benchmark
public byte[] codeLongArray() throws Exception {
return CoderBenchmarking.testCoder(coder, isWholeStream, longArray);
}
}
@@ -0,0 +1,41 @@
/*
* 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.beam.sdk.coders;

import org.apache.beam.sdk.util.CoderUtils;

import java.io.IOException;

/**
* Utilities for writing coder benchmarks.
*/
class CoderBenchmarking {

/**
* Encodes and decodes the given value using the specified Coder.
*
* @throws IOException if there are errors during encoding or decoding
*/
public static <T> T testCoder(
Coder<T> coder, boolean isWholeStream, T value) throws IOException {
Coder.Context context =
isWholeStream ? Coder.Context.OUTER : Coder.Context.NESTED;
byte[] encoded = CoderUtils.encodeToByteArray(coder, value, context);
return CoderUtils.decodeFromByteArray(coder, encoded, context);
}
}