/** * Create a gauge that will be part of a merged version of all column families. The global gauge * is defined as the globalGauge parameter */ protected <G, T> Gauge<T> createColumnFamilyGauge( String name, Gauge<T> gauge, Gauge<G> globalGauge) { Gauge<T> cfGauge = Metrics.newGauge(factory.createMetricName(name), gauge); if (register(name, cfGauge)) { Metrics.newGauge(globalNameFactory.createMetricName(name), globalGauge); } return cfGauge; }
/** * Create a histogram-like interface that will register both a CF, keyspace and global level * histogram and forward any updates to both */ protected ColumnFamilyHistogram createColumnFamilyHistogram( String name, Histogram keyspaceHistogram) { Histogram cfHistogram = Metrics.newHistogram(factory.createMetricName(name), true); register(name, cfHistogram); return new ColumnFamilyHistogram( cfHistogram, keyspaceHistogram, Metrics.newHistogram(globalNameFactory.createMetricName(name), true)); }
/** * Creates a counter that will also have a global counter thats the sum of all counters across * different column families */ protected Counter createColumnFamilyCounter(final String name) { Counter cfCounter = Metrics.newCounter(factory.createMetricName(name)); if (register(name, cfCounter)) { Metrics.newGauge( globalNameFactory.createMetricName(name), new Gauge<Long>() { public Long value() { long total = 0; for (Metric cfGauge : allColumnFamilyMetrics.get(name)) { total += ((Counter) cfGauge).count(); } return total; } }); } return cfCounter; }
/** Release all associated metrics. */ public void release() { for (String name : all) { allColumnFamilyMetrics .get(name) .remove(Metrics.defaultRegistry().allMetrics().get(factory.createMetricName(name))); Metrics.defaultRegistry().removeMetric(factory.createMetricName(name)); } readLatency.release(); writeLatency.release(); rangeLatency.release(); Metrics.defaultRegistry().removeMetric(factory.createMetricName("EstimatedRowSizeHistogram")); Metrics.defaultRegistry() .removeMetric(factory.createMetricName("EstimatedColumnCountHistogram")); Metrics.defaultRegistry().removeMetric(factory.createMetricName("KeyCacheHitRate")); Metrics.defaultRegistry().removeMetric(factory.createMetricName("CoordinatorReadLatency")); Metrics.defaultRegistry().removeMetric(factory.createMetricName("CoordinatorScanLatency")); Metrics.defaultRegistry().removeMetric(factory.createMetricName("WaitingOnFreeMemtableSpace")); }
/** * Creates metrics for given {@link ColumnFamilyStore}. * * @param cfs ColumnFamilyStore to measure metrics */ public ColumnFamilyMetrics(final ColumnFamilyStore cfs) { factory = new ColumnFamilyMetricNameFactory(cfs); memtableColumnsCount = createColumnFamilyGauge( "MemtableColumnsCount", new Gauge<Long>() { public Long value() { return cfs.getDataTracker().getView().getCurrentMemtable().getOperations(); } }); memtableOnHeapSize = createColumnFamilyGauge( "MemtableOnHeapSize", new Gauge<Long>() { public Long value() { return cfs.getDataTracker() .getView() .getCurrentMemtable() .getAllocator() .onHeap() .owns(); } }); memtableOffHeapSize = createColumnFamilyGauge( "MemtableOffHeapSize", new Gauge<Long>() { public Long value() { return cfs.getDataTracker() .getView() .getCurrentMemtable() .getAllocator() .offHeap() .owns(); } }); memtableLiveDataSize = createColumnFamilyGauge( "MemtableLiveDataSize", new Gauge<Long>() { public Long value() { return cfs.getDataTracker().getView().getCurrentMemtable().getLiveDataSize(); } }); allMemtablesOnHeapSize = createColumnFamilyGauge( "AllMemtablesHeapSize", new Gauge<Long>() { public Long value() { long size = 0; for (ColumnFamilyStore cfs2 : cfs.concatWithIndexes()) size += cfs2.getDataTracker() .getView() .getCurrentMemtable() .getAllocator() .onHeap() .owns(); return size; } }); allMemtablesOffHeapSize = createColumnFamilyGauge( "AllMemtablesOffHeapSize", new Gauge<Long>() { public Long value() { long size = 0; for (ColumnFamilyStore cfs2 : cfs.concatWithIndexes()) size += cfs2.getDataTracker() .getView() .getCurrentMemtable() .getAllocator() .offHeap() .owns(); return size; } }); allMemtablesLiveDataSize = createColumnFamilyGauge( "AllMemtablesLiveDataSize", new Gauge<Long>() { public Long value() { long size = 0; for (ColumnFamilyStore cfs2 : cfs.concatWithIndexes()) size += cfs2.getDataTracker().getView().getCurrentMemtable().getLiveDataSize(); return size; } }); memtableSwitchCount = createColumnFamilyCounter("MemtableSwitchCount"); estimatedRowSizeHistogram = Metrics.newGauge( factory.createMetricName("EstimatedRowSizeHistogram"), new Gauge<long[]>() { public long[] value() { long[] histogram = new long[90]; for (SSTableReader sstable : cfs.getSSTables()) { long[] rowSize = sstable.getEstimatedRowSize().getBuckets(false); for (int i = 0; i < histogram.length; i++) histogram[i] += rowSize[i]; } return histogram; } }); estimatedColumnCountHistogram = Metrics.newGauge( factory.createMetricName("EstimatedColumnCountHistogram"), new Gauge<long[]>() { public long[] value() { long[] histogram = new long[90]; for (SSTableReader sstable : cfs.getSSTables()) { long[] columnSize = sstable.getEstimatedColumnCount().getBuckets(false); for (int i = 0; i < histogram.length; i++) histogram[i] += columnSize[i]; } return histogram; } }); sstablesPerReadHistogram = createColumnFamilyHistogram( "SSTablesPerReadHistogram", cfs.keyspace.metric.sstablesPerReadHistogram); compressionRatio = createColumnFamilyGauge( "CompressionRatio", new Gauge<Double>() { public Double value() { double sum = 0; int total = 0; for (SSTableReader sstable : cfs.getSSTables()) { if (sstable.getCompressionRatio() != MetadataCollector.NO_COMPRESSION_RATIO) { sum += sstable.getCompressionRatio(); total++; } } return total != 0 ? sum / total : 0; } }, new Gauge<Double>() // global gauge { public Double value() { double sum = 0; int total = 0; for (Keyspace keyspace : Keyspace.all()) { for (SSTableReader sstable : keyspace.getAllSSTables()) { if (sstable.getCompressionRatio() != MetadataCollector.NO_COMPRESSION_RATIO) { sum += sstable.getCompressionRatio(); total++; } } } return total != 0 ? sum / total : 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 = createColumnFamilyCounter("PendingFlushes"); pendingCompactions = createColumnFamilyGauge( "PendingCompactions", new Gauge<Integer>() { public Integer value() { return cfs.getCompactionStrategy().getEstimatedRemainingTasks(); } }); liveSSTableCount = createColumnFamilyGauge( "LiveSSTableCount", new Gauge<Integer>() { public Integer value() { return cfs.getDataTracker().getSSTables().size(); } }); liveDiskSpaceUsed = createColumnFamilyCounter("LiveDiskSpaceUsed"); totalDiskSpaceUsed = createColumnFamilyCounter("TotalDiskSpaceUsed"); minRowSize = createColumnFamilyGauge( "MinRowSize", new Gauge<Long>() { public Long value() { long min = 0; for (SSTableReader sstable : cfs.getSSTables()) { if (min == 0 || sstable.getEstimatedRowSize().min() < min) min = sstable.getEstimatedRowSize().min(); } return min; } }, new Gauge<Long>() // global gauge { public Long value() { long min = Long.MAX_VALUE; for (Metric cfGauge : allColumnFamilyMetrics.get("MinRowSize")) { min = Math.min(min, ((Gauge<? extends Number>) cfGauge).value().longValue()); } return min; } }); maxRowSize = createColumnFamilyGauge( "MaxRowSize", new Gauge<Long>() { public Long value() { long max = 0; for (SSTableReader sstable : cfs.getSSTables()) { if (sstable.getEstimatedRowSize().max() > max) max = sstable.getEstimatedRowSize().max(); } return max; } }, new Gauge<Long>() // global gauge { public Long value() { long max = 0; for (Metric cfGauge : allColumnFamilyMetrics.get("MaxRowSize")) { max = Math.max(max, ((Gauge<? extends Number>) cfGauge).value().longValue()); } return max; } }); meanRowSize = createColumnFamilyGauge( "MeanRowSize", new Gauge<Long>() { public Long value() { long sum = 0; long count = 0; for (SSTableReader sstable : cfs.getSSTables()) { long n = sstable.getEstimatedRowSize().count(); sum += sstable.getEstimatedRowSize().mean() * n; count += n; } return count > 0 ? sum / count : 0; } }, new Gauge<Long>() // global gauge { public Long value() { long sum = 0; long count = 0; for (Keyspace keyspace : Keyspace.all()) { for (SSTableReader sstable : keyspace.getAllSSTables()) { long n = sstable.getEstimatedRowSize().count(); sum += sstable.getEstimatedRowSize().mean() * n; count += n; } } return count > 0 ? sum / count : 0; } }); bloomFilterFalsePositives = createColumnFamilyGauge( "BloomFilterFalsePositives", new Gauge<Long>() { public Long value() { long count = 0L; for (SSTableReader sstable : cfs.getSSTables()) count += sstable.getBloomFilterFalsePositiveCount(); return count; } }); recentBloomFilterFalsePositives = createColumnFamilyGauge( "RecentBloomFilterFalsePositives", new Gauge<Long>() { public Long value() { long count = 0L; for (SSTableReader sstable : cfs.getSSTables()) count += sstable.getRecentBloomFilterFalsePositiveCount(); return count; } }); bloomFilterFalseRatio = createColumnFamilyGauge( "BloomFilterFalseRatio", new Gauge<Double>() { public Double value() { long falseCount = 0L; long trueCount = 0L; for (SSTableReader sstable : cfs.getSSTables()) { falseCount += sstable.getBloomFilterFalsePositiveCount(); trueCount += sstable.getBloomFilterTruePositiveCount(); } if (falseCount == 0L && trueCount == 0L) return 0d; return (double) falseCount / (trueCount + falseCount); } }, new Gauge<Double>() // global gauge { public Double value() { long falseCount = 0L; long trueCount = 0L; for (Keyspace keyspace : Keyspace.all()) { for (SSTableReader sstable : keyspace.getAllSSTables()) { falseCount += sstable.getBloomFilterFalsePositiveCount(); trueCount += sstable.getBloomFilterTruePositiveCount(); } } if (falseCount == 0L && trueCount == 0L) return 0d; return (double) falseCount / (trueCount + falseCount); } }); recentBloomFilterFalseRatio = createColumnFamilyGauge( "RecentBloomFilterFalseRatio", new Gauge<Double>() { public Double value() { long falseCount = 0L; long trueCount = 0L; for (SSTableReader sstable : cfs.getSSTables()) { falseCount += sstable.getRecentBloomFilterFalsePositiveCount(); trueCount += sstable.getRecentBloomFilterTruePositiveCount(); } if (falseCount == 0L && trueCount == 0L) return 0d; return (double) falseCount / (trueCount + falseCount); } }, new Gauge<Double>() // global gauge { public Double value() { long falseCount = 0L; long trueCount = 0L; for (Keyspace keyspace : Keyspace.all()) { for (SSTableReader sstable : keyspace.getAllSSTables()) { falseCount += sstable.getRecentBloomFilterFalsePositiveCount(); trueCount += sstable.getRecentBloomFilterTruePositiveCount(); } } if (falseCount == 0L && trueCount == 0L) return 0d; return (double) falseCount / (trueCount + falseCount); } }); bloomFilterDiskSpaceUsed = createColumnFamilyGauge( "BloomFilterDiskSpaceUsed", new Gauge<Long>() { public Long value() { long total = 0; for (SSTableReader sst : cfs.getSSTables()) total += sst.getBloomFilterSerializedSize(); return total; } }); speculativeRetries = createColumnFamilyCounter("SpeculativeRetries"); keyCacheHitRate = Metrics.newGauge( factory.createMetricName("KeyCacheHitRate"), new RatioGauge() { protected double getNumerator() { long hits = 0L; for (SSTableReader sstable : cfs.getSSTables()) hits += sstable.getKeyCacheHit(); return hits; } protected double getDenominator() { long requests = 0L; for (SSTableReader sstable : cfs.getSSTables()) requests += sstable.getKeyCacheRequest(); return Math.max(requests, 1); // to avoid NaN. } }); tombstoneScannedHistogram = createColumnFamilyHistogram( "TombstoneScannedHistogram", cfs.keyspace.metric.tombstoneScannedHistogram); liveScannedHistogram = createColumnFamilyHistogram( "LiveScannedHistogram", cfs.keyspace.metric.liveScannedHistogram); coordinatorReadLatency = Metrics.newTimer( factory.createMetricName("CoordinatorReadLatency"), TimeUnit.MICROSECONDS, TimeUnit.SECONDS); coordinatorScanLatency = Metrics.newTimer( factory.createMetricName("CoordinatorScanLatency"), TimeUnit.MICROSECONDS, TimeUnit.SECONDS); waitingOnFreeMemtableSpace = Metrics.newTimer( factory.createMetricName("WaitingOnFreeMemtableSpace"), TimeUnit.MICROSECONDS, TimeUnit.SECONDS); trueSnapshotsSize = createColumnFamilyGauge( "SnapshotsSize", new Gauge<Long>() { public Long value() { return cfs.trueSnapshotsSize(); } }); rowCacheHitOutOfRange = createColumnFamilyCounter("RowCacheHitOutOfRange"); rowCacheHit = createColumnFamilyCounter("RowCacheHit"); rowCacheMiss = createColumnFamilyCounter("RowCacheMiss"); }