Repository: kafka Updated Branches: refs/heads/0.11.0 358ccf093 -> 296dfb5bf
MINOR: Remove redundant volatile write in RecordHeaders The JMH benchmark included shows that the redundant volatile write causes the constructor of `ProducerRecord` to take more than 50% longer: ProducerRecordBenchmark.constructorBenchmark avgt 15 24.136 ± 1.458 ns/op (before) ProducerRecordBenchmark.constructorBenchmark avgt 15 14.904 ± 0.231 ns/op (after) Author: Ismael Juma <[email protected]> Reviewers: Jason Gustafson <[email protected]> Closes #3233 from ijuma/remove-volatile-write-in-records-header-constructor (cherry picked from commit c7bc8f7d8c0101103853b5f6778edb04d1f41b76) Signed-off-by: Jason Gustafson <[email protected]> Project: http://git-wip-us.apache.org/repos/asf/kafka/repo Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/296dfb5b Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/296dfb5b Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/296dfb5b Branch: refs/heads/0.11.0 Commit: 296dfb5bf48863c77d49fbece3a87b163c86ce4f Parents: 358ccf0 Author: Ismael Juma <[email protected]> Authored: Sun Jun 4 10:48:34 2017 -0700 Committer: Jason Gustafson <[email protected]> Committed: Sun Jun 4 10:48:43 2017 -0700 ---------------------------------------------------------------------- checkstyle/import-control.xml | 1 + .../common/header/internals/RecordHeaders.java | 2 +- gradle/findbugs-exclude.xml | 1 + .../jmh/producer/ProducerRecordBenchmark.java | 47 ++++++++++++++++++++ 4 files changed, 50 insertions(+), 1 deletion(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/kafka/blob/296dfb5b/checkstyle/import-control.xml ---------------------------------------------------------------------- diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index f83f6e7..26f4a77 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -216,6 +216,7 @@ <allow pkg="org.openjdk.jmh.runner.options" /> <allow pkg="org.openjdk.jmh.infra" /> <allow pkg="org.apache.kafka.common" /> + <allow pkg="org.apache.kafka.clients" /> <allow pkg="org.apache.kafka.streams" /> <allow pkg="org.github.jamm" /> </subpackage> http://git-wip-us.apache.org/repos/asf/kafka/blob/296dfb5b/clients/src/main/java/org/apache/kafka/common/header/internals/RecordHeaders.java ---------------------------------------------------------------------- diff --git a/clients/src/main/java/org/apache/kafka/common/header/internals/RecordHeaders.java b/clients/src/main/java/org/apache/kafka/common/header/internals/RecordHeaders.java index afd991f..141c972 100644 --- a/clients/src/main/java/org/apache/kafka/common/header/internals/RecordHeaders.java +++ b/clients/src/main/java/org/apache/kafka/common/header/internals/RecordHeaders.java @@ -30,7 +30,7 @@ import org.apache.kafka.common.utils.AbstractIterator; public class RecordHeaders implements Headers { private final List<Header> headers; - private volatile boolean isReadOnly = false; + private volatile boolean isReadOnly; public RecordHeaders() { this((Iterable<Header>) null); http://git-wip-us.apache.org/repos/asf/kafka/blob/296dfb5b/gradle/findbugs-exclude.xml ---------------------------------------------------------------------- diff --git a/gradle/findbugs-exclude.xml b/gradle/findbugs-exclude.xml index eec9268..9262a78 100644 --- a/gradle/findbugs-exclude.xml +++ b/gradle/findbugs-exclude.xml @@ -165,6 +165,7 @@ For a detailed description of findbugs bug categories, see http://findbugs.sourc <Or> <Package name="org.apache.kafka.jmh.cache.generated"/> <Package name="org.apache.kafka.jmh.record.generated"/> + <Package name="org.apache.kafka.jmh.producer.generated"/> </Or> </Match> http://git-wip-us.apache.org/repos/asf/kafka/blob/296dfb5b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerRecordBenchmark.java ---------------------------------------------------------------------- diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerRecordBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerRecordBenchmark.java new file mode 100644 index 0000000..e7b3982 --- /dev/null +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerRecordBenchmark.java @@ -0,0 +1,47 @@ +/* + * 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.kafka.jmh.producer; + +import org.apache.kafka.clients.producer.ProducerRecord; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Warmup; + +import java.util.concurrent.TimeUnit; + +@State(Scope.Benchmark) +@Fork(value = 1) +@Warmup(iterations = 5) +@Measurement(iterations = 15) +@BenchmarkMode(Mode.AverageTime) +@OutputTimeUnit(TimeUnit.NANOSECONDS) +public class ProducerRecordBenchmark { + + @Benchmark + @OutputTimeUnit(TimeUnit.NANOSECONDS) + public ProducerRecord<String, String> constructorBenchmark() { + return new ProducerRecord("topic", "value"); + } + +}
