package org.apache.cassandra.utils;
import java.lang.management.ManagementFactory;
import java.util.Map;
import javax.management.*;
import org.apache.cassandra.cache.*;
import org.apache.cassandra.metrics.ThreadPoolMetrics;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.RowIndexEntry;
import org.apache.cassandra.db.compaction.CompactionManager;
import org.apache.cassandra.net.MessagingService;
import org.apache.cassandra.service.CacheService;
public class StatusLogger
{
private static final Logger logger = LoggerFactory.getLogger(StatusLogger.class);
public static void log()
{
MBeanServer server = ManagementFactory.getPlatformMBeanServer();
logger.info(String.format("%-25s%10s%10s%15s%10s%18s", "Pool Name", "Active", "Pending", "Completed", "Blocked", "All Time Blocked"));
for (Map.Entry<String, String> tpool : ThreadPoolMetrics.getJmxThreadPools(server).entries())
{
logger.info(String.format("%-25s%10s%10s%15s%10s%18s%n",
tpool.getValue(),
ThreadPoolMetrics.getJmxMetric(server, tpool.getKey(), tpool.getValue(), "ActiveTasks"),
ThreadPoolMetrics.getJmxMetric(server, tpool.getKey(), tpool.getValue(), "PendingTasks"),
ThreadPoolMetrics.getJmxMetric(server, tpool.getKey(), tpool.getValue(), "CompletedTasks"),
ThreadPoolMetrics.getJmxMetric(server, tpool.getKey(), tpool.getValue(), "CurrentlyBlockedTasks"),
ThreadPoolMetrics.getJmxMetric(server, tpool.getKey(), tpool.getValue(), "TotalBlockedTasks")));
}
logger.info(String.format("%-25s%10s%10s",
"CompactionManager", CompactionManager.instance.getActiveCompactions(), CompactionManager.instance.getPendingTasks()));
int pendingLargeMessages = 0;
for (int n : MessagingService.instance().getLargeMessagePendingTasks().values())
{
pendingLargeMessages += n;
}
int pendingSmallMessages = 0;
for (int n : MessagingService.instance().getSmallMessagePendingTasks().values())
{
pendingSmallMessages += n;
}
logger.info(String.format("%-25s%10s%10s",
"MessagingService", "n/a", pendingLargeMessages + "/" + pendingSmallMessages));
AutoSavingCache<KeyCacheKey, RowIndexEntry> keyCache = CacheService.instance.keyCache;
AutoSavingCache<RowCacheKey, IRowCacheEntry> rowCache = CacheService.instance.rowCache;
int keyCacheKeysToSave = DatabaseDescriptor.getKeyCacheKeysToSave();
int rowCacheKeysToSave = DatabaseDescriptor.getRowCacheKeysToSave();
logger.info(String.format("%-25s%10s%25s%25s",
"Cache Type", "Size", "Capacity", "KeysToSave"));
logger.info(String.format("%-25s%10s%25s%25s",
"KeyCache",
keyCache.weightedSize(),
keyCache.getCapacity(),
keyCacheKeysToSave == Integer.MAX_VALUE ? "all" : keyCacheKeysToSave));
logger.info(String.format("%-25s%10s%25s%25s",
"RowCache",
rowCache.weightedSize(),
rowCache.getCapacity(),
rowCacheKeysToSave == Integer.MAX_VALUE ? "all" : rowCacheKeysToSave));
logger.info(String.format("%-25s%20s", "Table", "Memtable ops,data"));
for (ColumnFamilyStore cfs : ColumnFamilyStore.all())
{
logger.info(String.format("%-25s%20s",
cfs.keyspace.getName() + "." + cfs.name,
cfs.metric.memtableColumnsCount.getValue() + "," + cfs.metric.memtableLiveDataSize.getValue()));
}
}
}