Skip to content

Commit 36e427a

Browse files
authored
chore(metrics): unify object wal metrics (#2784)
Signed-off-by: Robin Han <[email protected]>
1 parent cee57fd commit 36e427a

File tree

3 files changed

+2
-122
lines changed

3 files changed

+2
-122
lines changed

core/src/main/scala/kafka/log/stream/s3/telemetry/TelemetryManager.java

Lines changed: 0 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -34,7 +34,6 @@
3434
import com.automq.stream.s3.metrics.MetricsConfig;
3535
import com.automq.stream.s3.metrics.MetricsLevel;
3636
import com.automq.stream.s3.metrics.S3StreamMetricsManager;
37-
import com.automq.stream.s3.wal.metrics.ObjectWALMetricsManager;
3837

3938
import org.apache.commons.lang3.StringUtils;
4039
import org.apache.commons.lang3.tuple.Pair;
@@ -180,10 +179,6 @@ protected void initializeMetricsManager(Meter meter) {
180179

181180
S3StreamKafkaMetricsManager.configure(new MetricsConfig(metricsLevel(), Attributes.empty(), kafkaConfig.s3ExporterReportIntervalMs()));
182181
S3StreamKafkaMetricsManager.initMetrics(meter, TelemetryConstants.KAFKA_METRICS_PREFIX);
183-
184-
// kraft controller may not have s3WALPath config.
185-
ObjectWALMetricsManager.initMetrics(meter, TelemetryConstants.KAFKA_WAL_METRICS_PREFIX);
186-
187182
TableTopicMetricsManager.initMetrics(meter);
188183

189184
this.oTelHistogramReporter.start(meter);

s3stream/src/main/java/com/automq/stream/s3/wal/impl/object/DefaultWriter.java

Lines changed: 2 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@
2121

2222
import com.automq.stream.ByteBufSeqAlloc;
2323
import com.automq.stream.s3.ByteBufAlloc;
24+
import com.automq.stream.s3.metrics.stats.StorageOperationStats;
2425
import com.automq.stream.s3.model.StreamRecordBatch;
2526
import com.automq.stream.s3.operator.ObjectStorage;
2627
import com.automq.stream.s3.wal.AppendResult;
@@ -34,7 +35,6 @@
3435
import com.automq.stream.s3.wal.exception.RuntimeIOException;
3536
import com.automq.stream.s3.wal.exception.WALFencedException;
3637
import com.automq.stream.s3.wal.impl.DefaultRecordOffset;
37-
import com.automq.stream.s3.wal.metrics.ObjectWALMetricsManager;
3838
import com.automq.stream.s3.wal.util.WALUtil;
3939
import com.automq.stream.utils.FutureUtil;
4040
import com.automq.stream.utils.Systems;
@@ -123,9 +123,6 @@ public DefaultWriter(Time time, ObjectStorage objectStorage, ObjectWALConfig con
123123
if (!(config.openMode() == OpenMode.READ_WRITE || config.openMode() == OpenMode.FAILOVER)) {
124124
throw new IllegalArgumentException("The open mode must be READ_WRITE or FAILOVER, but got " + config.openMode());
125125
}
126-
ObjectWALMetricsManager.setInflightUploadCountSupplier(() -> (long) uploadingBulks.size());
127-
ObjectWALMetricsManager.setBufferedDataInBytesSupplier(bufferedDataBytes::get);
128-
ObjectWALMetricsManager.setObjectDataInBytesSupplier(objectDataBytes::get);
129126
}
130127

131128
public void start() {
@@ -239,7 +236,6 @@ public RecordOffset confirmOffset() {
239236

240237
public CompletableFuture<AppendResult> append0(
241238
StreamRecordBatch streamRecordBatch) throws OverCapacityException, WALFencedException {
242-
long startTime = time.nanoseconds();
243239
checkWriteStatus();
244240

245241
if (bufferedDataBytes.get() > config.maxUnflushedBytes()) {
@@ -273,10 +269,7 @@ public CompletableFuture<AppendResult> append0(
273269
bufferedDataBytes.addAndGet(-dataSize);
274270
if (throwable != null) {
275271
LOGGER.error("Failed to append record to S3 WAL", throwable);
276-
} else {
277-
ObjectWALMetricsManager.recordOperationDataSize(dataSize, "append");
278272
}
279-
ObjectWALMetricsManager.recordOperationLatency(time.nanoseconds() - startTime, "append", throwable == null);
280273
});
281274
}
282275

@@ -386,12 +379,11 @@ private void uploadBulk0() {
386379
FutureUtil.propagate(objectStorage.write(writeOptions, path, objectBuffer), bulk.uploadCf);
387380
long finalLastRecordOffset = lastRecordOffset;
388381
bulk.uploadCf.whenCompleteAsync((rst, ex) -> {
389-
ObjectWALMetricsManager.recordOperationLatency(time.nanoseconds() - startTime, "upload", ex == null);
390382
if (ex != null) {
391383
fenced = true;
392384
LOGGER.error("S3WAL upload {} fail", path, ex);
393385
} else {
394-
ObjectWALMetricsManager.recordOperationDataSize(objectLength, "upload");
386+
StorageOperationStats.getInstance().appendWALWriteStats.record(time.nanoseconds() - startTime);
395387
lastRecordOffset2object.put(finalLastRecordOffset, new WALObject(rst.bucket(), path, config.epoch(), firstOffset, endOffset, objectLength));
396388
objectDataBytes.addAndGet(objectLength);
397389
}
@@ -418,10 +410,8 @@ private void callback() {
418410
}
419411
// The inflight uploading bulks count was decreased, then trigger the upload of Bulk in waitingUploadBulks
420412
tryUploadBulkInWaiting();
421-
long commitStartTime = time.nanoseconds();
422413
return reservationService.verify(config.nodeId(), config.epoch(), config.openMode() == OpenMode.FAILOVER)
423414
.whenComplete((rst, ex) -> {
424-
ObjectWALMetricsManager.recordOperationLatency(time.nanoseconds() - commitStartTime, "commit", ex == null);
425415
if (ex != null) {
426416
LOGGER.error("Unexpected S3WAL lease check fail. Make the WAL fenced", ex);
427417
fenced = true;
@@ -484,7 +474,6 @@ public CompletableFuture<Void> trim0(long newStartOffset) throws WALFencedExcept
484474
checkStatus();
485475
List<ObjectStorage.ObjectPath> deleteObjectList = new ArrayList<>();
486476
AtomicLong deletedObjectSize = new AtomicLong();
487-
long startTime = time.nanoseconds();
488477
CompletableFuture<?> persistTrimOffsetCf;
489478
lock.writeLock().lock();
490479
try {
@@ -530,7 +519,6 @@ public CompletableFuture<Void> trim0(long newStartOffset) throws WALFencedExcept
530519
}
531520

532521
return objectStorage.delete(deleteObjectList).whenComplete((v, throwable) -> {
533-
ObjectWALMetricsManager.recordOperationLatency(time.nanoseconds() - startTime, "trim", throwable == null);
534522
objectDataBytes.addAndGet(-1 * deletedObjectSize.get());
535523
// Never fail the delete task, the under layer storage will retry forever.
536524
if (throwable != null) {

s3stream/src/main/java/com/automq/stream/s3/wal/metrics/ObjectWALMetricsManager.java

Lines changed: 0 additions & 103 deletions
This file was deleted.

0 commit comments

Comments
 (0)