Skip to content

Commit

Permalink
Add call point for ufs metrics
Browse files Browse the repository at this point in the history
  • Loading branch information
Kai-Zhang committed Nov 2, 2023
1 parent 701bc5e commit c114005
Show file tree
Hide file tree
Showing 6 changed files with 12 additions and 28 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import alluxio.grpc.FileSystemMasterCommonPOptions;
import alluxio.metrics.MetricKey;
import alluxio.metrics.MetricsSystem;
import alluxio.metrics.MultiDimensionalMetricsSystem;
import alluxio.util.CommonUtils;

import com.codahale.metrics.Counter;
Expand Down Expand Up @@ -268,6 +269,7 @@ private void writeInternal(byte[] b, int off, int len) throws IOException {
if (mUnderStorageType.isSyncPersist()) {
if (mUnderStorageOutputStream != null) {
mUnderStorageOutputStream.write(b, off, len);
MultiDimensionalMetricsSystem.UFS_DATA_ACCESS.labelValues("write").inc(len);
Metrics.BYTES_WRITTEN_TO_UFS.inc(len);
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -215,8 +215,8 @@ private byte[] readExternalPage(long position) {
totalBytesRead += bytesRead;
}
// Bytes read from external, may be larger than requests due to reading complete pages
MultiDimensionalMetricsSystem.UFS_DATA_ACCESS.labelValues("read").inc(totalBytesRead);
MetricsSystem.meter(MetricKey.CLIENT_CACHE_BYTES_READ_EXTERNAL.getName()).mark(totalBytesRead);
MultiDimensionalMetricsSystem.UFS_DATA_ACCESS.labelValues("read").observe(totalBytesRead);
if (totalBytesRead != pageSize) {
throw new FailedPreconditionRuntimeException(
"Failed to read complete page from external storage. Bytes read: "
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -59,26 +59,19 @@ public final class MultiDimensionalMetricsSystem {
.labelNames("method")
.build();

public static final Histogram UFS_DATA_ACCESS = Histogram.builder()
.name("alluxio_ufs_data_access")
.help("aggregated throughput of ufs access")
.unit(Unit.BYTES)
.labelNames("method")
.build();

public static final Summary UFS_DATA_ACCESS_LATENCY = Summary.builder()
.name("alluxio_ufs_data_access_latency")
.help("aggregated latency of ufs access")
.unit(Unit.SECONDS)
.labelNames("method")
.build();

public static final Counter META_OPERATION = Counter.builder()
.name("alluxio_meta_operation")
.help("counter of rpc calls of the meta operations")
.labelNames("op")
.build();

public static final Counter UFS_DATA_ACCESS = Counter.builder()
.name("alluxio_ufs_data_access")
.help("amount of the ufs access")
.unit(Unit.BYTES)
.labelNames("method")
.build();

public static final Counter CACHED_DATA_READ = Counter.builder()
.name("alluxio_cached_data_read")
.help("amount of the read cached data")
Expand Down Expand Up @@ -122,19 +115,10 @@ public static void initMetrics() {
// No essential metrics for the master for now.
break;
case WORKER:
PrometheusRegistry.defaultRegistry.register(DATA_ACCESS);
PrometheusRegistry.defaultRegistry.register(DATA_ACCESS_LATENCY);
PrometheusRegistry.defaultRegistry.register(UFS_DATA_ACCESS);
PrometheusRegistry.defaultRegistry.register(UFS_DATA_ACCESS_LATENCY);
PrometheusRegistry.defaultRegistry.register(META_OPERATION);
PrometheusRegistry.defaultRegistry.register(CACHED_DATA_READ);
PrometheusRegistry.defaultRegistry.register(CACHED_EVICTED_DATA);
PrometheusRegistry.defaultRegistry.register(CACHED_STORAGE);
PrometheusRegistry.defaultRegistry.register(CACHED_CAPACITY);
break;
case CLIENT:
PrometheusRegistry.defaultRegistry.register(DATA_ACCESS);
PrometheusRegistry.defaultRegistry.register(DATA_ACCESS_LATENCY);
PrometheusRegistry.defaultRegistry.register(UFS_DATA_ACCESS);
PrometheusRegistry.defaultRegistry.register(META_OPERATION);
PrometheusRegistry.defaultRegistry.register(CACHED_DATA_READ);
PrometheusRegistry.defaultRegistry.register(CACHED_EVICTED_DATA);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@
import alluxio.client.file.cache.CacheManager;
import alluxio.client.file.cache.PageId;
import alluxio.grpc.WritePType;
import alluxio.metrics.MultiDimensionalMetricsSystem;
import alluxio.network.protocol.databuffer.DataBuffer;
import alluxio.worker.block.io.BlockWriter;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -287,8 +287,8 @@ private void runInternal() {
int readableBytes = buf.readableBytes();
mContext.setPosToWrite(mContext.getPosToWrite() + readableBytes);
writeBuf(mContext, mChannel, buf, mContext.getPosToWrite());
incrementMetrics(readableBytes);
mContext.recordAccessMetric(readableBytes);
incrementMetrics(readableBytes);
} catch (Exception e) {
LOG.error("Failed to write packet for request {}", mContext.getRequest(), e);
Throwables.propagateIfPossible(e);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -695,7 +695,6 @@ private void sendData(RequestContext requestContext, DataBuffer dataBuffer,
mChannelEventQueue.add(WriteFutureResolved.failure(future.cause()));
return;
}
//TODO(bowen): add num bytes read metrics
mChannelEventQueue.put(WriteFutureResolved.success(length));
});
if (requestContext.bytesPending()
Expand Down

0 comments on commit c114005

Please sign in to comment.