Skip to content

Commit

Permalink
This closes #553
Browse files Browse the repository at this point in the history
  • Loading branch information
bchambers committed Jul 2, 2016
2 parents c834ecd + fa8bf32 commit 88db3be
Show file tree
Hide file tree
Showing 9 changed files with 699 additions and 6 deletions.
Expand Up @@ -123,7 +123,7 @@ <InputT, OutputT> Object createInstance(
* @param c the {@link org.apache.beam.sdk.transforms.DoFnWithContext.ProcessContext}
* to pass to {@link ProcessElement}.
*/
abstract <InputT, OutputT> void invokeProcessElement(
public abstract <InputT, OutputT> void invokeProcessElement(
DoFnWithContext<InputT, OutputT> fn,
DoFnWithContext<InputT, OutputT>.ProcessContext c,
ExtraContextFactory<InputT, OutputT> extra);
Expand All @@ -135,7 +135,7 @@ abstract <InputT, OutputT> void invokeProcessElement(
* @param c the {@link org.apache.beam.sdk.transforms.DoFnWithContext.Context}
* to pass to {@link StartBundle}.
*/
<InputT, OutputT> void invokeStartBundle(
public <InputT, OutputT> void invokeStartBundle(
DoFnWithContext<InputT, OutputT> fn,
DoFnWithContext<InputT, OutputT>.Context c,
ExtraContextFactory<InputT, OutputT> extra) {
Expand All @@ -149,7 +149,7 @@ <InputT, OutputT> void invokeStartBundle(
* @param c the {@link org.apache.beam.sdk.transforms.DoFnWithContext.Context}
* to pass to {@link FinishBundle}.
*/
abstract <InputT, OutputT> void invokeFinishBundle(
public abstract <InputT, OutputT> void invokeFinishBundle(
DoFnWithContext<InputT, OutputT> fn,
DoFnWithContext<InputT, OutputT>.Context c,
ExtraContextFactory<InputT, OutputT> extra);
Expand Down Expand Up @@ -430,15 +430,15 @@ private boolean usesContext(Class<?> context) {
}

@Override
<InputT, OutputT> void invokeProcessElement(
public <InputT, OutputT> void invokeProcessElement(
DoFnWithContext<InputT, OutputT> fn,
DoFnWithContext<InputT, OutputT>.ProcessContext c,
ExtraContextFactory<InputT, OutputT> extra) {
invoke(processElement, fn, c, extra, processElementArgs);
}

@Override
<InputT, OutputT> void invokeStartBundle(
public <InputT, OutputT> void invokeStartBundle(
DoFnWithContext<InputT, OutputT> fn,
DoFnWithContext<InputT, OutputT>.Context c,
ExtraContextFactory<InputT, OutputT> extra) {
Expand All @@ -449,7 +449,7 @@ <InputT, OutputT> void invokeStartBundle(
}

@Override
<InputT, OutputT> void invokeFinishBundle(
public <InputT, OutputT> void invokeFinishBundle(
DoFnWithContext<InputT, OutputT> fn,
DoFnWithContext<InputT, OutputT>.Context c,
ExtraContextFactory<InputT, OutputT> extra) {
Expand Down
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

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-microbenchmarks</artifactId>
<name>Apache Beam :: SDKs :: Java :: Microbenchmarks</name>
<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,121 @@
/*
* 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.microbenchmarks.coders;

import org.apache.beam.sdk.coders.AvroCoder;
import org.apache.beam.sdk.coders.DefaultCoder;

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 {@link 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,66 @@
/*
* 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.microbenchmarks.coders;

import org.apache.beam.sdk.coders.ByteArrayCoder;

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 {@link 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 comments on commit 88db3be

Please sign in to comment.