Cleanup JVM info and stats

Remove reflection since its not needed with Java 7, remove lastGcInfo since its not used, and move to only provide getters
closes #10553
This commit is contained in:
Shay Banon 2015-04-12 16:55:01 +02:00
parent fcbe3f92b7
commit e2a05bb43f
12 changed files with 85 additions and 521 deletions

View File

@ -548,13 +548,13 @@ public class ClusterStatsNodes implements ToXContent, Streamable {
if (js == null) {
return;
}
if (js.threads() != null) {
threads += js.threads().count();
if (js.getThreads() != null) {
threads += js.getThreads().getCount();
}
maxUptime = Math.max(maxUptime, js.uptime().millis());
if (js.mem() != null) {
heapUsed += js.mem().getHeapUsed().bytes();
heapMax += js.mem().getHeapMax().bytes();
maxUptime = Math.max(maxUptime, js.getUptime().millis());
if (js.getMem() != null) {
heapUsed += js.getMem().getHeapUsed().bytes();
heapMax += js.getMem().getHeapMax().bytes();
}
}
@ -640,9 +640,9 @@ public class ClusterStatsNodes implements ToXContent, Streamable {
JvmVersion(JvmInfo jvmInfo) {
version = jvmInfo.version();
vmName = jvmInfo.vmName();
vmVersion = jvmInfo.vmVersion();
vmVendor = jvmInfo.vmVendor();
vmName = jvmInfo.getVmName();
vmVersion = jvmInfo.getVmVersion();
vmVendor = jvmInfo.getVmVendor();
}
JvmVersion() {

View File

@ -182,7 +182,7 @@ public class Bootstrap {
}
// warn if running using the client VM
if (JvmInfo.jvmInfo().vmName().toLowerCase(Locale.ROOT).contains("client")) {
if (JvmInfo.jvmInfo().getVmName().toLowerCase(Locale.ROOT).contains("client")) {
ESLogger logger = Loggers.getLogger(Bootstrap.class);
logger.warn("jvm uses the client vm, make sure to run `java` with the server vm for best performance by adding `-server` to the command line");
}

View File

@ -168,9 +168,9 @@ public class NettyHttpServerTransport extends AbstractLifecycleComponent<HttpSer
this.detailedErrorsEnabled = settings.getAsBoolean(SETTING_HTTP_DETAILED_ERRORS_ENABLED, true);
long defaultReceiverPredictor = 512 * 1024;
if (JvmInfo.jvmInfo().mem().directMemoryMax().bytes() > 0) {
if (JvmInfo.jvmInfo().getMem().getDirectMemoryMax().bytes() > 0) {
// we can guess a better default...
long l = (long) ((0.3 * JvmInfo.jvmInfo().mem().directMemoryMax().bytes()) / workerCount);
long l = (long) ((0.3 * JvmInfo.jvmInfo().getMem().getDirectMemoryMax().bytes()) / workerCount);
defaultReceiverPredictor = Math.min(defaultReceiverPredictor, Math.max(l, 64 * 1024));
}

View File

@ -77,7 +77,7 @@ public class IndexingMemoryController extends AbstractLifecycleComponent<Indexin
String indexingBufferSetting = this.settings.get("indices.memory.index_buffer_size", "10%");
if (indexingBufferSetting.endsWith("%")) {
double percent = Double.parseDouble(indexingBufferSetting.substring(0, indexingBufferSetting.length() - 1));
indexingBuffer = new ByteSizeValue((long) (((double) JvmInfo.jvmInfo().mem().heapMax().bytes()) * (percent / 100)));
indexingBuffer = new ByteSizeValue((long) (((double) JvmInfo.jvmInfo().getMem().getHeapMax().bytes()) * (percent / 100)));
ByteSizeValue minIndexingBuffer = this.settings.getAsBytesSize("indices.memory.min_index_buffer_size", new ByteSizeValue(48, ByteSizeUnit.MB));
ByteSizeValue maxIndexingBuffer = this.settings.getAsBytesSize("indices.memory.max_index_buffer_size", null);
@ -99,7 +99,7 @@ public class IndexingMemoryController extends AbstractLifecycleComponent<Indexin
String translogBufferSetting = this.settings.get("indices.memory.translog_buffer_size", "1%");
if (translogBufferSetting.endsWith("%")) {
double percent = Double.parseDouble(translogBufferSetting.substring(0, translogBufferSetting.length() - 1));
translogBuffer = new ByteSizeValue((long) (((double) JvmInfo.jvmInfo().mem().heapMax().bytes()) * (percent / 100)));
translogBuffer = new ByteSizeValue((long) (((double) JvmInfo.jvmInfo().getMem().getHeapMax().bytes()) * (percent / 100)));
ByteSizeValue minTranslogBuffer = this.settings.getAsBytesSize("indices.memory.min_translog_buffer_size", new ByteSizeValue(256, ByteSizeUnit.KB));
ByteSizeValue maxTranslogBuffer = this.settings.getAsBytesSize("indices.memory.max_translog_buffer_size", null);

View File

@ -208,76 +208,40 @@ public class JvmInfo implements Streamable, Serializable, ToXContent {
}
}
public String vmName() {
return vmName;
}
public String getVmName() {
return vmName;
}
public String vmVersion() {
return vmVersion;
return this.vmName;
}
public String getVmVersion() {
return vmVersion;
}
public String vmVendor() {
return vmVendor;
return this.vmVersion;
}
public String getVmVendor() {
return vmVendor;
}
public long startTime() {
return startTime;
return this.vmVendor;
}
public long getStartTime() {
return startTime;
}
public Mem mem() {
return mem;
return this.startTime;
}
public Mem getMem() {
return mem();
}
public String[] inputArguments() {
return inputArguments;
return this.mem;
}
public String[] getInputArguments() {
return inputArguments;
}
public String bootClassPath() {
return bootClassPath;
return this.inputArguments;
}
public String getBootClassPath() {
return bootClassPath;
}
public String classPath() {
return classPath;
return this.bootClassPath;
}
public String getClassPath() {
return classPath;
}
public Map<String, String> systemProperties() {
return systemProperties;
return this.classPath;
}
public Map<String, String> getSystemProperties() {
return systemProperties;
return this.systemProperties;
}
@Override
@ -396,50 +360,24 @@ public class JvmInfo implements Streamable, Serializable, ToXContent {
Mem() {
}
public ByteSizeValue heapInit() {
public ByteSizeValue getHeapInit() {
return new ByteSizeValue(heapInit);
}
public ByteSizeValue getHeapInit() {
return heapInit();
}
public ByteSizeValue heapMax() {
public ByteSizeValue getHeapMax() {
return new ByteSizeValue(heapMax);
}
public ByteSizeValue getHeapMax() {
return heapMax();
}
public ByteSizeValue nonHeapInit() {
public ByteSizeValue getNonHeapInit() {
return new ByteSizeValue(nonHeapInit);
}
public ByteSizeValue getNonHeapInit() {
return nonHeapInit();
}
public ByteSizeValue nonHeapMax() {
public ByteSizeValue getNonHeapMax() {
return new ByteSizeValue(nonHeapMax);
}
public ByteSizeValue getNonHeapMax() {
return nonHeapMax();
}
public ByteSizeValue directMemoryMax() {
return new ByteSizeValue(directMemoryMax);
}
public ByteSizeValue getDirectMemoryMax() {
return directMemoryMax();
}
public static Mem readMem(StreamInput in) throws IOException {
Mem mem = new Mem();
mem.readFrom(in);
return mem;
return new ByteSizeValue(directMemoryMax);
}
@Override

View File

@ -29,9 +29,7 @@ import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.FutureUtils;
import org.elasticsearch.threadpool.ThreadPool;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ScheduledFuture;
import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds;
@ -107,7 +105,7 @@ public class JvmMonitorService extends AbstractLifecycleComponent<JvmMonitorServ
this.gcThresholds = gcThresholds.immutableMap();
logger.debug("enabled [{}], last_gc_enabled [{}], interval [{}], gc_threshold [{}]", enabled, JvmStats.isLastGcEnabled(), interval, this.gcThresholds);
logger.debug("enabled [{}], interval [{}], gc_threshold [{}]", enabled, interval, this.gcThresholds);
}
@Override
@ -133,18 +131,14 @@ public class JvmMonitorService extends AbstractLifecycleComponent<JvmMonitorServ
private class JvmMonitor implements Runnable {
private JvmStats lastJvmStats = jvmStats();
private long seq = 0;
private final Set<DeadlockAnalyzer.Deadlock> lastSeenDeadlocks = new HashSet<>();
public JvmMonitor() {
}
@Override
public void run() {
try {
// monitorDeadlock();
monitorLongGc();
} catch (Throwable t) {
logger.debug("failed to monitor", t);
@ -155,8 +149,8 @@ public class JvmMonitorService extends AbstractLifecycleComponent<JvmMonitorServ
seq++;
JvmStats currentJvmStats = jvmStats();
for (int i = 0; i < currentJvmStats.gc().collectors().length; i++) {
GarbageCollector gc = currentJvmStats.gc().collectors()[i];
for (int i = 0; i < currentJvmStats.getGc().getCollectors().length; i++) {
GarbageCollector gc = currentJvmStats.getGc().getCollectors()[i];
GarbageCollector prevGc = lastJvmStats.gc.collectors[i];
// no collection has happened
@ -169,44 +163,22 @@ public class JvmMonitorService extends AbstractLifecycleComponent<JvmMonitorServ
continue;
}
GcThreshold gcThreshold = gcThresholds.get(gc.name());
GcThreshold gcThreshold = gcThresholds.get(gc.getName());
if (gcThreshold == null) {
gcThreshold = gcThresholds.get("default");
}
if (gc.lastGc() != null && prevGc.lastGc() != null) {
GarbageCollector.LastGc lastGc = gc.lastGc();
if (lastGc.startTime == prevGc.lastGc().startTime()) {
// we already handled this one...
continue;
}
// Ignore any duration > 1hr; getLastGcInfo occasionally returns total crap
if (lastGc.duration().hoursFrac() > 1) {
continue;
}
if (lastGc.duration().millis() > gcThreshold.warnThreshold) {
logger.warn("[last_gc][{}][{}][{}] duration [{}], collections [{}], total [{}]/[{}], reclaimed [{}], leaving [{}][{}]/[{}]",
gc.name(), seq, gc.getCollectionCount(), lastGc.duration(), collections, TimeValue.timeValueMillis(collectionTime), gc.collectionTime(), lastGc.reclaimed(), lastGc.afterUsed(), lastGc.max());
} else if (lastGc.duration().millis() > gcThreshold.infoThreshold) {
logger.info("[last_gc][{}][{}][{}] duration [{}], collections [{}], total [{}]/[{}], reclaimed [{}], leaving [{}]/[{}]",
gc.name(), seq, gc.getCollectionCount(), lastGc.duration(), collections, TimeValue.timeValueMillis(collectionTime), gc.collectionTime(), lastGc.reclaimed(), lastGc.afterUsed(), lastGc.max());
} else if (lastGc.duration().millis() > gcThreshold.debugThreshold && logger.isDebugEnabled()) {
logger.debug("[last_gc][{}][{}][{}] duration [{}], collections [{}], total [{}]/[{}], reclaimed [{}], leaving [{}]/[{}]",
gc.name(), seq, gc.getCollectionCount(), lastGc.duration(), collections, TimeValue.timeValueMillis(collectionTime), gc.collectionTime(), lastGc.reclaimed(), lastGc.afterUsed(), lastGc.max());
}
}
long avgCollectionTime = collectionTime / collections;
if (avgCollectionTime > gcThreshold.warnThreshold) {
logger.warn("[gc][{}][{}][{}] duration [{}], collections [{}]/[{}], total [{}]/[{}], memory [{}]->[{}]/[{}], all_pools {}",
gc.name(), seq, gc.collectionCount(), TimeValue.timeValueMillis(collectionTime), collections, TimeValue.timeValueMillis(currentJvmStats.timestamp() - lastJvmStats.timestamp()), TimeValue.timeValueMillis(collectionTime), gc.collectionTime(), lastJvmStats.mem().heapUsed(), currentJvmStats.mem().heapUsed(), JvmInfo.jvmInfo().mem().heapMax(), buildPools(lastJvmStats, currentJvmStats));
gc.getName(), seq, gc.getCollectionCount(), TimeValue.timeValueMillis(collectionTime), collections, TimeValue.timeValueMillis(currentJvmStats.getTimestamp() - lastJvmStats.getTimestamp()), TimeValue.timeValueMillis(collectionTime), gc.getCollectionTime(), lastJvmStats.getMem().getHeapUsed(), currentJvmStats.getMem().getHeapUsed(), JvmInfo.jvmInfo().getMem().getHeapMax(), buildPools(lastJvmStats, currentJvmStats));
} else if (avgCollectionTime > gcThreshold.infoThreshold) {
logger.info("[gc][{}][{}][{}] duration [{}], collections [{}]/[{}], total [{}]/[{}], memory [{}]->[{}]/[{}], all_pools {}",
gc.name(), seq, gc.collectionCount(), TimeValue.timeValueMillis(collectionTime), collections, TimeValue.timeValueMillis(currentJvmStats.timestamp() - lastJvmStats.timestamp()), TimeValue.timeValueMillis(collectionTime), gc.collectionTime(), lastJvmStats.mem().heapUsed(), currentJvmStats.mem().heapUsed(), JvmInfo.jvmInfo().mem().heapMax(), buildPools(lastJvmStats, currentJvmStats));
gc.getName(), seq, gc.getCollectionCount(), TimeValue.timeValueMillis(collectionTime), collections, TimeValue.timeValueMillis(currentJvmStats.getTimestamp() - lastJvmStats.getTimestamp()), TimeValue.timeValueMillis(collectionTime), gc.getCollectionTime(), lastJvmStats.getMem().getHeapUsed(), currentJvmStats.getMem().getHeapUsed(), JvmInfo.jvmInfo().getMem().getHeapMax(), buildPools(lastJvmStats, currentJvmStats));
} else if (avgCollectionTime > gcThreshold.debugThreshold && logger.isDebugEnabled()) {
logger.debug("[gc][{}][{}][{}] duration [{}], collections [{}]/[{}], total [{}]/[{}], memory [{}]->[{}]/[{}], all_pools {}",
gc.name(), seq, gc.collectionCount(), TimeValue.timeValueMillis(collectionTime), collections, TimeValue.timeValueMillis(currentJvmStats.timestamp() - lastJvmStats.timestamp()), TimeValue.timeValueMillis(collectionTime), gc.collectionTime(), lastJvmStats.mem().heapUsed(), currentJvmStats.mem().heapUsed(), JvmInfo.jvmInfo().mem().heapMax(), buildPools(lastJvmStats, currentJvmStats));
gc.getName(), seq, gc.getCollectionCount(), TimeValue.timeValueMillis(collectionTime), collections, TimeValue.timeValueMillis(currentJvmStats.getTimestamp() - lastJvmStats.getTimestamp()), TimeValue.timeValueMillis(collectionTime), gc.getCollectionTime(), lastJvmStats.getMem().getHeapUsed(), currentJvmStats.getMem().getHeapUsed(), JvmInfo.jvmInfo().getMem().getHeapMax(), buildPools(lastJvmStats, currentJvmStats));
}
}
lastJvmStats = currentJvmStats;
@ -214,16 +186,16 @@ public class JvmMonitorService extends AbstractLifecycleComponent<JvmMonitorServ
private String buildPools(JvmStats prev, JvmStats current) {
StringBuilder sb = new StringBuilder();
for (JvmStats.MemoryPool currentPool : current.mem()) {
for (JvmStats.MemoryPool currentPool : current.getMem()) {
JvmStats.MemoryPool prevPool = null;
for (JvmStats.MemoryPool pool : prev.mem()) {
for (JvmStats.MemoryPool pool : prev.getMem()) {
if (pool.getName().equals(currentPool.getName())) {
prevPool = pool;
break;
}
}
sb.append("{[").append(currentPool.name())
.append("] [").append(prevPool == null ? "?" : prevPool.used()).append("]->[").append(currentPool.used()).append("]/[").append(currentPool.getMax()).append("]}");
sb.append("{[").append(currentPool.getName())
.append("] [").append(prevPool == null ? "?" : prevPool.getUsed()).append("]->[").append(currentPool.getUsed()).append("]/[").append(currentPool.getMax()).append("]}");
}
return sb.toString();
}

View File

@ -51,7 +51,7 @@ public class JvmService extends AbstractComponent {
}
public synchronized JvmStats stats() {
if ((System.currentTimeMillis() - jvmStats.timestamp()) > refreshInterval.millis()) {
if ((System.currentTimeMillis() - jvmStats.getTimestamp()) > refreshInterval.millis()) {
jvmStats = JvmStats.jvmStats();
}
return jvmStats;

View File

@ -45,91 +45,14 @@ import java.util.concurrent.TimeUnit;
*/
public class JvmStats implements Streamable, Serializable, ToXContent {
private static boolean enableLastGc;
public static boolean isLastGcEnabled() {
return enableLastGc;
}
private final static RuntimeMXBean runtimeMXBean;
private final static MemoryMXBean memoryMXBean;
private final static ThreadMXBean threadMXBean;
private static Method managementFactoryPlatformMXBeansMethod;
private static Method getLastGcInfoMethod;
private static Method getMemoryUsageBeforeGcMethod;
private static Method getMemoryUsageAfterGcMethod;
private static Method getStartTimeMethod;
private static Method getEndTimeMethod;
private static Method getDurationMethod;
private static boolean bufferPoolsEnabled;
private static Class bufferPoolMXBeanClass;
private static Method bufferPoolMXBeanNameMethod;
private static Method bufferPoolMXBeanCountMethod;
private static Method bufferPoolMXBeanTotalCapacityMethod;
private static Method bufferPoolMXBeanMemoryUsedMethod;
static {
runtimeMXBean = ManagementFactory.getRuntimeMXBean();
memoryMXBean = ManagementFactory.getMemoryMXBean();
threadMXBean = ManagementFactory.getThreadMXBean();
try {
managementFactoryPlatformMXBeansMethod = ManagementFactory.class.getMethod("getPlatformMXBeans", Class.class);
} catch (Throwable e) {
managementFactoryPlatformMXBeansMethod = null;
}
try {
bufferPoolMXBeanClass = Class.forName("java.lang.management.BufferPoolMXBean");
bufferPoolMXBeanNameMethod = bufferPoolMXBeanClass.getMethod("getName");
bufferPoolMXBeanCountMethod = bufferPoolMXBeanClass.getMethod("getCount");
bufferPoolMXBeanTotalCapacityMethod = bufferPoolMXBeanClass.getMethod("getTotalCapacity");
bufferPoolMXBeanMemoryUsedMethod = bufferPoolMXBeanClass.getMethod("getMemoryUsed");
bufferPoolsEnabled = true;
} catch (Throwable t) {
bufferPoolsEnabled = false;
}
JvmInfo info = JvmInfo.jvmInfo();
boolean defaultEnableLastGc = false;
if (info.versionAsInteger() == 170) {
defaultEnableLastGc = info.versionUpdatePack() >= 4;
} else if (info.versionAsInteger() > 170) {
defaultEnableLastGc = true;
}
// always disable lastG, some reports it gives are strange...
defaultEnableLastGc = false;
boolean enableLastGc = Booleans.parseBoolean(System.getProperty("monitor.jvm.enable_last_gc"), defaultEnableLastGc);
if (enableLastGc) {
try {
Class sunGcClass = Class.forName("com.sun.management.GarbageCollectorMXBean");
Class gcInfoClass = Class.forName("com.sun.management.GcInfo");
getLastGcInfoMethod = sunGcClass.getDeclaredMethod("getLastGcInfo");
getLastGcInfoMethod.setAccessible(true);
getMemoryUsageBeforeGcMethod = gcInfoClass.getDeclaredMethod("getMemoryUsageBeforeGc");
getMemoryUsageBeforeGcMethod.setAccessible(true);
getMemoryUsageAfterGcMethod = gcInfoClass.getDeclaredMethod("getMemoryUsageAfterGc");
getMemoryUsageAfterGcMethod.setAccessible(true);
getStartTimeMethod = gcInfoClass.getDeclaredMethod("getStartTime");
getStartTimeMethod.setAccessible(true);
getEndTimeMethod = gcInfoClass.getDeclaredMethod("getEndTime");
getEndTimeMethod.setAccessible(true);
getDurationMethod = gcInfoClass.getDeclaredMethod("getDuration");
getDurationMethod.setAccessible(true);
} catch (Throwable ex) {
enableLastGc = false;
}
}
JvmStats.enableLastGc = enableLastGc;
}
public static JvmStats jvmStats() {
@ -183,66 +106,26 @@ public class JvmStats implements Streamable, Serializable, ToXContent {
stats.gc.collectors[i].name = GcNames.getByGcName(gcMxBean.getName(), gcMxBean.getName());
stats.gc.collectors[i].collectionCount = gcMxBean.getCollectionCount();
stats.gc.collectors[i].collectionTime = gcMxBean.getCollectionTime();
if (enableLastGc) {
}
try {
Object lastGcInfo = getLastGcInfoMethod.invoke(gcMxBean);
if (lastGcInfo != null) {
Map<String, MemoryUsage> usageBeforeGc = (Map<String, MemoryUsage>) getMemoryUsageBeforeGcMethod.invoke(lastGcInfo);
Map<String, MemoryUsage> usageAfterGc = (Map<String, MemoryUsage>) getMemoryUsageAfterGcMethod.invoke(lastGcInfo);
long startTime = (Long) getStartTimeMethod.invoke(lastGcInfo);
long endTime = (Long) getEndTimeMethod.invoke(lastGcInfo);
long duration = (Long) getDurationMethod.invoke(lastGcInfo);
long previousMemoryUsed = 0;
long memoryUsed = 0;
long memoryMax = 0;
for (Map.Entry<String, MemoryUsage> entry : usageBeforeGc.entrySet()) {
previousMemoryUsed += entry.getValue().getUsed();
}
for (Map.Entry<String, MemoryUsage> entry : usageAfterGc.entrySet()) {
MemoryUsage mu = entry.getValue();
memoryUsed += mu.getUsed();
memoryMax += mu.getMax();
}
stats.gc.collectors[i].lastGc = new GarbageCollector.LastGc(startTime, endTime, memoryMax, previousMemoryUsed, memoryUsed, duration);
}
} catch (Exception e) {
// e.printStackTrace();
}
}
}
if (bufferPoolsEnabled) {
try {
List bufferPools = (List) managementFactoryPlatformMXBeansMethod.invoke(null, bufferPoolMXBeanClass);
List<BufferPoolMXBean> bufferPools = ManagementFactory.getPlatformMXBeans(BufferPoolMXBean.class);
stats.bufferPools = new ArrayList<>(bufferPools.size());
for (Object bufferPool : bufferPools) {
String name = (String) bufferPoolMXBeanNameMethod.invoke(bufferPool);
Long count = (Long) bufferPoolMXBeanCountMethod.invoke(bufferPool);
Long totalCapacity = (Long) bufferPoolMXBeanTotalCapacityMethod.invoke(bufferPool);
Long memoryUsed = (Long) bufferPoolMXBeanMemoryUsedMethod.invoke(bufferPool);
stats.bufferPools.add(new BufferPool(name, count, totalCapacity, memoryUsed));
for (BufferPoolMXBean bufferPool : bufferPools) {
stats.bufferPools.add(new BufferPool(bufferPool.getName(), bufferPool.getCount(), bufferPool.getTotalCapacity(), bufferPool.getMemoryUsed()));
}
} catch (Throwable t) {
//t.printStackTrace();
}
// buffer pools are not available
}
return stats;
}
long timestamp = -1;
long uptime;
Mem mem;
Threads threads;
GarbageCollectors gc;
List<BufferPool> bufferPools;
private JvmStats() {
@ -253,44 +136,24 @@ public class JvmStats implements Streamable, Serializable, ToXContent {
this.uptime = uptime;
}
public long timestamp() {
return timestamp;
}
public long getTimestamp() {
return timestamp;
}
public TimeValue uptime() {
public TimeValue getUptime() {
return new TimeValue(uptime);
}
public TimeValue getUptime() {
return uptime();
}
public Mem mem() {
public Mem getMem() {
return this.mem;
}
public Mem getMem() {
return mem();
}
public Threads threads() {
public Threads getThreads() {
return threads;
}
public Threads getThreads() {
return threads();
}
public GarbageCollectors gc() {
return gc;
}
public GarbageCollectors getGc() {
return gc();
return gc;
}
@Override
@ -302,8 +165,8 @@ public class JvmStats implements Streamable, Serializable, ToXContent {
builder.startObject(Fields.MEM);
builder.byteSizeField(Fields.HEAP_USED_IN_BYTES, Fields.HEAP_USED, mem.heapUsed);
if (mem.heapUsedPercent() >= 0) {
builder.field(Fields.HEAP_USED_PERCENT, mem.heapUsedPercent());
if (mem.getHeapUsedPercent() >= 0) {
builder.field(Fields.HEAP_USED_PERCENT, mem.getHeapUsedPercent());
}
builder.byteSizeField(Fields.HEAP_COMMITTED_IN_BYTES, Fields.HEAP_COMMITTED, mem.heapCommitted);
builder.byteSizeField(Fields.HEAP_MAX_IN_BYTES, Fields.HEAP_MAX, mem.heapMax);
@ -312,7 +175,7 @@ public class JvmStats implements Streamable, Serializable, ToXContent {
builder.startObject(Fields.POOLS);
for (MemoryPool pool : mem) {
builder.startObject(pool.name(), XContentBuilder.FieldCaseConversion.NONE);
builder.startObject(pool.getName(), XContentBuilder.FieldCaseConversion.NONE);
builder.byteSizeField(Fields.USED_IN_BYTES, Fields.USED, pool.used);
builder.byteSizeField(Fields.MAX_IN_BYTES, Fields.MAX, pool.max);
@ -327,8 +190,8 @@ public class JvmStats implements Streamable, Serializable, ToXContent {
}
if (threads != null) {
builder.startObject(Fields.THREADS);
builder.field(Fields.COUNT, threads.count());
builder.field(Fields.PEAK_COUNT, threads.peakCount());
builder.field(Fields.COUNT, threads.getCount());
builder.field(Fields.PEAK_COUNT, threads.getPeakCount());
builder.endObject();
}
if (gc != null) {
@ -336,8 +199,8 @@ public class JvmStats implements Streamable, Serializable, ToXContent {
builder.startObject(Fields.COLLECTORS);
for (GarbageCollector collector : gc) {
builder.startObject(collector.name(), XContentBuilder.FieldCaseConversion.NONE);
builder.field(Fields.COLLECTION_COUNT, collector.collectionCount());
builder.startObject(collector.getName(), XContentBuilder.FieldCaseConversion.NONE);
builder.field(Fields.COLLECTION_COUNT, collector.getCollectionCount());
builder.timeValueField(Fields.COLLECTION_TIME_IN_MILLIS, Fields.COLLECTION_TIME, collector.collectionTime);
builder.endObject();
}
@ -349,8 +212,8 @@ public class JvmStats implements Streamable, Serializable, ToXContent {
if (bufferPools != null) {
builder.startObject(Fields.BUFFER_POOLS);
for (BufferPool bufferPool : bufferPools) {
builder.startObject(bufferPool.name(), XContentBuilder.FieldCaseConversion.NONE);
builder.field(Fields.COUNT, bufferPool.count());
builder.startObject(bufferPool.getName(), XContentBuilder.FieldCaseConversion.NONE);
builder.field(Fields.COUNT, bufferPool.getCount());
builder.byteSizeField(Fields.USED_IN_BYTES, Fields.USED, bufferPool.used);
builder.byteSizeField(Fields.TOTAL_CAPACITY_IN_BYTES, Fields.TOTAL_CAPACITY, bufferPool.totalCapacity);
builder.endObject();
@ -484,7 +347,7 @@ public class JvmStats implements Streamable, Serializable, ToXContent {
}
}
public GarbageCollector[] collectors() {
public GarbageCollector[] getCollectors() {
return this.collectors;
}
@ -496,114 +359,9 @@ public class JvmStats implements Streamable, Serializable, ToXContent {
public static class GarbageCollector implements Streamable, Serializable {
public static class LastGc implements Streamable {
long startTime;
long endTime;
long max;
long beforeUsed;
long afterUsed;
long duration;
LastGc() {
}
public LastGc(long startTime, long endTime, long max, long beforeUsed, long afterUsed, long duration) {
this.startTime = startTime;
this.endTime = endTime;
this.max = max;
this.beforeUsed = beforeUsed;
this.afterUsed = afterUsed;
this.duration = duration;
}
public long startTime() {
return this.startTime;
}
public long getStartTime() {
return startTime();
}
public long endTime() {
return this.endTime;
}
public long getEndTime() {
return endTime();
}
public ByteSizeValue max() {
return new ByteSizeValue(max);
}
public ByteSizeValue getMax() {
return max();
}
public ByteSizeValue afterUsed() {
return new ByteSizeValue(afterUsed);
}
public ByteSizeValue getAfterUsed() {
return afterUsed();
}
public ByteSizeValue beforeUsed() {
return new ByteSizeValue(beforeUsed);
}
public ByteSizeValue getBeforeUsed() {
return beforeUsed();
}
public ByteSizeValue reclaimed() {
return new ByteSizeValue(beforeUsed - afterUsed);
}
public ByteSizeValue getReclaimed() {
return reclaimed();
}
public TimeValue duration() {
return new TimeValue(this.duration);
}
public TimeValue getDuration() {
return duration();
}
public static LastGc readLastGc(StreamInput in) throws IOException {
LastGc lastGc = new LastGc();
lastGc.readFrom(in);
return lastGc;
}
@Override
public void readFrom(StreamInput in) throws IOException {
startTime = in.readVLong();
endTime = in.readVLong();
max = in.readVLong();
beforeUsed = in.readVLong();
afterUsed = in.readVLong();
duration = in.readVLong();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeVLong(startTime);
out.writeVLong(endTime);
out.writeVLong(max);
out.writeVLong(beforeUsed);
out.writeVLong(afterUsed);
out.writeVLong(duration);
}
}
String name;
long collectionCount;
long collectionTime;
LastGc lastGc;
GarbageCollector() {
}
@ -619,9 +377,6 @@ public class JvmStats implements Streamable, Serializable, ToXContent {
name = in.readString();
collectionCount = in.readVLong();
collectionTime = in.readVLong();
if (in.readBoolean()) {
lastGc = LastGc.readLastGc(in);
}
}
@Override
@ -629,44 +384,18 @@ public class JvmStats implements Streamable, Serializable, ToXContent {
out.writeString(name);
out.writeVLong(collectionCount);
out.writeVLong(collectionTime);
if (lastGc == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
lastGc.writeTo(out);
}
}
public String name() {
return name;
}
public String getName() {
return name();
}
public long collectionCount() {
return collectionCount;
return this.name;
}
public long getCollectionCount() {
return collectionCount();
}
public TimeValue collectionTime() {
return new TimeValue(collectionTime, TimeUnit.MILLISECONDS);
return this.collectionCount;
}
public TimeValue getCollectionTime() {
return collectionTime();
}
public LastGc lastGc() {
return this.lastGc;
}
public LastGc getLastGc() {
return lastGc();
return new TimeValue(collectionTime, TimeUnit.MILLISECONDS);
}
}
@ -678,20 +407,12 @@ public class JvmStats implements Streamable, Serializable, ToXContent {
Threads() {
}
public int count() {
public int getCount() {
return count;
}
public int getCount() {
return count();
}
public int peakCount() {
return peakCount;
}
public int getPeakCount() {
return peakCount();
return peakCount;
}
public static Threads readThreads(StreamInput in) throws IOException {
@ -740,44 +461,24 @@ public class JvmStats implements Streamable, Serializable, ToXContent {
return pool;
}
public String name() {
return this.name;
}
public String getName() {
return this.name;
}
public ByteSizeValue used() {
public ByteSizeValue getUsed() {
return new ByteSizeValue(used);
}
public ByteSizeValue getUsed() {
return used();
}
public ByteSizeValue max() {
public ByteSizeValue getMax() {
return new ByteSizeValue(max);
}
public ByteSizeValue getMax() {
return max();
}
public ByteSizeValue peakUsed() {
public ByteSizeValue getPeakUsed() {
return new ByteSizeValue(peakUsed);
}
public ByteSizeValue getPeakUsed() {
return peakUsed();
}
public ByteSizeValue peakMax() {
return new ByteSizeValue(peakMax);
}
public ByteSizeValue getPeakMax() {
return peakMax();
return new ByteSizeValue(peakMax);
}
@Override
@ -849,67 +550,37 @@ public class JvmStats implements Streamable, Serializable, ToXContent {
}
}
public ByteSizeValue heapCommitted() {
public ByteSizeValue getHeapCommitted() {
return new ByteSizeValue(heapCommitted);
}
public ByteSizeValue getHeapCommitted() {
return heapCommitted();
}
public ByteSizeValue heapUsed() {
return new ByteSizeValue(heapUsed);
}
public ByteSizeValue getHeapUsed() {
return heapUsed();
}
/**
* returns the maximum heap size. 0 bytes signals unknown.
*/
public ByteSizeValue heapMax() {
return new ByteSizeValue(heapMax);
return new ByteSizeValue(heapUsed);
}
/**
* returns the maximum heap size. 0 bytes signals unknown.
*/
public ByteSizeValue getHeapMax() {
return heapMax();
return new ByteSizeValue(heapMax);
}
/**
* returns the heap usage in percent. -1 signals unknown.
*/
public short heapUsedPercent() {
public short getHeapUsedPercent() {
if (heapMax == 0) {
return -1;
}
return (short) (heapUsed * 100 / heapMax);
}
/**
* returns the heap usage in percent. -1 signals unknown.
*/
public short getHeapUsedPrecent() {
return heapUsedPercent();
}
public ByteSizeValue nonHeapCommitted() {
public ByteSizeValue getNonHeapCommitted() {
return new ByteSizeValue(nonHeapCommitted);
}
public ByteSizeValue getNonHeapCommitted() {
return nonHeapCommitted();
}
public ByteSizeValue nonHeapUsed() {
return new ByteSizeValue(nonHeapUsed);
}
public ByteSizeValue getNonHeapUsed() {
return nonHeapUsed();
return new ByteSizeValue(nonHeapUsed);
}
}
@ -930,36 +601,20 @@ public class JvmStats implements Streamable, Serializable, ToXContent {
this.used = used;
}
public String name() {
return this.name;
}
public String getName() {
return this.name;
}
public long count() {
return this.count;
}
public long getCount() {
return this.count;
}
public ByteSizeValue totalCapacity() {
public ByteSizeValue getTotalCapacity() {
return new ByteSizeValue(totalCapacity);
}
public ByteSizeValue getTotalCapacity() {
return totalCapacity();
}
public ByteSizeValue used() {
return new ByteSizeValue(used);
}
public ByteSizeValue getUsed() {
return used();
return new ByteSizeValue(used);
}
@Override

View File

@ -249,7 +249,7 @@ public class RestNodesAction extends AbstractCatAction {
table.addCell(jvmInfo == null ? null : jvmInfo.version());
table.addCell(fsStats == null ? null : fsStats.getTotal().getAvailable());
table.addCell(jvmStats == null ? null : jvmStats.getMem().getHeapUsed());
table.addCell(jvmStats == null ? null : jvmStats.getMem().getHeapUsedPrecent());
table.addCell(jvmStats == null ? null : jvmStats.getMem().getHeapUsedPercent());
table.addCell(jvmInfo == null ? null : jvmInfo.getMem().getHeapMax());
table.addCell(osStats == null ? null : osStats.getMem() == null ? null : osStats.getMem().used());
table.addCell(osStats == null ? null : osStats.getMem() == null ? null : osStats.getMem().usedPercent());
@ -260,7 +260,7 @@ public class RestNodesAction extends AbstractCatAction {
table.addCell(processInfo == null ? null : processInfo.getMaxFileDescriptors());
table.addCell(osStats == null ? null : osStats.getLoadAverage().length < 1 ? null : String.format(Locale.ROOT, "%.2f", osStats.getLoadAverage()[0]));
table.addCell(jvmStats == null ? null : jvmStats.uptime());
table.addCell(jvmStats == null ? null : jvmStats.getUptime());
table.addCell(node.clientNode() ? "c" : node.dataNode() ? "d" : "-");
table.addCell(masterId == null ? "x" : masterId.equals(node.id()) ? "*" : node.masterNode() ? "m" : "-");
table.addCell(node.name());

View File

@ -77,7 +77,6 @@ import java.nio.channels.CancelledKeyException;
import java.util.*;
import java.util.concurrent.*;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
@ -195,9 +194,9 @@ public class NettyTransport extends AbstractLifecycleComponent<Transport> implem
}
long defaultReceiverPredictor = 512 * 1024;
if (JvmInfo.jvmInfo().mem().directMemoryMax().bytes() > 0) {
if (JvmInfo.jvmInfo().getMem().getDirectMemoryMax().bytes() > 0) {
// we can guess a better default...
long l = (long) ((0.3 * JvmInfo.jvmInfo().mem().directMemoryMax().bytes()) / workerCount);
long l = (long) ((0.3 * JvmInfo.jvmInfo().getMem().getDirectMemoryMax().bytes()) / workerCount);
defaultReceiverPredictor = Math.min(defaultReceiverPredictor, Math.max(l, 64 * 1024));
}

View File

@ -36,7 +36,7 @@ import java.io.StreamCorruptedException;
*/
public class SizeHeaderFrameDecoder extends FrameDecoder {
private static final long NINETY_PER_HEAP_SIZE = (long) (JvmInfo.jvmInfo().mem().heapMax().bytes() * 0.9);
private static final long NINETY_PER_HEAP_SIZE = (long) (JvmInfo.jvmInfo().getMem().getHeapMax().bytes() * 0.9);
@Override
protected Object decode(ChannelHandlerContext ctx, Channel channel, ChannelBuffer buffer) throws Exception {

View File

@ -127,9 +127,9 @@ public class ScrollSearchBenchmark {
}
if (scrollRequestCounter % 20 == 0) {
long avgTimeSpent = sumTimeSpent / 20;
JvmStats.Mem mem = JvmStats.jvmStats().mem();
JvmStats.Mem mem = JvmStats.jvmStats().getMem();
System.out.printf(Locale.ENGLISH, "Cursor location=%d, avg time spent=%d ms\n", (requestSize * scrollRequestCounter), (avgTimeSpent));
System.out.printf(Locale.ENGLISH, "heap max=%s, used=%s, percentage=%d\n", mem.getHeapMax(), mem.getHeapUsed(), mem.getHeapUsedPrecent());
System.out.printf(Locale.ENGLISH, "heap max=%s, used=%s, percentage=%d\n", mem.getHeapMax(), mem.getHeapUsed(), mem.getHeapUsedPercent());
sumTimeSpent = 0;
}
if (searchResponse.getHits().hits().length == 0) {