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

[KafkaIO] Update tracker and watermark for non-visible progress #34202

Open
wants to merge 9 commits into
base: master
Choose a base branch
from
Open
1 change: 1 addition & 0 deletions build.gradle.kts
Original file line number Diff line number Diff line change
Expand Up @@ -304,6 +304,7 @@ tasks.register("javaPreCommit") {
dependsOn(":sdks:java:io:contextualtextio:build")
dependsOn(":sdks:java:io:expansion-service:build")
dependsOn(":sdks:java:io:file-based-io-tests:build")
dependsOn(":sdks:java:io:kafka:jmh:build")
dependsOn(":sdks:java:io:sparkreceiver:3:build")
dependsOn(":sdks:java:io:synthetic:build")
dependsOn(":sdks:java:io:xml:build")
Expand Down
31 changes: 31 additions & 0 deletions sdks/java/io/kafka/jmh/build.gradle
Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
/*
* 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.
*/

plugins { id 'org.apache.beam.module' }

applyJavaNature(
automaticModuleName: 'org.apache.beam.sdk.io.kafka.jmh',
enableJmh: true,
publish: false)

description = "Apache Beam :: SDKs :: Java :: IO :: Kafka :: JMH"
ext.summary = "This contains JMH benchmarks for the Kafka IO connector for Beam Java"

dependencies {
implementation project(path: ":sdks:java:io:kafka")
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,127 @@
/*
* 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.io.kafka.jmh;

import java.util.Random;
import java.util.concurrent.TimeUnit;
import org.apache.beam.sdk.io.kafka.KafkaIOUtils;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Group;
import org.openjdk.jmh.annotations.GroupThreads;
import org.openjdk.jmh.annotations.Level;
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.Setup;
import org.openjdk.jmh.annotations.State;
import org.openjdk.jmh.annotations.TearDown;
import org.openjdk.jmh.annotations.Warmup;
import org.openjdk.jmh.infra.IterationParams;
import org.openjdk.jmh.infra.ThreadParams;

@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.NANOSECONDS)
@Warmup(batchSize = KafkaIOUtilsBenchmark.SIZE)
@Measurement(batchSize = KafkaIOUtilsBenchmark.SIZE)
public class KafkaIOUtilsBenchmark {
static final int READERS = 2;
static final int WRITERS = 1;
static final int SIZE = 1024;

@State(Scope.Thread)
public static class ProducerState {
private int[] values;
private int idx;

@Setup(Level.Iteration)
public void setup(final IterationParams ip, final ThreadParams tp) {
values =
new Random(299792458 + ip.getCount() + tp.getThreadIndex())
.ints(KafkaIOUtilsBenchmark.SIZE, 0, 100)
.toArray();
idx = 0;
}

@TearDown(Level.Invocation)
public void tearDown(final IterationParams ip, final ThreadParams tp) {
idx = (idx + 1) % KafkaIOUtilsBenchmark.SIZE;
}
}

@State(Scope.Group)
public static class PlainAccumulatorState {
// As implemented before 2.64.0.
// Note that numUpdates may overflow and count back from Long.MIN_VALUE.
static class MovingAvg {
private static final int MOVING_AVG_WINDOW = 1000;
private double avg = 0;
private long numUpdates = 0;

void update(double quantity) {
numUpdates++;
avg += (quantity - avg) / Math.min(MOVING_AVG_WINDOW, numUpdates);
}

double get() {
return avg;
}
}

private MovingAvg accumulator;

@Setup(Level.Iteration)
public void setup(final IterationParams ip, final ThreadParams tp) {
accumulator = new MovingAvg();
}
}

@State(Scope.Group)
public static class AtomicAccumulatorState {
private final KafkaIOUtils.MovingAvg accumulator = new KafkaIOUtils.MovingAvg();
}

@Benchmark
@Group("Plain")
@GroupThreads(KafkaIOUtilsBenchmark.WRITERS)
public void plainWrite(final PlainAccumulatorState as, final ProducerState ps) {
as.accumulator.update(ps.values[ps.idx]);
}

@Benchmark
@Group("Plain")
@GroupThreads(KafkaIOUtilsBenchmark.READERS)
public double plainRead(final PlainAccumulatorState as) {
return as.accumulator.get();
}

@Benchmark
@Group("Atomic")
@GroupThreads(KafkaIOUtilsBenchmark.WRITERS)
public void atomicWrite(final AtomicAccumulatorState as, final ProducerState ps) {
as.accumulator.update(ps.values[ps.idx]);
}

@Benchmark
@Group("Atomic")
@GroupThreads(KafkaIOUtilsBenchmark.READERS)
public double atomicRead(final AtomicAccumulatorState as) {
return as.accumulator.get();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,20 @@
/*
* 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.
*/

/** Benchmarks for KafkaIO. */
package org.apache.beam.sdk.io.kafka.jmh;
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,12 @@

import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument;

import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
import java.nio.charset.StandardCharsets;
import java.util.HashMap;
import java.util.Map;
import java.util.Random;
import java.util.concurrent.atomic.AtomicLongFieldUpdater;
import org.apache.beam.sdk.transforms.SerializableFunction;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.primitives.Longs;
Expand Down Expand Up @@ -129,19 +131,80 @@ static Map<String, Object> getOffsetConsumerConfig(
return offsetConsumerConfig;
}

// Maintains approximate average over last 1000 elements
static class MovingAvg {
/*
* Attempt to prevent false sharing by padding to at least 64 bytes.
* object header: 4, 8, 12 or 16 bytes
* alignment: at least 8 bytes
*/
@SuppressFBWarnings("UUF_UNUSED_FIELD")
private static class MovingAvgPadding {
byte p000, p001, p002, p003, p004, p005, p006, p007;
byte p010, p011, p012, p013, p014, p015, p016, p017;
byte p020, p021, p022, p023, p024, p025, p026, p027;
byte p030, p031, p032, p033, p034, p035, p036, p037;
byte p040, p041, p042, p043, p044, p045, p046, p047;
byte p050, p051, p052, p053, p054, p055, p056, p057;
byte p060, p061, p062, p063, p064, p065, p066, p067;
}

// The accumulator's fields should be padded to at least 128 bytes (at least 1 or 2
// cache lines).
private static class MovingAvgFields extends MovingAvgPadding {
private static final int MOVING_AVG_WINDOW = 1000;
private double avg = 0;

private static final AtomicLongFieldUpdater<MovingAvgFields> AVG =
AtomicLongFieldUpdater.newUpdater(MovingAvgFields.class, "avg");

private volatile long avg = 0;
private long numUpdates = 0;

void update(double quantity) {
numUpdates++;
avg += (quantity - avg) / Math.min(MOVING_AVG_WINDOW, numUpdates);
protected double getAvg() {
return Double.longBitsToDouble(avg);
}

protected void setAvg(final double value) {
AVG.lazySet(this, Double.doubleToRawLongBits(value));
}

protected long incrementAndGetNumUpdates() {
final long nextNumUpdates = Math.min(MOVING_AVG_WINDOW, numUpdates + 1);
numUpdates = nextNumUpdates;
return nextNumUpdates;
}
}

/*
* Maintains approximate average over last 1000 elements.
* Usage is only thread-safe for a single producer and multiple consumers.
*
* Attempt to prevent false sharing by padding to 64 bytes.
* avg: 8 bytes
* numUpdates: 8 bytes
* alignment: at least 8 bytes
*
* Visibility and ordering of non-volatile loads/stores on numUpdates is guaranteed by volatile loads/stores on avg.
* Sanity of visibility is only useful when the writer thread changes since avg is the only field that can be shared between multiple concurrent threads.
*/
@SuppressFBWarnings("UUF_UNUSED_FIELD")
public static final class MovingAvg extends MovingAvgFields {
byte p100, p101, p102, p103, p104, p105, p106, p107;
byte p110, p111, p112, p113, p114, p115, p116, p117;
byte p120, p121, p122, p123, p124, p125, p126, p127;
byte p130, p131, p132, p133, p134, p135, p136, p137;
byte p140, p141, p142, p143, p144, p145, p146, p147;
byte p150, p151, p152, p153, p154, p155, p156, p157;

public void update(final double quantity) {
final double prevAvg = getAvg(); // volatile load (acquire)

final long nextNumUpdates = incrementAndGetNumUpdates(); // normal load/store
final double nextAvg = prevAvg + (quantity - prevAvg) / nextNumUpdates; // normal load/store

setAvg(nextAvg); // ordered store (release)
}

double get() {
return avg;
public double get() {
return getAvg(); // volatile load (acquire)
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -151,7 +151,6 @@ public boolean start() throws IOException {
@Override
public boolean advance() throws IOException {
/* Read first record (if any). we need to loop here because :
* - (a) some records initially need to be skipped if they are before consumedOffset
* - (b) if curBatch is empty, we want to fetch next batch and then advance.
* - (c) curBatch is an iterator of iterators. we interleave the records from each.
* curBatch.next() might return an empty iterator.
Expand All @@ -173,26 +172,6 @@ public boolean advance() throws IOException {
elementsReadBySplit.inc();

ConsumerRecord<byte[], byte[]> rawRecord = pState.recordIter.next();
long expected = pState.nextOffset;
long offset = rawRecord.offset();

if (offset < expected) { // -- (a)
// this can happen when compression is enabled in Kafka (seems to be fixed in 0.10)
// should we check if the offset is way off from consumedOffset (say > 1M)?
LOG.warn(
"{}: ignoring already consumed offset {} for {}",
this,
offset,
pState.topicPartition);
continue;
}

long offsetGap = offset - expected; // could be > 0 when Kafka log compaction is enabled.

if (curRecord == null) {
LOG.info("{}: first record offset {}", name, offset);
offsetGap = 0;
}

// Apply user deserializers. User deserializers might throw, which will be propagated up
// and 'curRecord' remains unchanged. The runner should close this reader.
Expand All @@ -219,7 +198,7 @@ public boolean advance() throws IOException {
int recordSize =
(rawRecord.key() == null ? 0 : rawRecord.key().length)
+ (rawRecord.value() == null ? 0 : rawRecord.value().length);
pState.recordConsumed(offset, recordSize, offsetGap);
pState.recordConsumed(rawRecord.offset(), recordSize);
bytesRead.inc(recordSize);
bytesReadBySplit.inc(recordSize);

Expand Down Expand Up @@ -470,8 +449,6 @@ private static class PartitionState<K, V> {
private Iterator<ConsumerRecord<byte[], byte[]>> recordIter = Collections.emptyIterator();

private KafkaIOUtils.MovingAvg avgRecordSize = new KafkaIOUtils.MovingAvg();
private KafkaIOUtils.MovingAvg avgOffsetGap =
new KafkaIOUtils.MovingAvg(); // > 0 only when log compaction is enabled.

PartitionState(
TopicPartition partition, long nextOffset, TimestampPolicy<K, V> timestampPolicy) {
Expand All @@ -487,13 +464,10 @@ public TopicPartition topicPartition() {
return topicPartition;
}

// Update consumedOffset, avgRecordSize, and avgOffsetGap
void recordConsumed(long offset, int size, long offsetGap) {
// Update consumedOffset and avgRecordSize
void recordConsumed(long offset, int size) {
nextOffset = offset + 1;

// This is always updated from single thread. Probably not worth making atomic.
avgRecordSize.update(size);
avgOffsetGap.update(offsetGap);
}

synchronized void setLatestOffset(long latestOffset, Instant fetchTime) {
Expand Down Expand Up @@ -521,7 +495,7 @@ synchronized long backlogMessageCount() {
if (latestOffset < 0 || nextOffset < 0) {
return UnboundedReader.BACKLOG_UNKNOWN;
}
double remaining = (latestOffset - nextOffset) / (1 + avgOffsetGap.get());
double remaining = latestOffset - nextOffset;
return Math.max(0, (long) Math.ceil(remaining));
}

Expand Down
Loading
Loading