You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2018/12/04 01:40:04 UTC

[GitHub] dclim closed pull request #6588: autosize processing buffers based on direct memory sizing by default

dclim closed pull request #6588: autosize processing buffers based on direct memory sizing by default
URL: https://github.com/apache/incubator-druid/pull/6588
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/core/src/main/java/org/apache/druid/java/util/common/concurrent/ExecutorServiceConfig.java b/core/src/main/java/org/apache/druid/java/util/common/concurrent/ExecutorServiceConfig.java
index acfe8ca29f5..6d6327fbdd2 100644
--- a/core/src/main/java/org/apache/druid/java/util/common/concurrent/ExecutorServiceConfig.java
+++ b/core/src/main/java/org/apache/druid/java/util/common/concurrent/ExecutorServiceConfig.java
@@ -19,6 +19,7 @@
 
 package org.apache.druid.java.util.common.concurrent;
 
+import org.apache.druid.utils.JvmUtils;
 import org.skife.config.Config;
 import org.skife.config.Default;
 
@@ -38,7 +39,7 @@ public int getNumThreads()
     if (numThreadsConfigured != DEFAULT_NUM_THREADS) {
       return numThreadsConfigured;
     } else {
-      return Math.max(Runtime.getRuntime().availableProcessors() - 1, 1);
+      return Math.max(JvmUtils.getRuntimeInfo().getAvailableProcessors() - 1, 1);
     }
   }
 
diff --git a/core/src/main/java/org/apache/druid/java/util/emitter/core/BaseHttpEmittingConfig.java b/core/src/main/java/org/apache/druid/java/util/emitter/core/BaseHttpEmittingConfig.java
index 76e3a90e3ff..46d25a4dd91 100644
--- a/core/src/main/java/org/apache/druid/java/util/emitter/core/BaseHttpEmittingConfig.java
+++ b/core/src/main/java/org/apache/druid/java/util/emitter/core/BaseHttpEmittingConfig.java
@@ -21,6 +21,7 @@
 
 import com.fasterxml.jackson.annotation.JsonProperty;
 import org.apache.druid.java.util.common.Pair;
+import org.apache.druid.utils.JvmUtils;
 
 import javax.validation.constraints.Min;
 
@@ -33,7 +34,8 @@
   public static final int DEFAULT_MAX_BATCH_SIZE;
   public static final int DEFAULT_BATCH_QUEUE_SIZE_LIMIT;
   static {
-    Pair<Integer, Integer> batchConfigPair = getDefaultBatchSizeAndLimit(Runtime.getRuntime().maxMemory());
+    Pair<Integer, Integer> batchConfigPair =
+        getDefaultBatchSizeAndLimit(JvmUtils.getRuntimeInfo().getMaxHeapSizeBytes());
     DEFAULT_MAX_BATCH_SIZE = batchConfigPair.lhs;
     DEFAULT_BATCH_QUEUE_SIZE_LIMIT = batchConfigPair.rhs;
   }
diff --git a/core/src/main/java/org/apache/druid/java/util/http/client/HttpClientConfig.java b/core/src/main/java/org/apache/druid/java/util/http/client/HttpClientConfig.java
index 158c21660a2..6c2e80154ae 100644
--- a/core/src/main/java/org/apache/druid/java/util/http/client/HttpClientConfig.java
+++ b/core/src/main/java/org/apache/druid/java/util/http/client/HttpClientConfig.java
@@ -19,6 +19,7 @@
 
 package org.apache.druid.java.util.http.client;
 
+import org.apache.druid.utils.JvmUtils;
 import org.joda.time.Duration;
 import org.joda.time.Period;
 
@@ -70,7 +71,7 @@ public String getEncodingString()
   private static final int DEFAULT_BOSS_COUNT = 1;
 
   // Default from SelectorUtil.DEFAULT_IO_THREADS, which is private:
-  private static final int DEFAULT_WORKER_COUNT = Runtime.getRuntime().availableProcessors() * 2;
+  private static final int DEFAULT_WORKER_COUNT = JvmUtils.getRuntimeInfo().getAvailableProcessors() * 2;
 
   private static final Duration DEFAULT_UNUSED_CONNECTION_TIMEOUT_DURATION = new Period("PT4M").toStandardDuration();
 
diff --git a/core/src/main/java/org/apache/druid/utils/JvmUtils.java b/core/src/main/java/org/apache/druid/utils/JvmUtils.java
new file mode 100644
index 00000000000..9319937903b
--- /dev/null
+++ b/core/src/main/java/org/apache/druid/utils/JvmUtils.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.utils;
+
+import com.google.inject.Inject;
+
+import java.lang.management.ManagementFactory;
+import java.lang.management.ThreadMXBean;
+
+public class JvmUtils
+{
+  @Inject
+  private static RuntimeInfo runtimeInfo = new RuntimeInfo();
+
+  public static RuntimeInfo getRuntimeInfo()
+  {
+    return runtimeInfo;
+  }
+
+  private static final ThreadMXBean THREAD_MX_BEAN = ManagementFactory.getThreadMXBean();
+
+  public static boolean isThreadCpuTimeEnabled()
+  {
+    return THREAD_MX_BEAN.isThreadCpuTimeSupported() && THREAD_MX_BEAN.isThreadCpuTimeEnabled();
+  }
+
+  public static long safeGetThreadCpuTime()
+  {
+    if (!isThreadCpuTimeEnabled()) {
+      return 0L;
+    } else {
+      return getCurrentThreadCpuTime();
+    }
+  }
+
+  /**
+   * Returns the total CPU time for current thread.
+   * This method should be called after verifying that cpu time measurement for current thread is supported by JVM
+   *
+   * @return total CPU time for the current thread in nanoseconds.
+   *
+   * @throws UnsupportedOperationException if the Java virtual machine does not support CPU time measurement for
+   * the current thread.
+   */
+  public static long getCurrentThreadCpuTime()
+  {
+    return THREAD_MX_BEAN.getCurrentThreadCpuTime();
+  }
+}
diff --git a/core/src/main/java/org/apache/druid/common/utils/VMUtils.java b/core/src/main/java/org/apache/druid/utils/RuntimeInfo.java
similarity index 63%
rename from core/src/main/java/org/apache/druid/common/utils/VMUtils.java
rename to core/src/main/java/org/apache/druid/utils/RuntimeInfo.java
index b18f3260301..1dba77d57de 100644
--- a/core/src/main/java/org/apache/druid/common/utils/VMUtils.java
+++ b/core/src/main/java/org/apache/druid/utils/RuntimeInfo.java
@@ -17,47 +17,35 @@
  * under the License.
  */
 
-package org.apache.druid.common.utils;
+package org.apache.druid.utils;
 
 import org.apache.druid.java.util.common.UOE;
 
-import java.lang.management.ManagementFactory;
-import java.lang.management.ThreadMXBean;
 import java.lang.reflect.InvocationTargetException;
 
-public class VMUtils
+public class RuntimeInfo
 {
-  private static final ThreadMXBean THREAD_MX_BEAN = ManagementFactory.getThreadMXBean();
+  public int getAvailableProcessors()
+  {
+    return Runtime.getRuntime().availableProcessors();
+  }
 
-  public static boolean isThreadCpuTimeEnabled()
+  public long getMaxHeapSizeBytes()
   {
-    return THREAD_MX_BEAN.isThreadCpuTimeSupported() && THREAD_MX_BEAN.isThreadCpuTimeEnabled();
+    return Runtime.getRuntime().maxMemory();
   }
 
-  public static long safeGetThreadCpuTime()
+  public long getTotalHeapSizeBytes()
   {
-    if (!isThreadCpuTimeEnabled()) {
-      return 0L;
-    } else {
-      return getCurrentThreadCpuTime();
-    }
+    return Runtime.getRuntime().totalMemory();
   }
 
-  /**
-   * Returns the total CPU time for current thread.
-   * This method should be called after verifying that cpu time measurement for current thread is supported by JVM
-   *
-   * @return total CPU time for the current thread in nanoseconds.
-   *
-   * @throws UnsupportedOperationException if the Java virtual machine does not support CPU time measurement for
-   * the current thread.
-   */
-  public static long getCurrentThreadCpuTime()
+  public long getFreeHeapSizeBytes()
   {
-    return THREAD_MX_BEAN.getCurrentThreadCpuTime();
+    return Runtime.getRuntime().freeMemory();
   }
 
-  public static long getMaxDirectMemory() throws UnsupportedOperationException
+  public long getDirectMemorySizeBytes()
   {
     try {
       Class<?> vmClass = Class.forName("sun.misc.VM");
diff --git a/core/src/test/java/org/apache/druid/common/utils/VMUtilsTest.java b/core/src/test/java/org/apache/druid/utils/JvmUtilsTest.java
similarity index 89%
rename from core/src/test/java/org/apache/druid/common/utils/VMUtilsTest.java
rename to core/src/test/java/org/apache/druid/utils/JvmUtilsTest.java
index 92e9d98db4f..c9e7599e0b1 100644
--- a/core/src/test/java/org/apache/druid/common/utils/VMUtilsTest.java
+++ b/core/src/test/java/org/apache/druid/utils/JvmUtilsTest.java
@@ -17,18 +17,18 @@
  * under the License.
  */
 
-package org.apache.druid.common.utils;
+package org.apache.druid.utils;
 
 import org.junit.Assert;
 import org.junit.Test;
 
-public class VMUtilsTest
+public class JvmUtilsTest
 {
   @Test
   public void testgetMaxDirectMemory()
   {
     try {
-      long maxMemory = VMUtils.getMaxDirectMemory();
+      long maxMemory = JvmUtils.getRuntimeInfo().getDirectMemorySizeBytes();
       Assert.assertTrue((maxMemory > 0));
     }
     catch (UnsupportedOperationException expected) {
diff --git a/docs/content/configuration/index.md b/docs/content/configuration/index.md
index f30c555eb44..333ee12f1a9 100644
--- a/docs/content/configuration/index.md
+++ b/docs/content/configuration/index.md
@@ -1058,7 +1058,7 @@ Processing properties set on the Middlemanager will be passed through to Peons.
 
 |Property|Description|Default|
 |--------|-----------|-------|
-|`druid.processing.buffer.sizeBytes`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Historical and Realtime nodes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed.|1073741824 (1GB)|
+|`druid.processing.buffer.sizeBytes`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Historical and Realtime nodes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed.|auto (max 1GB)|
 |`druid.processing.buffer.poolCacheMaxCount`|processing buffer pool caches the buffers for later use, this is the maximum count cache will grow to. note that pool can create more buffers than it can cache if necessary.|Integer.MAX_VALUE|
 |`druid.processing.formatString`|Realtime and historical nodes use this format string to name their processing threads.|processing-%s|
 |`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy v2) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`|
@@ -1209,7 +1209,7 @@ The broker uses processing configs for nested groupBy queries. And, optionally,
 
 |Property|Description|Default|
 |--------|-----------|-------|
-|`druid.processing.buffer.sizeBytes`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Historical and Realtime nodes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed.|1073741824 (1GB)|
+|`druid.processing.buffer.sizeBytes`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Historical and Realtime nodes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed.|auto (max 1GB)|
 |`druid.processing.buffer.poolCacheMaxCount`|processing buffer pool caches the buffers for later use, this is the maximum count cache will grow to. note that pool can create more buffers than it can cache if necessary.|Integer.MAX_VALUE|
 |`druid.processing.formatString`|Realtime and historical nodes use this format string to name their processing threads.|processing-%s|
 |`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy v2) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`|
@@ -1334,7 +1334,7 @@ Druid uses Jetty to serve HTTP requests.
 
 |Property|Description|Default|
 |--------|-----------|-------|
-|`druid.processing.buffer.sizeBytes`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Historical and Realtime nodes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed.|1073741824 (1GB)|
+|`druid.processing.buffer.sizeBytes`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Historical and Realtime nodes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed.|auto (max 1GB)|
 |`druid.processing.buffer.poolCacheMaxCount`|processing buffer pool caches the buffers for later use, this is the maximum count cache will grow to. note that pool can create more buffers than it can cache if necessary.|Integer.MAX_VALUE|
 |`druid.processing.formatString`|Realtime and historical nodes use this format string to name their processing threads.|processing-%s|
 |`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy v2) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`|
diff --git a/docs/content/configuration/realtime.md b/docs/content/configuration/realtime.md
index b75fca5879a..9f383f4e6a1 100644
--- a/docs/content/configuration/realtime.md
+++ b/docs/content/configuration/realtime.md
@@ -40,7 +40,7 @@ The realtime node uses several of the global configs in [Configuration](../confi
 
 |Property|Description|Default|
 |--------|-----------|-------|
-|`druid.processing.buffer.sizeBytes`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Historical and Realtime nodes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed.|1073741824 (1GB)|
+|`druid.processing.buffer.sizeBytes`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Historical and Realtime nodes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed.|auto (max 1GB)|
 |`druid.processing.formatString`|Realtime and historical nodes use this format string to name their processing threads.|processing-%s|
 |`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy v2) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`|
 |`druid.processing.numThreads`|The number of processing threads to have available for parallel processing of segments. Our rule of thumb is `num_cores - 1`, which means that even under heavy load there will still be one core available to do background tasks like talking with ZooKeeper and pulling down segments. If only one core is available, this property defaults to the value `1`.|Number of cores - 1 (or 1)|
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/worker/config/WorkerConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/worker/config/WorkerConfig.java
index 11a346926a0..e574a0d151c 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/worker/config/WorkerConfig.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/worker/config/WorkerConfig.java
@@ -21,6 +21,7 @@
 
 import com.fasterxml.jackson.annotation.JsonProperty;
 import org.apache.druid.server.DruidNode;
+import org.apache.druid.utils.JvmUtils;
 
 import javax.validation.constraints.Min;
 import javax.validation.constraints.NotNull;
@@ -39,7 +40,7 @@
 
   @JsonProperty
   @Min(1)
-  private int capacity = Math.max(1, Runtime.getRuntime().availableProcessors() - 1);
+  private int capacity = Math.max(1, JvmUtils.getRuntimeInfo().getAvailableProcessors() - 1);
 
   public String getIp()
   {
diff --git a/processing/src/main/java/org/apache/druid/guice/GuiceInjectors.java b/processing/src/main/java/org/apache/druid/guice/GuiceInjectors.java
index c40498e6b3f..77e39ec3e8e 100644
--- a/processing/src/main/java/org/apache/druid/guice/GuiceInjectors.java
+++ b/processing/src/main/java/org/apache/druid/guice/GuiceInjectors.java
@@ -40,6 +40,7 @@
         new DruidGuiceExtensions(),
         new JacksonModule(),
         new PropertiesModule(Arrays.asList("common.runtime.properties", "runtime.properties")),
+        new RuntimeInfoModule(),
         new ConfigModule(),
         new NullHandlingModule(),
         binder -> {
diff --git a/processing/src/main/java/org/apache/druid/guice/RuntimeInfoModule.java b/processing/src/main/java/org/apache/druid/guice/RuntimeInfoModule.java
new file mode 100644
index 00000000000..c09c1330ccb
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/guice/RuntimeInfoModule.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.guice;
+
+import com.google.inject.Binder;
+import com.google.inject.Module;
+import org.apache.druid.utils.RuntimeInfo;
+
+public class RuntimeInfoModule implements Module
+{
+  @Override
+  public void configure(Binder binder)
+  {
+    binder.requestStaticInjection(RuntimeInfo.class);
+  }
+}
diff --git a/processing/src/main/java/org/apache/druid/query/CPUTimeMetricQueryRunner.java b/processing/src/main/java/org/apache/druid/query/CPUTimeMetricQueryRunner.java
index 67fbdf82836..5bbe9417363 100644
--- a/processing/src/main/java/org/apache/druid/query/CPUTimeMetricQueryRunner.java
+++ b/processing/src/main/java/org/apache/druid/query/CPUTimeMetricQueryRunner.java
@@ -20,12 +20,12 @@
 package org.apache.druid.query;
 
 import com.google.common.base.Supplier;
-import org.apache.druid.common.utils.VMUtils;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.java.util.common.guava.SequenceWrapper;
 import org.apache.druid.java.util.common.guava.Sequences;
 import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.utils.JvmUtils;
 
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicLong;
@@ -46,7 +46,7 @@ private CPUTimeMetricQueryRunner(
       boolean report
   )
   {
-    if (!VMUtils.isThreadCpuTimeEnabled()) {
+    if (!JvmUtils.isThreadCpuTimeEnabled()) {
       throw new ISE("Cpu time must enabled");
     }
     this.delegate = delegate;
@@ -69,12 +69,12 @@ private CPUTimeMetricQueryRunner(
           @Override
           public <RetType> RetType wrap(Supplier<RetType> sequenceProcessing)
           {
-            final long start = VMUtils.getCurrentThreadCpuTime();
+            final long start = JvmUtils.getCurrentThreadCpuTime();
             try {
               return sequenceProcessing.get();
             }
             finally {
-              cpuTimeAccumulator.addAndGet(VMUtils.getCurrentThreadCpuTime() - start);
+              cpuTimeAccumulator.addAndGet(JvmUtils.getCurrentThreadCpuTime() - start);
             }
           }
 
@@ -100,7 +100,7 @@ public void after(boolean isDone, Throwable thrown)
       boolean report
   )
   {
-    if (!VMUtils.isThreadCpuTimeEnabled()) {
+    if (!JvmUtils.isThreadCpuTimeEnabled()) {
       return delegate;
     } else {
       return new CPUTimeMetricQueryRunner<>(delegate, queryToolChest, emitter, accumulator, report);
diff --git a/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java b/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java
index f91957e4721..b35f29743f4 100644
--- a/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java
+++ b/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java
@@ -20,17 +20,57 @@
 package org.apache.druid.query;
 
 import org.apache.druid.java.util.common.concurrent.ExecutorServiceConfig;
+import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.segment.column.ColumnConfig;
+import org.apache.druid.utils.JvmUtils;
 import org.skife.config.Config;
 
+import java.util.concurrent.atomic.AtomicReference;
+
 public abstract class DruidProcessingConfig extends ExecutorServiceConfig implements ColumnConfig
 {
+  private static final Logger log = new Logger(DruidProcessingConfig.class);
+
   public static final int DEFAULT_NUM_MERGE_BUFFERS = -1;
+  public static final int DEFAULT_PROCESSING_BUFFER_SIZE_BYTES = -1;
+  public static final int MAX_DEFAULT_PROCESSING_BUFFER_SIZE_BYTES = 1024 * 1024 * 1024;
+
+  private AtomicReference<Integer> computedBufferSizeBytes = new AtomicReference<>();
 
   @Config({"druid.computation.buffer.size", "${base_path}.buffer.sizeBytes"})
+  public int intermediateComputeSizeBytesConfigured()
+  {
+    return DEFAULT_PROCESSING_BUFFER_SIZE_BYTES;
+  }
+
   public int intermediateComputeSizeBytes()
   {
-    return 1024 * 1024 * 1024;
+    int sizeBytesConfigured = intermediateComputeSizeBytesConfigured();
+    if (sizeBytesConfigured != DEFAULT_PROCESSING_BUFFER_SIZE_BYTES) {
+      return sizeBytesConfigured;
+    } else if (computedBufferSizeBytes.get() != null) {
+      return computedBufferSizeBytes.get();
+    }
+
+    long directSizeBytes = JvmUtils.getRuntimeInfo().getDirectMemorySizeBytes();
+
+    int numProcessingThreads = getNumThreads();
+    int numMergeBuffers = getNumMergeBuffers();
+    int totalNumBuffers = numMergeBuffers + numProcessingThreads;
+    int sizePerBuffer = (int) ((double) directSizeBytes / (double) (totalNumBuffers + 1));
+
+    final int computedSizePerBuffer = Math.min(sizePerBuffer, MAX_DEFAULT_PROCESSING_BUFFER_SIZE_BYTES);
+    if (computedBufferSizeBytes.compareAndSet(null, computedSizePerBuffer)) {
+      log.info(
+          "Auto sizing buffers to [%,d] bytes each for [%,d] processing and [%,d] merge buffers " +
+          "out of [%,d] max direct memory",
+          computedSizePerBuffer,
+          numProcessingThreads,
+          numMergeBuffers,
+          directSizeBytes
+      );
+    }
+    return computedSizePerBuffer;
   }
 
   @Config({"druid.computation.buffer.poolCacheMaxCount", "${base_path}.buffer.poolCacheMaxCount"})
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/ParallelCombiner.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/ParallelCombiner.java
index 0ef98c6913d..7f142fc94ca 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/ParallelCombiner.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/ParallelCombiner.java
@@ -261,9 +261,10 @@ public ByteBuffer get()
     throw new ISE(
         "Cannot find a proper leaf combine degree for the combining tree. "
         + "Each node of the combining tree requires a buffer of [%d] bytes. "
-        + "Try increasing druid.processing.buffer.sizeBytes for larger buffer or "
+        + "Try increasing druid.processing.buffer.sizeBytes (currently [%d] bytes) for larger buffer or "
         + "druid.query.groupBy.intermediateCombineDegree for a smaller tree",
-        requiredMinimumBufferCapacity
+        requiredMinimumBufferCapacity,
+        combineBuffer.capacity()
     );
   }
 
diff --git a/processing/src/main/java/org/apache/druid/query/lookup/LookupConfig.java b/processing/src/main/java/org/apache/druid/query/lookup/LookupConfig.java
index 6746e08b4b3..dd804beeb4b 100644
--- a/processing/src/main/java/org/apache/druid/query/lookup/LookupConfig.java
+++ b/processing/src/main/java/org/apache/druid/query/lookup/LookupConfig.java
@@ -22,6 +22,7 @@
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.utils.JvmUtils;
 
 import javax.validation.constraints.Min;
 import java.util.Objects;
@@ -38,7 +39,7 @@
 
   @Min(1)
   @JsonProperty("numLookupLoadingThreads")
-  private int numLookupLoadingThreads = Math.max(1, Runtime.getRuntime().availableProcessors() / 2);
+  private int numLookupLoadingThreads = Math.max(1, JvmUtils.getRuntimeInfo().getAvailableProcessors() / 2);
 
   @Min(1)
   @JsonProperty("coordinatorFetchRetries")
diff --git a/processing/src/test/java/org/apache/druid/query/DruidProcessingConfigTest.java b/processing/src/test/java/org/apache/druid/query/DruidProcessingConfigTest.java
index e00a2db1a3f..4a08f78009f 100644
--- a/processing/src/test/java/org/apache/druid/query/DruidProcessingConfigTest.java
+++ b/processing/src/test/java/org/apache/druid/query/DruidProcessingConfigTest.java
@@ -20,39 +20,106 @@
 package org.apache.druid.query;
 
 import com.google.common.collect.ImmutableMap;
+import com.google.inject.Guice;
+import com.google.inject.Injector;
 import org.apache.druid.java.util.common.config.Config;
+import org.apache.druid.utils.JvmUtils;
+import org.apache.druid.utils.RuntimeInfo;
 import org.junit.Assert;
 import org.junit.Test;
 import org.skife.config.ConfigurationObjectFactory;
 
+import java.util.Map;
 import java.util.Properties;
 
 /**
  */
 public class DruidProcessingConfigTest
 {
+  private static final long bufferSize = 1024L * 1024L * 1024L;
+  private static final int numProcessors = 4;
+  private static final long directSize = bufferSize * (3L + 2L + 1L);
+  private static final long heapSize = bufferSize * 2L;
+
+  private static Injector makeInjector(int numProcessors, long directMemorySize, long heapSize)
+  {
+    return makeInjector(numProcessors, directMemorySize, heapSize, new Properties(), null);
+  }
+
+  private static Injector makeInjector(
+      int numProcessors,
+      long directMemorySize,
+      long heapSize,
+      Properties props,
+      Map<String, String> replacements
+  )
+  {
+    return Guice.createInjector(
+        binder -> {
+          binder.bind(RuntimeInfo.class).toInstance(new MockRuntimeInfo(numProcessors, directMemorySize, heapSize));
+          binder.requestStaticInjection(JvmUtils.class);
+          ConfigurationObjectFactory factory = Config.createFactory(props);
+          DruidProcessingConfig config;
+          if (replacements != null) {
+            config = factory.buildWithReplacements(
+                DruidProcessingConfig.class,
+                replacements
+            );
+          } else {
+            config = factory.build(DruidProcessingConfig.class);
+          }
+          binder.bind(ConfigurationObjectFactory.class).toInstance(factory);
+          binder.bind(DruidProcessingConfig.class).toInstance(config);
+        }
+    );
+  }
 
   @Test
-  public void testDeserialization()
+  public void testDefaultsMultiProcessor()
   {
-    ConfigurationObjectFactory factory = Config.createFactory(new Properties());
+    Injector injector = makeInjector(numProcessors, directSize, heapSize);
+    DruidProcessingConfig config = injector.getInstance(DruidProcessingConfig.class);
+
+    Assert.assertEquals(Integer.MAX_VALUE, config.poolCacheMaxCount());
+    Assert.assertEquals(numProcessors - 1, config.getNumThreads());
+    Assert.assertEquals(Math.max(2, config.getNumThreads() / 4), config.getNumMergeBuffers());
+    Assert.assertEquals(0, config.columnCacheSizeBytes());
+    Assert.assertFalse(config.isFifo());
+    Assert.assertEquals(System.getProperty("java.io.tmpdir"), config.getTmpDir());
+    Assert.assertEquals(bufferSize, config.intermediateComputeSizeBytes());
+  }
 
-    //with defaults
-    DruidProcessingConfig config = factory.build(DruidProcessingConfig.class);
+  @Test
+  public void testDefaultsSingleProcessor()
+  {
+    Injector injector = makeInjector(1, bufferSize * 4L, heapSize);
+    DruidProcessingConfig config = injector.getInstance(DruidProcessingConfig.class);
 
-    Assert.assertEquals(1024 * 1024 * 1024, config.intermediateComputeSizeBytes());
     Assert.assertEquals(Integer.MAX_VALUE, config.poolCacheMaxCount());
-    if (Runtime.getRuntime().availableProcessors() == 1) {
-      Assert.assertTrue(config.getNumThreads() == 1);
-    } else {
-      Assert.assertTrue(config.getNumThreads() < Runtime.getRuntime().availableProcessors());
-    }
+    Assert.assertTrue(config.getNumThreads() == 1);
     Assert.assertEquals(Math.max(2, config.getNumThreads() / 4), config.getNumMergeBuffers());
     Assert.assertEquals(0, config.columnCacheSizeBytes());
     Assert.assertFalse(config.isFifo());
     Assert.assertEquals(System.getProperty("java.io.tmpdir"), config.getTmpDir());
+    Assert.assertEquals(bufferSize, config.intermediateComputeSizeBytes());
+  }
 
-    //with non-defaults
+  @Test
+  public void testDefaultsLargeDirect()
+  {
+    // test that auto sized buffer is no larger than 1
+    Injector injector = makeInjector(1, bufferSize * 100L, heapSize);
+    DruidProcessingConfig config = injector.getInstance(DruidProcessingConfig.class);
+
+    Assert.assertEquals(
+        DruidProcessingConfig.MAX_DEFAULT_PROCESSING_BUFFER_SIZE_BYTES,
+        config.intermediateComputeSizeBytes()
+    );
+  }
+
+  @Test
+  public void testReplacements()
+  {
     Properties props = new Properties();
     props.setProperty("druid.processing.buffer.sizeBytes", "1");
     props.setProperty("druid.processing.buffer.poolCacheMaxCount", "1");
@@ -61,10 +128,16 @@ public void testDeserialization()
     props.setProperty("druid.processing.fifo", "true");
     props.setProperty("druid.processing.tmpDir", "/test/path");
 
-    factory = Config.createFactory(props);
-    config = factory.buildWithReplacements(DruidProcessingConfig.class, ImmutableMap.of("base_path", "druid.processing"));
+    Injector injector = makeInjector(
+        numProcessors,
+        directSize,
+        heapSize,
+        props,
+        ImmutableMap.of("base_path", "druid.processing")
+    );
+    DruidProcessingConfig config = injector.getInstance(DruidProcessingConfig.class);
 
-    Assert.assertEquals(1, config.intermediateComputeSizeBytes());
+    Assert.assertEquals(1, config.intermediateComputeSizeBytes()); // heh
     Assert.assertEquals(1, config.poolCacheMaxCount());
     Assert.assertEquals(256, config.getNumThreads());
     Assert.assertEquals(64, config.getNumMergeBuffers());
@@ -72,4 +145,36 @@ public void testDeserialization()
     Assert.assertTrue(config.isFifo());
     Assert.assertEquals("/test/path", config.getTmpDir());
   }
+
+  static class MockRuntimeInfo extends RuntimeInfo
+  {
+    private final int availableProcessors;
+    private final long maxHeapSize;
+    private final long directSize;
+
+    MockRuntimeInfo(int availableProcessors, long directSize, long maxHeapSize)
+    {
+      this.availableProcessors = availableProcessors;
+      this.directSize = directSize;
+      this.maxHeapSize = maxHeapSize;
+    }
+
+    @Override
+    public int getAvailableProcessors()
+    {
+      return availableProcessors;
+    }
+
+    @Override
+    public long getMaxHeapSizeBytes()
+    {
+      return maxHeapSize;
+    }
+
+    @Override
+    public long getDirectMemorySizeBytes()
+    {
+      return directSize;
+    }
+  }
 }
diff --git a/server/src/main/java/org/apache/druid/client/cache/CaffeineCache.java b/server/src/main/java/org/apache/druid/client/cache/CaffeineCache.java
index ec18cd2d7ab..209518435b2 100644
--- a/server/src/main/java/org/apache/druid/client/cache/CaffeineCache.java
+++ b/server/src/main/java/org/apache/druid/client/cache/CaffeineCache.java
@@ -31,6 +31,7 @@
 import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.java.util.emitter.service.ServiceEmitter;
 import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
+import org.apache.druid.utils.JvmUtils;
 
 import java.nio.ByteBuffer;
 import java.util.Map;
@@ -68,7 +69,7 @@ public static CaffeineCache create(final CaffeineCacheConfig config, final Execu
     if (config.getSizeInBytes() >= 0) {
       builder.maximumWeight(config.getSizeInBytes());
     } else {
-      builder.maximumWeight(Math.min(MAX_DEFAULT_BYTES, Runtime.getRuntime().maxMemory() / 10));
+      builder.maximumWeight(Math.min(MAX_DEFAULT_BYTES, JvmUtils.getRuntimeInfo().getMaxHeapSizeBytes() / 10));
     }
     builder
         .weigher((NamedKey key, byte[] value) -> value.length
diff --git a/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java b/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java
index ea53dbbea5e..fae4c1049fb 100644
--- a/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java
+++ b/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java
@@ -34,7 +34,6 @@
 import org.apache.druid.collections.DefaultBlockingPool;
 import org.apache.druid.collections.NonBlockingPool;
 import org.apache.druid.collections.StupidPool;
-import org.apache.druid.common.utils.VMUtils;
 import org.apache.druid.guice.annotations.Global;
 import org.apache.druid.guice.annotations.Merging;
 import org.apache.druid.guice.annotations.Processing;
@@ -49,6 +48,7 @@
 import org.apache.druid.query.MetricsEmittingExecutorService;
 import org.apache.druid.query.PrioritizedExecutorService;
 import org.apache.druid.server.metrics.MetricsModule;
+import org.apache.druid.utils.JvmUtils;
 
 import java.nio.ByteBuffer;
 import java.util.concurrent.ExecutorService;
@@ -138,7 +138,7 @@ public ExecutorService getProcessingExecutorService(
   private void verifyDirectMemory(DruidProcessingConfig config)
   {
     try {
-      final long maxDirectMemory = VMUtils.getMaxDirectMemory();
+      final long maxDirectMemory = JvmUtils.getRuntimeInfo().getDirectMemorySizeBytes();
       final long memoryNeeded = (long) config.intermediateComputeSizeBytes() *
                                 (config.getNumMergeBuffers() + config.getNumThreads() + 1);
 
diff --git a/server/src/main/java/org/apache/druid/guice/http/DruidHttpClientConfig.java b/server/src/main/java/org/apache/druid/guice/http/DruidHttpClientConfig.java
index d0944bf4f04..7e09f8e059a 100644
--- a/server/src/main/java/org/apache/druid/guice/http/DruidHttpClientConfig.java
+++ b/server/src/main/java/org/apache/druid/guice/http/DruidHttpClientConfig.java
@@ -21,6 +21,7 @@
 
 import com.fasterxml.jackson.annotation.JsonProperty;
 import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.utils.JvmUtils;
 import org.joda.time.Duration;
 import org.joda.time.Period;
 
@@ -43,7 +44,7 @@
 
   @JsonProperty
   @Min(1)
-  private int numMaxThreads = Math.max(10, (Runtime.getRuntime().availableProcessors() * 17) / 16 + 2) + 30;
+  private int numMaxThreads = Math.max(10, (JvmUtils.getRuntimeInfo().getAvailableProcessors() * 17) / 16 + 2) + 30;
 
   @JsonProperty
   @Min(1)
diff --git a/server/src/main/java/org/apache/druid/segment/indexing/TuningConfig.java b/server/src/main/java/org/apache/druid/segment/indexing/TuningConfig.java
index 8d3e1b129f4..6d02ca42ac1 100644
--- a/server/src/main/java/org/apache/druid/segment/indexing/TuningConfig.java
+++ b/server/src/main/java/org/apache/druid/segment/indexing/TuningConfig.java
@@ -21,6 +21,7 @@
 
 import com.fasterxml.jackson.annotation.JsonSubTypes;
 import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import org.apache.druid.utils.JvmUtils;
 
 /**
  */
@@ -37,5 +38,5 @@
   // We initially estimated this to be 1/3(max jvm memory), but bytesCurrentlyInMemory only
   // tracks active index and not the index being flushed to disk, to account for that
   // we halved default to 1/6(max jvm memory)
-  long DEFAULT_MAX_BYTES_IN_MEMORY = Runtime.getRuntime().maxMemory() / 6;
+  long DEFAULT_MAX_BYTES_IN_MEMORY = JvmUtils.getRuntimeInfo().getMaxHeapSizeBytes() / 6;
 }
diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java
index 8d80f609cb2..8e426e0f6f8 100644
--- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java
+++ b/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java
@@ -35,7 +35,6 @@
 import org.apache.druid.client.cache.CachePopulatorStats;
 import org.apache.druid.common.guava.ThreadRenamingCallable;
 import org.apache.druid.common.guava.ThreadRenamingRunnable;
-import org.apache.druid.common.utils.VMUtils;
 import org.apache.druid.concurrent.TaskThreadPriority;
 import org.apache.druid.data.input.Committer;
 import org.apache.druid.data.input.InputRow;
@@ -76,6 +75,7 @@
 import org.apache.druid.timeline.DataSegment;
 import org.apache.druid.timeline.VersionedIntervalTimeline;
 import org.apache.druid.timeline.partition.SingleElementPartitionChunk;
+import org.apache.druid.utils.JvmUtils;
 import org.joda.time.DateTime;
 import org.joda.time.Duration;
 import org.joda.time.Interval;
@@ -331,7 +331,7 @@ public void doRun()
             handed off instead of individual segments being handed off (that is, if one of the set succeeds in handing
             off and the others fail, the real-time would believe that it needs to re-ingest the data).
              */
-            long persistThreadCpuTime = VMUtils.safeGetThreadCpuTime();
+            long persistThreadCpuTime = JvmUtils.safeGetThreadCpuTime();
             try {
               for (Pair<FireHydrant, Interval> pair : indexesToPersist) {
                 metrics.incrementRowOutputCount(
@@ -345,7 +345,7 @@ handed off instead of individual segments being handed off (that is, if one of t
               throw e;
             }
             finally {
-              metrics.incrementPersistCpuTime(VMUtils.safeGetThreadCpuTime() - persistThreadCpuTime);
+              metrics.incrementPersistCpuTime(JvmUtils.safeGetThreadCpuTime() - persistThreadCpuTime);
               metrics.incrementNumPersists();
               metrics.incrementPersistTimeMillis(persistStopwatch.elapsed(TimeUnit.MILLISECONDS));
               persistStopwatch.stop();
@@ -415,7 +415,7 @@ public void doRun()
                   }
                 }
               }
-              final long mergeThreadCpuTime = VMUtils.safeGetThreadCpuTime();
+              final long mergeThreadCpuTime = JvmUtils.safeGetThreadCpuTime();
               mergeStopwatch = Stopwatch.createStarted();
 
               final File mergedFile;
@@ -447,7 +447,7 @@ public void doRun()
               }
 
               // emit merge metrics before publishing segment
-              metrics.incrementMergeCpuTime(VMUtils.safeGetThreadCpuTime() - mergeThreadCpuTime);
+              metrics.incrementMergeCpuTime(JvmUtils.safeGetThreadCpuTime() - mergeThreadCpuTime);
               metrics.incrementMergeTimeMillis(mergeStopwatch.elapsed(TimeUnit.MILLISECONDS));
 
               log.info("Pushing [%s] to deep storage", sink.getSegment().getIdentifier());
diff --git a/server/src/main/java/org/apache/druid/server/StatusResource.java b/server/src/main/java/org/apache/druid/server/StatusResource.java
index b96d2bf6b60..44b31ac43e2 100644
--- a/server/src/main/java/org/apache/druid/server/StatusResource.java
+++ b/server/src/main/java/org/apache/druid/server/StatusResource.java
@@ -29,6 +29,8 @@
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.server.http.security.ConfigResourceFilter;
 import org.apache.druid.server.http.security.StateResourceFilter;
+import org.apache.druid.utils.JvmUtils;
+import org.apache.druid.utils.RuntimeInfo;
 
 import javax.inject.Inject;
 import javax.servlet.http.HttpServletRequest;
@@ -103,7 +105,7 @@ public Status(Collection<DruidModule> modules)
     {
       this.version = getDruidVersion();
       this.modules = getExtensionVersions(modules);
-      this.memory = new Memory(Runtime.getRuntime());
+      this.memory = new Memory(JvmUtils.getRuntimeInfo());
     }
 
     private String getDruidVersion()
@@ -215,13 +217,15 @@ public String toString()
     final long totalMemory;
     final long freeMemory;
     final long usedMemory;
+    final long directMemory;
 
-    public Memory(Runtime runtime)
+    public Memory(RuntimeInfo runtime)
     {
-      maxMemory = runtime.maxMemory();
-      totalMemory = runtime.totalMemory();
-      freeMemory = runtime.freeMemory();
+      maxMemory = runtime.getMaxHeapSizeBytes();
+      totalMemory = runtime.getTotalHeapSizeBytes();
+      freeMemory = runtime.getFreeHeapSizeBytes();
       usedMemory = totalMemory - freeMemory;
+      directMemory = runtime.getDirectMemorySizeBytes();
     }
 
     @JsonProperty
@@ -247,5 +251,11 @@ public long getUsedMemory()
     {
       return usedMemory;
     }
+
+    @JsonProperty
+    public long getDirectMemory()
+    {
+      return directMemory;
+    }
   }
 }
diff --git a/server/src/main/java/org/apache/druid/server/initialization/ServerConfig.java b/server/src/main/java/org/apache/druid/server/initialization/ServerConfig.java
index 11a9248d83f..cc86003c913 100644
--- a/server/src/main/java/org/apache/druid/server/initialization/ServerConfig.java
+++ b/server/src/main/java/org/apache/druid/server/initialization/ServerConfig.java
@@ -20,6 +20,7 @@
 package org.apache.druid.server.initialization;
 
 import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.druid.utils.JvmUtils;
 import org.joda.time.Period;
 
 import javax.validation.constraints.Max;
@@ -38,7 +39,7 @@
 
   @JsonProperty
   @Min(1)
-  private int numThreads = Math.max(10, (Runtime.getRuntime().availableProcessors() * 17) / 16 + 2) + 30;
+  private int numThreads = Math.max(10, (JvmUtils.getRuntimeInfo().getAvailableProcessors() * 17) / 16 + 2) + 30;
 
   @JsonProperty
   @Min(1)
diff --git a/server/src/test/java/org/apache/druid/guice/DruidProcessingModuleTest.java b/server/src/test/java/org/apache/druid/guice/DruidProcessingModuleTest.java
index 0b084564509..add2d297aae 100644
--- a/server/src/test/java/org/apache/druid/guice/DruidProcessingModuleTest.java
+++ b/server/src/test/java/org/apache/druid/guice/DruidProcessingModuleTest.java
@@ -46,4 +46,20 @@ public int intermediateComputeSizeBytes()
     });
   }
 
+  @Test
+  public void testMemoryCheckIsChillByDefaultIfNothingSet()
+  {
+    DruidProcessingConfig config = new DruidProcessingConfig()
+    {
+      @Override
+      public String getFormatString()
+      {
+        return "processing-test-%s";
+      }
+    };
+
+    DruidProcessingModule module = new DruidProcessingModule();
+    module.getIntermediateResultsPool(config);
+  }
 }
+
diff --git a/services/src/main/java/org/apache/druid/cli/GuiceRunnable.java b/services/src/main/java/org/apache/druid/cli/GuiceRunnable.java
index fd169505461..090f6a3c81b 100644
--- a/services/src/main/java/org/apache/druid/cli/GuiceRunnable.java
+++ b/services/src/main/java/org/apache/druid/cli/GuiceRunnable.java
@@ -29,6 +29,7 @@
 import org.apache.druid.java.util.common.lifecycle.Lifecycle;
 import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.server.log.StartupLoggingConfig;
+import org.apache.druid.utils.JvmUtils;
 
 import java.util.List;
 import java.util.Properties;
@@ -79,10 +80,11 @@ public Lifecycle initLifecycle(Injector injector)
       final StartupLoggingConfig startupLoggingConfig = injector.getInstance(StartupLoggingConfig.class);
 
       log.info(
-          "Starting up with processors[%,d], memory[%,d], maxMemory[%,d].",
-          Runtime.getRuntime().availableProcessors(),
-          Runtime.getRuntime().totalMemory(),
-          Runtime.getRuntime().maxMemory()
+          "Starting up with processors[%,d], memory[%,d], maxMemory[%,d], directMemory[%,d].",
+          JvmUtils.getRuntimeInfo().getAvailableProcessors(),
+          JvmUtils.getRuntimeInfo().getTotalHeapSizeBytes(),
+          JvmUtils.getRuntimeInfo().getMaxHeapSizeBytes(),
+          JvmUtils.getRuntimeInfo().getDirectMemorySizeBytes()
       );
 
       if (startupLoggingConfig.isLogProperties()) {


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org