in src/java/org/apache/cassandra/tools/nodetool/stats/TableStatsHolder.java [192:448]
private void initializeKeyspaces(NodeProbe probe, boolean ignore, List<String> tableNames)
{
OptionFilter filter = new OptionFilter(ignore, tableNames);
ArrayListMultimap<String, ColumnFamilyStoreMBean> selectedTableMbeans = ArrayListMultimap.create();
Map<String, StatsKeyspace> keyspaceStats = new HashMap<>();
// get a list of table stores
Iterator<Map.Entry<String, ColumnFamilyStoreMBean>> tableMBeans = probe.getColumnFamilyStoreMBeanProxies();
while (tableMBeans.hasNext())
{
Map.Entry<String, ColumnFamilyStoreMBean> entry = tableMBeans.next();
String keyspaceName = entry.getKey();
ColumnFamilyStoreMBean tableProxy = entry.getValue();
if (filter.isKeyspaceIncluded(keyspaceName))
{
StatsKeyspace stats = keyspaceStats.get(keyspaceName);
if (stats == null)
{
stats = new StatsKeyspace(probe, keyspaceName);
stats.initialize();
keyspaceStats.put(keyspaceName, stats);
}
stats.add(tableProxy);
if (filter.isTableIncluded(keyspaceName, tableProxy.getTableName()))
selectedTableMbeans.put(keyspaceName, tableProxy);
}
}
numberOfTables = selectedTableMbeans.size();
// make sure all specified keyspace and tables exist
filter.verifyKeyspaces(probe.getKeyspaces());
filter.verifyTables();
// get metrics of keyspace
for (Map.Entry<String, Collection<ColumnFamilyStoreMBean>> entry : selectedTableMbeans.asMap().entrySet())
{
String keyspaceName = entry.getKey();
Collection<ColumnFamilyStoreMBean> tables = entry.getValue();
StatsKeyspace statsKeyspace = keyspaceStats.get(keyspaceName);
// get metrics of table statistics for this keyspace
for (ColumnFamilyStoreMBean table : tables)
{
String tableName = table.getTableName();
StatsTable statsTable = new StatsTable();
statsTable.fullName = keyspaceName + "." + tableName;
statsTable.keyspaceName = keyspaceName;
statsTable.tableName = tableName;
statsTable.isIndex = tableName.contains(".");
statsTable.sstableCount = probe.getColumnFamilyMetric(keyspaceName, tableName, "LiveSSTableCount");
statsTable.oldSSTableCount = probe.getColumnFamilyMetric(keyspaceName, tableName, "OldVersionSSTableCount");
Long sstableSize = (Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "MaxSSTableSize");
statsTable.maxSSTableSize = sstableSize == null ? 0 : sstableSize;
int[] leveledSStables = table.getSSTableCountPerLevel();
if (leveledSStables != null)
{
statsTable.isLeveledSstable = true;
for (int level = 0; level < leveledSStables.length; level++)
{
int count = leveledSStables[level];
long maxCount = 4L; // for L0
if (level > 0)
maxCount = (long) Math.pow(table.getLevelFanoutSize(), level);
// show max threshold for level when exceeded
statsTable.sstablesInEachLevel.add(count + ((count > maxCount) ? "/" + maxCount : ""));
}
}
statsTable.sstableCountPerTWCSBucket = table.getSSTableCountPerTWCSBucket();
long[] leveledSSTablesBytes = table.getPerLevelSizeBytes();
if (leveledSSTablesBytes != null)
{
statsTable.isLeveledSstable = true;
for (int level = 0; level < leveledSSTablesBytes.length; level++)
{
long size = leveledSSTablesBytes[level];
statsTable.sstableBytesInEachLevel.add(FileUtils.stringifyFileSize(size, humanReadable));
}
}
if (locationCheck)
statsTable.isInCorrectLocation = !table.hasMisplacedSSTables();
Long memtableOffHeapSize = null;
Long bloomFilterOffHeapSize = null;
Long indexSummaryOffHeapSize = null;
Long compressionMetadataOffHeapSize = null;
Long offHeapSize = null;
Double percentRepaired = null;
Long bytesRepaired = null;
Long bytesUnrepaired = null;
Long bytesPendingRepair = null;
Double sstableCompressionRatio = null;
Double bloomFilterFalseRatio = null;
try
{
memtableOffHeapSize = (Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "MemtableOffHeapSize");
bloomFilterOffHeapSize = (Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "BloomFilterOffHeapMemoryUsed");
indexSummaryOffHeapSize = (Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "IndexSummaryOffHeapMemoryUsed");
compressionMetadataOffHeapSize = (Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "CompressionMetadataOffHeapMemoryUsed");
offHeapSize = memtableOffHeapSize + bloomFilterOffHeapSize + indexSummaryOffHeapSize + compressionMetadataOffHeapSize;
percentRepaired = (Double) probe.getColumnFamilyMetric(keyspaceName, tableName, "PercentRepaired");
bytesRepaired = (Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "BytesRepaired");
bytesUnrepaired = (Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "BytesUnrepaired");
bytesPendingRepair = (Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "BytesPendingRepair");
sstableCompressionRatio = (Double) probe.getColumnFamilyMetric(keyspaceName, tableName, "CompressionRatio");
bloomFilterFalseRatio = (Double) probe.getColumnFamilyMetric(keyspaceName, tableName, "RecentBloomFilterFalseRatio");
}
catch (RuntimeException e)
{
// offheap-metrics introduced in 2.1.3 - older versions do not have the appropriate mbeans
if (!(e.getCause() instanceof InstanceNotFoundException))
throw e;
}
statsTable.spaceUsedLive = FileUtils.stringifyFileSize((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "LiveDiskSpaceUsed"), humanReadable);
statsTable.spaceUsedTotal = FileUtils.stringifyFileSize((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "TotalDiskSpaceUsed"), humanReadable);
statsTable.spaceUsedBySnapshotsTotal = FileUtils.stringifyFileSize((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "SnapshotsSize"), humanReadable);
maybeAddTWCSWindowWithMaxDuration(statsTable, probe, keyspaceName, tableName);
if (offHeapSize != null)
{
statsTable.offHeapUsed = true;
statsTable.offHeapMemoryUsedTotal = FileUtils.stringifyFileSize(offHeapSize, humanReadable);
}
if (percentRepaired != null)
{
statsTable.percentRepaired = Math.round(100 * percentRepaired) / 100.0;
}
statsTable.bytesRepaired = bytesRepaired != null ? bytesRepaired : 0;
statsTable.bytesUnrepaired = bytesUnrepaired != null ? bytesUnrepaired : 0;
statsTable.bytesPendingRepair = bytesPendingRepair != null ? bytesPendingRepair : 0;
statsTable.sstableCompressionRatio = sstableCompressionRatio != null ? sstableCompressionRatio : Double.NaN;
Object estimatedPartitionCount = probe.getColumnFamilyMetric(keyspaceName, tableName, "EstimatedPartitionCount");
if (Long.valueOf(-1L).equals(estimatedPartitionCount))
{
estimatedPartitionCount = 0L;
}
statsTable.numberOfPartitionsEstimate = estimatedPartitionCount;
statsTable.memtableCellCount = probe.getColumnFamilyMetric(keyspaceName, tableName, "MemtableColumnsCount");
statsTable.memtableDataSize = FileUtils.stringifyFileSize((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "MemtableLiveDataSize"), humanReadable);
if (memtableOffHeapSize != null)
{
statsTable.memtableOffHeapUsed = true;
statsTable.memtableOffHeapMemoryUsed = FileUtils.stringifyFileSize(memtableOffHeapSize, humanReadable);
}
statsTable.memtableSwitchCount = probe.getColumnFamilyMetric(keyspaceName, tableName, "MemtableSwitchCount");
statsTable.speculativeRetries = probe.getColumnFamilyMetric(keyspaceName, tableName, "SpeculativeRetries");
statsTable.localReadCount = ((CassandraMetricsRegistry.JmxTimerMBean) probe.getColumnFamilyMetric(keyspaceName, tableName, "ReadLatency")).getCount();
double localReadLatency = ((CassandraMetricsRegistry.JmxTimerMBean) probe.getColumnFamilyMetric(keyspaceName, tableName, "ReadLatency")).getMean() / 1000;
double localRLatency = localReadLatency > 0 ? localReadLatency : Double.NaN;
statsTable.localReadLatencyMs = localRLatency;
statsTable.localWriteCount = ((CassandraMetricsRegistry.JmxTimerMBean) probe.getColumnFamilyMetric(keyspaceName, tableName, "WriteLatency")).getCount();
double localWriteLatency = ((CassandraMetricsRegistry.JmxTimerMBean) probe.getColumnFamilyMetric(keyspaceName, tableName, "WriteLatency")).getMean() / 1000;
double localWLatency = localWriteLatency > 0 ? localWriteLatency : Double.NaN;
statsTable.localReadWriteRatio = statsTable.localWriteCount > 0 ? statsTable.localReadCount / (double) statsTable.localWriteCount : 0;
statsTable.localWriteLatencyMs = localWLatency;
statsTable.pendingFlushes = probe.getColumnFamilyMetric(keyspaceName, tableName, "PendingFlushes");
statsTable.bloomFilterFalsePositives = probe.getColumnFamilyMetric(keyspaceName, tableName, "BloomFilterFalsePositives");
statsTable.bloomFilterFalseRatio = bloomFilterFalseRatio != null ? bloomFilterFalseRatio : Double.NaN;
statsTable.bloomFilterSpaceUsed = FileUtils.stringifyFileSize((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "BloomFilterDiskSpaceUsed"), humanReadable);
if (bloomFilterOffHeapSize != null)
{
statsTable.bloomFilterOffHeapUsed = true;
statsTable.bloomFilterOffHeapMemoryUsed = FileUtils.stringifyFileSize(bloomFilterOffHeapSize, humanReadable);
}
if (indexSummaryOffHeapSize != null)
{
statsTable.indexSummaryOffHeapUsed = true;
statsTable.indexSummaryOffHeapMemoryUsed = FileUtils.stringifyFileSize(indexSummaryOffHeapSize, humanReadable);
}
if (compressionMetadataOffHeapSize != null)
{
statsTable.compressionMetadataOffHeapUsed = true;
statsTable.compressionMetadataOffHeapMemoryUsed = FileUtils.stringifyFileSize(compressionMetadataOffHeapSize, humanReadable);
}
statsTable.compactedPartitionMinimumBytes = (Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "MinPartitionSize");
statsTable.compactedPartitionMaximumBytes = (Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "MaxPartitionSize");
statsTable.compactedPartitionMeanBytes = (Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "MeanPartitionSize");
CassandraMetricsRegistry.JmxHistogramMBean histogram = (CassandraMetricsRegistry.JmxHistogramMBean) probe.getColumnFamilyMetric(keyspaceName, tableName, "LiveScannedHistogram");
statsTable.averageLiveCellsPerSliceLastFiveMinutes = histogram.getMean();
statsTable.maximumLiveCellsPerSliceLastFiveMinutes = histogram.getMax();
histogram = (CassandraMetricsRegistry.JmxHistogramMBean) probe.getColumnFamilyMetric(keyspaceName, tableName, "TombstoneScannedHistogram");
statsTable.averageTombstonesPerSliceLastFiveMinutes = histogram.getMean();
statsTable.maximumTombstonesPerSliceLastFiveMinutes = histogram.getMax();
statsTable.droppableTombstoneRatio = probe.getDroppableTombstoneRatio(keyspaceName, tableName);
statsTable.topSizePartitions = format(table.getTopSizePartitions(), humanReadable);
if (table.getTopSizePartitionsLastUpdate() != null)
statsTable.topSizePartitionsLastUpdate = millisToDateString(table.getTopSizePartitionsLastUpdate());
statsTable.topTombstonePartitions = table.getTopTombstonePartitions();
if (table.getTopTombstonePartitionsLastUpdate() != null)
statsTable.topTombstonePartitionsLastUpdate = millisToDateString(table.getTopTombstonePartitionsLastUpdate());
if (!SchemaConstants.isSystemKeyspace(keyspaceName))
{
Object totalIndexCount = probe.getSaiMetric(keyspaceName, tableName, "TotalIndexCount");
statsTable.saiTotalIndexCount = (totalIndexCount != null) ? (int) totalIndexCount : 0;
if (statsTable.saiTotalIndexCount > 0)
{
Object queryLatencyMetric = probe.getSaiMetric(keyspaceName, tableName, "QueryLatency");
double queryLatency = getMetricMean(queryLatencyMetric);
statsTable.saiQueryLatencyMs = queryLatency > 0 ? queryLatency : Double.NaN;
Object PostFilteringReadLatency = probe.getSaiMetric(keyspaceName, tableName, "PostFilteringReadLatency");
double postfilteringreadlatency = getMetricMean(PostFilteringReadLatency);
statsTable.saiPostFilteringReadLatencyMs = postfilteringreadlatency > 0 ? postfilteringreadlatency : Double.NaN;
Object diskUsedBytes = probe.getSaiMetric(keyspaceName, tableName, "DiskUsedBytes");
long saidiskusedbytes = (diskUsedBytes != null) ? (long) diskUsedBytes : 0L;
statsTable.saiDiskUsedBytes = FileUtils.stringifyFileSize(saidiskusedbytes, humanReadable);
Object SSTableIndexesHit = probe.getSaiMetric(keyspaceName, tableName, "SSTableIndexesHit");
statsTable.saiSSTableIndexesHit = getMetricMean(SSTableIndexesHit);
Object IndexSegmentsHit = probe.getSaiMetric(keyspaceName, tableName, "IndexSegmentsHit");
statsTable.saiIndexSegmentsHit = getMetricMean(IndexSegmentsHit);
Object RowsFiltered = probe.getSaiMetric(keyspaceName, tableName, "RowsFiltered");
statsTable.saiRowsFiltered = getMetricMean(RowsFiltered);
Object totalQueryTimeouts = probe.getSaiMetric(keyspaceName, tableName, "TotalQueryTimeouts");
statsTable.saiTotalQueryTimeouts = (totalQueryTimeouts != null) ? (Long) totalQueryTimeouts : 0L;
Object totalQueryableIndexCount = probe.getSaiMetric(keyspaceName, tableName, "TotalQueryableIndexCount");
int saiTotalQueryableIndexCount = (totalQueryableIndexCount != null) ? (int) totalQueryableIndexCount : 0;
statsTable.saiTotalQueryableIndexRatio = String.format("%d/%d", saiTotalQueryableIndexCount, statsTable.saiTotalIndexCount);
}
}
statsKeyspace.tables.add(statsTable);
}
keyspaces.add(statsKeyspace);
}
}