package org.apache.cassandra.metrics;
import java.nio.ByteBuffer;
import java.util.*;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.TimeUnit;
import com.google.common.collect.Maps;
import com.codahale.metrics.*;
import com.codahale.metrics.Timer;
import org.apache.cassandra.config.Schema;
import org.apache.cassandra.config.SchemaConstants;
import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.Keyspace;
import org.apache.cassandra.db.Memtable;
import org.apache.cassandra.db.lifecycle.SSTableSet;
import org.apache.cassandra.index.SecondaryIndexManager;
import org.apache.cassandra.io.compress.CompressionMetadata;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
import org.apache.cassandra.utils.EstimatedHistogram;
import org.apache.cassandra.utils.TopKSampler;
import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
public class TableMetrics
{
public static final long[] EMPTY = new long[0];
public final Gauge<Long> memtableOnHeapSize;
public final Gauge<Long> memtableOffHeapSize;
public final Gauge<Long> memtableLiveDataSize;
public final Gauge<Long> allMemtablesOnHeapSize;
public final Gauge<Long> allMemtablesOffHeapSize;
public final Gauge<Long> allMemtablesLiveDataSize;
public final Gauge<Long> memtableColumnsCount;
public final Counter memtableSwitchCount;
public final Gauge<Double> compressionRatio;
public final Gauge<long[]> estimatedPartitionSizeHistogram;
public final Gauge<Long> estimatedPartitionCount;
public final Gauge<long[]> estimatedColumnCountHistogram;
public final TableHistogram sstablesPerReadHistogram;
public final LatencyMetrics readLatency;
public final LatencyMetrics rangeLatency;
public final LatencyMetrics writeLatency;
public final Counter pendingFlushes;
public final Counter bytesFlushed;
public final Counter compactionBytesWritten;
public final Gauge<Integer> pendingCompactions;
public final Gauge<Integer> liveSSTableCount;
public final Counter liveDiskSpaceUsed;
public final Counter totalDiskSpaceUsed;
public final Gauge<Long> minPartitionSize;
public final Gauge<Long> maxPartitionSize;
public final Gauge<Long> meanPartitionSize;
public final Gauge<Long> bloomFilterFalsePositives;
public final Gauge<Long> recentBloomFilterFalsePositives;
public final Gauge<Double> bloomFilterFalseRatio;
public final Gauge<Double> recentBloomFilterFalseRatio;
public final Gauge<Long> bloomFilterDiskSpaceUsed;
public final Gauge<Long> bloomFilterOffHeapMemoryUsed;
public final Gauge<Long> indexSummaryOffHeapMemoryUsed;
public final Gauge<Long> compressionMetadataOffHeapMemoryUsed;
public final Gauge<Double> keyCacheHitRate;
public final TableHistogram tombstoneScannedHistogram;
public final TableHistogram liveScannedHistogram;
public final TableHistogram colUpdateTimeDeltaHistogram;
public final TableTimer viewLockAcquireTime;
public final TableTimer viewReadTime;
public final Gauge<Long> trueSnapshotsSize;
public final Counter rowCacheHitOutOfRange;
public final Counter rowCacheHit;
public final Counter rowCacheMiss;
public final LatencyMetrics casPrepare;
public final LatencyMetrics casPropose;
public final LatencyMetrics casCommit;
public final Gauge<Double> percentRepaired;
public final Timer coordinatorReadLatency;
public final Timer coordinatorScanLatency;
public final Histogram waitingOnFreeMemtableSpace;
public final Counter droppedMutations;
private final MetricNameFactory factory;
private final MetricNameFactory aliasFactory;
private static final MetricNameFactory globalFactory = new AllTableMetricNameFactory("Table");
private static final MetricNameFactory globalAliasFactory = new AllTableMetricNameFactory("ColumnFamily");
public final Counter speculativeRetries;
public final static LatencyMetrics globalReadLatency = new LatencyMetrics(globalFactory, globalAliasFactory, "Read");
public final static LatencyMetrics globalWriteLatency = new LatencyMetrics(globalFactory, globalAliasFactory, "Write");
public final static LatencyMetrics globalRangeLatency = new LatencyMetrics(globalFactory, globalAliasFactory, "Range");
public final static Gauge<Double> globalPercentRepaired = Metrics.register(globalFactory.createMetricName("PercentRepaired"),
new Gauge<Double>()
{
public Double getValue()
{
double repaired = 0;
double total = 0;
for (String keyspace : Schema.instance.getNonSystemKeyspaces())
{
Keyspace k = Schema.instance.getKeyspaceInstance(keyspace);
if (SchemaConstants.DISTRIBUTED_KEYSPACE_NAME.equals(k.getName()))
continue;
if (k.getReplicationStrategy().getReplicationFactor() < 2)
continue;
for (ColumnFamilyStore cf : k.getColumnFamilyStores())
{
if (!SecondaryIndexManager.isIndexColumnFamily(cf.name))
{
for (SSTableReader sstable : cf.getSSTables(SSTableSet.CANONICAL))
{
if (sstable.isRepaired())
{
repaired += sstable.uncompressedLength();
}
total += sstable.uncompressedLength();
}
}
}
}
return total > 0 ? (repaired / total) * 100 : 100.0;
}
});
public final Meter readRepairRequests;
public final Meter shortReadProtectionRequests;
public final Map<Sampler, TopKSampler<ByteBuffer>> samplers;
public final static ConcurrentMap<String, Set<Metric>> allTableMetrics = Maps.newConcurrentMap();
public final static Map<String, String> all = Maps.newHashMap();
private interface GetHistogram
{
EstimatedHistogram getHistogram(SSTableReader reader);
}
private static long[] combineHistograms(Iterable<SSTableReader> sstables, GetHistogram getHistogram)
{
Iterator<SSTableReader> iterator = sstables.iterator();
if (!iterator.hasNext())
{
return EMPTY;
}
long[] firstBucket = getHistogram.getHistogram(iterator.next()).getBuckets(false);
long[] values = new long[firstBucket.length];
System.arraycopy(firstBucket, 0, values, 0, values.length);
while (iterator.hasNext())
{
long[] nextBucket = getHistogram.getHistogram(iterator.next()).getBuckets(false);
if (nextBucket.length > values.length)
{
long[] newValues = new long[nextBucket.length];
System.arraycopy(firstBucket, 0, newValues, 0, firstBucket.length);
for (int i = 0; i < newValues.length; i++)
{
newValues[i] += nextBucket[i];
}
values = newValues;
}
else
{
for (int i = 0; i < values.length; i++)
{
values[i] += nextBucket[i];
}
}
}
return values;
}
public TableMetrics(final ColumnFamilyStore cfs)
{
factory = new TableMetricNameFactory(cfs, "Table");
aliasFactory = new TableMetricNameFactory(cfs, "ColumnFamily");
samplers = Maps.newHashMap();
for (Sampler sampler : Sampler.values())
{
samplers.put(sampler, new TopKSampler<>());
}
memtableColumnsCount = createTableGauge("MemtableColumnsCount", new Gauge<Long>()
{
public Long getValue()
{
return cfs.getTracker().getView().getCurrentMemtable().getOperations();
}
});
memtableOnHeapSize = createTableGauge("MemtableOnHeapSize", new Gauge<Long>()
{
public Long getValue()
{
return cfs.getTracker().getView().getCurrentMemtable().getAllocator().onHeap().owns();
}
});
memtableOffHeapSize = createTableGauge("MemtableOffHeapSize", new Gauge<Long>()
{
public Long getValue()
{
return cfs.getTracker().getView().getCurrentMemtable().getAllocator().offHeap().owns();
}
});
memtableLiveDataSize = createTableGauge("MemtableLiveDataSize", new Gauge<Long>()
{
public Long getValue()
{
return cfs.getTracker().getView().getCurrentMemtable().getLiveDataSize();
}
});
allMemtablesOnHeapSize = createTableGauge("AllMemtablesHeapSize", new Gauge<Long>()
{
public Long getValue()
{
long size = 0;
for (ColumnFamilyStore cfs2 : cfs.concatWithIndexes())
size += cfs2.getTracker().getView().getCurrentMemtable().getAllocator().onHeap().owns();
return size;
}
});
allMemtablesOffHeapSize = createTableGauge("AllMemtablesOffHeapSize", new Gauge<Long>()
{
public Long getValue()
{
long size = 0;
for (ColumnFamilyStore cfs2 : cfs.concatWithIndexes())
size += cfs2.getTracker().getView().getCurrentMemtable().getAllocator().offHeap().owns();
return size;
}
});
allMemtablesLiveDataSize = createTableGauge("AllMemtablesLiveDataSize", new Gauge<Long>()
{
public Long getValue()
{
long size = 0;
for (ColumnFamilyStore cfs2 : cfs.concatWithIndexes())
size += cfs2.getTracker().getView().getCurrentMemtable().getLiveDataSize();
return size;
}
});
memtableSwitchCount = createTableCounter("MemtableSwitchCount");
estimatedPartitionSizeHistogram = Metrics.register(factory.createMetricName("EstimatedPartitionSizeHistogram"),
aliasFactory.createMetricName("EstimatedRowSizeHistogram"),
new Gauge<long[]>()
{
public long[] getValue()
{
return combineHistograms(cfs.getSSTables(SSTableSet.CANONICAL), new GetHistogram()
{
public EstimatedHistogram getHistogram(SSTableReader reader)
{
return reader.getEstimatedPartitionSize();
}
});
}
});
estimatedPartitionCount = Metrics.register(factory.createMetricName("EstimatedPartitionCount"),
aliasFactory.createMetricName("EstimatedRowCount"),
new Gauge<Long>()
{
public Long getValue()
{
long memtablePartitions = 0;
for (Memtable memtable : cfs.getTracker().getView().getAllMemtables())
memtablePartitions += memtable.partitionCount();
return SSTableReader.getApproximateKeyCount(cfs.getSSTables(SSTableSet.CANONICAL)) + memtablePartitions;
}
});
estimatedColumnCountHistogram = Metrics.register(factory.createMetricName("EstimatedColumnCountHistogram"),
aliasFactory.createMetricName("EstimatedColumnCountHistogram"),
new Gauge<long[]>()
{
public long[] getValue()
{
return combineHistograms(cfs.getSSTables(SSTableSet.CANONICAL), new GetHistogram()
{
public EstimatedHistogram getHistogram(SSTableReader reader)
{
return reader.getEstimatedColumnCount();
}
});
}
});
sstablesPerReadHistogram = createTableHistogram("SSTablesPerReadHistogram", cfs.keyspace.metric.sstablesPerReadHistogram, true);
compressionRatio = createTableGauge("CompressionRatio", new Gauge<Double>()
{
public Double getValue()
{
return computeCompressionRatio(cfs.getSSTables(SSTableSet.CANONICAL));
}
}, new Gauge<Double>()
{
public Double getValue()
{
List<SSTableReader> sstables = new ArrayList<>();
Keyspace.all().forEach(ks -> sstables.addAll(ks.getAllSSTables(SSTableSet.CANONICAL)));
return computeCompressionRatio(sstables);
}
});
percentRepaired = createTableGauge("PercentRepaired", new Gauge<Double>()
{
public Double getValue()
{
double repaired = 0;
double total = 0;
for (SSTableReader sstable : cfs.getSSTables(SSTableSet.CANONICAL))
{
if (sstable.isRepaired())
{
repaired += sstable.uncompressedLength();
}
total += sstable.uncompressedLength();
}
return total > 0 ? (repaired / total) * 100 : 100.0;
}
});
readLatency = new LatencyMetrics(factory, "Read", cfs.keyspace.metric.readLatency, globalReadLatency);
writeLatency = new LatencyMetrics(factory, "Write", cfs.keyspace.metric.writeLatency, globalWriteLatency);
rangeLatency = new LatencyMetrics(factory, "Range", cfs.keyspace.metric.rangeLatency, globalRangeLatency);
pendingFlushes = createTableCounter("PendingFlushes");
bytesFlushed = createTableCounter("BytesFlushed");
compactionBytesWritten = createTableCounter("CompactionBytesWritten");
pendingCompactions = createTableGauge("PendingCompactions", new Gauge<Integer>()
{
public Integer getValue()
{
return cfs.getCompactionStrategyManager().getEstimatedRemainingTasks();
}
});
liveSSTableCount = createTableGauge("LiveSSTableCount", new Gauge<Integer>()
{
public Integer getValue()
{
return cfs.getTracker().getView().liveSSTables().size();
}
});
liveDiskSpaceUsed = createTableCounter("LiveDiskSpaceUsed");
totalDiskSpaceUsed = createTableCounter("TotalDiskSpaceUsed");
minPartitionSize = createTableGauge("MinPartitionSize", "MinRowSize", new Gauge<Long>()
{
public Long getValue()
{
long min = 0;
for (SSTableReader sstable : cfs.getSSTables(SSTableSet.CANONICAL))
{
if (min == 0 || sstable.getEstimatedPartitionSize().min() < min)
min = sstable.getEstimatedPartitionSize().min();
}
return min;
}
}, new Gauge<Long>()
{
public Long getValue()
{
long min = Long.MAX_VALUE;
for (Metric cfGauge : allTableMetrics.get("MinPartitionSize"))
{
min = Math.min(min, ((Gauge<? extends Number>) cfGauge).getValue().longValue());
}
return min;
}
});
maxPartitionSize = createTableGauge("MaxPartitionSize", "MaxRowSize", new Gauge<Long>()
{
public Long getValue()
{
long max = 0;
for (SSTableReader sstable : cfs.getSSTables(SSTableSet.CANONICAL))
{
if (sstable.getEstimatedPartitionSize().max() > max)
max = sstable.getEstimatedPartitionSize().max();
}
return max;
}
}, new Gauge<Long>()
{
public Long getValue()
{
long max = 0;
for (Metric cfGauge : allTableMetrics.get("MaxPartitionSize"))
{
max = Math.max(max, ((Gauge<? extends Number>) cfGauge).getValue().longValue());
}
return max;
}
});
meanPartitionSize = createTableGauge("MeanPartitionSize", "MeanRowSize", new Gauge<Long>()
{
public Long getValue()
{
long sum = 0;
long count = 0;
for (SSTableReader sstable : cfs.getSSTables(SSTableSet.CANONICAL))
{
long n = sstable.getEstimatedPartitionSize().count();
sum += sstable.getEstimatedPartitionSize().mean() * n;
count += n;
}
return count > 0 ? sum / count : 0;
}
}, new Gauge<Long>()
{
public Long getValue()
{
long sum = 0;
long count = 0;
for (Keyspace keyspace : Keyspace.all())
{
for (SSTableReader sstable : keyspace.getAllSSTables(SSTableSet.CANONICAL))
{
long n = sstable.getEstimatedPartitionSize().count();
sum += sstable.getEstimatedPartitionSize().mean() * n;
count += n;
}
}
return count > 0 ? sum / count : 0;
}
});
bloomFilterFalsePositives = createTableGauge("BloomFilterFalsePositives", new Gauge<Long>()
{
public Long getValue()
{
long count = 0L;
for (SSTableReader sstable: cfs.getSSTables(SSTableSet.LIVE))
count += sstable.getBloomFilterFalsePositiveCount();
return count;
}
});
recentBloomFilterFalsePositives = createTableGauge("RecentBloomFilterFalsePositives", new Gauge<Long>()
{
public Long getValue()
{
long count = 0L;
for (SSTableReader sstable : cfs.getSSTables(SSTableSet.LIVE))
count += sstable.getRecentBloomFilterFalsePositiveCount();
return count;
}
});
bloomFilterFalseRatio = createTableGauge("BloomFilterFalseRatio", new Gauge<Double>()
{
public Double getValue()
{
long falseCount = 0L;
long trueCount = 0L;
for (SSTableReader sstable : cfs.getSSTables(SSTableSet.LIVE))
{
falseCount += sstable.getBloomFilterFalsePositiveCount();
trueCount += sstable.getBloomFilterTruePositiveCount();
}
if (falseCount == 0L && trueCount == 0L)
return 0d;
return (double) falseCount / (trueCount + falseCount);
}
}, new Gauge<Double>()
{
public Double getValue()
{
long falseCount = 0L;
long trueCount = 0L;
for (Keyspace keyspace : Keyspace.all())
{
for (SSTableReader sstable : keyspace.getAllSSTables(SSTableSet.LIVE))
{
falseCount += sstable.getBloomFilterFalsePositiveCount();
trueCount += sstable.getBloomFilterTruePositiveCount();
}
}
if (falseCount == 0L && trueCount == 0L)
return 0d;
return (double) falseCount / (trueCount + falseCount);
}
});
recentBloomFilterFalseRatio = createTableGauge("RecentBloomFilterFalseRatio", new Gauge<Double>()
{
public Double getValue()
{
long falseCount = 0L;
long trueCount = 0L;
for (SSTableReader sstable: cfs.getSSTables(SSTableSet.LIVE))
{
falseCount += sstable.getRecentBloomFilterFalsePositiveCount();
trueCount += sstable.getRecentBloomFilterTruePositiveCount();
}
if (falseCount == 0L && trueCount == 0L)
return 0d;
return (double) falseCount / (trueCount + falseCount);
}
}, new Gauge<Double>()
{
public Double getValue()
{
long falseCount = 0L;
long trueCount = 0L;
for (Keyspace keyspace : Keyspace.all())
{
for (SSTableReader sstable : keyspace.getAllSSTables(SSTableSet.LIVE))
{
falseCount += sstable.getRecentBloomFilterFalsePositiveCount();
trueCount += sstable.getRecentBloomFilterTruePositiveCount();
}
}
if (falseCount == 0L && trueCount == 0L)
return 0d;
return (double) falseCount / (trueCount + falseCount);
}
});
bloomFilterDiskSpaceUsed = createTableGauge("BloomFilterDiskSpaceUsed", new Gauge<Long>()
{
public Long getValue()
{
long total = 0;
for (SSTableReader sst : cfs.getSSTables(SSTableSet.CANONICAL))
total += sst.getBloomFilterSerializedSize();
return total;
}
});
bloomFilterOffHeapMemoryUsed = createTableGauge("BloomFilterOffHeapMemoryUsed", new Gauge<Long>()
{
public Long getValue()
{
long total = 0;
for (SSTableReader sst : cfs.getSSTables(SSTableSet.LIVE))
total += sst.getBloomFilterOffHeapSize();
return total;
}
});
indexSummaryOffHeapMemoryUsed = createTableGauge("IndexSummaryOffHeapMemoryUsed", new Gauge<Long>()
{
public Long getValue()
{
long total = 0;
for (SSTableReader sst : cfs.getSSTables(SSTableSet.LIVE))
total += sst.getIndexSummaryOffHeapSize();
return total;
}
});
compressionMetadataOffHeapMemoryUsed = createTableGauge("CompressionMetadataOffHeapMemoryUsed", new Gauge<Long>()
{
public Long getValue()
{
long total = 0;
for (SSTableReader sst : cfs.getSSTables(SSTableSet.LIVE))
total += sst.getCompressionMetadataOffHeapSize();
return total;
}
});
speculativeRetries = createTableCounter("SpeculativeRetries");
keyCacheHitRate = Metrics.register(factory.createMetricName("KeyCacheHitRate"),
aliasFactory.createMetricName("KeyCacheHitRate"),
new RatioGauge()
{
@Override
public Ratio getRatio()
{
return Ratio.of(getNumerator(), getDenominator());
}
protected double getNumerator()
{
long hits = 0L;
for (SSTableReader sstable : cfs.getSSTables(SSTableSet.LIVE))
hits += sstable.getKeyCacheHit();
return hits;
}
protected double getDenominator()
{
long requests = 0L;
for (SSTableReader sstable : cfs.getSSTables(SSTableSet.LIVE))
requests += sstable.getKeyCacheRequest();
return Math.max(requests, 1);
}
});
tombstoneScannedHistogram = createTableHistogram("TombstoneScannedHistogram", cfs.keyspace.metric.tombstoneScannedHistogram, false);
liveScannedHistogram = createTableHistogram("LiveScannedHistogram", cfs.keyspace.metric.liveScannedHistogram, false);
colUpdateTimeDeltaHistogram = createTableHistogram("ColUpdateTimeDeltaHistogram", cfs.keyspace.metric.colUpdateTimeDeltaHistogram, false);
coordinatorReadLatency = Metrics.timer(factory.createMetricName("CoordinatorReadLatency"));
coordinatorScanLatency = Metrics.timer(factory.createMetricName("CoordinatorScanLatency"));
waitingOnFreeMemtableSpace = Metrics.histogram(factory.createMetricName("WaitingOnFreeMemtableSpace"), false);
if (cfs.metadata.isView())
{
viewLockAcquireTime = null;
viewReadTime = null;
}
else
{
viewLockAcquireTime = createTableTimer("ViewLockAcquireTime", cfs.keyspace.metric.viewLockAcquireTime);
viewReadTime = createTableTimer("ViewReadTime", cfs.keyspace.metric.viewReadTime);
}
trueSnapshotsSize = createTableGauge("SnapshotsSize", new Gauge<Long>()
{
public Long getValue()
{
return cfs.trueSnapshotsSize();
}
});
rowCacheHitOutOfRange = createTableCounter("RowCacheHitOutOfRange");
rowCacheHit = createTableCounter("RowCacheHit");
rowCacheMiss = createTableCounter("RowCacheMiss");
droppedMutations = createTableCounter("DroppedMutations");
casPrepare = new LatencyMetrics(factory, "CasPrepare", cfs.keyspace.metric.casPrepare);
casPropose = new LatencyMetrics(factory, "CasPropose", cfs.keyspace.metric.casPropose);
casCommit = new LatencyMetrics(factory, "CasCommit", cfs.keyspace.metric.casCommit);
readRepairRequests = Metrics.meter(factory.createMetricName("ReadRepairRequests"));
shortReadProtectionRequests = Metrics.meter(factory.createMetricName("ShortReadProtectionRequests"));
}
public void updateSSTableIterated(int count)
{
sstablesPerReadHistogram.update(count);
}
public void release()
{
for(Map.Entry<String, String> entry : all.entrySet())
{
CassandraMetricsRegistry.MetricName name = factory.createMetricName(entry.getKey());
CassandraMetricsRegistry.MetricName alias = aliasFactory.createMetricName(entry.getValue());
final Metric metric = Metrics.getMetrics().get(name.getMetricName());
if (metric != null)
{
allTableMetrics.get(entry.getKey()).remove(metric);
Metrics.remove(name, alias);
}
}
readLatency.release();
writeLatency.release();
rangeLatency.release();
Metrics.remove(factory.createMetricName("EstimatedPartitionSizeHistogram"), aliasFactory.createMetricName("EstimatedRowSizeHistogram"));
Metrics.remove(factory.createMetricName("EstimatedPartitionCount"), aliasFactory.createMetricName("EstimatedRowCount"));
Metrics.remove(factory.createMetricName("EstimatedColumnCountHistogram"), aliasFactory.createMetricName("EstimatedColumnCountHistogram"));
Metrics.remove(factory.createMetricName("KeyCacheHitRate"), aliasFactory.createMetricName("KeyCacheHitRate"));
Metrics.remove(factory.createMetricName("CoordinatorReadLatency"), aliasFactory.createMetricName("CoordinatorReadLatency"));
Metrics.remove(factory.createMetricName("CoordinatorScanLatency"), aliasFactory.createMetricName("CoordinatorScanLatency"));
Metrics.remove(factory.createMetricName("WaitingOnFreeMemtableSpace"), aliasFactory.createMetricName("WaitingOnFreeMemtableSpace"));
}
protected <T extends Number> Gauge<T> createTableGauge(final String name, Gauge<T> gauge)
{
return createTableGauge(name, gauge, new Gauge<Long>()
{
public Long getValue()
{
long total = 0;
for (Metric cfGauge : allTableMetrics.get(name))
{
total = total + ((Gauge<? extends Number>) cfGauge).getValue().longValue();
}
return total;
}
});
}
protected <G,T> Gauge<T> createTableGauge(String name, Gauge<T> gauge, Gauge<G> globalGauge)
{
return createTableGauge(name, name, gauge, globalGauge);
}
protected <G,T> Gauge<T> createTableGauge(String name, String alias, Gauge<T> gauge, Gauge<G> globalGauge)
{
Gauge<T> cfGauge = Metrics.register(factory.createMetricName(name), aliasFactory.createMetricName(alias), gauge);
if (register(name, alias, cfGauge))
{
Metrics.register(globalFactory.createMetricName(name), globalAliasFactory.createMetricName(alias), globalGauge);
}
return cfGauge;
}
protected Counter createTableCounter(final String name)
{
return createTableCounter(name, name);
}
protected Counter createTableCounter(final String name, final String alias)
{
Counter cfCounter = Metrics.counter(factory.createMetricName(name), aliasFactory.createMetricName(alias));
if (register(name, alias, cfCounter))
{
Metrics.register(globalFactory.createMetricName(name),
globalAliasFactory.createMetricName(alias),
new Gauge<Long>()
{
public Long getValue()
{
long total = 0;
for (Metric cfGauge : allTableMetrics.get(name))
{
total += ((Counter) cfGauge).getCount();
}
return total;
}
});
}
return cfCounter;
}
private static Double computeCompressionRatio(Iterable<SSTableReader> sstables)
{
double compressedLengthSum = 0;
double dataLengthSum = 0;
for (SSTableReader sstable : sstables)
{
if (sstable.compression)
{
assert sstable.openReason != SSTableReader.OpenReason.EARLY;
CompressionMetadata compressionMetadata = sstable.getCompressionMetadata();
compressedLengthSum += compressionMetadata.compressedFileLength;
dataLengthSum += compressionMetadata.dataLength;
}
}
return dataLengthSum != 0 ? compressedLengthSum / dataLengthSum : MetadataCollector.NO_COMPRESSION_RATIO;
}
protected TableHistogram createTableHistogram(String name, Histogram keyspaceHistogram, boolean considerZeroes)
{
return createTableHistogram(name, name, keyspaceHistogram, considerZeroes);
}
protected TableHistogram createTableHistogram(String name, String alias, Histogram keyspaceHistogram, boolean considerZeroes)
{
Histogram cfHistogram = Metrics.histogram(factory.createMetricName(name), aliasFactory.createMetricName(alias), considerZeroes);
register(name, alias, cfHistogram);
return new TableHistogram(cfHistogram,
keyspaceHistogram,
Metrics.histogram(globalFactory.createMetricName(name),
globalAliasFactory.createMetricName(alias),
considerZeroes));
}
protected TableTimer createTableTimer(String name, Timer keyspaceTimer)
{
return createTableTimer(name, name, keyspaceTimer);
}
protected TableTimer createTableTimer(String name, String alias, Timer keyspaceTimer)
{
Timer cfTimer = Metrics.timer(factory.createMetricName(name), aliasFactory.createMetricName(alias));
register(name, alias, cfTimer);
return new TableTimer(cfTimer,
keyspaceTimer,
Metrics.timer(globalFactory.createMetricName(name),
globalAliasFactory.createMetricName(alias)));
}
private boolean register(String name, String alias, Metric metric)
{
boolean ret = allTableMetrics.putIfAbsent(name, ConcurrentHashMap.newKeySet()) == null;
allTableMetrics.get(name).add(metric);
all.put(name, alias);
return ret;
}
public static class TableHistogram
{
public final Histogram[] all;
public final Histogram cf;
private TableHistogram(Histogram cf, Histogram keyspace, Histogram global)
{
this.cf = cf;
this.all = new Histogram[]{cf, keyspace, global};
}
public void update(long i)
{
for(Histogram histo : all)
{
histo.update(i);
}
}
}
public static class TableTimer
{
public final Timer[] all;
public final Timer cf;
private TableTimer(Timer cf, Timer keyspace, Timer global)
{
this.cf = cf;
this.all = new Timer[]{cf, keyspace, global};
}
public void update(long i, TimeUnit unit)
{
for(Timer timer : all)
{
timer.update(i, unit);
}
}
}
static class TableMetricNameFactory implements MetricNameFactory
{
private final String keyspaceName;
private final String tableName;
private final boolean isIndex;
private final String type;
TableMetricNameFactory(ColumnFamilyStore cfs, String type)
{
this.keyspaceName = cfs.keyspace.getName();
this.tableName = cfs.name;
this.isIndex = cfs.isIndex();
this.type = type;
}
public CassandraMetricsRegistry.MetricName createMetricName(String metricName)
{
String groupName = TableMetrics.class.getPackage().getName();
String type = isIndex ? "Index" + this.type : this.type;
StringBuilder mbeanName = new StringBuilder();
mbeanName.append(groupName).append(":");
mbeanName.append("type=").append(type);
mbeanName.append(",keyspace=").append(keyspaceName);
mbeanName.append(",scope=").append(tableName);
mbeanName.append(",name=").append(metricName);
return new CassandraMetricsRegistry.MetricName(groupName, type, metricName, keyspaceName + "." + tableName, mbeanName.toString());
}
}
static class AllTableMetricNameFactory implements MetricNameFactory
{
private final String type;
public AllTableMetricNameFactory(String type)
{
this.type = type;
}
public CassandraMetricsRegistry.MetricName createMetricName(String metricName)
{
String groupName = TableMetrics.class.getPackage().getName();
StringBuilder mbeanName = new StringBuilder();
mbeanName.append(groupName).append(":");
mbeanName.append("type=").append(type);
mbeanName.append(",name=").append(metricName);
return new CassandraMetricsRegistry.MetricName(groupName, type, metricName, "all", mbeanName.toString());
}
}
public enum Sampler
{
READS, WRITES
}
}