From 41bf2000bf6cdf88c1683f3028b073d173313fae Mon Sep 17 00:00:00 2001 From: Shichao Nie Date: Fri, 29 Dec 2023 15:26:51 +0800 Subject: [PATCH] feat(s3stream): reuse attributes to reduce cost during recording metrics Signed-off-by: Shichao Nie --- .../stream/s3/metrics/AttributesCache.java | 116 +++++++++++++++++ .../s3/metrics/S3StreamMetricsManager.java | 118 +++++------------- .../s3/metrics/AttributesCacheTest.java | 59 +++++++++ .../s3/metrics/StreamMetricsManagerTest.java | 58 --------- 4 files changed, 206 insertions(+), 145 deletions(-) create mode 100644 s3stream/src/main/java/com/automq/stream/s3/metrics/AttributesCache.java create mode 100644 s3stream/src/test/java/com/automq/stream/s3/metrics/AttributesCacheTest.java delete mode 100644 s3stream/src/test/java/com/automq/stream/s3/metrics/StreamMetricsManagerTest.java diff --git a/s3stream/src/main/java/com/automq/stream/s3/metrics/AttributesCache.java b/s3stream/src/main/java/com/automq/stream/s3/metrics/AttributesCache.java new file mode 100644 index 000000000..29437de0a --- /dev/null +++ b/s3stream/src/main/java/com/automq/stream/s3/metrics/AttributesCache.java @@ -0,0 +1,116 @@ +/* + * 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.metrics; + +import com.automq.stream.s3.metrics.operations.S3ObjectStage; +import com.automq.stream.s3.metrics.operations.S3Operation; +import com.automq.stream.s3.metrics.operations.S3Stage; +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.common.AttributesBuilder; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +public class AttributesCache { + public static final AttributesCache INSTANCE = new AttributesCache(); + private final Map attributesMap = new ConcurrentHashMap<>(); + private Attributes defaultAttributes = null; + + private AttributesCache() { + } + + public void setDefaultAttributes(Attributes defaultAttributes) { + this.defaultAttributes = defaultAttributes; + } + + public Attributes defaultAttributes() { + return defaultAttributes == null ? Attributes.builder().build() : defaultAttributes; + } + + public Attributes getAttributes(S3Operation operation, long size, boolean isSuccess) { + String sizeLabelName = getObjectBucketLabel(size); + String key = operation.getUniqueKey() + "-" + sizeLabelName + "-" + isSuccess; + return attributesMap.computeIfAbsent(key, k -> buildAttributes(operation, sizeLabelName, isSuccess)); + } + + public Attributes getAttributes(S3Stage stage) { + String key = stage.getOperation().getUniqueKey() + "-" + stage.getName(); + return attributesMap.computeIfAbsent(key, k -> buildAttributes(stage)); + } + + public Attributes getAttributes(S3Operation operation, String status) { + String key = operation.getUniqueKey() + "-" + status; + return attributesMap.computeIfAbsent(key, k -> buildAttributes(operation, status)); + } + + public Attributes getAttributes(String source) { + String key = "AllocateByteBuf-" + source; + return attributesMap.computeIfAbsent(key, k -> buildAttributes(source)); + } + + public Attributes getAttributes(S3ObjectStage stage) { + String key = "S3ObjectStage-" + stage.getName(); + return attributesMap.computeIfAbsent(key, k -> buildAttributes(stage)); + } + + String getObjectBucketLabel(long objectSize) { + int index = (int) Math.ceil(Math.log((double) objectSize / (16 * 1024)) / Math.log(2)); + index = Math.min(S3StreamMetricsConstant.OBJECT_SIZE_BUCKET_NAMES.length - 1, Math.max(0, index)); + return S3StreamMetricsConstant.OBJECT_SIZE_BUCKET_NAMES[index]; + } + + private Attributes buildAttributes(S3Operation operation, String sizeLabelName, boolean isSuccess) { + AttributesBuilder attributesBuilder = defaultAttributes().toBuilder() + .put(S3StreamMetricsConstant.LABEL_OPERATION_TYPE, operation.getType().getName()) + .put(S3StreamMetricsConstant.LABEL_OPERATION_NAME, operation.getName()) + .put(S3StreamMetricsConstant.LABEL_STATUS, isSuccess ? "success" : "failed"); + if (operation == S3Operation.GET_OBJECT || operation == S3Operation.PUT_OBJECT || operation == S3Operation.UPLOAD_PART) { + attributesBuilder.put(S3StreamMetricsConstant.LABEL_SIZE_NAME, sizeLabelName); + } + return attributesBuilder.build(); + } + + private Attributes buildAttributes(S3Stage stage) { + return defaultAttributes().toBuilder() + .put(S3StreamMetricsConstant.LABEL_OPERATION_TYPE, stage.getOperation().getType().getName()) + .put(S3StreamMetricsConstant.LABEL_OPERATION_NAME, stage.getOperation().getName()) + .put(S3StreamMetricsConstant.LABEL_STAGE, stage.getName()) + .build(); + } + + private Attributes buildAttributes(S3Operation operation, String status) { + return defaultAttributes().toBuilder() + .put(S3StreamMetricsConstant.LABEL_OPERATION_TYPE, operation.getType().getName()) + .put(S3StreamMetricsConstant.LABEL_OPERATION_NAME, operation.getName()) + .put(S3StreamMetricsConstant.LABEL_STATUS, status) + .build(); + } + + private Attributes buildAttributes(S3ObjectStage stage) { + return defaultAttributes().toBuilder() + .put(S3StreamMetricsConstant.LABEL_STAGE, stage.getName()) + .build(); + } + + private Attributes buildAttributes(String source) { + return defaultAttributes().toBuilder() + .put(S3StreamMetricsConstant.LABEL_ALLOCATE_BYTE_BUF_SOURCE, source) + .build(); + } + +} diff --git a/s3stream/src/main/java/com/automq/stream/s3/metrics/S3StreamMetricsManager.java b/s3stream/src/main/java/com/automq/stream/s3/metrics/S3StreamMetricsManager.java index 0ad7636fa..3cf44471e 100644 --- a/s3stream/src/main/java/com/automq/stream/s3/metrics/S3StreamMetricsManager.java +++ b/s3stream/src/main/java/com/automq/stream/s3/metrics/S3StreamMetricsManager.java @@ -21,7 +21,6 @@ import com.automq.stream.s3.metrics.operations.S3Operation; import com.automq.stream.s3.metrics.operations.S3Stage; import com.automq.stream.s3.network.AsyncNetworkBandwidthLimiter; -import io.opentelemetry.api.common.Attributes; import io.opentelemetry.api.common.AttributesBuilder; import io.opentelemetry.api.metrics.LongCounter; import io.opentelemetry.api.metrics.LongHistogram; @@ -33,7 +32,6 @@ public class S3StreamMetricsManager { private static LongCounter s3DownloadSizeInTotal = new NoopLongCounter(); private static LongCounter s3UploadSizeInTotal = new NoopLongCounter(); - private static LongCounter operationNumInTotal = new NoopLongCounter(); private static LongHistogram operationLatency = new NoopLongHistogram(); private static LongCounter objectNumInTotal = new NoopLongCounter(); private static LongHistogram objectStageCost = new NoopLongHistogram(); @@ -71,10 +69,9 @@ public class S3StreamMetricsManager { private static Supplier availableInflightS3ReadQuotaSupplier = () -> 0; private static Supplier availableInflightS3WriteQuotaSupplier = () -> 0; private static Supplier inflightWALUploadTasksCountSupplier = () -> 0; - private static Supplier attributesBuilderSupplier = null; public static void initAttributesBuilder(Supplier attributesBuilderSupplier) { - S3StreamMetricsManager.attributesBuilderSupplier = attributesBuilderSupplier; + AttributesCache.INSTANCE.setDefaultAttributes(attributesBuilderSupplier.get().build()); } public static void initMetrics(Meter meter) { @@ -90,9 +87,6 @@ public static void initMetrics(Meter meter, String prefix) { .setDescription("S3 upload size") .setUnit("bytes") .build(); - operationNumInTotal = meter.counterBuilder(prefix + S3StreamMetricsConstant.OPERATION_COUNT_METRIC_NAME) - .setDescription("Operations count") - .build(); operationLatency = meter.histogramBuilder(prefix + S3StreamMetricsConstant.OPERATION_LATENCY_METRIC_NAME) .setDescription("Operations latency") .setUnit("nanoseconds") @@ -130,20 +124,20 @@ public static void initMetrics(Meter meter, String prefix) { .setDescription("Network inbound available bandwidth") .setUnit("bytes") .ofLongs() - .buildWithCallback(result -> result.record(networkInboundAvailableBandwidthSupplier.get(), newAttributesBuilder().build())); + .buildWithCallback(result -> result.record(networkInboundAvailableBandwidthSupplier.get(), AttributesCache.INSTANCE.defaultAttributes())); networkOutboundAvailableBandwidth = meter.gaugeBuilder(prefix + S3StreamMetricsConstant.NETWORK_OUTBOUND_AVAILABLE_BANDWIDTH_METRIC_NAME) .setDescription("Network outbound available bandwidth") .setUnit("bytes") .ofLongs() - .buildWithCallback(result -> result.record(networkOutboundAvailableBandwidthSupplier.get(), newAttributesBuilder().build())); + .buildWithCallback(result -> result.record(networkOutboundAvailableBandwidthSupplier.get(), AttributesCache.INSTANCE.defaultAttributes())); networkInboundLimiterQueueSize = meter.gaugeBuilder(prefix + S3StreamMetricsConstant.NETWORK_INBOUND_LIMITER_QUEUE_SIZE_METRIC_NAME) .setDescription("Network inbound limiter queue size") .ofLongs() - .buildWithCallback(result -> result.record((long) networkInboundLimiterQueueSizeSupplier.get(), newAttributesBuilder().build())); + .buildWithCallback(result -> result.record((long) networkInboundLimiterQueueSizeSupplier.get(), AttributesCache.INSTANCE.defaultAttributes())); networkOutboundLimiterQueueSize = meter.gaugeBuilder(prefix + S3StreamMetricsConstant.NETWORK_OUTBOUND_LIMITER_QUEUE_SIZE_METRIC_NAME) .setDescription("Network outbound limiter queue size") .ofLongs() - .buildWithCallback(result -> result.record((long) networkOutboundLimiterQueueSizeSupplier.get(), newAttributesBuilder().build())); + .buildWithCallback(result -> result.record((long) networkOutboundLimiterQueueSizeSupplier.get(), AttributesCache.INSTANCE.defaultAttributes())); networkInboundLimiterQueueTime = meter.histogramBuilder(prefix + S3StreamMetricsConstant.NETWORK_INBOUND_LIMITER_QUEUE_TIME_METRIC_NAME) .setDescription("Network inbound limiter queue time") .setUnit("nanoseconds") @@ -169,38 +163,38 @@ public static void initMetrics(Meter meter, String prefix) { deltaWalStartOffset = meter.gaugeBuilder(prefix + S3StreamMetricsConstant.WAL_START_OFFSET) .setDescription("Delta WAL start offset") .ofLongs() - .buildWithCallback(result -> result.record(deltaWalStartOffsetSupplier.get(), newAttributesBuilder().build())); + .buildWithCallback(result -> result.record(deltaWalStartOffsetSupplier.get(), AttributesCache.INSTANCE.defaultAttributes())); deltaWalTrimmedOffset = meter.gaugeBuilder(prefix + S3StreamMetricsConstant.WAL_TRIMMED_OFFSET) .setDescription("Delta WAL trimmed offset") .ofLongs() - .buildWithCallback(result -> result.record(deltaWalTrimmedOffsetSupplier.get(), newAttributesBuilder().build())); + .buildWithCallback(result -> result.record(deltaWalTrimmedOffsetSupplier.get(), AttributesCache.INSTANCE.defaultAttributes())); deltaWalCacheSize = meter.gaugeBuilder(prefix + S3StreamMetricsConstant.DELTA_WAL_CACHE_SIZE) .setDescription("Delta WAL cache size") .setUnit("bytes") .ofLongs() - .buildWithCallback(result -> result.record(deltaWALCacheSizeSupplier.get(), newAttributesBuilder().build())); + .buildWithCallback(result -> result.record(deltaWALCacheSizeSupplier.get(), AttributesCache.INSTANCE.defaultAttributes())); blockCacheSize = meter.gaugeBuilder(prefix + S3StreamMetricsConstant.BLOCK_CACHE_SIZE) .setDescription("Block cache size") .setUnit("bytes") .ofLongs() - .buildWithCallback(result -> result.record(blockCacheSizeSupplier.get(), newAttributesBuilder().build())); + .buildWithCallback(result -> result.record(blockCacheSizeSupplier.get(), AttributesCache.INSTANCE.defaultAttributes())); availableInflightReadAheadSize = meter.gaugeBuilder(prefix + S3StreamMetricsConstant.AVAILABLE_INFLIGHT_READ_AHEAD_SIZE_METRIC_NAME) .setDescription("Available inflight read ahead size") .setUnit("bytes") .ofLongs() - .buildWithCallback(result -> result.record((long) availableInflightReadAheadSizeSupplier.get(), newAttributesBuilder().build())); + .buildWithCallback(result -> result.record((long) availableInflightReadAheadSizeSupplier.get(), AttributesCache.INSTANCE.defaultAttributes())); availableInflightS3ReadQuota = meter.gaugeBuilder(prefix + S3StreamMetricsConstant.AVAILABLE_S3_INFLIGHT_READ_QUOTA_METRIC_NAME) .setDescription("Available inflight S3 read quota") .ofLongs() - .buildWithCallback(result -> result.record((long) availableInflightS3ReadQuotaSupplier.get(), newAttributesBuilder().build())); + .buildWithCallback(result -> result.record((long) availableInflightS3ReadQuotaSupplier.get(), AttributesCache.INSTANCE.defaultAttributes())); availableInflightS3WriteQuota = meter.gaugeBuilder(prefix + S3StreamMetricsConstant.AVAILABLE_S3_INFLIGHT_WRITE_QUOTA_METRIC_NAME) .setDescription("Available inflight S3 write quota") .ofLongs() - .buildWithCallback(result -> result.record((long) availableInflightS3WriteQuotaSupplier.get(), newAttributesBuilder().build())); + .buildWithCallback(result -> result.record((long) availableInflightS3WriteQuotaSupplier.get(), AttributesCache.INSTANCE.defaultAttributes())); inflightWALUploadTasksCount = meter.gaugeBuilder(prefix + S3StreamMetricsConstant.INFLIGHT_WAL_UPLOAD_TASKS_COUNT_METRIC_NAME) .setDescription("Inflight upload WAL tasks count") .ofLongs() - .buildWithCallback(result -> result.record((long) inflightWALUploadTasksCountSupplier.get(), newAttributesBuilder().build())); + .buildWithCallback(result -> result.record((long) inflightWALUploadTasksCountSupplier.get(), AttributesCache.INSTANCE.defaultAttributes())); compactionReadSizeInTotal = meter.counterBuilder(prefix + S3StreamMetricsConstant.COMPACTION_READ_SIZE_METRIC_NAME) .setDescription("Compaction read size") .setUnit("bytes") @@ -211,13 +205,6 @@ public static void initMetrics(Meter meter, String prefix) { .build(); } - private static AttributesBuilder newAttributesBuilder() { - if (attributesBuilderSupplier != null) { - return attributesBuilderSupplier.get(); - } - return Attributes.builder(); - } - public static void registerNetworkLimiterSupplier(AsyncNetworkBandwidthLimiter.Type type, Supplier networkAvailableBandwidthSupplier, Supplier networkLimiterQueueSizeSupplier) { @@ -233,7 +220,6 @@ public static void registerNetworkLimiterSupplier(AsyncNetworkBandwidthLimiter.T } } - //TODO: 各broker当前stream数量、各stream流量?、各broker的s3 object number, size public static void registerDeltaWalOffsetSupplier(Supplier deltaWalStartOffsetSupplier, Supplier deltaWalTrimmedOffsetSupplier) { S3StreamMetricsManager.deltaWalStartOffsetSupplier = deltaWalStartOffsetSupplier; @@ -265,19 +251,11 @@ public static void registerInflightWALUploadTasksCountSupplier(Supplier } public static void recordS3UploadSize(long value) { - s3UploadSizeInTotal.add(value, newAttributesBuilder().build()); + s3UploadSizeInTotal.add(value, AttributesCache.INSTANCE.defaultAttributes()); } public static void recordS3DownloadSize(long value) { - s3DownloadSizeInTotal.add(value, newAttributesBuilder().build()); - } - - public static void recordOperationNum(long value, S3Operation operation) { - Attributes attributes = newAttributesBuilder() - .put(S3StreamMetricsConstant.LABEL_OPERATION_TYPE, operation.getType().getName()) - .put(S3StreamMetricsConstant.LABEL_OPERATION_NAME, operation.getName()) - .build(); - operationNumInTotal.add(value, attributes); + s3DownloadSizeInTotal.add(value, AttributesCache.INSTANCE.defaultAttributes()); } public static void recordOperationLatency(long value, S3Operation operation) { @@ -293,99 +271,65 @@ public static void recordOperationLatency(long value, S3Operation operation, lon } public static void recordOperationLatency(long value, S3Operation operation, long size, boolean isSuccess) { - AttributesBuilder attributesBuilder = newAttributesBuilder() - .put(S3StreamMetricsConstant.LABEL_OPERATION_TYPE, operation.getType().getName()) - .put(S3StreamMetricsConstant.LABEL_OPERATION_NAME, operation.getName()) - .put(S3StreamMetricsConstant.LABEL_STATUS, isSuccess ? "success" : "failed"); - if (operation == S3Operation.GET_OBJECT || operation == S3Operation.PUT_OBJECT || operation == S3Operation.UPLOAD_PART) { - attributesBuilder.put(S3StreamMetricsConstant.LABEL_SIZE_NAME, getObjectBucketLabel(size)); - } - operationLatency.record(value, attributesBuilder.build()); + operationLatency.record(value, AttributesCache.INSTANCE.getAttributes(operation, size, isSuccess)); } public static void recordStageLatency(long value, S3Stage stage) { - Attributes attributes = newAttributesBuilder() - .put(S3StreamMetricsConstant.LABEL_OPERATION_TYPE, stage.getOperation().getType().getName()) - .put(S3StreamMetricsConstant.LABEL_OPERATION_NAME, stage.getOperation().getName()) - .put(S3StreamMetricsConstant.LABEL_STAGE, stage.getName()) - .build(); - operationLatency.record(value, attributes); + operationLatency.record(value, AttributesCache.INSTANCE.getAttributes(stage)); } public static void recordReadCacheLatency(long value, S3Operation operation, boolean isCacheHit) { - Attributes attributes = newAttributesBuilder() - .put(S3StreamMetricsConstant.LABEL_OPERATION_TYPE, operation.getType().getName()) - .put(S3StreamMetricsConstant.LABEL_OPERATION_NAME, operation.getName()) - .put(S3StreamMetricsConstant.LABEL_STATUS, isCacheHit ? "hit" : "miss") - .build(); - operationLatency.record(value, attributes); + operationLatency.record(value, AttributesCache.INSTANCE.getAttributes(operation, isCacheHit ? "hit" : "miss")); } public static void recordReadAheadLatency(long value, S3Operation operation, boolean isSync) { - Attributes attributes = newAttributesBuilder() - .put(S3StreamMetricsConstant.LABEL_OPERATION_TYPE, operation.getType().getName()) - .put(S3StreamMetricsConstant.LABEL_OPERATION_NAME, operation.getName()) - .put(S3StreamMetricsConstant.LABEL_STATUS, isSync ? "sync" : "async") - .build(); - operationLatency.record(value, attributes); - } - - public static String getObjectBucketLabel(long objectSize) { - int index = (int) Math.ceil(Math.log((double) objectSize / (16 * 1024)) / Math.log(2)); - index = Math.min(S3StreamMetricsConstant.OBJECT_SIZE_BUCKET_NAMES.length - 1, Math.max(0, index)); - return S3StreamMetricsConstant.OBJECT_SIZE_BUCKET_NAMES[index]; + operationLatency.record(value, AttributesCache.INSTANCE.getAttributes(operation, isSync ? "sync" : "async")); } public static void recordObjectNum(long value) { - objectNumInTotal.add(value, newAttributesBuilder().build()); + objectNumInTotal.add(value, AttributesCache.INSTANCE.defaultAttributes()); } public static void recordObjectStageCost(long value, S3ObjectStage stage) { - Attributes attributes = newAttributesBuilder() - .put(S3StreamMetricsConstant.LABEL_STAGE, stage.getName()) - .build(); - objectStageCost.record(value, attributes); + objectStageCost.record(value, AttributesCache.INSTANCE.getAttributes(stage)); } public static void recordObjectUploadSize(long value) { - objectUploadSize.record(value, newAttributesBuilder().build()); + objectUploadSize.record(value, AttributesCache.INSTANCE.defaultAttributes()); } public static void recordObjectDownloadSize(long value) { - objectDownloadSize.record(value, newAttributesBuilder().build()); + objectDownloadSize.record(value, AttributesCache.INSTANCE.defaultAttributes()); } public static void recordNetworkInboundUsage(long value) { - networkInboundUsageInTotal.add(value, newAttributesBuilder().build()); + networkInboundUsageInTotal.add(value, AttributesCache.INSTANCE.defaultAttributes()); } public static void recordNetworkOutboundUsage(long value) { - networkOutboundUsageInTotal.add(value, newAttributesBuilder().build()); + networkOutboundUsageInTotal.add(value, AttributesCache.INSTANCE.defaultAttributes()); } public static void recordNetworkLimiterQueueTime(long value, AsyncNetworkBandwidthLimiter.Type type) { switch (type) { - case INBOUND -> networkInboundLimiterQueueTime.record(value, newAttributesBuilder().build()); - case OUTBOUND -> networkOutboundLimiterQueueTime.record(value, newAttributesBuilder().build()); + case INBOUND -> networkInboundLimiterQueueTime.record(value, AttributesCache.INSTANCE.defaultAttributes()); + case OUTBOUND -> networkOutboundLimiterQueueTime.record(value, AttributesCache.INSTANCE.defaultAttributes()); } } public static void recordAllocateByteBufSize(long value, String source) { - Attributes attributes = newAttributesBuilder() - .put(S3StreamMetricsConstant.LABEL_ALLOCATE_BYTE_BUF_SOURCE, source) - .build(); - allocateByteBufSize.record(value, attributes); + allocateByteBufSize.record(value, AttributesCache.INSTANCE.getAttributes(source)); } public static void recordReadAheadSize(long value) { - readAheadSize.record(value, newAttributesBuilder().build()); + readAheadSize.record(value, AttributesCache.INSTANCE.defaultAttributes()); } public static void recordCompactionReadSizeIn(long value) { - compactionReadSizeInTotal.add(value, newAttributesBuilder().build()); + compactionReadSizeInTotal.add(value, AttributesCache.INSTANCE.defaultAttributes()); } public static void recordCompactionWriteSize(long value) { - compactionWriteSizeInTotal.add(value, newAttributesBuilder().build()); + compactionWriteSizeInTotal.add(value, AttributesCache.INSTANCE.defaultAttributes()); } } diff --git a/s3stream/src/test/java/com/automq/stream/s3/metrics/AttributesCacheTest.java b/s3stream/src/test/java/com/automq/stream/s3/metrics/AttributesCacheTest.java new file mode 100644 index 000000000..1ad09fba9 --- /dev/null +++ b/s3stream/src/test/java/com/automq/stream/s3/metrics/AttributesCacheTest.java @@ -0,0 +1,59 @@ +/* + * 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.metrics; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class AttributesCacheTest { + + @Test + public void testGetObjectSizeBucket() { + AttributesCache cache = AttributesCache.INSTANCE; + Assertions.assertEquals("16KB", cache.getObjectBucketLabel(8 * 1024)); + Assertions.assertEquals("16KB", cache.getObjectBucketLabel(16 * 1024)); + Assertions.assertEquals("32KB", cache.getObjectBucketLabel(17 * 1024)); + Assertions.assertEquals("32KB", cache.getObjectBucketLabel(32 * 1024)); + Assertions.assertEquals("64KB", cache.getObjectBucketLabel(33 * 1024)); + Assertions.assertEquals("64KB", cache.getObjectBucketLabel(64 * 1024)); + Assertions.assertEquals("128KB", cache.getObjectBucketLabel(65 * 1024)); + Assertions.assertEquals("128KB", cache.getObjectBucketLabel(128 * 1024)); + Assertions.assertEquals("256KB", cache.getObjectBucketLabel(129 * 1024)); + Assertions.assertEquals("256KB", cache.getObjectBucketLabel(256 * 1024)); + Assertions.assertEquals("512KB", cache.getObjectBucketLabel(257 * 1024)); + Assertions.assertEquals("512KB", cache.getObjectBucketLabel(512 * 1024)); + Assertions.assertEquals("1MB", cache.getObjectBucketLabel(513 * 1024)); + Assertions.assertEquals("1MB", cache.getObjectBucketLabel(1024 * 1024)); + Assertions.assertEquals("2MB", cache.getObjectBucketLabel(1025 * 1024)); + Assertions.assertEquals("2MB", cache.getObjectBucketLabel(2 * 1024 * 1024)); + Assertions.assertEquals("4MB", cache.getObjectBucketLabel(2 * 1024 * 1024 + 1)); + Assertions.assertEquals("4MB", cache.getObjectBucketLabel(4 * 1024 * 1024)); + Assertions.assertEquals("8MB", cache.getObjectBucketLabel(4 * 1024 * 1024 + 1)); + Assertions.assertEquals("8MB", cache.getObjectBucketLabel(8 * 1024 * 1024)); + Assertions.assertEquals("16MB", cache.getObjectBucketLabel(8 * 1024 * 1024 + 1)); + Assertions.assertEquals("16MB", cache.getObjectBucketLabel(16 * 1024 * 1024)); + Assertions.assertEquals("32MB", cache.getObjectBucketLabel(16 * 1024 * 1024 + 1)); + Assertions.assertEquals("32MB", cache.getObjectBucketLabel(32 * 1024 * 1024)); + Assertions.assertEquals("64MB", cache.getObjectBucketLabel(32 * 1024 * 1024 + 1)); + Assertions.assertEquals("64MB", cache.getObjectBucketLabel(64 * 1024 * 1024)); + Assertions.assertEquals("128MB", cache.getObjectBucketLabel(64 * 1024 * 1024 + 1)); + Assertions.assertEquals("128MB", cache.getObjectBucketLabel(128 * 1024 * 1024)); + Assertions.assertEquals("inf", cache.getObjectBucketLabel(128 * 1024 * 1024 + 1)); + Assertions.assertEquals("inf", cache.getObjectBucketLabel(1024 * 1024 * 1024)); + } +} diff --git a/s3stream/src/test/java/com/automq/stream/s3/metrics/StreamMetricsManagerTest.java b/s3stream/src/test/java/com/automq/stream/s3/metrics/StreamMetricsManagerTest.java deleted file mode 100644 index fb9002e3e..000000000 --- a/s3stream/src/test/java/com/automq/stream/s3/metrics/StreamMetricsManagerTest.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * 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.metrics; - -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; - -public class StreamMetricsManagerTest { - - @Test - public void testGetObjectSizeBucket() { - Assertions.assertEquals("16KB", S3StreamMetricsManager.getObjectBucketLabel(8 * 1024)); - Assertions.assertEquals("16KB", S3StreamMetricsManager.getObjectBucketLabel(16 * 1024)); - Assertions.assertEquals("32KB", S3StreamMetricsManager.getObjectBucketLabel(17 * 1024)); - Assertions.assertEquals("32KB", S3StreamMetricsManager.getObjectBucketLabel(32 * 1024)); - Assertions.assertEquals("64KB", S3StreamMetricsManager.getObjectBucketLabel(33 * 1024)); - Assertions.assertEquals("64KB", S3StreamMetricsManager.getObjectBucketLabel(64 * 1024)); - Assertions.assertEquals("128KB", S3StreamMetricsManager.getObjectBucketLabel(65 * 1024)); - Assertions.assertEquals("128KB", S3StreamMetricsManager.getObjectBucketLabel(128 * 1024)); - Assertions.assertEquals("256KB", S3StreamMetricsManager.getObjectBucketLabel(129 * 1024)); - Assertions.assertEquals("256KB", S3StreamMetricsManager.getObjectBucketLabel(256 * 1024)); - Assertions.assertEquals("512KB", S3StreamMetricsManager.getObjectBucketLabel(257 * 1024)); - Assertions.assertEquals("512KB", S3StreamMetricsManager.getObjectBucketLabel(512 * 1024)); - Assertions.assertEquals("1MB", S3StreamMetricsManager.getObjectBucketLabel(513 * 1024)); - Assertions.assertEquals("1MB", S3StreamMetricsManager.getObjectBucketLabel(1024 * 1024)); - Assertions.assertEquals("2MB", S3StreamMetricsManager.getObjectBucketLabel(1025 * 1024)); - Assertions.assertEquals("2MB", S3StreamMetricsManager.getObjectBucketLabel(2 * 1024 * 1024)); - Assertions.assertEquals("4MB", S3StreamMetricsManager.getObjectBucketLabel(2 * 1024 * 1024 + 1)); - Assertions.assertEquals("4MB", S3StreamMetricsManager.getObjectBucketLabel(4 * 1024 * 1024)); - Assertions.assertEquals("8MB", S3StreamMetricsManager.getObjectBucketLabel(4 * 1024 * 1024 + 1)); - Assertions.assertEquals("8MB", S3StreamMetricsManager.getObjectBucketLabel(8 * 1024 * 1024)); - Assertions.assertEquals("16MB", S3StreamMetricsManager.getObjectBucketLabel(8 * 1024 * 1024 + 1)); - Assertions.assertEquals("16MB", S3StreamMetricsManager.getObjectBucketLabel(16 * 1024 * 1024)); - Assertions.assertEquals("32MB", S3StreamMetricsManager.getObjectBucketLabel(16 * 1024 * 1024 + 1)); - Assertions.assertEquals("32MB", S3StreamMetricsManager.getObjectBucketLabel(32 * 1024 * 1024)); - Assertions.assertEquals("64MB", S3StreamMetricsManager.getObjectBucketLabel(32 * 1024 * 1024 + 1)); - Assertions.assertEquals("64MB", S3StreamMetricsManager.getObjectBucketLabel(64 * 1024 * 1024)); - Assertions.assertEquals("128MB", S3StreamMetricsManager.getObjectBucketLabel(64 * 1024 * 1024 + 1)); - Assertions.assertEquals("128MB", S3StreamMetricsManager.getObjectBucketLabel(128 * 1024 * 1024)); - Assertions.assertEquals("inf", S3StreamMetricsManager.getObjectBucketLabel(128 * 1024 * 1024 + 1)); - Assertions.assertEquals("inf", S3StreamMetricsManager.getObjectBucketLabel(1024 * 1024 * 1024)); - } -}