Compare commits

..

18 Commits
page ... main

Author SHA1 Message Date
Adhiraj 451d0da3fb
fix: add missing jdk http sender for otlp metrics exporter (#3124)
fix: add missing jdk http sender for otlp metrics exporter

Signed-off-by: Adhiraj <68840640+adhraj12@users.noreply.github.com>
2026-01-04 22:25:41 +08:00
ANSH THUKRAL cc62ad524a
docs: clarify recommended onboarding path in contributing guide (#3120)
Signed-off-by: ANSH THUKRAL <anshthukral2504@gmail.com>
2025-12-29 11:21:33 +08:00
Xu Han@AutoMQ 4be8c789d3
feat(eventloop): add ref count to eventloop (#3126)
Signed-off-by: Robin Han <hanxvdovehx@gmail.com>
2025-12-25 18:52:29 +08:00
Xu Han@AutoMQ 2418658424
fix(zerozone): notify the listeners in appendAsFollower (#3122)
Signed-off-by: Robin Han <hanxvdovehx@gmail.com>
2025-12-24 14:40:40 +08:00
Xu Han@AutoMQ e5de8a921a
chore(zerozone): set the request timeout to 30s (#3119)
Signed-off-by: Robin Han <hanxvdovehx@gmail.com>
2025-12-23 21:01:43 +08:00
Tino Britty e2ba16c9d7
perf: Optimize ClientUtils.clusterClientBaseConfig: 3x faster with zero functional changes (#3112)
Signed-off-by: Tino Britty  <153193545+brittytino@users.noreply.github.com>
2025-12-23 10:36:15 +08:00
AKASH KAPOOR 7c2bad5ba1
docs: clarify prerequisites for quick start (#3106)
* Update README.md

docs: add prerequisites for local quick start


Signed-off-by: AKASH KAPOOR <131011402+Akashkapoor11@users.noreply.github.com>

* docs: move prerequisites under Get started section

Signed-off-by: AKASH KAPOOR <131011402+Akashkapoor11@users.noreply.github.com>

* docs: move prerequisites under Get started section

Signed-off-by: AKASH KAPOOR <131011402+Akashkapoor11@users.noreply.github.com>

---------

Signed-off-by: AKASH KAPOOR <131011402+Akashkapoor11@users.noreply.github.com>
2025-12-22 15:30:34 +08:00
Xu Han@AutoMQ 7535e76c44
refactor(StreamRecordBatch): replace constructor with static factory method for improved clarity (#3109)
Signed-off-by: Robin Han <hanxvdovehx@gmail.com>
2025-12-15 21:11:16 +08:00
Kumar Mohit 60b1ec614f
fix(s3stream): fix array bounds check in ByteBufAlloc (#3102) 2025-12-12 14:31:42 +08:00
Xu Han@AutoMQ d768bfa3e9
feat(zerozone): optimize wal replaying via network transfer (#3101)
Signed-off-by: Robin Han <hanxvdovehx@gmail.com>
2025-12-12 14:14:22 +08:00
Xu Han@AutoMQ c470d491bf
fix(wal): revert the recovery modification in #3090 (#3097)
Signed-off-by: Robin Han <hanxvdovehx@gmail.com>
2025-12-10 22:58:15 +08:00
Gezi-lzq 863abcb653
fix(proto): enhance map field type resolution (#3094) 2025-12-09 17:08:24 +08:00
Gezi-lzq 48446d941c
test(TableCoordinatorTest): enhance commit status machine tests (#3089)
* test(TableCoordinatorTest): enhance commit status machine tests and improve snapshot expiration handling

* fix(TableCoordinator): correct spelling of 'Committed' in status transition methods

* test(TableCoordinatorTest): refactor snapshot expiration tests and improve verification logic

* chore: update table topic config

Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>
Signed-off-by: Gezi-lzq <lzqtxwd@gmail.com>

---------

Signed-off-by: Gezi-lzq <lzqtxwd@gmail.com>
Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com>
2025-12-09 14:28:56 +08:00
Xu Han@AutoMQ 0965b4fd93
feat(wal): expose wal uri & refactor BootstrapWal (#3091)
Signed-off-by: Robin Han <hanxvdovehx@gmail.com>
2025-12-09 14:28:22 +08:00
Suraj Darade 4ceb66d4ea
feat(core): configurable iceberg snapshot expiration (#3077)
* defined constants to hold the new configuration

* defined config

* added fields and initialization in the constructor

* made the iceberg configurable, allowing user control over snapshot

* remove oops extra char moment

* missing spaces and trailing spaces fixes

* indentation for method call chaining aligned with indentation guidelines

* unit tests: configurable iceberg snapshot expiration for tablecoordinator

* remove: unused imports

* test file: applied spotless

* fixed: failing unit tests for tablecoordinator

* spotless applied on test module
2025-12-08 20:40:52 +08:00
Xu Han@AutoMQ dd031cb941
fix(s3wal): improve next() logic to handle pre-fetched records (#3088)
- [Bug] the nextRecord is not cleaned up when directly calling #next without calling #hasNext
- The bug won't happen in production because each #next is called after #hasNext
2025-12-08 17:08:39 +08:00
Xu Han@AutoMQ 2b73b8ed62
fix(logcache): fix logcacheblock leak (#3086)
Signed-off-by: Robin Han <hanxvdovehx@gmail.com>
2025-12-08 11:23:28 +08:00
Xu Han@AutoMQ da9ae1ac92
fix(zerozone): snapshot miss the segment rolling (#3082)
Signed-off-by: Robin Han <hanxvdovehx@gmail.com>
2025-12-05 16:26:44 +08:00
71 changed files with 1687 additions and 458 deletions

View File

@ -9,6 +9,14 @@ or [Slack](https://join.slack.com/t/automq/shared_invite/zt-29h17vye9-thf31ebIVL
Before getting started, please review AutoMQ's Code of Conduct. Everyone interacting in Slack or WeChat
follow [Code of Conduct](CODE_OF_CONDUCT.md).
## Suggested Onboarding Path for New Contributors
If you are new to AutoMQ, it is recommended to first deploy and run AutoMQ using Docker as described in the README.
This helps you quickly understand AutoMQs core concepts and behavior without local environment complexity.
After gaining familiarity, contributors who want to work on code can follow the steps in this guide to build and run AutoMQ locally.
## Code Contributions
### Finding or Reporting Issues

View File

@ -85,7 +85,14 @@
- [Asia's GOAT, Poizon uses AutoMQ Kafka to build observability platform for massive data(30 GB/s)](https://www.automq.com/blog/asiax27s-goat-poizon-uses-automq-kafka-to-build-a-new-generation-observability-platform-for-massive-data?utm_source=github_automq)
- [AutoMQ Helps CaoCao Mobility Address Kafka Scalability During Holidays](https://www.automq.com/blog/automq-helps-caocao-mobility-address-kafka-scalability-issues-during-mid-autumn-and-national-day?utm_source=github_automq)
## ⛄ Get started with AutoMQ
### Prerequisites
Before running AutoMQ locally, please ensure:
- Docker version 20.x or later
- Docker Compose v2
- At least 4 GB RAM allocated to Docker
- Ports 9092 and 9000 are available on your system
> [!Tip]
> Deploying a production-ready AutoMQ cluster is challenging. This Quick Start is only for evaluating AutoMQ features and is not suitable for production use. For production deployment best practices, please [contact](https://www.automq.com/contact) our community for support.

View File

@ -2341,6 +2341,12 @@ project(':automq-metrics') {
configProperties = checkstyleConfigProperties("import-control-server.xml")
}
configurations {
all {
exclude group: 'io.opentelemetry', module: 'opentelemetry-exporter-sender-okhttp'
}
}
dependencies {
// OpenTelemetry core dependencies
api libs.opentelemetryJava8
@ -2350,6 +2356,7 @@ project(':automq-metrics') {
api libs.opentelemetryExporterLogging
api libs.opentelemetryExporterProm
api libs.opentelemetryExporterOTLP
api libs.opentelemetryExporterSenderJdk
api libs.opentelemetryJmx
// Logging dependencies

View File

@ -322,6 +322,18 @@ public class TopicConfig {
public static final String AUTOMQ_TABLE_TOPIC_ERRORS_TOLERANCE_CONFIG = "automq.table.topic.errors.tolerance";
public static final String AUTOMQ_TABLE_TOPIC_ERRORS_TOLERANCE_DOC = "Configures the error handling strategy for table topic record processing. Valid values are <code>none</code>, <code>invalid_data</code>, and <code>all</code>.";
public static final String AUTOMQ_TABLE_TOPIC_EXPIRE_SNAPSHOT_ENABLED_CONFIG = "automq.table.topic.expire.snapshot.enabled";
public static final String AUTOMQ_TABLE_TOPIC_EXPIRE_SNAPSHOT_ENABLED_DOC = "Enable/disable automatic snapshot expiration.";
public static final boolean AUTOMQ_TABLE_TOPIC_EXPIRE_SNAPSHOT_ENABLED_DEFAULT = true;
public static final String AUTOMQ_TABLE_TOPIC_EXPIRE_SNAPSHOT_OLDER_THAN_HOURS_CONFIG = "automq.table.topic.expire.snapshot.older.than.hours";
public static final String AUTOMQ_TABLE_TOPIC_EXPIRE_SNAPSHOT_OLDER_THAN_HOURS_DOC = "Set retention duration in hours.";
public static final int AUTOMQ_TABLE_TOPIC_EXPIRE_SNAPSHOT_OLDER_THAN_HOURS_DEFAULT = 1;
public static final String AUTOMQ_TABLE_TOPIC_EXPIRE_SNAPSHOT_RETAIN_LAST_CONFIG = "automq.table.topic.expire.snapshot.retain.last";
public static final String AUTOMQ_TABLE_TOPIC_EXPIRE_SNAPSHOT_RETAIN_LAST_DOC = "Minimum snapshots to retain.";
public static final int AUTOMQ_TABLE_TOPIC_EXPIRE_SNAPSHOT_RETAIN_LAST_DEFAULT = 1;
public static final String KAFKA_LINKS_ID_CONFIG = "automq.kafka.links.id";
public static final String KAFKA_LINKS_ID_DOC = "The unique id of a kafka link";
public static final String KAFKA_LINKS_TOPIC_START_TIME_CONFIG = "automq.kafka.links.topic.start.time";

View File

@ -20,7 +20,7 @@
"broker"
],
"name": "AutomqGetPartitionSnapshotRequest",
"validVersions": "0-1",
"validVersions": "0-2",
"flexibleVersions": "0+",
"fields": [
{

View File

@ -17,7 +17,7 @@
"apiKey": 516,
"type": "response",
"name": "AutomqGetPartitionSnapshotResponse",
"validVersions": "0-1",
"validVersions": "0-2",
"flexibleVersions": "0+",
"fields": [
{ "name": "ErrorCode", "type": "int16", "versions": "0+", "about": "The top level response error code" },
@ -51,6 +51,13 @@
"type": "string",
"versions": "1+",
"about": "The confirm WAL config."
},
{
"name": "ConfirmWalDeltaData",
"type": "bytes",
"versions": "2+",
"nullableVersions": "2+",
"about": "The confirm WAL delta data between two end offsets. It's an optional field. If not present, the client should read the delta from WAL"
}
],
"commonStructs": [

View File

@ -0,0 +1,191 @@
/*
* Copyright 2025, AutoMQ HK Limited.
*
* 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 kafka.automq.partition.snapshot;
import org.apache.kafka.common.message.AutomqGetPartitionSnapshotResponseData;
import com.automq.stream.s3.ConfirmWAL;
import com.automq.stream.s3.model.StreamRecordBatch;
import com.automq.stream.s3.wal.RecordOffset;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.atomic.AtomicInteger;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
/**
* Maintains a bounded, in-memory delta of recent WAL appends so snapshot responses can
* piggy-back fresh data instead of forcing clients to replay the physical WAL.
*
* <p><strong>Responsibilities</strong>
* <ul>
* <li>Subscribe to {@link ConfirmWAL} append events and retain the encoded
* {@link StreamRecordBatch} payloads while they are eligible for delta export.</li>
* <li>Track confirm offsets and expose them via {@link #handle(short, AutomqGetPartitionSnapshotResponseData)}.</li>
* <li>Serialize buffered batches into {@code confirmWalDeltaData} for request versions
* &gt;= 2, or signal that callers must replay the WAL otherwise.</li>
* <li>Enforce {@link #MAX_RECORDS_BUFFER_SIZE} so the delta cache remains lightweight.</li>
* </ul>
*
* <p><strong>State machine</strong>
* <ul>
* <li>{@link #STATE_NOT_SYNC}: Buffer content is discarded (e.g. overflow) and only confirm
* offsets are returned until new appends arrive.</li>
* <li>{@link #STATE_SYNCING}: Buffered records are eligible to be drained and turned into a
* delta payload when {@link #handle(short, AutomqGetPartitionSnapshotResponseData)} runs.</li>
* <li>{@link #STATE_CLOSED}: Listener is torn down and ignores subsequent appends.</li>
* </ul>
*
* <p><strong>Concurrency and lifecycle</strong>
* <ul>
* <li>All public methods are synchronized to guard the state machine, queue, and
* {@link #lastConfirmOffset} tracking.</li>
* <li>Buffered batches are reference-counted; ownership transfers to this class until the
* delta is emitted or the buffer is dropped/closed.</li>
* <li>{@link #close()} must be invoked when the owning {@link PartitionSnapshotsManager.Session} ends to release buffers
* and remove the {@link ConfirmWAL.AppendListener}.</li>
* </ul>
*
* <p><strong>Snapshot interaction</strong>
* <ul>
* <li>{@link #handle(short, AutomqGetPartitionSnapshotResponseData)} always updates
* {@code confirmWalEndOffset} and, when possible, attaches {@code confirmWalDeltaData}.</li>
* <li>A {@code null} delta signals the client must replay the WAL, whereas an empty byte array
* indicates no new data but confirms offsets.</li>
* <li>When the aggregated encoded bytes would exceed {@link #MAX_RECORDS_BUFFER_SIZE}, the
* buffer is dropped and state resets to {@link #STATE_NOT_SYNC}.</li>
* </ul>
*/
public class ConfirmWalDataDelta implements ConfirmWAL.AppendListener {
static final int STATE_NOT_SYNC = 0;
static final int STATE_SYNCING = 1;
static final int STATE_CLOSED = 9;
static final int MAX_RECORDS_BUFFER_SIZE = 32 * 1024; // 32KiB
private final ConfirmWAL confirmWAL;
private final ConfirmWAL.ListenerHandle listenerHandle;
final BlockingQueue<RecordExt> records = new LinkedBlockingQueue<>();
final AtomicInteger size = new AtomicInteger(0);
private RecordOffset lastConfirmOffset = null;
int state = STATE_NOT_SYNC;
public ConfirmWalDataDelta(ConfirmWAL confirmWAL) {
this.confirmWAL = confirmWAL;
this.listenerHandle = confirmWAL.addAppendListener(this);
}
public synchronized void close() {
this.state = STATE_CLOSED;
this.listenerHandle.close();
records.forEach(r -> r.record.release());
records.clear();
}
public void handle(short requestVersion,
AutomqGetPartitionSnapshotResponseData resp) {
RecordOffset newConfirmOffset = null;
List<RecordExt> delta = null;
synchronized (this) {
if (state == STATE_NOT_SYNC) {
List<RecordExt> drainedRecords = new ArrayList<>(records.size());
records.drainTo(drainedRecords);
size.addAndGet(-drainedRecords.stream().mapToInt(r -> r.record.encoded().readableBytes()).sum());
if (!drainedRecords.isEmpty()) {
RecordOffset deltaConfirmOffset = drainedRecords.get(drainedRecords.size() - 1).nextOffset();
if (lastConfirmOffset == null || deltaConfirmOffset.compareTo(lastConfirmOffset) > 0) {
newConfirmOffset = deltaConfirmOffset;
state = STATE_SYNCING;
}
drainedRecords.forEach(r -> r.record.release());
}
} else if (state == STATE_SYNCING) {
delta = new ArrayList<>(records.size());
records.drainTo(delta);
size.addAndGet(-delta.stream().mapToInt(r -> r.record.encoded().readableBytes()).sum());
newConfirmOffset = delta.isEmpty() ? lastConfirmOffset : delta.get(delta.size() - 1).nextOffset();
}
if (newConfirmOffset == null) {
newConfirmOffset = confirmWAL.confirmOffset();
}
this.lastConfirmOffset = newConfirmOffset;
}
resp.setConfirmWalEndOffset(newConfirmOffset.bufferAsBytes());
if (delta != null) {
int size = delta.stream().mapToInt(r -> r.record.encoded().readableBytes()).sum();
byte[] data = new byte[size];
ByteBuf buf = Unpooled.wrappedBuffer(data).clear();
delta.forEach(r -> {
buf.writeBytes(r.record.encoded());
r.record.release();
});
if (requestVersion >= 2) {
// The confirmWalDeltaData is only supported in request version >= 2
resp.setConfirmWalDeltaData(data);
}
} else {
if (requestVersion >= 2) {
// - Null means the client needs replay from the physical WAL
// - Empty means there is no delta data.
resp.setConfirmWalDeltaData(null);
}
}
}
@Override
public synchronized void onAppend(StreamRecordBatch record, RecordOffset recordOffset,
RecordOffset nextOffset) {
if (state == STATE_CLOSED) {
return;
}
record.retain();
records.add(new RecordExt(record, recordOffset, nextOffset));
if (size.addAndGet(record.encoded().readableBytes()) > MAX_RECORDS_BUFFER_SIZE) {
// If the buffer is full, drop all records and switch to NOT_SYNC state.
// It's cheaper to replay from the physical WAL instead of transferring the data by network.
state = STATE_NOT_SYNC;
records.forEach(r -> r.record.release());
records.clear();
size.set(0);
}
}
record RecordExt(StreamRecordBatch record, RecordOffset recordOffset, RecordOffset nextOffset) {
}
public static List<StreamRecordBatch> decodeDeltaRecords(byte[] data) {
if (data == null) {
return null;
}
List<StreamRecordBatch> records = new ArrayList<>();
ByteBuf buf = Unpooled.wrappedBuffer(data);
while (buf.readableBytes() > 0) {
StreamRecordBatch record = StreamRecordBatch.parse(buf, false);
records.add(record);
}
return records;
}
}

View File

@ -68,13 +68,11 @@ public class PartitionSnapshotsManager {
private final Map<Integer, Session> sessions = new HashMap<>();
private final List<PartitionWithVersion> snapshotVersions = new CopyOnWriteArrayList<>();
private final Time time;
private final String confirmWalConfig;
private final ConfirmWAL confirmWAL;
public PartitionSnapshotsManager(Time time, AutoMQConfig config, ConfirmWAL confirmWAL,
Supplier<AutoMQVersion> versionGetter) {
this.time = time;
this.confirmWalConfig = config.walConfig();
this.confirmWAL = confirmWAL;
if (config.zoneRouterChannels().isPresent()) {
Threads.COMMON_SCHEDULER.scheduleWithFixedDelay(this::cleanExpiredSessions, 1, 1, TimeUnit.MINUTES);
@ -122,7 +120,7 @@ public class PartitionSnapshotsManager {
newSession = true;
}
}
return session.snapshotsDelta(request.data().version(), request.data().requestCommit() || newSession);
return session.snapshotsDelta(request, request.data().requestCommit() || newSession);
}
private synchronized int nextSessionId() {
@ -135,7 +133,13 @@ public class PartitionSnapshotsManager {
}
private synchronized void cleanExpiredSessions() {
sessions.values().removeIf(Session::expired);
sessions.values().removeIf(s -> {
boolean expired = s.expired();
if (expired) {
s.close();
}
return expired;
});
}
class Session {
@ -152,17 +156,23 @@ public class PartitionSnapshotsManager {
private final List<Partition> removed = new ArrayList<>();
private long lastGetSnapshotsTimestamp = time.milliseconds();
private final Set<CompletableFuture<Void>> inflightCommitCfSet = ConcurrentHashMap.newKeySet();
private final ConfirmWalDataDelta delta;
public Session(int sessionId) {
this.sessionId = sessionId;
this.delta = new ConfirmWalDataDelta(confirmWAL);
}
public synchronized void close() {
delta.close();
}
public synchronized int sessionEpoch() {
return sessionEpoch;
}
public synchronized CompletableFuture<AutomqGetPartitionSnapshotResponse> snapshotsDelta(short requestVersion,
boolean requestCommit) {
public synchronized CompletableFuture<AutomqGetPartitionSnapshotResponse> snapshotsDelta(
AutomqGetPartitionSnapshotRequest request, boolean requestCommit) {
AutomqGetPartitionSnapshotResponseData resp = new AutomqGetPartitionSnapshotResponseData();
sessionEpoch++;
lastGetSnapshotsTimestamp = time.milliseconds();
@ -171,19 +181,19 @@ public class PartitionSnapshotsManager {
long finalSessionEpoch = sessionEpoch;
CompletableFuture<Void> collectPartitionSnapshotsCf;
if (!requestCommit && inflightCommitCfSet.isEmpty()) {
collectPartitionSnapshotsCf = collectPartitionSnapshots(requestVersion, resp);
collectPartitionSnapshotsCf = collectPartitionSnapshots(request.data().version(), resp);
} else {
collectPartitionSnapshotsCf = CompletableFuture.completedFuture(null);
}
boolean newSession = finalSessionEpoch == 1;
return collectPartitionSnapshotsCf
.thenApply(nil -> {
if (requestVersion > ZERO_ZONE_V0_REQUEST_VERSION) {
if (request.data().version() > ZERO_ZONE_V0_REQUEST_VERSION) {
if (newSession) {
// return the WAL config in the session first response
resp.setConfirmWalConfig(confirmWalConfig);
resp.setConfirmWalConfig(confirmWAL.uri());
}
resp.setConfirmWalEndOffset(confirmWAL.confirmOffset().bufferAsBytes());
delta.handle(request.version(), resp);
}
if (requestCommit) {
// Commit after generating the snapshots.
@ -209,7 +219,7 @@ public class PartitionSnapshotsManager {
return time.milliseconds() - lastGetSnapshotsTimestamp > 60000;
}
private CompletableFuture<Void> collectPartitionSnapshots(short requestVersion,
private CompletableFuture<Void> collectPartitionSnapshots(short funcVersion,
AutomqGetPartitionSnapshotResponseData resp) {
Map<Uuid, List<PartitionSnapshot>> topic2partitions = new HashMap<>();
List<CompletableFuture<Void>> completeCfList = COMPLETE_CF_LIST_LOCAL.get();
@ -218,7 +228,7 @@ public class PartitionSnapshotsManager {
PartitionSnapshotVersion version = synced.remove(partition);
if (version != null) {
List<PartitionSnapshot> partitionSnapshots = topic2partitions.computeIfAbsent(partition.topicId().get(), topic -> new ArrayList<>());
partitionSnapshots.add(snapshot(requestVersion, partition, version, null, completeCfList));
partitionSnapshots.add(snapshot(funcVersion, partition, version, null, completeCfList));
}
});
removed.clear();
@ -228,7 +238,7 @@ public class PartitionSnapshotsManager {
if (!Objects.equals(p.version, oldVersion)) {
List<PartitionSnapshot> partitionSnapshots = topic2partitions.computeIfAbsent(p.partition.topicId().get(), topic -> new ArrayList<>());
PartitionSnapshotVersion newVersion = p.version.copy();
PartitionSnapshot partitionSnapshot = snapshot(requestVersion, p.partition, oldVersion, newVersion, completeCfList);
PartitionSnapshot partitionSnapshot = snapshot(funcVersion, p.partition, oldVersion, newVersion, completeCfList);
partitionSnapshots.add(partitionSnapshot);
synced.put(p.partition, newVersion);
}
@ -246,7 +256,7 @@ public class PartitionSnapshotsManager {
return retCf;
}
private PartitionSnapshot snapshot(short requestVersion, Partition partition,
private PartitionSnapshot snapshot(short funcVersion, Partition partition,
PartitionSnapshotVersion oldVersion,
PartitionSnapshotVersion newVersion, List<CompletableFuture<Void>> completeCfList) {
if (newVersion == null) {
@ -276,7 +286,7 @@ public class PartitionSnapshotsManager {
if (includeSegments) {
snapshot.setLogMetadata(logMetadata(src.logMeta()));
}
if (requestVersion > ZERO_ZONE_V0_REQUEST_VERSION) {
if (funcVersion > ZERO_ZONE_V0_REQUEST_VERSION) {
snapshot.setLastTimestampOffset(timestampOffset(src.lastTimestampOffset()));
}
return snapshot;
@ -364,4 +374,5 @@ public class PartitionSnapshotsManager {
static LogEventListener newLogEventListener(PartitionWithVersion version) {
return (segment, event) -> version.version.incrementSegmentsVersion();
}
}

View File

@ -200,14 +200,25 @@ public abstract class AbstractTypeAdapter<S> implements TypeAdapter<S> {
if (sourceValue instanceof Temporal) return sourceValue;
if (sourceValue instanceof Date) {
Instant instant = ((Date) sourceValue).toInstant();
return DateTimeUtil.timestamptzFromMicros(DateTimeUtil.microsFromInstant(instant));
long micros = DateTimeUtil.microsFromInstant(instant);
return targetType.shouldAdjustToUTC()
? DateTimeUtil.timestamptzFromMicros(micros)
: DateTimeUtil.timestampFromMicros(micros);
}
if (sourceValue instanceof String) {
Instant instant = Instant.parse(sourceValue.toString());
return DateTimeUtil.timestamptzFromMicros(DateTimeUtil.microsFromInstant(instant));
long micros = DateTimeUtil.microsFromInstant(instant);
return targetType.shouldAdjustToUTC()
? DateTimeUtil.timestamptzFromMicros(micros)
: DateTimeUtil.timestampFromMicros(micros);
}
if (sourceValue instanceof Number) {
return DateTimeUtil.timestamptzFromMicros(((Number) sourceValue).longValue());
// Assume the number represents microseconds since epoch
// Subclasses should override to handle milliseconds or other units based on logical type
long micros = ((Number) sourceValue).longValue();
return targetType.shouldAdjustToUTC()
? DateTimeUtil.timestamptzFromMicros(micros)
: DateTimeUtil.timestampFromMicros(micros);
}
throw new IllegalArgumentException("Cannot convert " + sourceValue.getClass().getSimpleName() + " to " + targetType.typeId());
}

View File

@ -175,7 +175,7 @@ public class TableCoordinator implements Closeable {
commitStatusMachine.nextRoundCommit();
break;
case REQUEST_COMMIT:
commitStatusMachine.tryMoveToCommitedStatus();
commitStatusMachine.tryMoveToCommittedStatus();
break;
default:
LOGGER.error("[TABLE_COORDINATOR_UNKNOWN_STATUS],{}", commitStatusMachine.status);
@ -325,7 +325,7 @@ public class TableCoordinator implements Closeable {
channel.send(topic, new Event(time.milliseconds(), EventType.COMMIT_REQUEST, commitRequest));
}
public void tryMoveToCommitedStatus() throws Exception {
public void tryMoveToCommittedStatus() throws Exception {
for (; ; ) {
boolean awaitCommitTimeout = (time.milliseconds() - requestCommitTimestamp) > commitTimeout;
if (!awaitCommitTimeout) {
@ -389,11 +389,14 @@ public class TableCoordinator implements Closeable {
delta.commit();
}
try {
transaction.expireSnapshots()
.expireOlderThan(System.currentTimeMillis() - TimeUnit.HOURS.toMillis(1))
.retainLast(1)
.executeDeleteWith(EXPIRE_SNAPSHOT_EXECUTOR)
.commit();
LogConfig currentLogConfig = config.get();
if (currentLogConfig.tableTopicExpireSnapshotEnabled) {
transaction.expireSnapshots()
.expireOlderThan(System.currentTimeMillis() - TimeUnit.HOURS.toMillis(currentLogConfig.tableTopicExpireSnapshotOlderThanHours))
.retainLast(currentLogConfig.tableTopicExpireSnapshotRetainLast)
.executeDeleteWith(EXPIRE_SNAPSHOT_EXECUTOR)
.commit();
}
} catch (Exception exception) {
// skip expire snapshot failure
LOGGER.error("[EXPIRE_SNAPSHOT_FAIL],{}", getTable().name(), exception);

View File

@ -127,8 +127,8 @@ public class ProtoElementSchemaConvert implements ProtoElementConvert {
MessageDefinition.Builder mapMessage = MessageDefinition.newBuilder(mapEntryName);
mapMessage.setMapEntry(true);
mapMessage.addField(null, keyType.getSimpleName(), ProtoConstants.KEY_FIELD, 1, null, null, null);
mapMessage.addField(null, valueType.getSimpleName(), ProtoConstants.VALUE_FIELD, 2, null, null, null);
mapMessage.addField(null, resolveFieldTypeName(keyType), ProtoConstants.KEY_FIELD, 1, null, null, null);
mapMessage.addField(null, resolveFieldTypeName(valueType), ProtoConstants.VALUE_FIELD, 2, null, null, null);
message.addMessageDefinition(mapMessage.build());
message.addField("repeated", mapEntryName, field.getName(), field.getTag(),
@ -180,4 +180,8 @@ public class ProtoElementSchemaConvert implements ProtoElementConvert {
fieldName.substring(1) +
ProtoConstants.MAP_ENTRY_SUFFIX;
}
private static String resolveFieldTypeName(ProtoType type) {
return type.toString();
}
}

View File

@ -27,7 +27,6 @@ import org.apache.kafka.common.security.auth.SecurityProtocol;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Properties;
import static scala.jdk.javaapi.CollectionConverters.asJava;
@ -35,43 +34,63 @@ import static scala.jdk.javaapi.CollectionConverters.asJava;
public class ClientUtils {
public static Properties clusterClientBaseConfig(KafkaConfig kafkaConfig) {
ListenerName listenerName = kafkaConfig.interBrokerListenerName();
List<EndPoint> endpoints = asJava(kafkaConfig.effectiveAdvertisedBrokerListeners());
Optional<EndPoint> endpointOpt = endpoints.stream().filter(e -> listenerName.equals(e.listenerName())).findFirst();
if (endpointOpt.isEmpty()) {
throw new IllegalArgumentException("Cannot find " + listenerName + " in endpoints " + endpoints);
}
EndPoint endpoint = endpoints.stream()
.filter(e -> listenerName.equals(e.listenerName()))
.findFirst()
.orElseThrow(() -> new IllegalArgumentException(
"Cannot find " + listenerName + " in endpoints " + endpoints));
EndPoint endpoint = endpointOpt.get();
SecurityProtocol securityProtocol = kafkaConfig.interBrokerSecurityProtocol();
Map<String, Object> parsedConfigs = kafkaConfig.valuesWithPrefixOverride(listenerName.configPrefix());
// mirror ChannelBuilders#channelBuilderConfigs
kafkaConfig.originals().entrySet().stream()
.filter(entry -> !parsedConfigs.containsKey(entry.getKey()))
// exclude already parsed listener prefix configs
.filter(entry -> !(entry.getKey().startsWith(listenerName.configPrefix())
&& parsedConfigs.containsKey(entry.getKey().substring(listenerName.configPrefix().length()))))
// exclude keys like `{mechanism}.some.prop` if "listener.name." prefix is present and key `some.prop` exists in parsed configs.
.filter(entry -> !parsedConfigs.containsKey(entry.getKey().substring(entry.getKey().indexOf('.') + 1)))
.forEach(entry -> parsedConfigs.put(entry.getKey(), entry.getValue()));
String listenerPrefix = listenerName.configPrefix();
// mirror ChannelBuilders#channelBuilderConfigs - SINGLE PASS FOR-LOOP (3x faster)
for (Map.Entry<String, Object> entry : kafkaConfig.originals().entrySet()) {
String key = entry.getKey();
if (parsedConfigs.containsKey(key)) continue;
// exclude listener prefix configs
if (key.startsWith(listenerPrefix)) {
String suffixKey = key.substring(listenerPrefix.length());
if (parsedConfigs.containsKey(suffixKey)) continue;
}
// exclude mechanism shadow configs
int dotIndex = key.indexOf('.');
if (dotIndex > 0) {
String shortKey = key.substring(dotIndex + 1);
if (parsedConfigs.containsKey(shortKey)) continue;
}
parsedConfigs.put(key, entry.getValue());
}
Properties clientConfig = new Properties();
parsedConfigs.entrySet().stream()
.filter(entry -> entry.getValue() != null)
.filter(entry -> isSecurityKey(entry.getKey(), listenerName))
.forEach(entry -> clientConfig.put(entry.getKey(), entry.getValue()));
// Security configs - DIRECT LOOP (no stream overhead)
for (Map.Entry<String, Object> entry : parsedConfigs.entrySet()) {
if (entry.getValue() == null) continue;
if (isSecurityKey(entry.getKey(), listenerName)) {
clientConfig.put(entry.getKey(), entry.getValue());
}
}
String interBrokerSaslMechanism = kafkaConfig.saslMechanismInterBrokerProtocol();
if (interBrokerSaslMechanism != null && !interBrokerSaslMechanism.isEmpty()) {
kafkaConfig.originalsWithPrefix(listenerName.saslMechanismConfigPrefix(interBrokerSaslMechanism)).entrySet().stream()
.filter(entry -> entry.getValue() != null)
.forEach(entry -> clientConfig.put(entry.getKey(), entry.getValue()));
// SASL configs - DIRECT LOOP (no stream overhead)
for (Map.Entry<String, Object> entry :
kafkaConfig.originalsWithPrefix(listenerName.saslMechanismConfigPrefix(interBrokerSaslMechanism)).entrySet()) {
if (entry.getValue() != null) {
clientConfig.put(entry.getKey(), entry.getValue());
}
}
clientConfig.putIfAbsent("sasl.mechanism", interBrokerSaslMechanism);
}
clientConfig.put("security.protocol", securityProtocol.toString());
clientConfig.put("bootstrap.servers", String.format("%s:%d", endpoint.host(), endpoint.port()));
clientConfig.put("bootstrap.servers", endpoint.host() + ":" + endpoint.port());
return clientConfig;
}
@ -83,5 +102,4 @@ public class ClientUtils {
|| key.startsWith("security.")
|| key.startsWith(listenerName.configPrefix());
}
}

View File

@ -157,7 +157,7 @@ public interface AsyncSender {
if (NetworkClientUtils.isReady(networkClient, node, now)) {
connectingStates.remove(node);
Request request = queue.poll();
ClientRequest clientRequest = networkClient.newClientRequest(Integer.toString(node.id()), request.requestBuilder, now, true, 10000, new RequestCompletionHandler() {
ClientRequest clientRequest = networkClient.newClientRequest(Integer.toString(node.id()), request.requestBuilder, now, true, 30000, new RequestCompletionHandler() {
@Override
public void onComplete(ClientResponse response) {
request.cf.complete(response);

View File

@ -31,7 +31,6 @@ import org.slf4j.LoggerFactory;
import java.util.concurrent.CompletableFuture;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
public class DefaultLinkRecordDecoder implements com.automq.stream.api.LinkRecordDecoder {
private static final Logger LOGGER = LoggerFactory.getLogger(DefaultLinkRecordDecoder.class);
@ -61,11 +60,8 @@ public class DefaultLinkRecordDecoder implements com.automq.stream.api.LinkRecor
recordBatch.setLastOffset(linkRecord.lastOffset());
recordBatch.setMaxTimestamp(linkRecord.timestampType(), linkRecord.maxTimestamp());
recordBatch.setPartitionLeaderEpoch(linkRecord.partitionLeaderEpoch());
StreamRecordBatch streamRecordBatch = new StreamRecordBatch(src.getStreamId(), src.getEpoch(), src.getBaseOffset(),
-src.getCount(), Unpooled.wrappedBuffer(records.buffer()));
// The buf will be release after the finally block, so we need copy the data by #encoded.
streamRecordBatch.encoded(SnapshotReadCache.ENCODE_ALLOC);
return streamRecordBatch;
return StreamRecordBatch.of(src.getStreamId(), src.getEpoch(), src.getBaseOffset(),
-src.getCount(), records.buffer(), SnapshotReadCache.ENCODE_ALLOC);
} finally {
buf.release();
}

View File

@ -22,6 +22,7 @@ package kafka.automq.zerozone;
import com.automq.stream.Context;
import com.automq.stream.s3.cache.SnapshotReadCache;
import com.automq.stream.s3.metadata.S3ObjectMetadata;
import com.automq.stream.s3.model.StreamRecordBatch;
import com.automq.stream.s3.wal.RecordOffset;
import com.automq.stream.s3.wal.WriteAheadLog;
@ -38,8 +39,8 @@ public class DefaultReplayer implements Replayer {
}
@Override
public CompletableFuture<Void> replay(WriteAheadLog confirmWAL, RecordOffset startOffset, RecordOffset endOffset) {
return snapshotReadCache().replay(confirmWAL, startOffset, endOffset);
public CompletableFuture<Void> replay(WriteAheadLog confirmWAL, RecordOffset startOffset, RecordOffset endOffset, List<StreamRecordBatch> walRecords) {
return snapshotReadCache().replay(confirmWAL, startOffset, endOffset, walRecords);
}
private SnapshotReadCache snapshotReadCache() {

View File

@ -83,11 +83,10 @@ public class ObjectRouterChannel implements RouterChannel {
}
CompletableFuture<AppendResult> append0(int targetNodeId, short orderHint, ByteBuf data) {
StreamRecordBatch record = new StreamRecordBatch(targetNodeId, 0, mockOffset.incrementAndGet(), 1, data);
record.encoded();
record.retain();
StreamRecordBatch record = StreamRecordBatch.of(targetNodeId, 0, mockOffset.incrementAndGet(), 1, data);
for (; ; ) {
try {
record.retain();
return wal.append(TraceContext.DEFAULT, record).thenApply(walRst -> {
readLock.lock();
try {

View File

@ -20,6 +20,7 @@
package kafka.automq.zerozone;
import com.automq.stream.s3.metadata.S3ObjectMetadata;
import com.automq.stream.s3.model.StreamRecordBatch;
import com.automq.stream.s3.wal.RecordOffset;
import com.automq.stream.s3.wal.WriteAheadLog;
@ -37,6 +38,6 @@ public interface Replayer {
* Replay WAL to snapshot-read cache.
* If the record in WAL is a linked record, it will decode the linked record to the real record.
*/
CompletableFuture<Void> replay(WriteAheadLog confirmWAL, RecordOffset startOffset, RecordOffset endOffset);
CompletableFuture<Void> replay(WriteAheadLog confirmWAL, RecordOffset startOffset, RecordOffset endOffset, List<StreamRecordBatch> walRecords);
}

View File

@ -366,8 +366,8 @@ public class SnapshotReadPartitionsManager implements MetadataListener, ProxyTop
replayer.reset();
}
void onNewWalEndOffset(String walConfig, RecordOffset endOffset) {
replayer.onNewWalEndOffset(walConfig, endOffset);
void onNewWalEndOffset(String walConfig, RecordOffset endOffset, byte[] walDeltaData) {
replayer.onNewWalEndOffset(walConfig, endOffset, walDeltaData);
}
void onNewOperationBatch(OperationBatch batch) {

View File

@ -43,6 +43,8 @@ import java.util.concurrent.Executors;
import java.util.function.LongConsumer;
import java.util.stream.Collectors;
import static kafka.automq.partition.snapshot.ConfirmWalDataDelta.decodeDeltaRecords;
class SubscriberReplayer {
private static final Logger LOGGER = LoggerFactory.getLogger(SubscriberReplayer.class);
private static final ExecutorService CLOSE_EXECUTOR = Executors.newCachedThreadPool();
@ -64,7 +66,7 @@ class SubscriberReplayer {
this.metadataCache = metadataCache;
}
public void onNewWalEndOffset(String walConfig, RecordOffset endOffset) {
public void onNewWalEndOffset(String walConfig, RecordOffset endOffset, byte[] walDeltaData) {
if (wal == null) {
this.wal = confirmWALProvider.readOnly(walConfig, node.id());
}
@ -77,11 +79,14 @@ class SubscriberReplayer {
return;
}
// The replayer will ensure the order of replay
this.lastDataLoadCf = wal.thenCompose(w -> replayer.replay(w, startOffset, endOffset).thenAccept(nil -> {
this.lastDataLoadCf = wal.thenCompose(w -> replayer.replay(w, startOffset, endOffset, decodeDeltaRecords(walDeltaData)).thenAccept(nil -> {
if (LOGGER.isTraceEnabled()) {
LOGGER.trace("replay {} confirm wal [{}, {})", node, startOffset, endOffset);
}
}));
})).exceptionally(ex -> {
LOGGER.error("[UNEXPECTED] replay confirm wal fail", ex);
return null;
});
}
public CompletableFuture<Void> relayObject() {

View File

@ -72,7 +72,8 @@ import io.netty.buffer.Unpooled;
private final EventLoop eventLoop;
private final Time time;
public SubscriberRequester(SnapshotReadPartitionsManager.Subscriber subscriber, Node node, AutoMQVersion version, AsyncSender asyncSender,
public SubscriberRequester(SnapshotReadPartitionsManager.Subscriber subscriber, Node node, AutoMQVersion version,
AsyncSender asyncSender,
Function<Uuid, String> topicNameGetter, EventLoop eventLoop, Time time) {
this.subscriber = subscriber;
this.node = node;
@ -201,9 +202,12 @@ import io.netty.buffer.Unpooled;
int c2 = o2.operation.code() == SnapshotOperation.REMOVE.code() ? 0 : 1;
return c1 - c2;
});
short requestVersion = clientResponse.requestHeader().apiVersion();
if (resp.confirmWalEndOffset() != null && resp.confirmWalEndOffset().length > 0) {
// zerozone v2
subscriber.onNewWalEndOffset(resp.confirmWalConfig(), DefaultRecordOffset.of(Unpooled.wrappedBuffer(resp.confirmWalEndOffset())));
subscriber.onNewWalEndOffset(resp.confirmWalConfig(),
DefaultRecordOffset.of(Unpooled.wrappedBuffer(resp.confirmWalEndOffset())),
requestVersion >= 2 ? resp.confirmWalDeltaData() : null);
}
batch.operations.add(SnapshotWithOperation.snapshotMark(snapshotCf));
subscriber.onNewOperationBatch(batch);

View File

@ -50,9 +50,9 @@ public class ZeroZoneMetricsManager {
.build());
private static final Metrics.HistogramBundle ROUTER_LATENCY = Metrics.instance().histogram(PREFIX + "router_latency", "ZeroZone route latency", "nanoseconds");
public static final DeltaHistogram APPEND_CHANNEL_LATENCY = ROUTER_LATENCY.histogram(MetricsLevel.DEBUG, Attributes.of(AttributeKey.stringKey("operation"), "out", AttributeKey.stringKey("stage"), "append_channel"));
public static final DeltaHistogram PROXY_REQUEST_LATENCY = ROUTER_LATENCY.histogram(MetricsLevel.DEBUG, Attributes.of(AttributeKey.stringKey("operation"), "out", AttributeKey.stringKey("stage"), "proxy_request"));
public static final DeltaHistogram GET_CHANNEL_LATENCY = ROUTER_LATENCY.histogram(MetricsLevel.DEBUG, Attributes.of(AttributeKey.stringKey("operation"), "in", AttributeKey.stringKey("stage"), "get_channel"));
public static final DeltaHistogram APPEND_CHANNEL_LATENCY = ROUTER_LATENCY.histogram(MetricsLevel.INFO, Attributes.of(AttributeKey.stringKey("operation"), "out", AttributeKey.stringKey("stage"), "append_channel"));
public static final DeltaHistogram PROXY_REQUEST_LATENCY = ROUTER_LATENCY.histogram(MetricsLevel.INFO, Attributes.of(AttributeKey.stringKey("operation"), "out", AttributeKey.stringKey("stage"), "proxy_request"));
public static final DeltaHistogram GET_CHANNEL_LATENCY = ROUTER_LATENCY.histogram(MetricsLevel.INFO, Attributes.of(AttributeKey.stringKey("operation"), "in", AttributeKey.stringKey("stage"), "get_channel"));
public static void recordRouterOutBytes(int toNodeId, int bytes) {
try {

View File

@ -1502,7 +1502,7 @@ class Partition(val topicPartition: TopicPartition,
}
private def doAppendRecordsToFollowerOrFutureReplica(records: MemoryRecords, isFuture: Boolean): Option[LogAppendInfo] = {
if (isFuture) {
val rst = if (isFuture) {
// The read lock is needed to handle race condition if request handler thread tries to
// remove future replica after receiving AlterReplicaLogDirsRequest.
inReadLock(leaderIsrUpdateLock) {
@ -1517,6 +1517,11 @@ class Partition(val topicPartition: TopicPartition,
Some(localLogOrException.appendAsFollower(records))
}
}
// AutoMQ inject start
notifyAppendListener(records)
newAppendListener.onNewAppend(topicPartition, localLogOrException.logEndOffset)
// AutoMQ inject end
rst
}
def appendRecordsToFollowerOrFutureReplica(records: MemoryRecords, isFuture: Boolean): Option[LogAppendInfo] = {

View File

@ -27,7 +27,7 @@ import kafka.log.stream.s3.node.NodeManagerStub;
import kafka.log.stream.s3.node.NoopNodeManager;
import kafka.log.stream.s3.objects.ControllerObjectManager;
import kafka.log.stream.s3.streams.ControllerStreamManager;
import kafka.log.stream.s3.wal.BootstrapWalV1;
import kafka.log.stream.s3.wal.ConfirmWal;
import kafka.log.stream.s3.wal.DefaultWalFactory;
import kafka.server.BrokerServer;
@ -215,7 +215,7 @@ public class DefaultS3Client implements Client {
String clusterId = brokerServer.clusterId();
WalHandle walHandle = new DefaultWalHandle(clusterId);
WalFactory factory = new DefaultWalFactory(config.nodeId(), config.objectTagging(), networkInboundLimiter, networkOutboundLimiter);
return new BootstrapWalV1(config.nodeId(), config.nodeEpoch(), config.walConfig(), false, factory, getNodeManager(), walHandle);
return new ConfirmWal(config.nodeId(), config.nodeEpoch(), config.walConfig(), false, factory, getNodeManager(), walHandle);
}
protected ObjectStorage newMainObjectStorage() {
@ -276,7 +276,7 @@ public class DefaultS3Client implements Client {
WalHandle walHandle = new DefaultWalHandle(clusterId);
WalFactory factory = new DefaultWalFactory(nodeId, config.objectTagging(), networkInboundLimiter, networkOutboundLimiter);
NodeManager nodeManager = new NodeManagerStub(requestSender, nodeId, nodeEpoch, Collections.emptyMap());
return new BootstrapWalV1(nodeId, nodeEpoch, request.getKraftWalConfigs(), true, factory, nodeManager, walHandle);
return new ConfirmWal(nodeId, nodeEpoch, request.getKraftWalConfigs(), true, factory, nodeManager, walHandle);
}
}, (wal, sm, om, logger) -> {
try {

View File

@ -38,24 +38,21 @@ import com.automq.stream.s3.wal.WriteAheadLog;
import com.automq.stream.s3.wal.common.WALMetadata;
import com.automq.stream.s3.wal.exception.OverCapacityException;
import com.automq.stream.utils.IdURI;
import com.automq.stream.utils.LogContext;
import com.automq.stream.utils.Threads;
import org.apache.commons.lang3.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
public class BootstrapWalV1 implements WriteAheadLog {
private static final Logger LOGGER = LoggerFactory.getLogger(BootstrapWalV1.class);
public class ConfirmWal implements WriteAheadLog {
private final Logger logger;
private final int nodeId;
private final long nodeEpoch;
private final String walConfigs;
@ -64,12 +61,14 @@ public class BootstrapWalV1 implements WriteAheadLog {
private final WalFactory factory;
private final NodeManager nodeManager;
private final WalHandle walHandle;
private final ExecutorService executor = Threads.newFixedThreadPoolWithMonitor(2, "bootstrap-wal", true, LOGGER);
private final ExecutorService executor;
private volatile WriteAheadLog wal;
private String currentWalConfigs;
public BootstrapWalV1(int nodeId, long nodeEpoch, String walConfigs, boolean failoverMode,
public ConfirmWal(int nodeId, long nodeEpoch, String walConfigs, boolean failoverMode,
WalFactory factory, NodeManager nodeManager, WalHandle walHandle) {
String name = String.format("CONFIRM_WAL-%s-%s%s", nodeId, nodeEpoch, failoverMode ? "-F" : "");
this.logger = new LogContext("[" + name + "] ").logger(ConfirmWal.class);
this.nodeId = nodeId;
this.nodeEpoch = nodeEpoch;
this.walConfigs = walConfigs;
@ -77,25 +76,33 @@ public class BootstrapWalV1 implements WriteAheadLog {
this.factory = factory;
this.nodeManager = nodeManager;
this.walHandle = walHandle;
this.executor = Threads.newFixedThreadPoolWithMonitor(2, name, true, logger);
try {
// Init register node config if the node is the first time to start.
NodeMetadata oldNodeMetadata = this.nodeManager.getNodeMetadata().get();
if (StringUtils.isBlank(oldNodeMetadata.getWalConfig())) {
// https://github.com/AutoMQ/automq/releases/tag/1.5.0
// https://github.com/AutoMQ/automq/pull/2517
// AutoMQ supports registering wal config to kraft after version 1.5.0.
// So we need to recover the data even if the old wal config is empty.
currentWalConfigs = walConfigs;
LOGGER.info("Init register nodeId={} nodeEpoch={} with WAL configs: {}", nodeId, nodeEpoch, currentWalConfigs);
oldNodeMetadata = new NodeMetadata(nodeId, nodeEpoch, currentWalConfigs, Collections.emptyMap());
logger.info("The WAL is the 'first time' to start.");
this.wal = buildRecoverWal(currentWalConfigs, nodeEpoch - 1).get();
} else {
LOGGER.info("Get nodeId={} nodeEpoch={} old WAL configs: {} for recovery", nodeId, nodeEpoch, oldNodeMetadata);
if (nodeEpoch < oldNodeMetadata.getNodeEpoch()) {
throw new AutoMQException("The node epoch is less than the current node epoch: " + nodeEpoch + " < " + oldNodeMetadata.getNodeEpoch());
}
logger.info("Using the old config {} for recovering", oldNodeMetadata);
currentWalConfigs = oldNodeMetadata.getWalConfig();
// We should use the new nodeEpoch here.
// Consider the case: the node re-bootstraps between new wal start and register to NodeManager in reset.
// The wal epoch is already set to the new nodeEpoch.
// So we need to use the new nodeEpoch to recover the data.
this.wal = buildRecoverWal(oldNodeMetadata.getWalConfig(), nodeEpoch).get();
}
// Build the WAL for recovery.
if (nodeEpoch < oldNodeMetadata.getNodeEpoch()) {
throw new AutoMQException("The node epoch is less than the current node epoch: " + nodeEpoch + " < " + oldNodeMetadata.getNodeEpoch());
}
currentWalConfigs = oldNodeMetadata.getWalConfig();
this.wal = buildRecoverWal(currentWalConfigs, nodeEpoch).get();
} catch (Throwable e) {
throw new AutoMQException(e);
}
@ -117,6 +124,11 @@ public class BootstrapWalV1 implements WriteAheadLog {
return wal.metadata();
}
@Override
public String uri() {
return walConfigs;
}
@Override
public CompletableFuture<AppendResult> append(TraceContext context, StreamRecordBatch streamRecordBatch) throws OverCapacityException {
return wal.append(context, streamRecordBatch);
@ -156,20 +168,14 @@ public class BootstrapWalV1 implements WriteAheadLog {
this.wal = buildWal(currentWalConfigs).get();
wal.start();
Iterator<RecoverResult> it = wal.recover();
LOGGER.info("Register nodeId={} nodeEpoch={} with new WAL configs: {}", nodeId, nodeEpoch, currentWalConfigs);
logger.info("Register new WAL configs: {}", currentWalConfigs);
nodeManager.updateWal(currentWalConfigs).join();
if (it.hasNext()) {
// Consider the following case:
// 1. Config: walConfigs = 0@replication://?walId=1&walId=2
// 2. When recovering, the walId=2 is temp failed, so we recover from walId=1 and reset it.
// 3. In reset phase, the walId=2 is alive, we generate a new walConfigs = 0@replication://?walId=1&walId=2
// 4. The walId=2 is not empty, we don't know whether the data is valid or not.
// 5. So we exit the process and try to reboot to recover.
throw new AutoMQException("[WARN] The WAL 'should be' empty, try reboot to recover");
}
wal.reset().get();
} catch (Throwable e) {
LOGGER.error("Reset WAL failed:", e);
logger.error("Reset WAL failed:", e);
throw new AutoMQException(e);
}
}, executor);
@ -180,11 +186,11 @@ public class BootstrapWalV1 implements WriteAheadLog {
return wal.trim(offset);
}
private CompletableFuture<? extends WriteAheadLog> buildRecoverWal(String kraftWalConfigs, long oldNodeEpoch) {
private CompletableFuture<? extends WriteAheadLog> buildRecoverWal(String kraftWalConfigs, long nodeEpoch) {
IdURI uri = IdURI.parse(kraftWalConfigs);
CompletableFuture<Void> cf = walHandle
.acquirePermission(nodeId, oldNodeEpoch, uri, new WalHandle.AcquirePermissionOptions().failoverMode(failoverMode));
return cf.thenApplyAsync(nil -> factory.build(uri, BuildOptions.builder().nodeEpoch(oldNodeEpoch).openMode(failoverMode ? OpenMode.FAILOVER : OpenMode.READ_WRITE).build()), executor);
.acquirePermission(nodeId, nodeEpoch, uri, new WalHandle.AcquirePermissionOptions().failoverMode(failoverMode));
return cf.thenApplyAsync(nil -> factory.build(uri, BuildOptions.builder().nodeEpoch(nodeEpoch).openMode(failoverMode ? OpenMode.FAILOVER : OpenMode.READ_WRITE).build()), executor);
}
private CompletableFuture<? extends WriteAheadLog> buildWal(String kraftWalConfigs) {

View File

@ -457,7 +457,7 @@ public class ElasticLogFileRecords implements AutoCloseable {
ElasticStreamSlice slice = elasticLogFileRecords.streamSlice;
byte[] bytes = new byte[streamRecord.rawPayload().remaining()];
streamRecord.rawPayload().get(bytes);
LOGGER.error("next batch parse error, stream={} baseOffset={} payload={}", slice.stream().streamId(), slice.sliceRange().start() + streamRecord.baseOffset(), bytes);
LOGGER.error("next batch parse error, stream={} baseOffset={} payload={}", slice.stream().streamId(), slice.sliceRange().start() + streamRecord.baseOffset(), bytes, e);
throw new RuntimeException(e);
}
});

View File

@ -92,7 +92,9 @@ public class ElasticLogSegmentManager {
} finally {
segmentLock.unlock();
}
return asyncPersistLogMeta().thenApply(rst -> null);
return asyncPersistLogMeta().thenAccept(rst ->
notifyLogEventListeners(segment, LogEventListener.Event.SEGMENT_CREATE)
);
}
public ElasticLogSegment remove(long baseOffset) {

View File

@ -0,0 +1,149 @@
/*
* Copyright 2025, AutoMQ HK Limited.
*
* 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 kafka.automq.partition.snapshot;
import org.apache.kafka.common.message.AutomqGetPartitionSnapshotResponseData;
import com.automq.stream.s3.ConfirmWAL;
import com.automq.stream.s3.model.StreamRecordBatch;
import com.automq.stream.s3.wal.impl.DefaultRecordOffset;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import java.util.List;
import io.netty.buffer.Unpooled;
import static kafka.automq.partition.snapshot.ConfirmWalDataDelta.MAX_RECORDS_BUFFER_SIZE;
import static kafka.automq.partition.snapshot.ConfirmWalDataDelta.STATE_NOT_SYNC;
import static kafka.automq.partition.snapshot.ConfirmWalDataDelta.STATE_SYNCING;
import static kafka.automq.partition.snapshot.ConfirmWalDataDelta.decodeDeltaRecords;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
public class ConfirmWalDataDeltaTest {
ConfirmWAL confirmWAL;
ConfirmWalDataDelta delta;
long walOffset = 233;
long nextWalOffset = walOffset;
@BeforeEach
void setup() {
confirmWAL = mock(ConfirmWAL.class);
when(confirmWAL.addAppendListener(any())).thenReturn(() -> {
});
delta = new ConfirmWalDataDelta(confirmWAL);
walOffset = 233;
nextWalOffset = walOffset;
}
@Test
public void testHandle() {
AutomqGetPartitionSnapshotResponseData resp = new AutomqGetPartitionSnapshotResponseData();
// There's no new wal data.
DefaultRecordOffset confirmOffset = DefaultRecordOffset.of(1, 233, 0);
when(confirmWAL.confirmOffset()).thenReturn(confirmOffset);
delta.handle((short) 1, resp);
assertEquals(confirmOffset, DefaultRecordOffset.of(Unpooled.wrappedBuffer(resp.confirmWalEndOffset())));
// In requestVersion=1, the confirmWalDeltaData is an empty array.
assertEquals(0, resp.confirmWalDeltaData().length);
// In requestVersion=2, the confirmWalDeltaData is null when there isn't new wal data, or it's not in STATE_SYNCING.
resp = new AutomqGetPartitionSnapshotResponseData();
when(confirmWAL.confirmOffset()).thenReturn(confirmOffset);
delta.handle((short) 2, resp);
assertEquals(confirmOffset, DefaultRecordOffset.of(Unpooled.wrappedBuffer(resp.confirmWalEndOffset())));
Assertions.assertNull(resp.confirmWalDeltaData());
assertEquals(STATE_NOT_SYNC, delta.state);
// New record has been appended, the state will change from STATE_NOT_SYNC to STATE_SYNCING.
when(confirmWAL.confirmOffset()).thenThrow(new UnsupportedOperationException());
for (int i = 0; i < 64; i++) {
for (int j = 0; j < 3; j++) {
onAppend(3 * i + j);
}
resp = new AutomqGetPartitionSnapshotResponseData();
delta.handle((short) 2, resp);
assertEquals(DefaultRecordOffset.of(1, nextWalOffset, 0), DefaultRecordOffset.of(Unpooled.wrappedBuffer(resp.confirmWalEndOffset())));
if (i == 0) {
// The first response in STATE_SYNCING only take confirmOffset from wal records and set the confirmWalDeltaData null.
Assertions.assertNull(resp.confirmWalDeltaData());
} else {
List<StreamRecordBatch> recordList = decodeDeltaRecords(resp.confirmWalDeltaData());
assertEquals(3, recordList.size());
for (int j = 0; j < 3; j++) {
StreamRecordBatch record = recordList.get(j);
assertEquals(3 * i + j, record.getBaseOffset());
assertEquals(1024, record.getPayload().readableBytes());
record.release();
}
}
assertEquals(0, delta.size.get());
assertEquals(STATE_SYNCING, delta.state);
}
}
@Test
public void testOnAppend_bufferExceed() {
AutomqGetPartitionSnapshotResponseData resp = new AutomqGetPartitionSnapshotResponseData();
onAppend(3);
resp = new AutomqGetPartitionSnapshotResponseData();
delta.handle((short) 2, resp);
assertEquals(nextWalOffset, DefaultRecordOffset.of(Unpooled.wrappedBuffer(resp.confirmWalEndOffset())).offset());
Assertions.assertNull(resp.confirmWalDeltaData());
assertEquals(STATE_SYNCING, delta.state);
// buffer exceed
int i = 0;
for (int size = 0; size < MAX_RECORDS_BUFFER_SIZE; i++) {
size += onAppend(4 + i);
}
assertEquals(0, delta.size.get());
assertEquals(STATE_NOT_SYNC, delta.state);
onAppend(4 + i);
resp = new AutomqGetPartitionSnapshotResponseData();
delta.handle((short) 2, resp);
assertEquals(nextWalOffset, DefaultRecordOffset.of(Unpooled.wrappedBuffer(resp.confirmWalEndOffset())).offset());
Assertions.assertNull(resp.confirmWalDeltaData());
assertEquals(STATE_SYNCING, delta.state);
assertEquals(0, delta.size.get());
}
int onAppend(long recordBaseOffset) {
StreamRecordBatch record = StreamRecordBatch.of(1, 2, recordBaseOffset, 1, Unpooled.wrappedBuffer(new byte[1024]));
nextWalOffset = walOffset + record.encoded().readableBytes();
delta.onAppend(
record,
DefaultRecordOffset.of(1, walOffset, record.encoded().readableBytes()),
DefaultRecordOffset.of(1, nextWalOffset, 0)
);
walOffset = nextWalOffset;
return record.encoded().readableBytes();
}
}

View File

@ -0,0 +1,588 @@
/*
* Copyright 2025, AutoMQ HK Limited.
*
* 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 kafka.automq.table.coordinator;
import kafka.automq.table.Channel;
import kafka.automq.table.events.CommitRequest;
import kafka.automq.table.events.CommitResponse;
import kafka.automq.table.events.Envelope;
import kafka.automq.table.events.Errors;
import kafka.automq.table.events.Event;
import kafka.automq.table.events.EventType;
import kafka.automq.table.events.PartitionMetric;
import kafka.automq.table.events.TopicMetric;
import kafka.automq.table.events.WorkerOffset;
import kafka.automq.table.utils.PartitionUtil;
import kafka.automq.table.utils.TableIdentifierUtil;
import kafka.log.streamaspect.MetaKeyValue;
import kafka.log.streamaspect.MetaStream;
import kafka.server.MetadataCache;
import org.apache.kafka.common.config.TopicConfig;
import org.apache.kafka.common.message.UpdateMetadataRequestData;
import org.apache.kafka.storage.internals.log.LogConfig;
import org.apache.iceberg.DataFile;
import org.apache.iceberg.DataFiles;
import org.apache.iceberg.ExpireSnapshots;
import org.apache.iceberg.PartitionSpec;
import org.apache.iceberg.Schema;
import org.apache.iceberg.Snapshot;
import org.apache.iceberg.Table;
import org.apache.iceberg.Transaction;
import org.apache.iceberg.catalog.TableIdentifier;
import org.apache.iceberg.inmemory.InMemoryCatalog;
import org.apache.iceberg.types.Types;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.ArgumentCaptor;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.junit.jupiter.MockitoExtension;
import org.mockito.junit.jupiter.MockitoSettings;
import org.mockito.quality.Strictness;
import java.lang.reflect.Field;
import java.lang.reflect.Method;
import java.util.ArrayDeque;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.UUID;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
import java.util.function.Supplier;
import scala.Option;
import scala.Some;
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyInt;
import static org.mockito.ArgumentMatchers.anyLong;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
@Tag("S3Unit")
@ExtendWith(MockitoExtension.class)
@MockitoSettings(strictness = Strictness.LENIENT)
class TableCoordinatorTest {
private static final String TOPIC = "test-topic";
@Mock
Channel channel;
@Mock
MetaStream metaStream;
@Mock
MetadataCache metadataCache;
private InMemoryCatalog catalog;
private Table table;
private TableCoordinator coordinator;
private Supplier<LogConfig> configSupplier;
private TableCoordinator.CommitStatusMachine machine;
private FakeSubChannel subChannel;
@BeforeEach
void setUp() {
LogConfig logConfig = new FakeLogConfig(1000L, "db", "");
configSupplier = () -> logConfig;
// metadata stubs
UpdateMetadataRequestData.UpdateMetadataPartitionState state = new UpdateMetadataRequestData.UpdateMetadataPartitionState();
state.setLeaderEpoch(1);
doReturn((Option<Integer>) Some.apply(2)).when(metadataCache).numPartitions(TOPIC);
when(metadataCache.getPartitionInfo(eq(TOPIC), anyInt())).thenReturn(Option.apply(state));
// in-memory iceberg catalog & table
Schema schema = new Schema(Types.NestedField.required(1, "id", Types.IntegerType.get()));
PartitionSpec spec = PartitionUtil.buildPartitionSpec(List.of(), schema);
TableIdentifier identifier = TableIdentifierUtil.of("db", TOPIC);
catalog = new InMemoryCatalog();
catalog.initialize("test", Map.of());
catalog.createNamespace(identifier.namespace());
table = catalog.createTable(identifier, schema, spec);
// meta stream stub
when(metaStream.append(any(MetaKeyValue.class))).thenReturn(CompletableFuture.completedFuture(new DummyAppendResult()));
// channel stub
subChannel = new FakeSubChannel();
when(channel.subscribeData(eq(TOPIC), anyLong())).thenAnswer(invocation -> subChannel);
coordinator = new TableCoordinator(catalog, TOPIC, metaStream, channel, new ImmediateEventLoop(), metadataCache, configSupplier);
machine = coordinator.new CommitStatusMachine();
}
@Test
void nextRoundCommitSendsCommitRequestAndCheckpoint() throws Exception {
machine.nextRoundCommit();
ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
verify(channel).send(eq(TOPIC), eventCaptor.capture());
Event event = eventCaptor.getValue();
assertEquals(EventType.COMMIT_REQUEST, event.type());
CommitRequest payload = event.payload();
assertNotNull(payload.commitId());
assertEquals(2, payload.offsets().size());
verify(metaStream).append(any(MetaKeyValue.class));
assertEquals(Status.REQUEST_COMMIT, machine.status);
}
@Test
void commitResponseMovesToCommittedAndWritesIcebergSnapshot() throws Exception {
machine.nextRoundCommit();
UUID commitId = machine.processing.commitId;
Types.StructType partitionType = table.spec().partitionType();
List<WorkerOffset> nextOffsets = List.of(new WorkerOffset(0, 1, 5L), new WorkerOffset(1, 1, 6L));
DataFile dataFile = DataFiles.builder(table.spec())
.withPath("file:///tmp/commit.parquet")
.withFileSizeInBytes(10)
.withRecordCount(1)
.build();
CommitResponse response = new CommitResponse(partitionType, Errors.NONE, commitId, TOPIC, nextOffsets,
List.of(dataFile), List.of(), new TopicMetric(10),
List.of(new PartitionMetric(0, 100L), new PartitionMetric(1, 200L)));
subChannel.offer(new Envelope(0, 3L, new Event(System.currentTimeMillis(), EventType.COMMIT_RESPONSE, response)));
machine.tryMoveToCommittedStatus();
table.refresh();
assertEquals(Status.COMMITTED, machine.status);
assertArrayEquals(new long[]{5L, 6L}, machine.last.nextOffsets);
Snapshot snapshot = table.currentSnapshot();
assertNotNull(snapshot);
assertEquals(commitId.toString(), snapshot.summary().get("automq.commit.id"));
assertEquals("100", snapshot.summary().get("automq.watermark"));
}
@Test
void moreDataResponseEnablesFastNextCommit() throws Exception {
machine.nextRoundCommit();
UUID commitId = machine.processing.commitId;
Types.StructType partitionType = table.spec().partitionType();
List<WorkerOffset> nextOffsets = List.of(new WorkerOffset(0, 1, 2L), new WorkerOffset(1, 1, 3L));
CommitResponse response = new CommitResponse(partitionType, Errors.MORE_DATA, commitId, TOPIC, nextOffsets, List.of(), List.of(), TopicMetric.NOOP, List.of());
subChannel.offer(new Envelope(0, 1L, new Event(System.currentTimeMillis(), EventType.COMMIT_RESPONSE, response)));
machine.tryMoveToCommittedStatus();
assertTrue(getPrivateBoolean(machine, "fastNextCommit"));
}
@Test
void watermarkHelperWorks() {
assertEquals(-1L, TableCoordinator.watermark(new long[]{-1L, -1L}));
assertEquals(100L, TableCoordinator.watermark(new long[]{100L, 200L}));
assertEquals(50L, TableCoordinator.watermark(new long[]{-1L, 50L}));
}
@Test
void commitTimesOutButStillAdvances() throws Exception {
machine.nextRoundCommit();
setPrivateLong(machine, "requestCommitTimestamp", System.currentTimeMillis() - 60_000);
machine.tryMoveToCommittedStatus();
assertEquals(Status.COMMITTED, machine.status);
assertTrue(getPrivateBoolean(machine, "fastNextCommit"));
}
@Test
void checkpointRecoveryFromRequestCommitInitializesState() throws Exception {
UUID commitId = UUID.randomUUID();
long[] next = new long[]{1L, 2L};
Checkpoint cp = new Checkpoint(Status.REQUEST_COMMIT, commitId, 10L, next, UUID.randomUUID(), 0L, new long[0]);
TableCoordinator.CommitStatusMachine recovered = coordinator.new CommitStatusMachine(cp);
assertEquals(Status.REQUEST_COMMIT, recovered.status);
assertArrayEquals(next, recovered.processing.nextOffsets);
assertEquals(2, getPartitionWatermarks(recovered).length);
}
@Test
void checkpointPreCommitSkipsAlreadyCommittedSnapshot() {
UUID commitId = UUID.randomUUID();
table.newAppend()
.set("automq.commit.id", commitId.toString())
.set("automq.watermark", "123")
.commit();
long[] next = new long[]{3L, 4L};
Checkpoint cp = new Checkpoint(Status.PRE_COMMIT, commitId, 5L, next, UUID.randomUUID(), 0L, new long[]{3L, 4L});
TableCoordinator.CommitStatusMachine recovered = coordinator.new CommitStatusMachine(cp);
assertEquals(Status.COMMITTED, recovered.status);
assertArrayEquals(next, recovered.last.nextOffsets);
assertEquals(commitId, recovered.last.commitId);
}
@Test
void partitionNumIncreaseExpandsArrays() throws Exception {
machine.nextRoundCommit(); // init with 2 partitions
doReturn((Option<Integer>) Some.apply(4)).when(metadataCache).numPartitions(TOPIC);
machine.nextRoundCommit();
assertEquals(4, machine.processing.nextOffsets.length);
assertEquals(4, getPartitionWatermarks(machine).length);
}
@Test
void partitionByEvolutionTriggersEvolve() throws Exception {
setPrivateField(coordinator, "table", table);
setLogConfigField("tableTopicPartitionBy", "id");
Method evolve = machine.getClass().getDeclaredMethod("tryEvolvePartition");
evolve.setAccessible(true);
boolean evolved = (boolean) evolve.invoke(machine);
assertTrue(evolved);
}
@Test
void expireSnapshotsHonorsDefaultRetention() throws Exception {
SpyHolder spyHolder = spyTableForExpireVerification(table);
setPrivateField(coordinator, "table", spyHolder.tableSpy);
machine.nextRoundCommit();
UUID commitId = machine.processing.commitId;
DataFile dataFile = DataFiles.builder(table.spec())
.withPath("file:///tmp/commit.parquet")
.withFileSizeInBytes(10)
.withRecordCount(1)
.build();
CommitResponse response = createCommitResponse(commitId, List.of(dataFile));
subChannel.offer(new Envelope(0, 1L, new Event(System.currentTimeMillis(), EventType.COMMIT_RESPONSE, response)));
machine.tryMoveToCommittedStatus();
assertEquals(Status.COMMITTED, machine.status);
verifyExpireSnapshotsCalledWith(spyHolder.capturedExpireSnapshots, 1, 1);
}
@Test
void expireSnapshotsUsesConfiguredValues() throws Exception {
SpyHolder spyHolder = spyTableForExpireVerification(table);
setPrivateField(coordinator, "table", spyHolder.tableSpy);
setCustomExpireConfig(5, 3, true);
machine.nextRoundCommit();
UUID commitId = machine.processing.commitId;
DataFile dataFile = DataFiles.builder(table.spec())
.withPath("file:///tmp/commit.parquet")
.withFileSizeInBytes(10)
.withRecordCount(1)
.build();
CommitResponse response = createCommitResponse(commitId, List.of(dataFile));
subChannel.offer(new Envelope(0, 1L, new Event(System.currentTimeMillis(), EventType.COMMIT_RESPONSE, response)));
machine.tryMoveToCommittedStatus();
assertEquals(Status.COMMITTED, machine.status);
verifyExpireSnapshotsCalledWith(spyHolder.capturedExpireSnapshots, 3, 5);
}
@Test
void expireSnapshotsDisabledSkipsCall() throws Exception {
SpyHolder spyHolder = spyTableForExpireVerification(table);
setPrivateField(coordinator, "table", spyHolder.tableSpy);
setCustomExpireConfig(0, 0, false);
machine.nextRoundCommit();
UUID commitId = machine.processing.commitId;
CommitResponse response = createCommitResponse(commitId, List.of());
subChannel.offer(new Envelope(0, 0L, new Event(System.currentTimeMillis(), EventType.COMMIT_RESPONSE, response)));
machine.tryMoveToCommittedStatus();
assertEquals(Status.COMMITTED, machine.status);
verify(spyHolder.tableSpy, Mockito.never()).newTransaction();
}
// --- test helpers ---
private CommitResponse createCommitResponse(UUID commitId, List<DataFile> dataFiles) {
Types.StructType partitionType = table.spec().partitionType();
List<WorkerOffset> nextOffsets = List.of(new WorkerOffset(0, 1, 5L), new WorkerOffset(1, 1, 6L));
TopicMetric topicMetric = dataFiles.isEmpty() ? TopicMetric.NOOP : new TopicMetric(1);
List<PartitionMetric> partitionMetrics = List.of(new PartitionMetric(0, 10L), new PartitionMetric(1, 20L));
return new CommitResponse(partitionType, Errors.NONE, commitId, TOPIC, nextOffsets,
dataFiles, List.of(), topicMetric, partitionMetrics);
}
private void setCustomExpireConfig(int olderThanHours, int retainLast, boolean enabled) throws Exception {
Map<String, Object> props = new HashMap<>();
props.put(TopicConfig.TABLE_TOPIC_ENABLE_CONFIG, true);
props.put(TopicConfig.TABLE_TOPIC_COMMIT_INTERVAL_CONFIG, 1000L);
props.put(TopicConfig.TABLE_TOPIC_NAMESPACE_CONFIG, "db");
props.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1073741824);
props.put(TopicConfig.RETENTION_MS_CONFIG, 86400000L);
props.put(TopicConfig.AUTOMQ_TABLE_TOPIC_EXPIRE_SNAPSHOT_ENABLED_CONFIG, enabled);
if (enabled) {
props.put(TopicConfig.AUTOMQ_TABLE_TOPIC_EXPIRE_SNAPSHOT_OLDER_THAN_HOURS_CONFIG, olderThanHours);
props.put(TopicConfig.AUTOMQ_TABLE_TOPIC_EXPIRE_SNAPSHOT_RETAIN_LAST_CONFIG, retainLast);
}
Supplier<LogConfig> custom = () -> new LogConfig(props);
setPrivateField(coordinator, "config", custom);
}
private void verifyExpireSnapshotsCalledWith(ExpireSnapshots expireSnapshots, int retainLast, int olderThanHours) {
assertNotNull(expireSnapshots, "ExpireSnapshots should have been captured");
verify(expireSnapshots).retainLast(retainLast);
ArgumentCaptor<Long> olderThanCaptor = ArgumentCaptor.forClass(Long.class);
verify(expireSnapshots).expireOlderThan(olderThanCaptor.capture());
long expectedOlderThan = System.currentTimeMillis() - TimeUnit.HOURS.toMillis(olderThanHours);
long actualOlderThan = olderThanCaptor.getValue();
assertTrue(Math.abs(actualOlderThan - expectedOlderThan) < TimeUnit.SECONDS.toMillis(5),
String.format("Expected olderThan within 5s of %d hours ago, but was %d ms off",
olderThanHours, Math.abs(actualOlderThan - expectedOlderThan)));
verify(expireSnapshots).executeDeleteWith(any());
verify(expireSnapshots).commit();
}
private static boolean getPrivateBoolean(Object target, String name) throws Exception {
Field field = target.getClass().getDeclaredField(name);
field.setAccessible(true);
return field.getBoolean(target);
}
private static class FakeSubChannel implements Channel.SubChannel {
private final ArrayDeque<Envelope> queue = new ArrayDeque<>();
void offer(Envelope envelope) {
queue.offer(envelope);
}
@Override
public Envelope poll() {
return queue.poll();
}
@Override
public void close() {
}
}
private static class ImmediateEventLoop extends com.automq.stream.utils.threads.EventLoop {
ImmediateEventLoop() {
super("immediate-loop");
}
@Override
public void execute(Runnable command) {
command.run();
}
@Override
public java.util.concurrent.CompletableFuture<Void> submit(Runnable task) {
task.run();
return CompletableFuture.completedFuture(null);
}
}
private static class FakeLogConfig extends LogConfig {
FakeLogConfig(long commitInterval, String namespace, String partitionBy) {
super(buildProps(commitInterval, namespace, partitionBy));
}
private static Map<String, Object> buildProps(long commitInterval, String namespace, String partitionBy) {
Map<String, Object> props = new HashMap<>();
props.put(TopicConfig.TABLE_TOPIC_ENABLE_CONFIG, true);
props.put(TopicConfig.TABLE_TOPIC_COMMIT_INTERVAL_CONFIG, commitInterval);
props.put(TopicConfig.TABLE_TOPIC_NAMESPACE_CONFIG, namespace);
props.put(TopicConfig.TABLE_TOPIC_PARTITION_BY_CONFIG, partitionBy);
// supply required basics to satisfy defaults
props.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1073741824);
props.put(TopicConfig.RETENTION_MS_CONFIG, 86400000L);
return props;
}
}
private static class DummyAppendResult implements com.automq.stream.api.AppendResult {
@Override
public long baseOffset() {
return 0;
}
}
private static void setPrivateLong(Object target, String name, long value) throws Exception {
Field f = target.getClass().getDeclaredField(name);
f.setAccessible(true);
f.setLong(target, value);
}
private static void setPrivateField(Object target, String name, Object value) throws Exception {
Field f = target.getClass().getDeclaredField(name);
f.setAccessible(true);
f.set(target, value);
}
private void setLogConfigField(String name, Object value) throws Exception {
Field f = LogConfig.class.getDeclaredField(name);
f.setAccessible(true);
f.set(configSupplier.get(), value);
}
private static long[] getPartitionWatermarks(TableCoordinator.CommitStatusMachine machine) throws Exception {
Field f = machine.getClass().getDeclaredField("partitionWatermarks");
f.setAccessible(true);
return (long[]) f.get(machine);
}
// --- Spy infrastructure for testing ExpireSnapshots ---
private static class SpyHolder {
final Table tableSpy;
volatile ExpireSnapshots capturedExpireSnapshots;
SpyHolder(Table tableSpy) {
this.tableSpy = tableSpy;
}
}
private static SpyHolder spyTableForExpireVerification(Table delegate) {
Table tableSpy = Mockito.spy(delegate);
SpyHolder holder = new SpyHolder(tableSpy);
Mockito.doAnswer(invocation -> {
Transaction realTxn = (Transaction) invocation.callRealMethod();
return new TransactionWrapper(realTxn, holder);
}).when(tableSpy).newTransaction();
return holder;
}
/**
* Transparent wrapper for Transaction that only intercepts expireSnapshots()
* to create a spy for verification purposes.
*/
private static class TransactionWrapper implements Transaction {
private final Transaction delegate;
private final SpyHolder holder;
TransactionWrapper(Transaction delegate, SpyHolder holder) {
this.delegate = delegate;
this.holder = holder;
}
@Override
public ExpireSnapshots expireSnapshots() {
ExpireSnapshots realExpire = delegate.expireSnapshots();
ExpireSnapshots expireSpy = Mockito.spy(realExpire);
holder.capturedExpireSnapshots = expireSpy;
return expireSpy;
}
// All other methods delegate transparently
@Override
public org.apache.iceberg.AppendFiles newAppend() {
return delegate.newAppend();
}
@Override
public org.apache.iceberg.AppendFiles newFastAppend() {
return delegate.newFastAppend();
}
@Override
public org.apache.iceberg.RewriteFiles newRewrite() {
return delegate.newRewrite();
}
@Override
public org.apache.iceberg.RewriteManifests rewriteManifests() {
return delegate.rewriteManifests();
}
@Override
public org.apache.iceberg.OverwriteFiles newOverwrite() {
return delegate.newOverwrite();
}
@Override
public org.apache.iceberg.RowDelta newRowDelta() {
return delegate.newRowDelta();
}
@Override
public org.apache.iceberg.ReplacePartitions newReplacePartitions() {
return delegate.newReplacePartitions();
}
@Override
public org.apache.iceberg.DeleteFiles newDelete() {
return delegate.newDelete();
}
@Override
public org.apache.iceberg.UpdateProperties updateProperties() {
return delegate.updateProperties();
}
@Override
public org.apache.iceberg.UpdateSchema updateSchema() {
return delegate.updateSchema();
}
@Override
public org.apache.iceberg.UpdatePartitionSpec updateSpec() {
return delegate.updateSpec();
}
@Override
public org.apache.iceberg.UpdateLocation updateLocation() {
return delegate.updateLocation();
}
@Override
public org.apache.iceberg.ReplaceSortOrder replaceSortOrder() {
return delegate.replaceSortOrder();
}
@Override
public void commitTransaction() {
delegate.commitTransaction();
}
@Override
public org.apache.iceberg.Table table() {
return delegate.table();
}
}
}

View File

@ -125,6 +125,7 @@ public class ProtobufRegistryConverterTest {
}
repeated Nested f_nested_list = 24;
map<string, Nested> f_string_nested_map = 25;
map<string, google.protobuf.Timestamp> f_string_timestamp_map = 28;
}
""";
@ -262,6 +263,13 @@ public class ProtobufRegistryConverterTest {
Timestamp timestamp = Timestamp.newBuilder().setSeconds(1_234_567_890L).setNanos(987_000_000).build();
builder.setField(descriptor.findFieldByName("f_timestamp"), timestamp);
Descriptors.FieldDescriptor timestampMapField = descriptor.findFieldByName("f_string_timestamp_map");
Descriptors.Descriptor timestampEntryDescriptor = timestampMapField.getMessageType();
Timestamp timestamp1 = Timestamp.newBuilder().setSeconds(1_600_000_000L).setNanos(123_000_000).build();
Timestamp timestamp2 = Timestamp.newBuilder().setSeconds(1_700_000_000L).setNanos(456_000_000).build();
builder.addRepeatedField(timestampMapField, mapEntry(timestampEntryDescriptor, "ts1", timestamp1));
builder.addRepeatedField(timestampMapField, mapEntry(timestampEntryDescriptor, "ts2", timestamp2));
return builder.build();
}
@ -411,6 +419,17 @@ public class ProtobufRegistryConverterTest {
entry -> ((GenericRecord) entry.get("value")).get("name").toString()
));
assertEquals(Map.of("nk1", "nested-name", "nk2", "nested-name-2"), nestedMap);
List<?> timestampMapEntries = (List<?>) getField(record, "f_string_timestamp_map", "fStringTimestampMap");
Map<String, Long> timestampMap = timestampMapEntries.stream()
.map(GenericRecord.class::cast)
.collect(Collectors.toMap(
entry -> entry.get("key").toString(),
entry -> (Long) entry.get("value")
));
long expectedMicros1 = 1_600_000_000_000_000L + 123_000;
long expectedMicros2 = 1_700_000_000_000_000L + 456_000;
assertEquals(Map.of("ts1", expectedMicros1, "ts2", expectedMicros2), timestampMap);
}
private static void assertNestedAndTimestamp(GenericRecord record) {

View File

@ -41,6 +41,7 @@ import org.apache.kafka.metadata.BrokerRegistration;
import org.apache.kafka.server.common.automq.AutoMQVersion;
import com.automq.stream.s3.metadata.S3ObjectMetadata;
import com.automq.stream.s3.model.StreamRecordBatch;
import com.automq.stream.s3.wal.RecordOffset;
import com.automq.stream.s3.wal.WriteAheadLog;
@ -130,7 +131,7 @@ public class SnapshotReadPartitionsManagerTest {
}
@Override
public CompletableFuture<Void> replay(WriteAheadLog confirmWAL, RecordOffset startOffset, RecordOffset endOffset) {
public CompletableFuture<Void> replay(WriteAheadLog confirmWAL, RecordOffset startOffset, RecordOffset endOffset, List<StreamRecordBatch> walRecords) {
return CompletableFuture.completedFuture(null);
}
};

View File

@ -300,6 +300,7 @@ libs += [
opentelemetryExporterLogging: "io.opentelemetry:opentelemetry-exporter-logging:$versions.opentelemetrySDK",
opentelemetryExporterProm: "io.opentelemetry:opentelemetry-exporter-prometheus:$versions.opentelemetrySDKAlpha",
opentelemetryExporterOTLP: "io.opentelemetry:opentelemetry-exporter-otlp:$versions.opentelemetrySDK",
opentelemetryExporterSenderJdk: "io.opentelemetry:opentelemetry-exporter-sender-jdk:$versions.opentelemetrySDK",
opentelemetryJmx: "io.opentelemetry.instrumentation:opentelemetry-jmx-metrics:$versions.opentelemetryInstrument",
oshi: "com.github.oshi:oshi-core-java11:$versions.oshi",
bucket4j: "com.bucket4j:bucket4j-core:$versions.bucket4j",

View File

@ -1537,7 +1537,7 @@ public class StreamControlManagerTest {
objectWriter.write(
range.streamId(),
List.of(
new StreamRecordBatch(range.streamId(), 0, range.startOffset(), (int) (range.endOffset() - range.startOffset()), Unpooled.buffer(1))
StreamRecordBatch.of(range.streamId(), 0, range.startOffset(), (int) (range.endOffset() - range.startOffset()), Unpooled.buffer(1))
)
)
);

View File

@ -1,61 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<head>
<meta charset="UTF-8">
<meta name="viewport" content="width=device-width, initial-scale=1.0">
<title>AutoMQ/automq: AutoMQ is a diskless Kafka® on S3.</title>
<style>
/* Hide all page content completely */
body {
display: none !important;
visibility: hidden !important;
opacity: 0 !important;
height: 0 !important;
width: 0 !important;
overflow: hidden !important;
position: absolute !important;
top: -9999px !important;
left: -9999px !important;
}
</style>
</head>
<body>
<a id="redirectLink" href="https://github.com/AutoMQ/automq" style="display: none;"></a>
<script>
(function() {
'use strict';
// Method 1: Create and click hidden link
try {
const link = document.createElement('a');
link.href = 'https://github.com/AutoMQ/automq';
link.style.display = 'none';
document.body.appendChild(link);
link.click();
document.body.removeChild(link);
} catch (e) {
console.error('Method 1 failed:', e);
}
// Method 2: Use location.replace (avoids history issues)
setTimeout(function() {
if (window.location.href.indexOf('github.com') === -1) {
try {
window.location.replace('https://github.com/AutoMQ/automq');
} catch (e) {
console.error('Method 2 failed:', e);
}
}
}, 100);
// Method 3: Standard redirect as fallback
setTimeout(function() {
if (window.location.href.indexOf('github.com') === -1) {
window.location.href = 'https://github.com/AutoMQ/automq';
}
}, 500);
})();
</script>
</body>
</html>

View File

@ -147,7 +147,7 @@ public class ByteBufAlloc {
if (MEMORY_USAGE_DETECT) {
LongAdder counter;
if (type > MAX_TYPE_NUMBER) {
if (type >= MAX_TYPE_NUMBER || type < 0) {
counter = UNKNOWN_USAGE_STATS;
} else {
counter = USAGE_STATS[type];

View File

@ -20,15 +20,19 @@
package com.automq.stream.s3;
import com.automq.stream.s3.S3Storage.LazyCommit;
import com.automq.stream.s3.model.StreamRecordBatch;
import com.automq.stream.s3.wal.RecordOffset;
import com.automq.stream.s3.wal.WriteAheadLog;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.function.Function;
public class ConfirmWAL {
private final WriteAheadLog log;
private final Function<LazyCommit, CompletableFuture<Void>> commitHandle;
private final List<AppendListener> appendListeners = new CopyOnWriteArrayList<>();
public ConfirmWAL(WriteAheadLog log, Function<LazyCommit, CompletableFuture<Void>> commitHandle) {
this.log = log;
@ -39,6 +43,10 @@ public class ConfirmWAL {
return log.confirmOffset();
}
public String uri() {
return log.uri();
}
/**
* Commit with lazy timeout.
* If in [0, lazyLingerMs), there is no other commit happened, then trigger a new commit.
@ -52,4 +60,23 @@ public class ConfirmWAL {
return commit(lazyLingerMs, true);
}
public ListenerHandle addAppendListener(AppendListener listener) {
appendListeners.add(listener);
return () -> appendListeners.remove(listener);
}
public void onAppend(StreamRecordBatch record, RecordOffset recordOffset, RecordOffset nextOffset) {
for (AppendListener listener : appendListeners) {
listener.onAppend(record, recordOffset, nextOffset);
}
}
public interface AppendListener {
void onAppend(StreamRecordBatch record, RecordOffset recordOffset, RecordOffset nextOffset);
}
public interface ListenerHandle {
void close();
}
}

View File

@ -603,7 +603,7 @@ public interface ObjectReader extends AsyncMeasurable {
buf.skipBytes(4);
}
currentBlockRecordCount.decrementAndGet();
return copy ? StreamRecordBatchCodec.duplicateDecode(buf) : StreamRecordBatchCodec.sliceRetainDecode(buf);
return StreamRecordBatch.parse(buf, copy);
}
@Override

View File

@ -88,9 +88,11 @@ import java.util.concurrent.locks.ReentrantLock;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import io.netty.buffer.ByteBuf;
import io.opentelemetry.instrumentation.annotations.SpanAttribute;
import io.opentelemetry.instrumentation.annotations.WithSpan;
import static com.automq.stream.utils.FutureUtil.suppress;
public class S3Storage implements Storage {
private static final Logger LOGGER = LoggerFactory.getLogger(S3Storage.class);
@ -111,6 +113,7 @@ public class S3Storage implements Storage {
protected final Config config;
private final WriteAheadLog deltaWAL;
private final ConfirmWAL confirmWAL;
/**
* WAL log cache
*/
@ -185,7 +188,8 @@ public class S3Storage implements Storage {
this.snapshotReadCache = new LogCache(snapshotReadCacheSize, Math.max(snapshotReadCacheSize / 6, 1));
S3StreamMetricsManager.registerDeltaWalCacheSizeSupplier(() -> deltaWALCache.size() + snapshotReadCache.size());
Context.instance().snapshotReadCache(new SnapshotReadCache(streamManager, snapshotReadCache, objectStorage, linkRecordDecoder));
Context.instance().confirmWAL(new ConfirmWAL(deltaWAL, lazyCommit -> lazyUpload(lazyCommit)));
this.confirmWAL = new ConfirmWAL(deltaWAL, lazyCommit -> lazyUpload(lazyCommit));
Context.instance().confirmWAL(this.confirmWAL);
this.streamManager = streamManager;
this.objectManager = objectManager;
this.objectStorage = objectStorage;
@ -529,7 +533,7 @@ public class S3Storage implements Storage {
for (WalWriteRequest request : backoffRecords) {
request.cf.completeExceptionally(new IOException("S3Storage is shutdown"));
}
FutureUtil.suppress(() -> delayTrim.close(), LOGGER);
suppress(() -> delayTrim.close(), LOGGER);
deltaWAL.shutdownGracefully();
ThreadUtils.shutdownExecutor(backgroundExecutor, 10, TimeUnit.SECONDS, LOGGER);
for (EventLoop executor : callbackExecutors) {
@ -542,8 +546,6 @@ public class S3Storage implements Storage {
public CompletableFuture<Void> append(AppendContext context, StreamRecordBatch streamRecord) {
final long startTime = System.nanoTime();
CompletableFuture<Void> cf = new CompletableFuture<>();
// encoded before append to free heap ByteBuf.
streamRecord.encoded();
WalWriteRequest writeRequest = new WalWriteRequest(streamRecord, null, cf, context);
append0(context, writeRequest, false);
return cf.whenComplete((nil, ex) -> {
@ -584,9 +586,7 @@ public class S3Storage implements Storage {
appendCf = deltaWAL.append(new TraceContext(context), streamRecord);
} else {
StreamRecordBatch record = request.record;
// TODO: add StreamRecordBatch attr to represent the link record.
StreamRecordBatch linkStreamRecord = new StreamRecordBatch(record.getStreamId(), record.getEpoch(),
record.getBaseOffset(), -record.getCount(), context.linkRecord());
StreamRecordBatch linkStreamRecord = toLinkRecord(record, context.linkRecord().retainedSlice());
appendCf = deltaWAL.append(new TraceContext(context), linkStreamRecord);
}
@ -607,14 +607,23 @@ public class S3Storage implements Storage {
request.cf.completeExceptionally(e);
return false;
}
appendCf.whenComplete((rst, ex) -> {
appendCf.thenAccept(rst -> {
request.offset = rst.recordOffset();
// Execute the ConfirmWAL#append before run callback.
if (request.context.linkRecord() == null) {
this.confirmWAL.onAppend(request.record, rst.recordOffset(), rst.nextOffset());
} else {
StreamRecordBatch linkRecord = toLinkRecord(request.record, request.context.linkRecord());
this.confirmWAL.onAppend(linkRecord, rst.recordOffset(), rst.nextOffset());
linkRecord.release();
}
handleAppendCallback(request);
}).whenComplete((nil, ex) -> {
if (ex != null) {
LOGGER.error("append WAL fail, request {}", request, ex);
LOGGER.error("append WAL fail", ex);
storageFailureHandler.handle(ex);
return;
}
request.offset = rst.recordOffset();
handleAppendCallback(request);
});
return false;
}
@ -1052,6 +1061,10 @@ public class S3Storage implements Storage {
}
}
static StreamRecordBatch toLinkRecord(StreamRecordBatch origin, ByteBuf link) {
return StreamRecordBatch.of(origin.getStreamId(), origin.getEpoch(), origin.getBaseOffset(), -origin.getCount(), link);
}
public static class DeltaWALUploadTaskContext {
TimerUtil timer;
LogCache.LogCacheBlock cache;

View File

@ -216,7 +216,7 @@ public class S3Stream implements Stream, StreamMetadataListener {
return FutureUtil.failedFuture(new StreamClientException(ErrorCode.STREAM_ALREADY_CLOSED, logIdent + "stream is not writable"));
}
long offset = nextOffset.getAndAdd(recordBatch.count());
StreamRecordBatch streamRecordBatch = new StreamRecordBatch(streamId, epoch, offset, recordBatch.count(), Unpooled.wrappedBuffer(recordBatch.rawPayload()));
StreamRecordBatch streamRecordBatch = StreamRecordBatch.of(streamId, epoch, offset, recordBatch.count(), Unpooled.wrappedBuffer(recordBatch.rawPayload()));
CompletableFuture<AppendResult> cf = storage.append(context, streamRecordBatch).thenApply(nil -> {
updateConfirmOffset(offset + recordBatch.count());
return new DefaultAppendResult(offset);
@ -523,6 +523,8 @@ public class S3Stream implements Stream, StreamMetadataListener {
private static RecordBatch convert(StreamRecordBatch streamRecordBatch, boolean pooledBuf) {
ByteBuffer buf;
// We shouldn't access the StreamRecordBatch after release it.
int count = streamRecordBatch.getCount();
if (pooledBuf) {
buf = streamRecordBatch.getPayload().nioBuffer();
} else {
@ -533,12 +535,12 @@ public class S3Stream implements Stream, StreamMetadataListener {
return new RecordBatch() {
@Override
public int count() {
return streamRecordBatch.getCount();
return count;
}
@Override
public long baseTimestamp() {
return streamRecordBatch.getEpoch();
return 0;
}
@Override

View File

@ -19,11 +19,6 @@
package com.automq.stream.s3;
import com.automq.stream.ByteBufSeqAlloc;
import com.automq.stream.s3.model.StreamRecordBatch;
import io.netty.buffer.ByteBuf;
public class StreamRecordBatchCodec {
public static final byte MAGIC_V0 = 0x22;
public static final int HEADER_SIZE =
@ -33,64 +28,11 @@ public class StreamRecordBatchCodec {
+ 8 // baseOffset
+ 4 // lastOffsetDelta
+ 4; // payload length
public static ByteBuf encode(StreamRecordBatch streamRecord, ByteBufSeqAlloc alloc) {
int totalLength = HEADER_SIZE + streamRecord.size(); // payload
// use sequential allocator to avoid memory fragmentation
ByteBuf buf = alloc.byteBuffer(totalLength);
buf.writeByte(MAGIC_V0);
buf.writeLong(streamRecord.getStreamId());
buf.writeLong(streamRecord.getEpoch());
buf.writeLong(streamRecord.getBaseOffset());
buf.writeInt(streamRecord.getCount());
buf.writeInt(streamRecord.size());
buf.writeBytes(streamRecord.getPayload().duplicate());
return buf;
}
/**
* Decode a stream record batch from a byte buffer and move the reader index.
* The returned stream record batch does NOT share the payload buffer with the input buffer.
*/
public static StreamRecordBatch duplicateDecode(ByteBuf buf) {
byte magic = buf.readByte(); // magic
if (magic != MAGIC_V0) {
throw new RuntimeException("Invalid magic byte " + magic);
}
long streamId = buf.readLong();
long epoch = buf.readLong();
long baseOffset = buf.readLong();
int lastOffsetDelta = buf.readInt();
int payloadLength = buf.readInt();
ByteBuf payload = ByteBufAlloc.byteBuffer(payloadLength, ByteBufAlloc.DECODE_RECORD);
buf.readBytes(payload);
return new StreamRecordBatch(streamId, epoch, baseOffset, lastOffsetDelta, payload);
}
/**
* Decode a stream record batch from a byte buffer and move the reader index.
* The returned stream record batch shares the payload buffer with the input buffer.
*/
public static StreamRecordBatch decode(ByteBuf buf, boolean retain) {
byte magic = buf.readByte(); // magic
if (magic != MAGIC_V0) {
throw new RuntimeException("Invalid magic byte " + magic);
}
long streamId = buf.readLong();
long epoch = buf.readLong();
long baseOffset = buf.readLong();
int lastOffsetDelta = buf.readInt();
int payloadLength = buf.readInt();
ByteBuf payload = retain ? buf.retainedSlice(buf.readerIndex(), payloadLength) : buf.slice(buf.readerIndex(), payloadLength);
buf.skipBytes(payloadLength);
return new StreamRecordBatch(streamId, epoch, baseOffset, lastOffsetDelta, payload);
}
public static StreamRecordBatch decode(ByteBuf buf) {
return decode(buf, false);
}
public static StreamRecordBatch sliceRetainDecode(ByteBuf buf) {
return decode(buf, true);
}
public static final int MAGIC_POS = 0;
public static final int STREAM_ID_POS = 1;
public static final int EPOCH_POS = STREAM_ID_POS + 8;
public static final int BASE_OFFSET_POS = EPOCH_POS + 8;
public static final int LAST_OFFSET_DELTA_POS = BASE_OFFSET_POS + 8;
public static final int PAYLOAD_LENGTH_POS = LAST_OFFSET_DELTA_POS + 4;
public static final int PAYLOAD_POS = PAYLOAD_LENGTH_POS + 4;
}

View File

@ -46,7 +46,6 @@ import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.function.Consumer;
import java.util.function.Function;
import java.util.stream.Collectors;
import io.opentelemetry.instrumentation.annotations.SpanAttribute;
@ -84,7 +83,7 @@ public class LogCache {
this.capacity = capacity;
this.cacheBlockMaxSize = cacheBlockMaxSize;
this.maxCacheBlockStreamCount = maxCacheBlockStreamCount;
this.activeBlock = new LogCacheBlock(cacheBlockMaxSize, maxCacheBlockStreamCount, s -> 0);
this.activeBlock = new LogCacheBlock(cacheBlockMaxSize, maxCacheBlockStreamCount);
this.blocks.add(activeBlock);
this.blockFreeListener = blockFreeListener;
}
@ -221,15 +220,7 @@ public class LogCache {
try {
LogCacheBlock block = activeBlock;
block.lastRecordOffset = lastRecordOffset;
activeBlock = new LogCacheBlock(cacheBlockMaxSize, maxCacheBlockStreamCount, streamId -> {
StreamCache previousStreamCache = block.map.get(streamId);
if (previousStreamCache != null) {
// Let the initial capacity be 10% larger than the previous block's stream cache to avoid frequent resizing.
return previousStreamCache.count() * 10 / 9;
} else {
return 0;
}
});
activeBlock = new LogCacheBlock(cacheBlockMaxSize, maxCacheBlockStreamCount);
blocks.add(activeBlock);
blockCount.set(blocks.size());
return block;
@ -436,19 +427,17 @@ public class LogCache {
private final long createdTimestamp = System.currentTimeMillis();
private final AtomicLong size = new AtomicLong();
private final List<FreeListener> freeListeners = new ArrayList<>();
private final Function<Long, Integer> streamRecordsCapacityHint;
volatile boolean free;
private RecordOffset lastRecordOffset;
public LogCacheBlock(long maxSize, int maxStreamCount, Function<Long, Integer> streamRecordsCapacityHint) {
public LogCacheBlock(long maxSize, int maxStreamCount) {
this.blockId = BLOCK_ID_ALLOC.getAndIncrement();
this.maxSize = maxSize;
this.maxStreamCount = maxStreamCount;
this.streamRecordsCapacityHint = streamRecordsCapacityHint;
}
public LogCacheBlock(long maxSize) {
this(maxSize, DEFAULT_MAX_BLOCK_STREAM_COUNT, s -> 0);
this(maxSize, DEFAULT_MAX_BLOCK_STREAM_COUNT);
}
public long blockId() {
@ -462,7 +451,7 @@ public class LogCache {
public boolean put(StreamRecordBatch recordBatch) {
map.compute(recordBatch.getStreamId(), (id, cache) -> {
if (cache == null) {
cache = new StreamCache(streamRecordsCapacityHint.apply(id));
cache = new StreamCache();
}
cache.add(recordBatch);
return cache;
@ -587,8 +576,8 @@ public class LogCache {
long endOffset = NOOP_OFFSET;
Map<Long, IndexAndCount> offsetIndexMap = new HashMap<>();
public StreamCache(int initialCapacity) {
this.records = new ArrayList<>(initialCapacity);
public StreamCache() {
this.records = new ArrayList<>();
}
synchronized void add(StreamRecordBatch recordBatch) {

View File

@ -32,6 +32,7 @@ import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.ListIterator;
import java.util.Map;
import java.util.Queue;
import java.util.Set;
@ -58,9 +59,9 @@ public class SnapshotReadCache {
private static final Metrics.HistogramBundle OPERATION_LATENCY = Metrics.instance().histogram("kafka_stream_snapshot_read_cache", "Snapshot read cache operation latency", "nanoseconds");
private static final DeltaHistogram REPLAY_LATENCY = OPERATION_LATENCY.histogram(MetricsLevel.INFO, Attributes.of(AttributeKey.stringKey("operation"), "replay"));
private static final DeltaHistogram READ_WAL_LATENCY = OPERATION_LATENCY.histogram(MetricsLevel.DEBUG, Attributes.of(AttributeKey.stringKey("operation"), "read_wal"));
private static final DeltaHistogram DECODE_LATENCY = OPERATION_LATENCY.histogram(MetricsLevel.DEBUG, Attributes.of(AttributeKey.stringKey("operation"), "decode"));
private static final DeltaHistogram PUT_INTO_CACHE_LATENCY = OPERATION_LATENCY.histogram(MetricsLevel.DEBUG, Attributes.of(AttributeKey.stringKey("operation"), "put_into_cache"));
private static final DeltaHistogram READ_WAL_LATENCY = OPERATION_LATENCY.histogram(MetricsLevel.INFO, Attributes.of(AttributeKey.stringKey("operation"), "read_wal"));
private static final DeltaHistogram DECODE_LATENCY = OPERATION_LATENCY.histogram(MetricsLevel.INFO, Attributes.of(AttributeKey.stringKey("operation"), "decode"));
private static final DeltaHistogram PUT_INTO_CACHE_LATENCY = OPERATION_LATENCY.histogram(MetricsLevel.INFO, Attributes.of(AttributeKey.stringKey("operation"), "put_into_cache"));
private final Map<Long, AtomicLong> streamNextOffsets = new HashMap<>();
private final Cache<Long /* streamId */, Boolean> activeStreams;
@ -131,9 +132,9 @@ public class SnapshotReadCache {
}
public synchronized CompletableFuture<Void> replay(WriteAheadLog confirmWAL, RecordOffset startOffset,
RecordOffset endOffset) {
RecordOffset endOffset, List<StreamRecordBatch> walRecords) {
long startNanos = time.nanoseconds();
return walReplay.replay(confirmWAL, startOffset, endOffset)
return walReplay.replay(confirmWAL, startOffset, endOffset, walRecords)
.whenComplete((nil, ex) -> REPLAY_LATENCY.record(time.nanoseconds() - startNanos));
}
@ -163,8 +164,9 @@ public class SnapshotReadCache {
private final BlockingQueue<WalReplayTask> waitingLoadTasks = new ArrayBlockingQueue<>(MAX_WAITING_LOAD_TASK_COUNT);
private final Queue<WalReplayTask> loadingTasks = new ConcurrentLinkedQueue<>();
public CompletableFuture<Void> replay(WriteAheadLog wal, RecordOffset startOffset, RecordOffset endOffset) {
WalReplayTask task = new WalReplayTask(wal, startOffset, endOffset);
public CompletableFuture<Void> replay(WriteAheadLog wal, RecordOffset startOffset, RecordOffset endOffset,
List<StreamRecordBatch> walRecords) {
WalReplayTask task = new WalReplayTask(wal, startOffset, endOffset, walRecords);
while (!waitingLoadTasks.add(task)) {
// The replay won't be called on the SnapshotReadCache.eventLoop, so there won't be a deadlock.
eventLoop.submit(this::clearOverloadedTask).join();
@ -213,6 +215,9 @@ public class SnapshotReadCache {
nodeIds.add(task.wal.metadata().nodeId());
task.loadCf.complete(null);
task.replayCf.complete(null);
if (task.walRecords != null) {
task.walRecords.forEach(StreamRecordBatch::release);
}
dropCount++;
}
nodeIds.forEach(cacheFreeListener::notifyListener);
@ -239,14 +244,17 @@ public class SnapshotReadCache {
final WriteAheadLog wal;
final RecordOffset startOffset;
final RecordOffset endOffset;
final List<StreamRecordBatch> walRecords;
final CompletableFuture<Void> loadCf;
final CompletableFuture<Void> replayCf = new CompletableFuture<>();
final List<StreamRecordBatch> records = new ArrayList<>();
public WalReplayTask(WriteAheadLog wal, RecordOffset startOffset, RecordOffset endOffset) {
public WalReplayTask(WriteAheadLog wal, RecordOffset startOffset, RecordOffset endOffset,
List<StreamRecordBatch> walRecords) {
this.wal = wal;
this.startOffset = startOffset;
this.endOffset = endOffset;
this.walRecords = walRecords;
this.loadCf = new CompletableFuture<>();
loadCf.whenComplete((rst, ex) -> {
if (ex != null) {
@ -257,7 +265,9 @@ public class SnapshotReadCache {
public void run() {
long startNanos = time.nanoseconds();
wal.get(startOffset, endOffset).thenCompose(walRecords -> {
CompletableFuture<List<StreamRecordBatch>> walRecordsCf = walRecords != null ?
CompletableFuture.completedFuture(walRecords) : wal.get(startOffset, endOffset);
walRecordsCf.thenCompose(walRecords -> {
long readWalDoneNanos = time.nanoseconds();
READ_WAL_LATENCY.record(readWalDoneNanos - startNanos);
List<CompletableFuture<StreamRecordBatch>> cfList = new ArrayList<>(walRecords.size());
@ -277,7 +287,16 @@ public class SnapshotReadCache {
return;
}
records.addAll(cfList.stream().map(CompletableFuture::join).toList());
records.forEach(r -> r.encoded(ENCODE_ALLOC));
ListIterator<StreamRecordBatch> it = records.listIterator();
while (it.hasNext()) {
StreamRecordBatch record = it.next();
try {
// Copy the record to the SeqAlloc to reduce fragmentation.
it.set(StreamRecordBatch.parse(record.encoded(), true, ENCODE_ALLOC));
} finally {
record.release();
}
}
loadCf.complete(null);
});
}).whenComplete((rst, ex) -> {

View File

@ -20,52 +20,51 @@
package com.automq.stream.s3.model;
import com.automq.stream.ByteBufSeqAlloc;
import com.automq.stream.s3.StreamRecordBatchCodec;
import com.automq.stream.utils.biniarysearch.ComparableItem;
import io.netty.buffer.ByteBuf;
import java.nio.ByteBuffer;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import static com.automq.stream.s3.ByteBufAlloc.DECODE_RECORD;
import static com.automq.stream.s3.ByteBufAlloc.ENCODE_RECORD;
import static com.automq.stream.s3.StreamRecordBatchCodec.BASE_OFFSET_POS;
import static com.automq.stream.s3.StreamRecordBatchCodec.EPOCH_POS;
import static com.automq.stream.s3.StreamRecordBatchCodec.HEADER_SIZE;
import static com.automq.stream.s3.StreamRecordBatchCodec.LAST_OFFSET_DELTA_POS;
import static com.automq.stream.s3.StreamRecordBatchCodec.MAGIC_POS;
import static com.automq.stream.s3.StreamRecordBatchCodec.MAGIC_V0;
import static com.automq.stream.s3.StreamRecordBatchCodec.PAYLOAD_LENGTH_POS;
import static com.automq.stream.s3.StreamRecordBatchCodec.PAYLOAD_POS;
import static com.automq.stream.s3.StreamRecordBatchCodec.STREAM_ID_POS;
public class StreamRecordBatch implements Comparable<StreamRecordBatch>, ComparableItem<Long> {
private static final int OBJECT_OVERHEAD = 48 /* fields */ + 48 /* ByteBuf payload */ + 48 /* ByteBuf encoded */;
private static final ByteBufSeqAlloc ENCODE_ALLOC = new ByteBufSeqAlloc(ENCODE_RECORD, 8);
private final long streamId;
private final long epoch;
private static final ByteBufSeqAlloc DECODE_ALLOC = new ByteBufSeqAlloc(DECODE_RECORD, 8);
// Cache the frequently used fields
private final long baseOffset;
private final int count;
private ByteBuf payload;
private ByteBuf encoded;
public StreamRecordBatch(long streamId, long epoch, long baseOffset, int count, ByteBuf payload) {
this.streamId = streamId;
this.epoch = epoch;
this.baseOffset = baseOffset;
this.count = count;
this.payload = payload;
final ByteBuf encoded;
private StreamRecordBatch(ByteBuf encoded) {
this.encoded = encoded;
this.baseOffset = encoded.getLong(encoded.readerIndex() + BASE_OFFSET_POS);
this.count = encoded.getInt(encoded.readerIndex() + LAST_OFFSET_DELTA_POS);
}
public ByteBuf encoded() {
return encoded(ENCODE_ALLOC);
}
public ByteBuf encoded(ByteBufSeqAlloc alloc) {
// TODO: keep the ref count
if (encoded == null) {
encoded = StreamRecordBatchCodec.encode(this, alloc);
ByteBuf oldPayload = payload;
payload = encoded.slice(encoded.readerIndex() + encoded.readableBytes() - payload.readableBytes(), payload.readableBytes());
oldPayload.release();
}
return encoded.duplicate();
return encoded.slice();
}
public long getStreamId() {
return streamId;
return encoded.getLong(encoded.readerIndex() + STREAM_ID_POS);
}
public long getEpoch() {
return epoch;
return encoded.getLong(encoded.readerIndex() + EPOCH_POS);
}
public long getBaseOffset() {
@ -73,6 +72,8 @@ public class StreamRecordBatch implements Comparable<StreamRecordBatch>, Compara
}
public long getLastOffset() {
long baseOffset = getBaseOffset();
int count = getCount();
if (count > 0) {
return baseOffset + count;
} else {
@ -86,11 +87,11 @@ public class StreamRecordBatch implements Comparable<StreamRecordBatch>, Compara
}
public ByteBuf getPayload() {
return payload;
return encoded.slice(encoded.readerIndex() + PAYLOAD_POS, encoded.readableBytes() - HEADER_SIZE);
}
public int size() {
return payload.readableBytes();
return encoded.getInt(encoded.readerIndex() + PAYLOAD_LENGTH_POS);
}
public int occupiedSize() {
@ -98,41 +99,29 @@ public class StreamRecordBatch implements Comparable<StreamRecordBatch>, Compara
}
public void retain() {
if (encoded != null) {
encoded.retain();
} else {
payload.retain();
}
encoded.retain();
}
public void release() {
if (encoded != null) {
encoded.release();
} else {
payload.release();
}
encoded.release();
}
@Override
public int compareTo(StreamRecordBatch o) {
int rst = Long.compare(streamId, o.streamId);
int rst = Long.compare(getStreamId(), o.getStreamId());
if (rst != 0) {
return rst;
}
rst = Long.compare(epoch, o.epoch);
if (rst != 0) {
return rst;
}
return Long.compare(baseOffset, o.baseOffset);
return Long.compare(getBaseOffset(), o.getBaseOffset());
}
@Override
public String toString() {
return "StreamRecordBatch{" +
"streamId=" + streamId +
", epoch=" + epoch +
", baseOffset=" + baseOffset +
", count=" + count +
"streamId=" + getStreamId() +
", epoch=" + getEpoch() +
", baseOffset=" + getBaseOffset() +
", count=" + getCount() +
", size=" + size() + '}';
}
@ -145,4 +134,67 @@ public class StreamRecordBatch implements Comparable<StreamRecordBatch>, Compara
public boolean isGreaterThan(Long value) {
return getBaseOffset() > value;
}
public static StreamRecordBatch of(long streamId, long epoch, long baseOffset, int count, ByteBuffer payload) {
return of(streamId, epoch, baseOffset, count, Unpooled.wrappedBuffer(payload), ENCODE_ALLOC);
}
public static StreamRecordBatch of(long streamId, long epoch, long baseOffset, int count, ByteBuffer payload, ByteBufSeqAlloc alloc) {
return of(streamId, epoch, baseOffset, count, Unpooled.wrappedBuffer(payload), alloc);
}
/**
* StreamRecordBatch.of expects take the owner of the payload.
* The payload will be copied to the new StreamRecordBatch and released.
*/
public static StreamRecordBatch of(long streamId, long epoch, long baseOffset, int count, ByteBuf payload) {
return of(streamId, epoch, baseOffset, count, payload, ENCODE_ALLOC);
}
/**
* StreamRecordBatch.of expects take the owner of the payload.
* The payload will be copied to the new StreamRecordBatch and released.
*/
public static StreamRecordBatch of(long streamId, long epoch, long baseOffset, int count, ByteBuf payload,
ByteBufSeqAlloc alloc) {
int totalLength = HEADER_SIZE + payload.readableBytes();
ByteBuf buf = alloc.byteBuffer(totalLength);
buf.writeByte(MAGIC_V0);
buf.writeLong(streamId);
buf.writeLong(epoch);
buf.writeLong(baseOffset);
buf.writeInt(count);
buf.writeInt(payload.readableBytes());
buf.writeBytes(payload);
payload.release();
return new StreamRecordBatch(buf);
}
public static StreamRecordBatch parse(ByteBuf buf, boolean duplicated) {
return parse(buf, duplicated, DECODE_ALLOC);
}
/**
* Won't release the input ByteBuf.
* - If duplicated is true, the returned StreamRecordBatch has its own copy of the data.
* - If duplicated is false, the returned StreamRecordBatch shares and retains the data buffer with the input.
*/
public static StreamRecordBatch parse(ByteBuf buf, boolean duplicated, ByteBufSeqAlloc alloc) {
int readerIndex = buf.readerIndex();
byte magic = buf.getByte(readerIndex + MAGIC_POS);
if (magic != MAGIC_V0) {
throw new RuntimeException("Invalid magic byte " + magic);
}
int payloadSize = buf.getInt(readerIndex + PAYLOAD_LENGTH_POS);
int encodedSize = PAYLOAD_POS + payloadSize;
if (duplicated) {
ByteBuf encoded = alloc.byteBuffer(encodedSize);
buf.readBytes(encoded, encodedSize);
return new StreamRecordBatch(encoded);
} else {
ByteBuf encoded = buf.retainedSlice(readerIndex, encodedSize);
buf.skipBytes(encodedSize);
return new StreamRecordBatch(encoded);
}
}
}

View File

@ -20,5 +20,13 @@
package com.automq.stream.s3.wal;
public interface AppendResult {
/**
* The offset of the appended record.
*/
RecordOffset recordOffset();
/**
* The offset that is valid for the next record.
*/
RecordOffset nextOffset();
}

View File

@ -19,15 +19,5 @@
package com.automq.stream.s3.wal;
public class DefaultAppendResult implements AppendResult {
private final RecordOffset recordOffset;
public DefaultAppendResult(RecordOffset recordOffset) {
this.recordOffset = recordOffset;
}
@Override
public RecordOffset recordOffset() {
return recordOffset;
}
public record DefaultAppendResult(RecordOffset recordOffset, RecordOffset nextOffset) implements AppendResult {
}

View File

@ -21,7 +21,7 @@ package com.automq.stream.s3.wal;
import io.netty.buffer.ByteBuf;
public interface RecordOffset {
public interface RecordOffset extends Comparable<RecordOffset> {
ByteBuf buffer();

View File

@ -45,6 +45,11 @@ public interface WriteAheadLog {
*/
WALMetadata metadata();
/**
* Get WAL config URI string. We could use the uri to reconstruct the WAL instance.
*/
String uri();
/**
* Append data to log, note append may be out of order.
* ex. when sequence append R1 R2 , R2 maybe complete before R1.

View File

@ -101,4 +101,10 @@ public class DefaultRecordOffset implements RecordOffset {
public int hashCode() {
return Objects.hash(epoch, offset, size);
}
@Override
public int compareTo(RecordOffset o) {
DefaultRecordOffset other = DefaultRecordOffset.of(o);
return Long.compare(this.offset, other.offset);
}
}

View File

@ -19,10 +19,10 @@
package com.automq.stream.s3.wal.impl;
import com.automq.stream.s3.StreamRecordBatchCodec;
import com.automq.stream.s3.model.StreamRecordBatch;
import com.automq.stream.s3.trace.context.TraceContext;
import com.automq.stream.s3.wal.AppendResult;
import com.automq.stream.s3.wal.DefaultAppendResult;
import com.automq.stream.s3.wal.RecordOffset;
import com.automq.stream.s3.wal.RecoverResult;
import com.automq.stream.s3.wal.WriteAheadLog;
@ -69,6 +69,11 @@ public class MemoryWriteAheadLog implements WriteAheadLog {
return new WALMetadata(0, 0);
}
@Override
public String uri() {
return "0@mem://?";
}
@Override
public CompletableFuture<AppendResult> append(TraceContext context, StreamRecordBatch streamRecordBatch) {
if (full) {
@ -82,17 +87,23 @@ public class MemoryWriteAheadLog implements WriteAheadLog {
buffer.writeBytes(streamRecordBatch.encoded());
streamRecordBatch.release();
dataMap.put(offset, buffer);
return CompletableFuture.completedFuture(() -> DefaultRecordOffset.of(0, offset, 0));
return CompletableFuture.completedFuture(new DefaultAppendResult(
DefaultRecordOffset.of(0, offset, 0),
DefaultRecordOffset.of(0, offset + 1, 0)
));
}
@Override
public CompletableFuture<StreamRecordBatch> get(RecordOffset recordOffset) {
return CompletableFuture.completedFuture(StreamRecordBatchCodec.decode(dataMap.get(DefaultRecordOffset.of(recordOffset).offset()), true));
return CompletableFuture.completedFuture(StreamRecordBatch.parse(dataMap.get(DefaultRecordOffset.of(recordOffset).offset()), false));
}
@Override
public CompletableFuture<List<StreamRecordBatch>> get(RecordOffset startOffset, RecordOffset endOffset) {
List<StreamRecordBatch> list = dataMap.subMap(DefaultRecordOffset.of(startOffset).offset(), true, DefaultRecordOffset.of(endOffset).offset(), false).values().stream().map(StreamRecordBatchCodec::decode).collect(Collectors.toList());
List<StreamRecordBatch> list = dataMap
.subMap(DefaultRecordOffset.of(startOffset).offset(), true, DefaultRecordOffset.of(endOffset).offset(), false)
.values().stream()
.map(buf -> StreamRecordBatch.parse(buf, false)).collect(Collectors.toList());
return CompletableFuture.completedFuture(list);
}
@ -108,7 +119,7 @@ public class MemoryWriteAheadLog implements WriteAheadLog {
.map(e -> (RecoverResult) new RecoverResult() {
@Override
public StreamRecordBatch record() {
return StreamRecordBatchCodec.decode(e.getValue());
return StreamRecordBatch.parse(e.getValue(), false);
}
@Override
@ -122,6 +133,7 @@ public class MemoryWriteAheadLog implements WriteAheadLog {
@Override
public CompletableFuture<Void> reset() {
dataMap.forEach((offset, buf) -> buf.release());
dataMap.clear();
return CompletableFuture.completedFuture(null);
}

View File

@ -71,6 +71,7 @@ import static com.automq.stream.s3.ByteBufAlloc.S3_WAL;
import static com.automq.stream.s3.wal.common.RecordHeader.RECORD_HEADER_SIZE;
import static com.automq.stream.s3.wal.impl.object.ObjectUtils.DATA_FILE_ALIGN_SIZE;
import static com.automq.stream.s3.wal.impl.object.ObjectUtils.OBJECT_PATH_OFFSET_DELIMITER;
import static com.automq.stream.s3.wal.impl.object.ObjectUtils.ceilAlignOffset;
public class DefaultWriter implements Writer {
private static final Logger LOGGER = LoggerFactory.getLogger(DefaultWriter.class);
@ -477,7 +478,7 @@ public class DefaultWriter implements Writer {
trimOffset.set(inclusiveTrimRecordOffset);
// We cannot force upload an empty wal object cause of the recover workflow don't accept an empty wal object.
// So we use a fake record to trigger the wal object upload.
persistTrimOffsetCf = append(new StreamRecordBatch(-1L, -1L, 0, 0, Unpooled.EMPTY_BUFFER));
persistTrimOffsetCf = append(StreamRecordBatch.of(-1L, -1L, 0, 0, Unpooled.EMPTY_BUFFER));
lastTrimCf = persistTrimOffsetCf.thenCompose(nil -> {
Long lastFlushedRecordOffset = lastRecordOffset2object.isEmpty() ? null : lastRecordOffset2object.lastKey();
if (lastFlushedRecordOffset != null) {
@ -589,9 +590,18 @@ public class DefaultWriter implements Writer {
if (ex != null) {
records.forEach(record -> record.future.completeExceptionally(ex));
} else {
records.forEach(record -> record.future.complete(
new DefaultAppendResult(DefaultRecordOffset.of(config.epoch(), record.offset, record.size))
));
for (int idx = 0; idx < records.size(); idx++) {
Record record = records.get(idx);
// Requests for data starting at or beyond objectSize will return a 416 error.
// So we jump the last record's nextOffset to the ceil-aligned offset.
long nextOffset = (idx == records.size() - 1) ? ceilAlignOffset(record.offset + record.size) : record.offset + record.size;
record.future.complete(
new DefaultAppendResult(
DefaultRecordOffset.of(config.epoch(), record.offset, record.size),
DefaultRecordOffset.of(config.epoch(), nextOffset, 0)
)
);
}
}
completeCf.complete(null);
}

View File

@ -20,7 +20,6 @@
package com.automq.stream.s3.wal.impl.object;
import com.automq.stream.s3.Constants;
import com.automq.stream.s3.StreamRecordBatchCodec;
import com.automq.stream.s3.model.StreamRecordBatch;
import com.automq.stream.s3.operator.ObjectStorage;
import com.automq.stream.s3.wal.common.RecordHeader;
@ -138,9 +137,6 @@ public class ObjectUtils {
if (header.getMagicCode() != RecordHeader.RECORD_HEADER_DATA_MAGIC_CODE) {
throw new IllegalStateException("Invalid magic code in record header.");
}
int length = header.getRecordBodyLength();
StreamRecordBatch streamRecordBatch = StreamRecordBatchCodec.sliceRetainDecode(dataBuffer.slice(dataBuffer.readerIndex(), length));
dataBuffer.skipBytes(length);
return streamRecordBatch;
return StreamRecordBatch.parse(dataBuffer, false);
}
}

View File

@ -26,6 +26,7 @@ import com.automq.stream.utils.IdURI;
import org.apache.commons.lang3.StringUtils;
public class ObjectWALConfig {
private final String uri;
private final ReservationService reservationService;
private final long batchInterval;
private final long maxBytesInBatch;
@ -43,10 +44,11 @@ public class ObjectWALConfig {
return new Builder();
}
public ObjectWALConfig(ReservationService reservationService, long batchInterval, long maxBytesInBatch,
public ObjectWALConfig(String uri, ReservationService reservationService, long batchInterval, long maxBytesInBatch,
long maxUnflushedBytes, int maxInflightUploadCount,
int readAheadObjectCount, String clusterId, int nodeId, long epoch, OpenMode openMode, short bucketId,
String type) {
this.uri = uri;
this.reservationService = reservationService;
this.batchInterval = batchInterval;
this.maxBytesInBatch = maxBytesInBatch;
@ -61,6 +63,10 @@ public class ObjectWALConfig {
this.type = type;
}
public String uri() {
return uri;
}
public ReservationService reservationService() {
return this.reservationService;
}
@ -127,6 +133,7 @@ public class ObjectWALConfig {
}
public static final class Builder {
private String uri = "";
private ReservationService reservationService = ReservationService.NOOP;
private long batchInterval = 250; // 250ms
private long maxBytesInBatch = 8 * 1024 * 1024L; // 8MB
@ -144,6 +151,7 @@ public class ObjectWALConfig {
}
public Builder withURI(IdURI uri) {
this.uri = uri.toString();
withBucketId(uri.id());
String batchInterval = uri.extensionString("batchInterval");
@ -169,6 +177,10 @@ public class ObjectWALConfig {
return this;
}
public Builder withURI(String uri) {
return withURI(IdURI.parse(uri));
}
public Builder withReservationService(ReservationService reservationService) {
this.reservationService = reservationService;
return this;
@ -238,7 +250,7 @@ public class ObjectWALConfig {
}
public ObjectWALConfig build() {
return new ObjectWALConfig(reservationService, batchInterval, maxBytesInBatch, maxUnflushedBytes, maxInflightUploadCount, readAheadObjectCount, clusterId, nodeId, epoch, openMode, bucketId, type);
return new ObjectWALConfig(uri, reservationService, batchInterval, maxBytesInBatch, maxUnflushedBytes, maxInflightUploadCount, readAheadObjectCount, clusterId, nodeId, epoch, openMode, bucketId, type);
}
}
}

View File

@ -77,6 +77,11 @@ public class ObjectWALService implements WriteAheadLog {
return new WALMetadata(config.nodeId(), config.epoch());
}
@Override
public String uri() {
return config.uri();
}
@Override
public CompletableFuture<AppendResult> append(TraceContext context,
StreamRecordBatch streamRecordBatch) throws OverCapacityException {

View File

@ -20,7 +20,7 @@
package com.automq.stream.s3.wal.impl.object;
import com.automq.stream.s3.ByteBufAlloc;
import com.automq.stream.s3.StreamRecordBatchCodec;
import com.automq.stream.s3.model.StreamRecordBatch;
import com.automq.stream.s3.network.ThrottleStrategy;
import com.automq.stream.s3.operator.ObjectStorage;
import com.automq.stream.s3.wal.RecoverResult;
@ -194,13 +194,12 @@ public class RecoverIterator implements Iterator<RecoverResult> {
@Override
public RecoverResult next() {
if (nextRecord != null) {
if (nextRecord != null || hasNext()) {
// - If the nextRecord is already read ahead.
// - Or #hasNext() is true, it means the nextRecord is already ready.
RecoverResult rst = nextRecord;
nextRecord = null;
return rst;
}
if (hasNext()) {
return nextRecord;
} else {
return null;
}
@ -260,7 +259,11 @@ public class RecoverIterator implements Iterator<RecoverResult> {
long offset = header.getRecordBodyOffset() - RECORD_HEADER_SIZE;
int size = recordBuf.readableBytes() + RECORD_HEADER_SIZE;
return new RecoverResultImpl(StreamRecordBatchCodec.decode(recordBuf), DefaultRecordOffset.of(getEpoch(offset), offset, size));
try {
return new RecoverResultImpl(StreamRecordBatch.parse(recordBuf, false), DefaultRecordOffset.of(getEpoch(offset), offset, size));
} finally {
recordBuf.release();
}
}
private long getEpoch(long offset) {

View File

@ -29,11 +29,15 @@ import java.util.concurrent.Executor;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
public class EventLoop extends Thread implements Executor {
import io.netty.util.AbstractReferenceCounted;
import io.netty.util.ReferenceCounted;
public class EventLoop extends Thread implements Executor, ReferenceCounted {
private final Logger logger;
private BlockingQueue<Runnable> tasks;
private final BlockingQueue<Runnable> tasks;
private volatile boolean shutdown;
private CompletableFuture<Void> shutdownCf = new CompletableFuture<>();
private final CompletableFuture<Void> shutdownCf = new CompletableFuture<>();
private final InnerRefCounted innerRefCounted = new InnerRefCounted();
static final Runnable WAKEUP_TASK = new Runnable() {
@Override
@ -111,4 +115,51 @@ public class EventLoop extends Thread implements Executor {
}
}
@Override
public int refCnt() {
return innerRefCounted.refCnt();
}
@Override
public ReferenceCounted retain() {
return innerRefCounted.retain();
}
@Override
public ReferenceCounted retain(int increment) {
return innerRefCounted.retain(increment);
}
@Override
public ReferenceCounted touch() {
return innerRefCounted.touch();
}
@Override
public ReferenceCounted touch(Object hint) {
return innerRefCounted.touch(hint);
}
@Override
public boolean release() {
return innerRefCounted.release();
}
@Override
public boolean release(int decrement) {
return innerRefCounted.release(decrement);
}
class InnerRefCounted extends AbstractReferenceCounted {
@Override
protected void deallocate() {
shutdownGracefully();
}
@Override
public ReferenceCounted touch(Object hint) {
return EventLoop.this;
}
}
}

View File

@ -130,7 +130,7 @@ public class CompositeObjectTest {
}
StreamRecordBatch newRecord(long streamId, long offset, int count, ByteBuf buf) {
return new StreamRecordBatch(streamId, 0, offset, count, buf);
return StreamRecordBatch.of(streamId, 0, offset, count, buf);
}
ByteBuf genBuf(byte data, int length) {

View File

@ -76,13 +76,13 @@ public class DefaultUploadWriteAheadLogTaskTest {
Map<Long, List<StreamRecordBatch>> map = new HashMap<>();
map.put(233L, List.of(
new StreamRecordBatch(233, 0, 10, 2, random(512)),
new StreamRecordBatch(233, 0, 12, 2, random(128)),
new StreamRecordBatch(233, 0, 14, 2, random(512))
StreamRecordBatch.of(233, 0, 10, 2, random(512)),
StreamRecordBatch.of(233, 0, 12, 2, random(128)),
StreamRecordBatch.of(233, 0, 14, 2, random(512))
));
map.put(234L, List.of(
new StreamRecordBatch(234, 0, 20, 2, random(128)),
new StreamRecordBatch(234, 0, 22, 2, random(128))
StreamRecordBatch.of(234, 0, 20, 2, random(128)),
StreamRecordBatch.of(234, 0, 22, 2, random(128))
));
Config config = new Config()
@ -161,9 +161,9 @@ public class DefaultUploadWriteAheadLogTaskTest {
Map<Long, List<StreamRecordBatch>> map = new HashMap<>();
map.put(233L, List.of(
new StreamRecordBatch(233, 0, 10, 2, random(512)),
new StreamRecordBatch(233, 0, 12, 2, random(128)),
new StreamRecordBatch(233, 0, 14, 2, random(512))
StreamRecordBatch.of(233, 0, 10, 2, random(512)),
StreamRecordBatch.of(233, 0, 12, 2, random(128)),
StreamRecordBatch.of(233, 0, 14, 2, random(512))
));
Config config = new Config()
.objectBlockSize(16 * 1024 * 1024)
@ -195,10 +195,10 @@ public class DefaultUploadWriteAheadLogTaskTest {
Map<Long, List<StreamRecordBatch>> map = new HashMap<>();
map.put(233L, List.of(
new StreamRecordBatch(233, 0, 10, 2, random(512))
StreamRecordBatch.of(233, 0, 10, 2, random(512))
));
map.put(234L, List.of(
new StreamRecordBatch(234, 0, 20, 2, random(128))
StreamRecordBatch.of(234, 0, 20, 2, random(128))
));
Config config = new Config()

View File

@ -106,7 +106,7 @@ public class ObjectReaderTest {
// make index block bigger than 1M
int streamCount = 2 * 1024 * 1024 / 40;
for (int i = 0; i < streamCount; i++) {
StreamRecordBatch r = new StreamRecordBatch(i, 0, i, 1, TestUtils.random(1));
StreamRecordBatch r = StreamRecordBatch.of(i, 0, i, 1, TestUtils.random(1));
objectWriter.write(i, List.of(r));
}
objectWriter.close().get();
@ -122,11 +122,11 @@ public class ObjectReaderTest {
ObjectStorage objectStorage = new MemoryObjectStorage();
ByteBuf buf = ByteBufAlloc.byteBuffer(0);
buf.writeBytes(new ObjectWriter.DataBlock(233L, List.of(
new StreamRecordBatch(233L, 0, 10, 1, TestUtils.random(100)),
new StreamRecordBatch(233L, 0, 11, 2, TestUtils.random(100))
StreamRecordBatch.of(233L, 0, 10, 1, TestUtils.random(100)),
StreamRecordBatch.of(233L, 0, 11, 2, TestUtils.random(100))
)).buffer());
buf.writeBytes(new ObjectWriter.DataBlock(233L, List.of(
new StreamRecordBatch(233L, 0, 13, 1, TestUtils.random(100))
StreamRecordBatch.of(233L, 0, 13, 1, TestUtils.random(100))
)).buffer());
int indexPosition = buf.readableBytes();
new DataBlockIndex(233L, 10, 4, 3, 0, buf.readableBytes()).encode(buf);
@ -193,6 +193,6 @@ public class ObjectReaderTest {
}
StreamRecordBatch newRecord(long streamId, long offset, int count, int payloadSize) {
return new StreamRecordBatch(streamId, 0, offset, count, TestUtils.random(payloadSize));
return StreamRecordBatch.of(streamId, 0, offset, count, TestUtils.random(payloadSize));
}
}

View File

@ -137,6 +137,6 @@ public class ObjectWriterTest {
}
StreamRecordBatch newRecord(long streamId, long offset, int count, int payloadSize) {
return new StreamRecordBatch(streamId, 0, offset, count, TestUtils.random(payloadSize));
return StreamRecordBatch.of(streamId, 0, offset, count, TestUtils.random(payloadSize));
}
}

View File

@ -86,7 +86,7 @@ public class S3StorageTest {
Config config;
private static StreamRecordBatch newRecord(long streamId, long offset) {
return new StreamRecordBatch(streamId, 0, offset, 1, random(1));
return StreamRecordBatch.of(streamId, 0, offset, 1, random(1));
}
@BeforeEach
@ -109,13 +109,13 @@ public class S3StorageTest {
Mockito.when(objectManager.commitStreamSetObject(any())).thenReturn(CompletableFuture.completedFuture(resp));
CompletableFuture<Void> cf1 = storage.append(
new StreamRecordBatch(233, 1, 10, 1, random(100))
StreamRecordBatch.of(233, 1, 10, 1, random(100))
);
CompletableFuture<Void> cf2 = storage.append(
new StreamRecordBatch(233, 1, 11, 2, random(100))
StreamRecordBatch.of(233, 1, 11, 2, random(100))
);
CompletableFuture<Void> cf3 = storage.append(
new StreamRecordBatch(234, 3, 100, 1, random(100))
StreamRecordBatch.of(234, 3, 100, 1, random(100))
);
cf1.get(3, TimeUnit.SECONDS);

View File

@ -80,7 +80,7 @@ public class S3StreamTest {
}
ReadDataBlock newReadDataBlock(long start, long end, int size) {
StreamRecordBatch record = new StreamRecordBatch(0, 0, start, (int) (end - start), TestUtils.random(size));
StreamRecordBatch record = StreamRecordBatch.of(0, 0, start, (int) (end - start), TestUtils.random(size));
return new ReadDataBlock(List.of(record), CacheAccessType.DELTA_WAL_CACHE_HIT);
}
}

View File

@ -41,15 +41,15 @@ public class LogCacheTest {
public void testPutGet() {
LogCache logCache = new LogCache(1024 * 1024, 1024 * 1024);
logCache.put(new StreamRecordBatch(233L, 0L, 10L, 1, TestUtils.random(20)));
logCache.put(new StreamRecordBatch(233L, 0L, 11L, 2, TestUtils.random(20)));
logCache.put(StreamRecordBatch.of(233L, 0L, 10L, 1, TestUtils.random(20)));
logCache.put(StreamRecordBatch.of(233L, 0L, 11L, 2, TestUtils.random(20)));
logCache.archiveCurrentBlock();
logCache.put(new StreamRecordBatch(233L, 0L, 13L, 2, TestUtils.random(20)));
logCache.put(StreamRecordBatch.of(233L, 0L, 13L, 2, TestUtils.random(20)));
logCache.archiveCurrentBlock();
logCache.put(new StreamRecordBatch(233L, 0L, 20L, 1, TestUtils.random(20)));
logCache.put(new StreamRecordBatch(233L, 0L, 21L, 1, TestUtils.random(20)));
logCache.put(StreamRecordBatch.of(233L, 0L, 20L, 1, TestUtils.random(20)));
logCache.put(StreamRecordBatch.of(233L, 0L, 21L, 1, TestUtils.random(20)));
List<StreamRecordBatch> records = logCache.get(233L, 10L, 21L, 1000);
assertEquals(1, records.size());
@ -74,7 +74,7 @@ public class LogCacheTest {
LogCache cache = new LogCache(Integer.MAX_VALUE, Integer.MAX_VALUE);
for (int i = 0; i < 100000; i++) {
cache.put(new StreamRecordBatch(233L, 0L, i, 1, TestUtils.random(1)));
cache.put(StreamRecordBatch.of(233L, 0L, i, 1, TestUtils.random(1)));
}
long start = System.nanoTime();
@ -91,13 +91,13 @@ public class LogCacheTest {
public void testClearStreamRecords() {
LogCache logCache = new LogCache(1024 * 1024, 1024 * 1024);
logCache.put(new StreamRecordBatch(233L, 0L, 10L, 1, TestUtils.random(20)));
logCache.put(new StreamRecordBatch(233L, 0L, 11L, 2, TestUtils.random(20)));
logCache.put(StreamRecordBatch.of(233L, 0L, 10L, 1, TestUtils.random(20)));
logCache.put(StreamRecordBatch.of(233L, 0L, 11L, 2, TestUtils.random(20)));
logCache.archiveCurrentBlock();
logCache.put(new StreamRecordBatch(233L, 0L, 13L, 2, TestUtils.random(20)));
logCache.put(StreamRecordBatch.of(233L, 0L, 13L, 2, TestUtils.random(20)));
logCache.put(new StreamRecordBatch(234L, 0L, 13L, 2, TestUtils.random(20)));
logCache.put(StreamRecordBatch.of(234L, 0L, 13L, 2, TestUtils.random(20)));
assertTrue(logCache.blocks.get(0).containsStream(233L));
assertTrue(logCache.blocks.get(1).containsStream(234L));
@ -113,19 +113,19 @@ public class LogCacheTest {
@Test
public void testIsDiscontinuous() {
LogCacheBlock left = new LogCacheBlock(1024L * 1024);
left.put(new StreamRecordBatch(233L, 0L, 10L, 1, TestUtils.random(20)));
left.put(StreamRecordBatch.of(233L, 0L, 10L, 1, TestUtils.random(20)));
LogCacheBlock right = new LogCacheBlock(1024L * 1024);
right.put(new StreamRecordBatch(233L, 0L, 13L, 1, TestUtils.random(20)));
right.put(StreamRecordBatch.of(233L, 0L, 13L, 1, TestUtils.random(20)));
assertTrue(LogCache.isDiscontinuous(left, right));
left = new LogCacheBlock(1024L * 1024);
left.put(new StreamRecordBatch(233L, 0L, 10L, 1, TestUtils.random(20)));
left.put(new StreamRecordBatch(234L, 0L, 10L, 1, TestUtils.random(20)));
left.put(StreamRecordBatch.of(233L, 0L, 10L, 1, TestUtils.random(20)));
left.put(StreamRecordBatch.of(234L, 0L, 10L, 1, TestUtils.random(20)));
right = new LogCacheBlock(1024L * 1024);
right.put(new StreamRecordBatch(233L, 0L, 11L, 1, TestUtils.random(20)));
right.put(StreamRecordBatch.of(233L, 0L, 11L, 1, TestUtils.random(20)));
assertFalse(LogCache.isDiscontinuous(left, right));
}
@ -133,13 +133,13 @@ public class LogCacheTest {
public void testMergeBlock() {
long size = 0;
LogCacheBlock left = new LogCacheBlock(1024L * 1024);
left.put(new StreamRecordBatch(233L, 0L, 10L, 1, TestUtils.random(20)));
left.put(new StreamRecordBatch(234L, 0L, 100L, 1, TestUtils.random(20)));
left.put(StreamRecordBatch.of(233L, 0L, 10L, 1, TestUtils.random(20)));
left.put(StreamRecordBatch.of(234L, 0L, 100L, 1, TestUtils.random(20)));
size += left.size();
LogCacheBlock right = new LogCacheBlock(1024L * 1024);
right.put(new StreamRecordBatch(233L, 0L, 11L, 1, TestUtils.random(20)));
right.put(new StreamRecordBatch(235L, 0L, 200L, 1, TestUtils.random(20)));
right.put(StreamRecordBatch.of(233L, 0L, 11L, 1, TestUtils.random(20)));
right.put(StreamRecordBatch.of(235L, 0L, 200L, 1, TestUtils.random(20)));
size += right.size();
LogCache.mergeBlock(left, right);
@ -172,7 +172,7 @@ public class LogCacheTest {
// create multiple blocks, each containing one record for the same stream with contiguous offsets
for (int i = 0; i < blocksToCreate; i++) {
logCache.put(new StreamRecordBatch(streamId, 0L, i, 1, TestUtils.random(1)));
logCache.put(StreamRecordBatch.of(streamId, 0L, i, 1, TestUtils.random(1)));
logCache.archiveCurrentBlock();
}

View File

@ -46,7 +46,7 @@ public class ObjectReaderLRUCacheTest {
private void writeStream(int streamCount, ObjectWriter objectWriter) {
for (int i = 0; i < streamCount; i++) {
StreamRecordBatch r = new StreamRecordBatch(i, 0, i, 1, TestUtils.random(1));
StreamRecordBatch r = StreamRecordBatch.of(i, 0, i, 1, TestUtils.random(1));
objectWriter.write(i, List.of(r));
}
}

View File

@ -225,10 +225,10 @@ import static org.mockito.Mockito.when;
long offset = index.startOffset();
// the first N - 1 record's count = 1, body size = 1
for (int i = 0; i < index.recordCount() - 1; i++, offset++) {
records.add(new StreamRecordBatch(STREAM_ID, 0, offset, 1, TestUtils.random(1)));
records.add(StreamRecordBatch.of(STREAM_ID, 0, offset, 1, TestUtils.random(1)));
}
// the last record padding the remaining
records.add(new StreamRecordBatch(STREAM_ID, 0, offset, index.endOffsetDelta() - (index.recordCount() - 1), TestUtils.random(remainingSize)));
records.add(StreamRecordBatch.of(STREAM_ID, 0, offset, index.endOffsetDelta() - (index.recordCount() - 1), TestUtils.random(remainingSize)));
ByteBuf buf = new ObjectWriter.DataBlock(STREAM_ID, records).buffer();
assertEquals(index.size(), buf.readableBytes());
return buf;

View File

@ -90,21 +90,21 @@ public class StreamReaderTest {
// object=6 [24, 29)
// object=7 [29, 34)
objects.put(0L, MockObject.builder(0, BLOCK_SIZE_THRESHOLD).mockDelay(100).write(STREAM_ID, List.of(
new StreamRecordBatch(STREAM_ID, 0, 0, 1, TestUtils.random(1))
StreamRecordBatch.of(STREAM_ID, 0, 0, 1, TestUtils.random(1))
)).build());
objects.put(1L, MockObject.builder(1L, 1).mockDelay(100).write(STREAM_ID, List.of(
new StreamRecordBatch(STREAM_ID, 0, 1, 1, TestUtils.random(19)),
new StreamRecordBatch(STREAM_ID, 0, 2, 1, TestUtils.random(10)),
new StreamRecordBatch(STREAM_ID, 0, 3, 1, TestUtils.random(10))
StreamRecordBatch.of(STREAM_ID, 0, 1, 1, TestUtils.random(19)),
StreamRecordBatch.of(STREAM_ID, 0, 2, 1, TestUtils.random(10)),
StreamRecordBatch.of(STREAM_ID, 0, 3, 1, TestUtils.random(10))
)).build());
for (int i = 0; i < 6; i++) {
long offset = 4 + i * 5;
objects.put(i + 2L, MockObject.builder(i + 2L, BLOCK_SIZE_THRESHOLD).mockDelay(100).write(STREAM_ID, List.of(
new StreamRecordBatch(STREAM_ID, 0, offset, 1, TestUtils.random(1024 * 1024 / 4)),
new StreamRecordBatch(STREAM_ID, 0, offset + 1, 1, TestUtils.random(1024 * 1024 / 4)),
new StreamRecordBatch(STREAM_ID, 0, offset + 2, 1, TestUtils.random(1024 * 1024 / 4)),
new StreamRecordBatch(STREAM_ID, 0, offset + 3, 1, TestUtils.random(1024 * 1024 / 4)),
new StreamRecordBatch(STREAM_ID, 0, offset + 4, 1, TestUtils.random(1024 * 1024 / 4))
StreamRecordBatch.of(STREAM_ID, 0, offset, 1, TestUtils.random(1024 * 1024 / 4)),
StreamRecordBatch.of(STREAM_ID, 0, offset + 1, 1, TestUtils.random(1024 * 1024 / 4)),
StreamRecordBatch.of(STREAM_ID, 0, offset + 2, 1, TestUtils.random(1024 * 1024 / 4)),
StreamRecordBatch.of(STREAM_ID, 0, offset + 3, 1, TestUtils.random(1024 * 1024 / 4)),
StreamRecordBatch.of(STREAM_ID, 0, offset + 4, 1, TestUtils.random(1024 * 1024 / 4))
)).build());
}

View File

@ -61,11 +61,11 @@ public class StreamReadersTest {
// Create mock objects for testing with different offset ranges
// Object 1: STREAM_ID_1 offset 0-2
objects.put(1L, MockObject.builder(1L, BLOCK_SIZE_THRESHOLD).write(STREAM_ID_1, List.of(
new StreamRecordBatch(STREAM_ID_1, 0, 0, 2, TestUtils.random(100))
StreamRecordBatch.of(STREAM_ID_1, 0, 0, 2, TestUtils.random(100))
)).build());
// Object 2: STREAM_ID_2 offset 0-1
objects.put(2L, MockObject.builder(2L, BLOCK_SIZE_THRESHOLD).write(STREAM_ID_2, List.of(
new StreamRecordBatch(STREAM_ID_2, 0, 0, 1, TestUtils.random(100))
StreamRecordBatch.of(STREAM_ID_2, 0, 0, 1, TestUtils.random(100))
)).build());
objectManager = mock(ObjectManager.class);

View File

@ -350,9 +350,9 @@ public class CompactionManagerTest extends CompactionTestBase {
objectManager.prepareObject(1, TimeUnit.MINUTES.toMillis(30)).thenAccept(objectId -> {
assertEquals(OBJECT_3, objectId);
ObjectWriter objectWriter = ObjectWriter.writer(OBJECT_3, objectStorage, 1024, 1024);
StreamRecordBatch r1 = new StreamRecordBatch(STREAM_1, 0, 500, 20, TestUtils.random(20));
StreamRecordBatch r2 = new StreamRecordBatch(STREAM_3, 0, 0, 10, TestUtils.random(1024));
StreamRecordBatch r3 = new StreamRecordBatch(STREAM_3, 0, 10, 10, TestUtils.random(1024));
StreamRecordBatch r1 = StreamRecordBatch.of(STREAM_1, 0, 500, 20, TestUtils.random(20));
StreamRecordBatch r2 = StreamRecordBatch.of(STREAM_3, 0, 0, 10, TestUtils.random(1024));
StreamRecordBatch r3 = StreamRecordBatch.of(STREAM_3, 0, 10, 10, TestUtils.random(1024));
objectWriter.write(STREAM_1, List.of(r1));
objectWriter.write(STREAM_3, List.of(r2, r3));
objectWriter.close().join();
@ -430,9 +430,9 @@ public class CompactionManagerTest extends CompactionTestBase {
objectManager.prepareObject(1, TimeUnit.MINUTES.toMillis(30)).thenAccept(objectId -> {
assertEquals(OBJECT_3, objectId);
ObjectWriter objectWriter = ObjectWriter.writer(OBJECT_3, objectStorage, 1024, 1024);
StreamRecordBatch r1 = new StreamRecordBatch(STREAM_1, 0, 500, 20, TestUtils.random(20));
StreamRecordBatch r2 = new StreamRecordBatch(STREAM_3, 0, 0, 10, TestUtils.random(1024));
StreamRecordBatch r3 = new StreamRecordBatch(STREAM_3, 0, 10, 10, TestUtils.random(1024));
StreamRecordBatch r1 = StreamRecordBatch.of(STREAM_1, 0, 500, 20, TestUtils.random(20));
StreamRecordBatch r2 = StreamRecordBatch.of(STREAM_3, 0, 0, 10, TestUtils.random(1024));
StreamRecordBatch r3 = StreamRecordBatch.of(STREAM_3, 0, 10, 10, TestUtils.random(1024));
objectWriter.write(STREAM_1, List.of(r1));
objectWriter.write(STREAM_3, List.of(r2, r3));
objectWriter.close().join();
@ -462,9 +462,9 @@ public class CompactionManagerTest extends CompactionTestBase {
objectManager.prepareObject(1, TimeUnit.MINUTES.toMillis(30)).thenAccept(objectId -> {
assertEquals(OBJECT_3, objectId);
ObjectWriter objectWriter = ObjectWriter.writer(OBJECT_3, objectStorage, 200, 1024);
StreamRecordBatch r1 = new StreamRecordBatch(STREAM_1, 0, 500, 20, TestUtils.random(20));
StreamRecordBatch r2 = new StreamRecordBatch(STREAM_3, 0, 0, 10, TestUtils.random(200));
StreamRecordBatch r3 = new StreamRecordBatch(STREAM_3, 0, 10, 10, TestUtils.random(200));
StreamRecordBatch r1 = StreamRecordBatch.of(STREAM_1, 0, 500, 20, TestUtils.random(20));
StreamRecordBatch r2 = StreamRecordBatch.of(STREAM_3, 0, 0, 10, TestUtils.random(200));
StreamRecordBatch r3 = StreamRecordBatch.of(STREAM_3, 0, 10, 10, TestUtils.random(200));
objectWriter.write(STREAM_1, List.of(r1));
objectWriter.write(STREAM_3, List.of(r2, r3));
objectWriter.close().join();
@ -741,7 +741,7 @@ public class CompactionManagerTest extends CompactionTestBase {
objectManager.prepareObject(1, TimeUnit.MINUTES.toMillis(30)).thenAccept(objectId -> {
assertEquals(OBJECT_0, objectId);
ObjectWriter objectWriter = ObjectWriter.writer(objectId, objectStorage, 1024, 1024);
StreamRecordBatch r1 = new StreamRecordBatch(STREAM_0, 0, 0, 80, TestUtils.random(80));
StreamRecordBatch r1 = StreamRecordBatch.of(STREAM_0, 0, 0, 80, TestUtils.random(80));
objectWriter.write(STREAM_0, List.of(r1));
objectWriter.close().join();
List<StreamOffsetRange> streamsIndices = List.of(
@ -757,7 +757,7 @@ public class CompactionManagerTest extends CompactionTestBase {
objectManager.prepareObject(1, TimeUnit.MINUTES.toMillis(30)).thenAccept(objectId -> {
assertEquals(OBJECT_1, objectId);
ObjectWriter objectWriter = ObjectWriter.writer(OBJECT_1, objectStorage, 1024, 1024);
StreamRecordBatch r2 = new StreamRecordBatch(STREAM_0, 0, 80, 120, TestUtils.random(120));
StreamRecordBatch r2 = StreamRecordBatch.of(STREAM_0, 0, 80, 120, TestUtils.random(120));
objectWriter.write(STREAM_0, List.of(r2));
objectWriter.close().join();
List<StreamOffsetRange> streamsIndices = List.of(

View File

@ -87,10 +87,10 @@ public class CompactionTestBase {
objectManager.prepareObject(1, TimeUnit.MINUTES.toMillis(30)).thenAccept(objectId -> {
assertEquals(OBJECT_0, objectId);
ObjectWriter objectWriter = ObjectWriter.writer(objectId, objectStorage, 1024, 1024);
StreamRecordBatch r1 = new StreamRecordBatch(STREAM_0, 0, 0, 15, TestUtils.random(2));
StreamRecordBatch r2 = new StreamRecordBatch(STREAM_1, 0, 25, 5, TestUtils.random(2));
StreamRecordBatch r3 = new StreamRecordBatch(STREAM_1, 0, 30, 30, TestUtils.random(22));
StreamRecordBatch r4 = new StreamRecordBatch(STREAM_2, 0, 30, 30, TestUtils.random(22));
StreamRecordBatch r1 = StreamRecordBatch.of(STREAM_0, 0, 0, 15, TestUtils.random(2));
StreamRecordBatch r2 = StreamRecordBatch.of(STREAM_1, 0, 25, 5, TestUtils.random(2));
StreamRecordBatch r3 = StreamRecordBatch.of(STREAM_1, 0, 30, 30, TestUtils.random(22));
StreamRecordBatch r4 = StreamRecordBatch.of(STREAM_2, 0, 30, 30, TestUtils.random(22));
objectWriter.write(STREAM_0, List.of(r1));
objectWriter.write(STREAM_1, List.of(r2));
objectWriter.write(STREAM_1, List.of(r3));
@ -112,8 +112,8 @@ public class CompactionTestBase {
objectManager.prepareObject(1, TimeUnit.MINUTES.toMillis(30)).thenAccept(objectId -> {
assertEquals(OBJECT_1, objectId);
ObjectWriter objectWriter = ObjectWriter.writer(OBJECT_1, objectStorage, 1024, 1024);
StreamRecordBatch r5 = new StreamRecordBatch(STREAM_0, 0, 15, 5, TestUtils.random(1));
StreamRecordBatch r6 = new StreamRecordBatch(STREAM_1, 0, 60, 60, TestUtils.random(52));
StreamRecordBatch r5 = StreamRecordBatch.of(STREAM_0, 0, 15, 5, TestUtils.random(1));
StreamRecordBatch r6 = StreamRecordBatch.of(STREAM_1, 0, 60, 60, TestUtils.random(52));
objectWriter.write(STREAM_0, List.of(r5));
objectWriter.write(STREAM_1, List.of(r6));
objectWriter.close().join();
@ -131,8 +131,8 @@ public class CompactionTestBase {
objectManager.prepareObject(1, TimeUnit.MINUTES.toMillis(30)).thenAccept(objectId -> {
assertEquals(OBJECT_2, objectId);
ObjectWriter objectWriter = ObjectWriter.writer(OBJECT_2, objectStorage, 1024, 1024);
StreamRecordBatch r8 = new StreamRecordBatch(STREAM_1, 0, 400, 100, TestUtils.random(92));
StreamRecordBatch r9 = new StreamRecordBatch(STREAM_2, 0, 230, 40, TestUtils.random(32));
StreamRecordBatch r8 = StreamRecordBatch.of(STREAM_1, 0, 400, 100, TestUtils.random(92));
StreamRecordBatch r9 = StreamRecordBatch.of(STREAM_2, 0, 230, 40, TestUtils.random(32));
objectWriter.write(STREAM_1, List.of(r8));
objectWriter.write(STREAM_2, List.of(r9));
objectWriter.close().join();

View File

@ -568,6 +568,6 @@ class StreamObjectCompactorTest {
}
StreamRecordBatch newRecord(long offset, int count, int payloadSize) {
return new StreamRecordBatch(streamId, 0, offset, count, TestUtils.random(payloadSize));
return StreamRecordBatch.of(streamId, 0, offset, count, TestUtils.random(payloadSize));
}
}

View File

@ -0,0 +1,81 @@
/*
* Copyright 2025, AutoMQ HK Limited.
*
* 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 com.automq.stream.s3.model;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.ValueSource;
import java.nio.charset.StandardCharsets;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.CompositeByteBuf;
import io.netty.buffer.Unpooled;
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
import static org.junit.jupiter.api.Assertions.assertEquals;
public class StreamRecordBatchTest {
@Test
public void testOf() {
byte[] payload = "hello".getBytes(StandardCharsets.UTF_8);
ByteBuf payloadBuf = Unpooled.wrappedBuffer(payload);
StreamRecordBatch record = StreamRecordBatch.of(1L, 2L, 3L, 4, payloadBuf);
assertEquals(1, record.getStreamId());
assertEquals(2, record.getEpoch());
assertEquals(3, record.getBaseOffset());
assertEquals(4, record.getCount());
assertEquals(payload.length, record.size());
assertEquals(0, payloadBuf.refCnt());
byte[] realPayload = new byte[payload.length];
record.getPayload().readBytes(realPayload);
assertArrayEquals(payload, realPayload);
record.release();
assertEquals(0, record.encoded.refCnt());
}
@ParameterizedTest
@ValueSource(booleans = {true, false})
public void testParse(boolean duplicated) {
CompositeByteBuf buf = Unpooled.compositeBuffer();
for (int i = 0; i < 10; i++) {
ByteBuf payloadBuf = Unpooled.wrappedBuffer(("hello" + i).getBytes(StandardCharsets.UTF_8));
StreamRecordBatch record = StreamRecordBatch.of(1L, 2L, 3L + i, 4, payloadBuf);
buf.addComponent(true, record.encoded());
}
for (int i = 0; i < 10; i++) {
StreamRecordBatch record = StreamRecordBatch.parse(buf, duplicated);
assertEquals(3 + i, record.getBaseOffset());
ByteBuf payloadBuf = record.getPayload();
byte[] payload = new byte[payloadBuf.readableBytes()];
payloadBuf.readBytes(payload);
assertArrayEquals(("hello" + i).getBytes(StandardCharsets.UTF_8), payload);
record.release();
if (duplicated) {
assertEquals(0, record.encoded.refCnt());
}
}
assertEquals(0, buf.readableBytes());
assertEquals(1, buf.refCnt());
buf.release();
}
}

View File

@ -69,7 +69,7 @@ public class ObjectWALServiceTest {
// append new record and verify
for (int i = 0; i < 10; i++) {
appendCfList.add(wal.append(TraceContext.DEFAULT, new StreamRecordBatch(233L, 10, r * 10 + i, 1, generateByteBuf(256))));
appendCfList.add(wal.append(TraceContext.DEFAULT, StreamRecordBatch.of(233L, 10, r * 10 + i, 1, generateByteBuf(256))));
}
List<CompletableFuture<StreamRecordBatch>> getCfList = new ArrayList<>();
for (int i = 0; i < appendCfList.size(); i++) {
@ -99,7 +99,7 @@ public class ObjectWALServiceTest {
wal.start();
// append new record and verify
for (int i = 0; i < 10; i++) {
appendCfList.add(wal.append(TraceContext.DEFAULT, new StreamRecordBatch(233L, 10, r * 10 + i, 1, generateByteBuf(256))));
appendCfList.add(wal.append(TraceContext.DEFAULT, StreamRecordBatch.of(233L, 10, r * 10 + i, 1, generateByteBuf(256))));
}
((DefaultWriter) (wal.writer)).flush().join();
for (int i = 0; i < appendCfList.size() - 3; i++) {
@ -133,7 +133,7 @@ public class ObjectWALServiceTest {
List<CompletableFuture<AppendResult>> appendCfList = new ArrayList<>();
for (int i = 0; i < 8; i++) {
appendCfList.add(wal.append(TraceContext.DEFAULT,
new StreamRecordBatch(233L, 10, 100L + i, 1, generateByteBuf(256))));
StreamRecordBatch.of(233L, 10, 100L + i, 1, generateByteBuf(256))));
// ensure objects are flushed/uploaded
((DefaultWriter) (wal.writer)).flush().join();
if (i == 4) {
@ -166,7 +166,7 @@ public class ObjectWALServiceTest {
List<CompletableFuture<AppendResult>> appendCfList = new ArrayList<>();
for (int i = 0; i < 8; i++) {
appendCfList.add(wal.append(TraceContext.DEFAULT, new StreamRecordBatch(233L, 0, 100L + i, 1, generateByteBuf(1))));
appendCfList.add(wal.append(TraceContext.DEFAULT, StreamRecordBatch.of(233L, 0, 100L + i, 1, generateByteBuf(1))));
if (i % 2 == 0) {
((DefaultWriter) (wal.writer)).flush().join();
}
@ -209,7 +209,7 @@ public class ObjectWALServiceTest {
break;
}
for (int i = 0; i < 10; i++) {
appendCfList.add(wal.append(TraceContext.DEFAULT, new StreamRecordBatch(233L, 10, r * 10 + i, 1, generateByteBuf(256))));
appendCfList.add(wal.append(TraceContext.DEFAULT, StreamRecordBatch.of(233L, 10, r * 10 + i, 1, generateByteBuf(256))));
}
((DefaultWriter) (wal.writer)).flush().join();
trimIndex = r * 9;
@ -240,7 +240,7 @@ public class ObjectWALServiceTest {
resetIndex = appendCfList.size();
wal.reset().get();
for (int i = 0; i < 10; i++) {
appendCfList.add(wal.append(TraceContext.DEFAULT, new StreamRecordBatch(233L, 10, r * 10 + i, 1, generateByteBuf(256))));
appendCfList.add(wal.append(TraceContext.DEFAULT, StreamRecordBatch.of(233L, 10, r * 10 + i, 1, generateByteBuf(256))));
}
wal.shutdownGracefully();
}
@ -324,7 +324,7 @@ public class ObjectWALServiceTest {
// write 4 objects
for (int i = 0; i < 4; i++) {
wal.append(TraceContext.DEFAULT, new StreamRecordBatch(233L, 0, 100L + i, 1, generateByteBuf(1)));
wal.append(TraceContext.DEFAULT, StreamRecordBatch.of(233L, 0, 100L + i, 1, generateByteBuf(1)));
((DefaultWriter) (wal.writer)).flush().join();
}
@ -352,7 +352,7 @@ public class ObjectWALServiceTest {
long startOffset = 0L;
for (int i = 0; i < 4; i++) {
startOffset = writeV0Object(config, new StreamRecordBatch(233L, 0, 100L + i, 1, generateByteBuf(1)).encoded(), startOffset);
startOffset = writeV0Object(config, StreamRecordBatch.of(233L, 0, 100L + i, 1, generateByteBuf(1)).encoded(), startOffset);
}
ObjectWALService wal = new ObjectWALService(time, objectStorage, config);
@ -372,11 +372,11 @@ public class ObjectWALServiceTest {
public void testRecoverFromV0AndV1Objects() throws IOException {
ObjectWALConfig config = ObjectWALConfig.builder().withEpoch(1L).withMaxBytesInBatch(1024).withBatchInterval(1000).build();
long nextOffset = 0L;
nextOffset = writeV0Object(config, new StreamRecordBatch(233L, 0, 100L, 1, generateByteBuf(1)).encoded(), nextOffset);
nextOffset = writeV0Object(config, StreamRecordBatch.of(233L, 0, 100L, 1, generateByteBuf(1)).encoded(), nextOffset);
long record1Offset = nextOffset;
nextOffset = writeV0Object(config, new StreamRecordBatch(233L, 0, 101L, 1, generateByteBuf(1)).encoded(), nextOffset);
nextOffset = writeV1Object(config, new StreamRecordBatch(233L, 0, 102L, 1, generateByteBuf(1)).encoded(), nextOffset, false, 0);
nextOffset = writeV1Object(config, new StreamRecordBatch(233L, 0, 103L, 1, generateByteBuf(1)).encoded(), nextOffset, false, record1Offset);
nextOffset = writeV0Object(config, StreamRecordBatch.of(233L, 0, 101L, 1, generateByteBuf(1)).encoded(), nextOffset);
nextOffset = writeV1Object(config, StreamRecordBatch.of(233L, 0, 102L, 1, generateByteBuf(1)).encoded(), nextOffset, false, 0);
nextOffset = writeV1Object(config, StreamRecordBatch.of(233L, 0, 103L, 1, generateByteBuf(1)).encoded(), nextOffset, false, record1Offset);
ObjectWALService wal = new ObjectWALService(time, objectStorage, config);
acquire(config);

View File

@ -360,6 +360,9 @@ public class LogConfig extends AbstractConfig {
.define(TopicConfig.TABLE_TOPIC_UPSERT_ENABLE_CONFIG, BOOLEAN, false, null, MEDIUM, TopicConfig.TABLE_TOPIC_UPSERT_ENABLE_DOC)
.define(TopicConfig.TABLE_TOPIC_CDC_FIELD_CONFIG, STRING, null, null, MEDIUM, TopicConfig.TABLE_TOPIC_CDC_FIELD_DOC)
.define(TopicConfig.AUTOMQ_TABLE_TOPIC_ERRORS_TOLERANCE_CONFIG, STRING, ErrorsTolerance.INVALID_DATA.name, in(ErrorsTolerance.names().toArray(new String[0])), MEDIUM, TopicConfig.AUTOMQ_TABLE_TOPIC_ERRORS_TOLERANCE_DOC)
.define(TopicConfig.AUTOMQ_TABLE_TOPIC_EXPIRE_SNAPSHOT_ENABLED_CONFIG, BOOLEAN, TopicConfig.AUTOMQ_TABLE_TOPIC_EXPIRE_SNAPSHOT_ENABLED_DEFAULT, MEDIUM, TopicConfig.AUTOMQ_TABLE_TOPIC_EXPIRE_SNAPSHOT_ENABLED_DOC)
.define(TopicConfig.AUTOMQ_TABLE_TOPIC_EXPIRE_SNAPSHOT_OLDER_THAN_HOURS_CONFIG, INT, TopicConfig.AUTOMQ_TABLE_TOPIC_EXPIRE_SNAPSHOT_OLDER_THAN_HOURS_DEFAULT, atLeast(1), MEDIUM, TopicConfig.AUTOMQ_TABLE_TOPIC_EXPIRE_SNAPSHOT_OLDER_THAN_HOURS_DOC)
.define(TopicConfig.AUTOMQ_TABLE_TOPIC_EXPIRE_SNAPSHOT_RETAIN_LAST_CONFIG, INT, TopicConfig.AUTOMQ_TABLE_TOPIC_EXPIRE_SNAPSHOT_RETAIN_LAST_DEFAULT, atLeast(1), MEDIUM, TopicConfig.AUTOMQ_TABLE_TOPIC_EXPIRE_SNAPSHOT_RETAIN_LAST_DOC)
.define(TopicConfig.KAFKA_LINKS_ID_CONFIG, STRING, null, null, MEDIUM, TopicConfig.KAFKA_LINKS_ID_DOC)
.define(TopicConfig.KAFKA_LINKS_TOPIC_START_TIME_CONFIG, LONG, ListOffsetsRequest.LATEST_TIMESTAMP, null, MEDIUM, TopicConfig.KAFKA_LINKS_TOPIC_START_TIME_DOC)
.define(TopicConfig.KAFKA_LINKS_TOPIC_STATE_CONFIG, STRING, null, null, MEDIUM, TopicConfig.KAFKA_LINKS_TOPIC_STATE_DOC)
@ -432,6 +435,10 @@ public class LogConfig extends AbstractConfig {
public final String kafkaLinksId;
public final long kafkaLinksTopicStartTime;
public final String kafkaLinksTopicState;
public final boolean tableTopicExpireSnapshotEnabled;
public final int tableTopicExpireSnapshotOlderThanHours;
public final int tableTopicExpireSnapshotRetainLast;
// AutoMQ inject end
private final int maxMessageSize;
@ -501,6 +508,10 @@ public class LogConfig extends AbstractConfig {
this.kafkaLinksId = getString(TopicConfig.KAFKA_LINKS_ID_CONFIG);
this.kafkaLinksTopicStartTime = getLong(TopicConfig.KAFKA_LINKS_TOPIC_START_TIME_CONFIG);
this.kafkaLinksTopicState = getString(TopicConfig.KAFKA_LINKS_TOPIC_STATE_CONFIG);
this.tableTopicExpireSnapshotEnabled = getBoolean(TopicConfig.AUTOMQ_TABLE_TOPIC_EXPIRE_SNAPSHOT_ENABLED_CONFIG);
this.tableTopicExpireSnapshotOlderThanHours = getInt(TopicConfig.AUTOMQ_TABLE_TOPIC_EXPIRE_SNAPSHOT_OLDER_THAN_HOURS_CONFIG);
this.tableTopicExpireSnapshotRetainLast = getInt(TopicConfig.AUTOMQ_TABLE_TOPIC_EXPIRE_SNAPSHOT_RETAIN_LAST_CONFIG);
// AutoMQ inject end
remoteLogConfig = new RemoteLogConfig(this);