Thanks to visit codestin.com
Credit goes to github.com

Skip to content

Commit e71fdce

Browse files
authored
[KafkaIO] Fix average record size data race and backlog estimation (#34165)
1 parent df5dd4e commit e71fdce

8 files changed

Lines changed: 355 additions & 94 deletions

File tree

build.gradle.kts

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -304,6 +304,7 @@ tasks.register("javaPreCommit") {
304304
dependsOn(":sdks:java:io:contextualtextio:build")
305305
dependsOn(":sdks:java:io:expansion-service:build")
306306
dependsOn(":sdks:java:io:file-based-io-tests:build")
307+
dependsOn(":sdks:java:io:kafka:jmh:build")
307308
dependsOn(":sdks:java:io:sparkreceiver:3:build")
308309
dependsOn(":sdks:java:io:synthetic:build")
309310
dependsOn(":sdks:java:io:xml:build")
Lines changed: 35 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,35 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* License); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an AS IS BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
19+
plugins {
20+
groovy
21+
id("org.apache.beam.module")
22+
}
23+
24+
val applyJavaNature: groovy.lang.Closure<Any?> by extra
25+
applyJavaNature(mapOf(
26+
"automaticModuleName" to "org.apache.beam.sdk.io.kafka.jmh",
27+
"enableJmh" to true,
28+
"publish" to false))
29+
30+
description = "Apache Beam :: SDKs :: Java :: IO :: Kafka :: JMH"
31+
val summary by extra("This contains JMH benchmarks for the Kafka IO connector for Beam Java")
32+
33+
dependencies {
34+
implementation(project(":sdks:java:io:kafka"))
35+
}
Lines changed: 235 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,235 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
package org.apache.beam.sdk.io.kafka.jmh;
19+
20+
import java.util.Random;
21+
import java.util.concurrent.TimeUnit;
22+
import org.apache.beam.sdk.io.kafka.KafkaIOUtils;
23+
import org.openjdk.jmh.annotations.Benchmark;
24+
import org.openjdk.jmh.annotations.BenchmarkMode;
25+
import org.openjdk.jmh.annotations.Group;
26+
import org.openjdk.jmh.annotations.Level;
27+
import org.openjdk.jmh.annotations.Mode;
28+
import org.openjdk.jmh.annotations.OutputTimeUnit;
29+
import org.openjdk.jmh.annotations.Scope;
30+
import org.openjdk.jmh.annotations.Setup;
31+
import org.openjdk.jmh.annotations.State;
32+
import org.openjdk.jmh.annotations.Threads;
33+
import org.openjdk.jmh.infra.IterationParams;
34+
import org.openjdk.jmh.infra.ThreadParams;
35+
36+
@BenchmarkMode(Mode.AverageTime)
37+
@OutputTimeUnit(TimeUnit.NANOSECONDS)
38+
@Threads(Threads.MAX)
39+
public class KafkaIOUtilsBenchmark {
40+
private static final int SIZE = 1024;
41+
42+
@State(Scope.Thread)
43+
public static class ProducerState {
44+
private int[] values;
45+
private int idx;
46+
47+
@Setup(Level.Iteration)
48+
public void setup(final IterationParams ip, final ThreadParams tp) {
49+
values = new Random(299792458 + ip.getCount()).ints(SIZE, 0, 100).toArray();
50+
idx = 0;
51+
}
52+
53+
int next() {
54+
final int value = values[idx];
55+
idx = (idx + 1) % SIZE;
56+
return value;
57+
}
58+
}
59+
60+
@State(Scope.Group)
61+
public static class PlainAccumulatorState {
62+
// As implemented before 2.64.0.
63+
// Note that numUpdates may overflow and count back from Long.MIN_VALUE.
64+
static class MovingAvg {
65+
private static final int MOVING_AVG_WINDOW = 1000;
66+
private double avg = 0;
67+
private long numUpdates = 0;
68+
69+
void update(double quantity) {
70+
numUpdates++;
71+
avg += (quantity - avg) / Math.min(MOVING_AVG_WINDOW, numUpdates);
72+
}
73+
74+
double get() {
75+
return avg;
76+
}
77+
}
78+
79+
MovingAvg accumulator;
80+
81+
@Setup(Level.Trial)
82+
public void setup() {
83+
accumulator = new MovingAvg();
84+
}
85+
}
86+
87+
@State(Scope.Group)
88+
public static class AtomicAccumulatorState {
89+
KafkaIOUtils.MovingAvg accumulator;
90+
91+
@Setup(Level.Trial)
92+
public void setup() {
93+
accumulator = new KafkaIOUtils.MovingAvg();
94+
}
95+
}
96+
97+
@State(Scope.Group)
98+
public static class VolatileAccumulatorState {
99+
// Atomic accumulator using only volatile reads and writes.
100+
static class MovingAvg {
101+
private static final int MOVING_AVG_WINDOW = 1000;
102+
103+
private volatile double avg = 0;
104+
private long numUpdates = 0;
105+
106+
void update(final double quantity) {
107+
final double prevAvg = avg;
108+
numUpdates = Math.min(MOVING_AVG_WINDOW, numUpdates + 1);
109+
avg = prevAvg + (quantity - prevAvg) / numUpdates;
110+
}
111+
112+
double get() {
113+
return avg;
114+
}
115+
}
116+
117+
MovingAvg accumulator;
118+
119+
@Setup(Level.Trial)
120+
public void setup() {
121+
accumulator = new MovingAvg();
122+
}
123+
}
124+
125+
@Benchmark
126+
@Group("WritePlain")
127+
public void plainWrite(final PlainAccumulatorState as, final ProducerState ps) {
128+
as.accumulator.update(ps.next());
129+
}
130+
131+
@Benchmark
132+
@Group("ReadPlain")
133+
public double plainRead(final PlainAccumulatorState as) {
134+
return as.accumulator.get();
135+
}
136+
137+
@Benchmark
138+
@Group("ReadAndWritePlain")
139+
public void plainWriteWhileReading(final PlainAccumulatorState as, final ProducerState ps) {
140+
as.accumulator.update(ps.next());
141+
}
142+
143+
@Benchmark
144+
@Group("ReadAndWritePlain")
145+
public double plainReadWhileWriting(final PlainAccumulatorState as) {
146+
return as.accumulator.get();
147+
}
148+
149+
@Benchmark
150+
@Group("WriteSynchronizedPlain")
151+
public void synchronizedPlainWrite(final PlainAccumulatorState as, final ProducerState ps) {
152+
final PlainAccumulatorState.MovingAvg accumulator = as.accumulator;
153+
final int value = ps.next();
154+
synchronized (accumulator) {
155+
accumulator.update(value);
156+
}
157+
}
158+
159+
@Benchmark
160+
@Group("ReadSynchronizedPlain")
161+
public double synchronizedPlainRead(final PlainAccumulatorState as) {
162+
final PlainAccumulatorState.MovingAvg accumulator = as.accumulator;
163+
synchronized (accumulator) {
164+
return accumulator.get();
165+
}
166+
}
167+
168+
@Benchmark
169+
@Group("ReadAndWriteSynchronizedPlain")
170+
public void synchronizedPlainWriteWhileReading(
171+
final PlainAccumulatorState as, final ProducerState ps) {
172+
final PlainAccumulatorState.MovingAvg accumulator = as.accumulator;
173+
final int value = ps.next();
174+
synchronized (accumulator) {
175+
accumulator.update(value);
176+
}
177+
}
178+
179+
@Benchmark
180+
@Group("ReadAndWriteSynchronizedPlain")
181+
public double synchronizedPlainReadWhileWriting(final PlainAccumulatorState as) {
182+
final PlainAccumulatorState.MovingAvg accumulator = as.accumulator;
183+
synchronized (accumulator) {
184+
return accumulator.get();
185+
}
186+
}
187+
188+
@Benchmark
189+
@Group("WriteAtomic")
190+
public void atomicWrite(final AtomicAccumulatorState as, final ProducerState ps) {
191+
as.accumulator.update(ps.next());
192+
}
193+
194+
@Benchmark
195+
@Group("ReadAtomic")
196+
public double atomicRead(final AtomicAccumulatorState as) {
197+
return as.accumulator.get();
198+
}
199+
200+
@Benchmark
201+
@Group("ReadAndWriteAtomic")
202+
public void atomicWriteWhileReading(final AtomicAccumulatorState as, final ProducerState ps) {
203+
as.accumulator.update(ps.next());
204+
}
205+
206+
@Benchmark
207+
@Group("ReadAndWriteAtomic")
208+
public double atomicReadWhileWriting(final AtomicAccumulatorState as) {
209+
return as.accumulator.get();
210+
}
211+
212+
@Benchmark
213+
@Group("WriteVolatile")
214+
public void volatileWrite(final VolatileAccumulatorState as, final ProducerState ps) {
215+
as.accumulator.update(ps.next());
216+
}
217+
218+
@Benchmark
219+
@Group("ReadVolatile")
220+
public double volatileRead(final VolatileAccumulatorState as) {
221+
return as.accumulator.get();
222+
}
223+
224+
@Benchmark
225+
@Group("ReadAndWriteVolatile")
226+
public void volatileWriteWhileReading(final VolatileAccumulatorState as, final ProducerState ps) {
227+
as.accumulator.update(ps.next());
228+
}
229+
230+
@Benchmark
231+
@Group("ReadAndWriteVolatile")
232+
public double volatileReadWhileWriting(final VolatileAccumulatorState as) {
233+
return as.accumulator.get();
234+
}
235+
}
Lines changed: 20 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,20 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
19+
/** Benchmarks for KafkaIO. */
20+
package org.apache.beam.sdk.io.kafka.jmh;

sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIOUtils.java

Lines changed: 34 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@
2323
import java.util.HashMap;
2424
import java.util.Map;
2525
import java.util.Random;
26+
import java.util.concurrent.atomic.AtomicLongFieldUpdater;
2627
import org.apache.beam.sdk.transforms.SerializableFunction;
2728
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;
2829
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.primitives.Longs;
@@ -129,19 +130,43 @@ static Map<String, Object> getOffsetConsumerConfig(
129130
return offsetConsumerConfig;
130131
}
131132

132-
// Maintains approximate average over last 1000 elements
133-
static class MovingAvg {
133+
/*
134+
* Maintains approximate average over last 1000 elements.
135+
* Usage is only thread-safe for a single producer and multiple consumers.
136+
*/
137+
public static final class MovingAvg {
138+
private static final AtomicLongFieldUpdater<MovingAvg> AVG =
139+
AtomicLongFieldUpdater.newUpdater(MovingAvg.class, "avg");
134140
private static final int MOVING_AVG_WINDOW = 1000;
135-
private double avg = 0;
136-
private long numUpdates = 0;
137141

138-
void update(double quantity) {
139-
numUpdates++;
140-
avg += (quantity - avg) / Math.min(MOVING_AVG_WINDOW, numUpdates);
142+
private volatile long avg;
143+
private long numUpdates;
144+
145+
private double getAvg() {
146+
return Double.longBitsToDouble(avg);
147+
}
148+
149+
private void setAvg(final double value) {
150+
AVG.lazySet(this, Double.doubleToRawLongBits(value));
151+
}
152+
153+
private long incrementAndGetNumUpdates() {
154+
final long nextNumUpdates = Math.min(MOVING_AVG_WINDOW, numUpdates + 1);
155+
numUpdates = nextNumUpdates;
156+
return nextNumUpdates;
157+
}
158+
159+
public void update(final double quantity) {
160+
final double prevAvg = getAvg(); // volatile load (acquire)
161+
162+
final long nextNumUpdates = incrementAndGetNumUpdates(); // normal load/store
163+
final double nextAvg = prevAvg + (quantity - prevAvg) / nextNumUpdates; // normal load/store
164+
165+
setAvg(nextAvg); // ordered store (release)
141166
}
142167

143-
double get() {
144-
return avg;
168+
public double get() {
169+
return getAvg(); // volatile load (acquire)
145170
}
146171
}
147172

0 commit comments

Comments
 (0)