You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by "paul-rogers (via GitHub)" <gi...@apache.org> on 2023/02/24 23:27:07 UTC

[GitHub] [druid] paul-rogers commented on a diff in pull request #13846: Suggested memory calculation in case NOT_ENOUGH_MEMORY_FAULT is thrown.

paul-rogers commented on code in PR #13846:
URL: https://github.com/apache/druid/pull/13846#discussion_r1117801467


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/NotEnoughMemoryFault.java:
##########
@@ -45,19 +47,27 @@ public NotEnoughMemoryFault(
   {
     super(
         CODE,
-        "Not enough memory (total = %,d; usable = %,d; server workers = %,d; server threads = %,d)",
+        "Not enough memory. Required alteast %,d bytes. (total = %,d bytes; usable = %,d bytes; server workers = %,d; server threads = %,d). Increase JVM memory with the -xmx option",

Review Comment:
   Nit: `atleast` -> `at least`
   
   Note that there seems to be a convention that all interpolated values be enclosed in angle brackets. Seems silly sometimes, bug some folks feel strongly about it.
   
   Increasing JVM memory is not always possible. The other solution is to _decrease_ the number of workers. This is the problem with making suggestions: there are multiple ways to solve the problem.



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerMemoryParameters.java:
##########
@@ -431,6 +466,26 @@ private static long memoryPerBundle(
     return memoryForBundles / bundleCount;
   }
 
+  /**
+   * Used for estimating the usable memory for better exception messages when {@link NotEnoughMemoryFault} is thrown.
+   */
+  private static long estimateUsableMemory(
+      final int numWorkersInJvm,
+      final int numProcessingThreadsInJvm,
+      final long estimatedEachBundleMemory
+  )
+  {
+    final int bundleCount = numWorkersInJvm + numProcessingThreadsInJvm;
+    return estimateUsableMemory(numWorkersInJvm, estimatedEachBundleMemory * bundleCount);
+
+  }
+
+  private static long estimateUsableMemory(final int numWorkersInJvm, final long estimatedTotalBundleMemory)
+  {
+    final long estimatedWorkerMemory = numWorkersInJvm * PARTITION_STATS_MEMORY_MAX_BYTES;

Review Comment:
   Nit: Now that you've worked out the math, would be great to explain the reasoning. For example, why is the partition stats large enough to worry about? Why isn't there any other per-worker overhead to consider?



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerMemoryParameters.java:
##########
@@ -439,11 +494,19 @@ private static long memoryNeededForInputChannels(final int numInputWorkers)
   }
 
   /**
-   * Amount of heap memory available for our usage.
+   * Amount of heap memory available for our usage. Any computation changes done to this method should also be done in its corresponding method {@link WorkerMemoryParameters#estimateTotalMemoryInJvmFromUsableMemory}
+   */
+  private static long computeUsableMemoryInJvm(final long maxMemory, final long totalLookupFootprint)
+  {
+    return (long) ((maxMemory - totalLookupFootprint) * USABLE_MEMORY_FRACTION);
+  }
+
+  /**
+   * Estimate amount of heap memory to use in case usable memory is provided. This method is used for bettter exception messages when {@link NotEnoughMemoryFault} is thrown.
    */
-  private static long computeUsableMemoryInJvm(final Injector injector)
+  private static long estimateTotalMemoryInJvmFromUsableMemory(long usuableMemeory, final long totalLookupFootprint)

Review Comment:
   This would also benefit from explanation. On the surface, it appears we're estimating the memory in the JVM. But, of course, we don't have to estimate that: we know that. So, maybe we're estimating the amount of memory the JVM _would need_ to handle the given workload. Can we add a note, or change the name, to express that, if that is, indeed, what we're doing?



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerMemoryParameters.java:
##########
@@ -200,15 +207,30 @@ public static WorkerMemoryParameters createProductionInstanceForWorker(
    *                                  the task capacity.
    * @param numProcessingThreadsInJvm size of the processing thread pool in the JVM.
    * @param numInputWorkers           number of workers across input stages that need to be merged together.
+   * @param totalLookUpFootprint      estimated size of the lookups loaded by the process.
    */
   public static WorkerMemoryParameters createInstance(
       final long maxMemoryInJvm,
-      final long usableMemoryInJvm,
       final int numWorkersInJvm,
       final int numProcessingThreadsInJvm,
-      final int numInputWorkers
+      final int numInputWorkers,
+      final long totalLookUpFootprint
   )
   {
+    Preconditions.checkArgument(maxMemoryInJvm > 0, "Max memory passed: [%s] should be > 0", maxMemoryInJvm);
+    Preconditions.checkArgument(numWorkersInJvm > 0, "Number of workers: [%s] in jvm should be > 0", numWorkersInJvm);

Review Comment:
   Nit: `%d` for numbers. Here and below.
   
   For extra credit, since the values are likely to be big, include comma separators: `%,d`.



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerMemoryParameters.java:
##########
@@ -439,11 +494,19 @@ private static long memoryNeededForInputChannels(final int numInputWorkers)
   }
 
   /**
-   * Amount of heap memory available for our usage.
+   * Amount of heap memory available for our usage. Any computation changes done to this method should also be done in its corresponding method {@link WorkerMemoryParameters#estimateTotalMemoryInJvmFromUsableMemory}
+   */
+  private static long computeUsableMemoryInJvm(final long maxMemory, final long totalLookupFootprint)
+  {
+    return (long) ((maxMemory - totalLookupFootprint) * USABLE_MEMORY_FRACTION);

Review Comment:
   What does `USABLE_MEMORY_FRACTION` represent? Should it apply to lookups? That is, if `USABLE_MEMORY_FRACTION` represents memory given over to Java, process overhead, etc., then we should apply that to total memory, and then subtract lookups, since lookups are subject to the same overhead as worker memory. Else, maybe explain the reasoning.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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