Compare commits
18 Commits
| Author | SHA1 | Date |
|---|---|---|
|
|
451d0da3fb | |
|
|
cc62ad524a | |
|
|
4be8c789d3 | |
|
|
2418658424 | |
|
|
e5de8a921a | |
|
|
e2ba16c9d7 | |
|
|
7c2bad5ba1 | |
|
|
7535e76c44 | |
|
|
60b1ec614f | |
|
|
d768bfa3e9 | |
|
|
c470d491bf | |
|
|
863abcb653 | |
|
|
48446d941c | |
|
|
0965b4fd93 | |
|
|
4ceb66d4ea | |
|
|
dd031cb941 | |
|
|
2b73b8ed62 | |
|
|
da9ae1ac92 |
|
|
@ -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 AutoMQ’s 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
|
||||
|
|
|
|||
|
|
@ -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.
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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";
|
||||
|
|
|
|||
|
|
@ -20,7 +20,7 @@
|
|||
"broker"
|
||||
],
|
||||
"name": "AutomqGetPartitionSnapshotRequest",
|
||||
"validVersions": "0-1",
|
||||
"validVersions": "0-2",
|
||||
"flexibleVersions": "0+",
|
||||
"fields": [
|
||||
{
|
||||
|
|
|
|||
|
|
@ -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": [
|
||||
|
|
|
|||
|
|
@ -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
|
||||
* >= 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;
|
||||
}
|
||||
}
|
||||
|
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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());
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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);
|
||||
|
|
|
|||
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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());
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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);
|
||||
|
|
|
|||
|
|
@ -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();
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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() {
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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);
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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) {
|
||||
|
|
|
|||
|
|
@ -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() {
|
||||
|
|
|
|||
|
|
@ -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);
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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] = {
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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) {
|
||||
|
|
@ -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);
|
||||
}
|
||||
});
|
||||
|
|
|
|||
|
|
@ -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) {
|
||||
|
|
|
|||
|
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -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) {
|
||||
|
|
|
|||
|
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
|
|
|||
|
|
@ -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",
|
||||
|
|
|
|||
|
|
@ -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))
|
||||
)
|
||||
)
|
||||
);
|
||||
|
|
|
|||
|
|
@ -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>
|
||||
|
|
@ -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];
|
||||
|
|
|
|||
|
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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;
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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;
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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) {
|
||||
|
|
|
|||
|
|
@ -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) -> {
|
||||
|
|
|
|||
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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();
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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();
|
||||
|
||||
|
|
|
|||
|
|
@ -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.
|
||||
|
|
|
|||
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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);
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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);
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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) {
|
||||
|
|
|
|||
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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) {
|
||||
|
|
|
|||
|
|
@ -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()
|
||||
|
|
|
|||
|
|
@ -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));
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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));
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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);
|
||||
|
|
|
|||
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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();
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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));
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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;
|
||||
|
|
|
|||
|
|
@ -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());
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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);
|
||||
|
|
|
|||
|
|
@ -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(
|
||||
|
|
|
|||
|
|
@ -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();
|
||||
|
|
|
|||
|
|
@ -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));
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -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);
|
||||
|
|
|
|||
|
|
@ -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);
|
||||
|
|
|
|||
Loading…
Reference in New Issue