提交 1fa4508e 编写于 作者: R Rajan 提交者: GitHub

reduce collection instances while generating metrics (#284)

reduce collection instances while generating metrics
上级 1d5ec83a
...@@ -15,13 +15,9 @@ ...@@ -15,13 +15,9 @@
*/ */
package com.yahoo.pulsar.broker.stats; package com.yahoo.pulsar.broker.stats;
import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Map.Entry;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import com.yahoo.pulsar.common.policies.data.PersistentSubscriptionStats;
import com.yahoo.pulsar.common.policies.data.ReplicatorStats;
public class NamespaceStats { public class NamespaceStats {
......
...@@ -168,7 +168,7 @@ abstract class AbstractMetrics { ...@@ -168,7 +168,7 @@ abstract class AbstractMetrics {
return createMetrics(dimensionMap); return createMetrics(dimensionMap);
} }
protected void populateBucketEntries(Map<String, List<Double>> map, String mkey, double[] boundaries, protected void populateBucketEntries(Map<String, Double> map, String mkey, double[] boundaries,
long[] bucketValues) { long[] bucketValues) {
// bucket values should be one more that the boundaries to have the last element as OVERFLOW // bucket values should be one more that the boundaries to have the last element as OVERFLOW
...@@ -191,11 +191,8 @@ abstract class AbstractMetrics { ...@@ -191,11 +191,8 @@ abstract class AbstractMetrics {
value = (bucketValues == null) ? 0.0D : (double) bucketValues[i]; value = (bucketValues == null) ? 0.0D : (double) bucketValues[i];
if (!map.containsKey(bucketKey)) { Double val = map.getOrDefault(bucketKey, 0.0);
map.put(bucketKey, Lists.newArrayList(value)); map.put(bucketKey, val + value);
} else {
map.get(bucketKey).add(value);
}
} }
} }
...@@ -207,6 +204,11 @@ abstract class AbstractMetrics { ...@@ -207,6 +204,11 @@ abstract class AbstractMetrics {
} }
} }
protected void populateAggregationMapWithSum(Map<String, Double> map, String mkey, double value) {
Double val = map.getOrDefault(mkey, 0.0);
map.put(mkey, val + value);
}
protected void populateMaxMap(Map<String, Long> map, String mkey, long value) { protected void populateMaxMap(Map<String, Long> map, String mkey, long value) {
Long existingValue = map.get(mkey); Long existingValue = map.get(mkey);
if (existingValue == null || value > existingValue) { if (existingValue == null || value > existingValue) {
......
...@@ -31,12 +31,14 @@ import io.netty.buffer.PooledByteBufAllocator; ...@@ -31,12 +31,14 @@ import io.netty.buffer.PooledByteBufAllocator;
public class ManagedLedgerCacheMetrics extends AbstractMetrics { public class ManagedLedgerCacheMetrics extends AbstractMetrics {
private List<Metrics> metrics;
public ManagedLedgerCacheMetrics(PulsarService pulsar) { public ManagedLedgerCacheMetrics(PulsarService pulsar) {
super(pulsar); super(pulsar);
this.metrics = Lists.newArrayList();
} }
@Override @Override
public List<Metrics> generate() { public synchronized List<Metrics> generate() {
// get the ML cache stats bean // get the ML cache stats bean
...@@ -87,7 +89,9 @@ public class ManagedLedgerCacheMetrics extends AbstractMetrics { ...@@ -87,7 +89,9 @@ public class ManagedLedgerCacheMetrics extends AbstractMetrics {
m.put("brk_ml_cache_pool_active_allocations_normal", activeAllocationsNormal); m.put("brk_ml_cache_pool_active_allocations_normal", activeAllocationsNormal);
m.put("brk_ml_cache_pool_active_allocations_huge", activeAllocationsHuge); m.put("brk_ml_cache_pool_active_allocations_huge", activeAllocationsHuge);
return Lists.newArrayList(m); metrics.clear();
metrics.add(m);
return metrics;
} }
......
...@@ -29,87 +29,87 @@ import com.yahoo.pulsar.broker.stats.Metrics; ...@@ -29,87 +29,87 @@ import com.yahoo.pulsar.broker.stats.Metrics;
public class ManagedLedgerMetrics extends AbstractMetrics { public class ManagedLedgerMetrics extends AbstractMetrics {
private List<Metrics> metricsCollection;
private Map<Metrics, List<ManagedLedgerImpl>> ledgersByDimensionMap;
// temp map to prepare aggregation metrics
private Map<String, Double> tempAggregatedMetricsMap;
public ManagedLedgerMetrics(PulsarService pulsar) { public ManagedLedgerMetrics(PulsarService pulsar) {
super(pulsar); super(pulsar);
this.metricsCollection = Lists.newArrayList();
this.ledgersByDimensionMap = Maps.newHashMap();
this.tempAggregatedMetricsMap = Maps.newHashMap();
} }
@Override @Override
public List<Metrics> generate() { public synchronized List<Metrics> generate() {
// get the current snapshot of ledgers by NS dimension // get the current snapshot of ledgers by NS dimension
return aggregate(groupLedgersByDimension());
Map<Metrics, List<ManagedLedgerImpl>> ledgersByDimension = groupLedgersByDimension();
return aggregate(ledgersByDimension);
} }
/** /**
* Aggregation by namespace * Aggregation by namespace (not thread-safe)
* *
* @param ledgersByDimension * @param ledgersByDimension
* @return * @return
*/ */
private List<Metrics> aggregate(Map<Metrics, List<ManagedLedgerImpl>> ledgersByDimension) { private List<Metrics> aggregate(Map<Metrics, List<ManagedLedgerImpl>> ledgersByDimension) {
List<Metrics> metricsCollection = Lists.newArrayList(); metricsCollection.clear();
for (Entry<Metrics, List<ManagedLedgerImpl>> e : ledgersByDimension.entrySet()) { for (Entry<Metrics, List<ManagedLedgerImpl>> e : ledgersByDimension.entrySet()) {
Metrics metrics = e.getKey(); Metrics metrics = e.getKey();
List<ManagedLedgerImpl> ledgers = e.getValue(); List<ManagedLedgerImpl> ledgers = e.getValue();
// prepare aggregation map // prepare aggregation map
tempAggregatedMetricsMap.clear();
Map<String, List<Double>> aggregatedMetricsMap = Maps.newHashMap();
// generate the collections by each metrics and then apply the aggregation // generate the collections by each metrics and then apply the aggregation
for (ManagedLedgerImpl ledger : ledgers) { for (ManagedLedgerImpl ledger : ledgers) {
ManagedLedgerMXBean lStats = ledger.getStats(); ManagedLedgerMXBean lStats = ledger.getStats();
populateAggregationMap(aggregatedMetricsMap, "brk_ml_AddEntryBytesRate", lStats.getAddEntryBytesRate()); populateAggregationMapWithSum(tempAggregatedMetricsMap, "brk_ml_AddEntryBytesRate", lStats.getAddEntryBytesRate());
populateAggregationMap(aggregatedMetricsMap, "brk_ml_AddEntryErrors", populateAggregationMapWithSum(tempAggregatedMetricsMap, "brk_ml_AddEntryErrors",
(double) lStats.getAddEntryErrors()); (double) lStats.getAddEntryErrors());
populateAggregationMap(aggregatedMetricsMap, "brk_ml_AddEntryMessagesRate", populateAggregationMapWithSum(tempAggregatedMetricsMap, "brk_ml_AddEntryMessagesRate",
lStats.getAddEntryMessagesRate()); lStats.getAddEntryMessagesRate());
populateAggregationMap(aggregatedMetricsMap, "brk_ml_AddEntrySucceed", populateAggregationMapWithSum(tempAggregatedMetricsMap, "brk_ml_AddEntrySucceed",
(double) lStats.getAddEntrySucceed()); (double) lStats.getAddEntrySucceed());
populateAggregationMap(aggregatedMetricsMap, "brk_ml_NumberOfMessagesInBacklog", populateAggregationMapWithSum(tempAggregatedMetricsMap, "brk_ml_NumberOfMessagesInBacklog",
(double) lStats.getNumberOfMessagesInBacklog()); (double) lStats.getNumberOfMessagesInBacklog());
populateAggregationMap(aggregatedMetricsMap, "brk_ml_ReadEntriesBytesRate", populateAggregationMapWithSum(tempAggregatedMetricsMap, "brk_ml_ReadEntriesBytesRate",
lStats.getReadEntriesBytesRate()); lStats.getReadEntriesBytesRate());
populateAggregationMap(aggregatedMetricsMap, "brk_ml_ReadEntriesErrors", populateAggregationMapWithSum(tempAggregatedMetricsMap, "brk_ml_ReadEntriesErrors",
(double) lStats.getReadEntriesErrors()); (double) lStats.getReadEntriesErrors());
populateAggregationMap(aggregatedMetricsMap, "brk_ml_ReadEntriesRate", lStats.getReadEntriesRate()); populateAggregationMapWithSum(tempAggregatedMetricsMap, "brk_ml_ReadEntriesRate", lStats.getReadEntriesRate());
populateAggregationMap(aggregatedMetricsMap, "brk_ml_ReadEntriesSucceeded", populateAggregationMapWithSum(tempAggregatedMetricsMap, "brk_ml_ReadEntriesSucceeded",
(double) lStats.getReadEntriesSucceeded()); (double) lStats.getReadEntriesSucceeded());
populateAggregationMap(aggregatedMetricsMap, "brk_ml_StoredMessagesSize", populateAggregationMapWithSum(tempAggregatedMetricsMap, "brk_ml_StoredMessagesSize",
(double) lStats.getStoredMessagesSize()); (double) lStats.getStoredMessagesSize());
// handle bucket entries initialization here // handle bucket entries initialization here
populateBucketEntries(aggregatedMetricsMap, "brk_ml_AddEntryLatencyBuckets", ENTRY_LATENCY_BUCKETS_MS, populateBucketEntries(tempAggregatedMetricsMap, "brk_ml_AddEntryLatencyBuckets",
lStats.getAddEntryLatencyBuckets()); ENTRY_LATENCY_BUCKETS_MS, lStats.getAddEntryLatencyBuckets());
populateBucketEntries(aggregatedMetricsMap, "brk_ml_LedgerSwitchLatencyBuckets", populateBucketEntries(tempAggregatedMetricsMap, "brk_ml_LedgerSwitchLatencyBuckets",
ENTRY_LATENCY_BUCKETS_MS, lStats.getLedgerSwitchLatencyBuckets()); ENTRY_LATENCY_BUCKETS_MS, lStats.getLedgerSwitchLatencyBuckets());
populateBucketEntries(aggregatedMetricsMap, "brk_ml_EntrySizeBuckets", ENTRY_SIZE_BUCKETS_BYTES, populateBucketEntries(tempAggregatedMetricsMap, "brk_ml_EntrySizeBuckets", ENTRY_SIZE_BUCKETS_BYTES,
lStats.getEntrySizeBuckets()); lStats.getEntrySizeBuckets());
populateAggregationMap(aggregatedMetricsMap, "brk_ml_MarkDeleteRate", lStats.getMarkDeleteRate()); populateAggregationMapWithSum(tempAggregatedMetricsMap, "brk_ml_MarkDeleteRate", lStats.getMarkDeleteRate());
} }
// SUM up collections of each metrics // SUM up collections of each metrics
for (Entry<String, List<Double>> ma : aggregatedMetricsMap.entrySet()) { for (Entry<String, Double> ma : tempAggregatedMetricsMap.entrySet()) {
// sum metrics.put(ma.getKey(), ma.getValue());
String metricsName = ma.getKey();
Double metricsValue = sum(ma.getValue());
metrics.put(metricsName, metricsValue);
} }
metricsCollection.add(metrics); metricsCollection.add(metrics);
...@@ -119,21 +119,19 @@ public class ManagedLedgerMetrics extends AbstractMetrics { ...@@ -119,21 +119,19 @@ public class ManagedLedgerMetrics extends AbstractMetrics {
} }
/** /**
* Build a map of dimensions key to list of destination stats * Build a map of dimensions key to list of destination stats (not thread-safe)
* <p> * <p>
* *
* @return * @return
*/ */
private Map<Metrics, List<ManagedLedgerImpl>> groupLedgersByDimension() { private Map<Metrics, List<ManagedLedgerImpl>> groupLedgersByDimension() {
Map<Metrics, List<ManagedLedgerImpl>> ledgersByDimensionMap = Maps.newHashMap(); ledgersByDimensionMap.clear();
// get the current destinations statistics from StatsBrokerFilter // get the current destinations statistics from StatsBrokerFilter
// Map : destination-name->dest-stat // Map : destination-name->dest-stat
Map<String, ManagedLedgerImpl> ledgersMap = getManagedLedgers(); for (Entry<String, ManagedLedgerImpl> e : getManagedLedgers().entrySet()) {
for (Entry<String, ManagedLedgerImpl> e : ledgersMap.entrySet()) {
String ledgerName = e.getKey(); String ledgerName = e.getKey();
ManagedLedgerImpl ledger = e.getValue(); ManagedLedgerImpl ledger = e.getValue();
......
/**
* Copyright 2016 Yahoo Inc.
*
* Licensed 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.yahoo.pulsar.broker.stats;
import java.util.List;
import java.util.Map.Entry;
import java.util.concurrent.TimeUnit;
import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl;
import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
import org.apache.bookkeeper.mledger.impl.ManagedLedgerMBeanImpl;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;
import com.yahoo.pulsar.broker.service.BrokerTestBase;
import com.yahoo.pulsar.broker.stats.metrics.ManagedLedgerMetrics;
import com.yahoo.pulsar.client.api.Producer;
import junit.framework.Assert;
/**
*/
public class ManagedLedgerMetricsTest extends BrokerTestBase {
@BeforeClass
@Override
protected void setup() throws Exception {
super.baseSetup();
}
@AfterClass
@Override
protected void cleanup() throws Exception {
super.internalCleanup();
}
@Test
public void testManagedLedgerMetrics() throws Exception {
ManagedLedgerMetrics metrics = new ManagedLedgerMetrics(pulsar);
final String addEntryRateKey = "brk_ml_AddEntryMessagesRate";
List<Metrics> list1 = metrics.generate();
Assert.assertTrue(list1.isEmpty());
Producer producer = pulsarClient.createProducer("persistent://my-property/use/my-ns/my-topic1");
for (int i = 0; i < 10; i++) {
String message = "my-message-" + i;
producer.send(message.getBytes());
}
for (Entry<String, ManagedLedgerImpl> ledger : ((ManagedLedgerFactoryImpl) pulsar.getManagedLedgerFactory())
.getManagedLedgers().entrySet()) {
ManagedLedgerMBeanImpl stats = (ManagedLedgerMBeanImpl) ledger.getValue().getStats();
stats.refreshStats(1, TimeUnit.SECONDS);
}
List<Metrics> list2 = metrics.generate();
Assert.assertEquals(list2.get(0).getMetrics().get(addEntryRateKey), 10.0D);
for (int i = 0; i < 5; i++) {
String message = "my-message-" + i;
producer.send(message.getBytes());
}
for (Entry<String, ManagedLedgerImpl> ledger : ((ManagedLedgerFactoryImpl) pulsar.getManagedLedgerFactory())
.getManagedLedgers().entrySet()) {
ManagedLedgerMBeanImpl stats = (ManagedLedgerMBeanImpl) ledger.getValue().getStats();
stats.refreshStats(1, TimeUnit.SECONDS);
}
List<Metrics> list3 = metrics.generate();
Assert.assertEquals(list3.get(0).getMetrics().get(addEntryRateKey), 5.0D);
}
}
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册