tillrohrmann commented on a change in pull request #9760: 
[FLINK-13982][runtime] Implement memory calculation logics
URL: https://github.com/apache/flink/pull/9760#discussion_r333990070
 
 

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/TaskExecutorResourceUtils.java
 ##########
 @@ -0,0 +1,659 @@
+/*
+ * 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.flink.runtime.clusterframework;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.configuration.NettyShuffleEnvironmentOptions;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.runtime.util.ConfigurationParserUtils;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Utility class for TaskExecutor memory configurations.
+ *
+ * <p>A TaskExecutor's memory consists of the following components.
+ * <ul>
+ *     <li>Framework Heap Memory</li>
+ *     <li>Task Heap Memory</li>
+ *     <li>Task Off-Heap Memory</li>
+ *     <li>Shuffle Memory</li>
+ *     <li>Managed Memory</li>
+ *     <ul>
+ *         <li>On-Heap Managed Memory</li>
+ *         <li>Off-Heap Managed Memory</li>
+ *     </ul>
+ *     <li>JVM Metaspace</li>
+ *     <li>JVM Overhead</li>
+ * </ul>
+ * Among all the components, Framework Heap Memory, Task Heap Memory and 
On-Heap Managed Memory use on heap memory,
+ * while the rest use off heap memory. We use Total Process Memory to refer to 
all the memory components, while Total
+ * Flink Memory refering to all the components except JVM Metaspace and JVM 
Overhead.
+ *
+ * <p>The relationships of TaskExecutor memory components are shown below.
+ * <pre>
+ *               ┌ ─ ─ Total Process Memory  ─ ─ ┐
+ *                ┌ ─ ─ Total Flink Memory  ─ ─ ┐
+ *               │ ┌───────────────────────────┐ │
+ *                ││   Framework Heap Memory   ││  ─┐
+ *               │ └───────────────────────────┘ │  │
+ *                │┌───────────────────────────┐│   │
+ *               │ │     Task Heap Memory      │ │ ─┤
+ *                │└───────────────────────────┘│   │
+ *               │ ┌───────────────────────────┐ │  │
+ *            ┌─  ││   Task Off-Heap Memory    ││   │
+ *            │  │ └───────────────────────────┘ │  ├─ On-Heap
+ *            │   │┌───────────────────────────┐│   │
+ *            ├─ │ │      Shuffle Memory       │ │  │
+ *            │   │└───────────────────────────┘│   │
+ *            │  │ ┌───── Managed Memory ──────┐ │  │
+ *            │   ││┌─────────────────────────┐││   │
+ *            │  │ ││ On-Heap Managed Memory  ││ │ ─┘
+ *            │   ││├─────────────────────────┤││
+ *  Off-Heap ─┼─ │ ││ Off-Heap Managed Memory ││ │
+ *            │   ││└─────────────────────────┘││
+ *            │  │ └───────────────────────────┘ │
+ *            │   └ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┘
+ *            │  │┌─────────────────────────────┐│
+ *            ├─  │        JVM Metaspace        │
+ *            │  │└─────────────────────────────┘│
+ *            │   ┌─────────────────────────────┐
+ *            └─ ││        JVM Overhead         ││
+ *                └─────────────────────────────┘
+ *               └ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┘
+ * </pre>
+ */
+public class TaskExecutorResourceUtils {
+
+       private TaskExecutorResourceUtils() {}
+
+       // 
------------------------------------------------------------------------
+       //  Generating JVM Parameters
+       // 
------------------------------------------------------------------------
+
+       public static String generateJvmParametersStr(final 
TaskExecutorResourceSpec taskExecutorResourceSpec) {
+               final MemorySize jvmHeapSize = 
taskExecutorResourceSpec.getFrameworkHeapSize()
+                       .add(taskExecutorResourceSpec.getTaskHeapSize())
+                       
.add(taskExecutorResourceSpec.getOnHeapManagedMemorySize());
+               final MemorySize jvmDirectSize = 
taskExecutorResourceSpec.getTaskOffHeapSize()
+                       .add(taskExecutorResourceSpec.getShuffleMemSize());
+               final MemorySize jvmMetaspaceSize = 
taskExecutorResourceSpec.getJvmMetaspaceSize();
+
+               return "-Xmx" + jvmHeapSize.getBytes()
+                       + " -Xms" + jvmHeapSize.getBytes()
+                       + " -XX:MaxDirectMemorySize=" + jvmDirectSize.getBytes()
+                       + " -XX:MetaspaceSize=" + jvmMetaspaceSize.getBytes();
+       }
+
+       // 
------------------------------------------------------------------------
+       //  Generating Dynamic Config Options
+       // 
------------------------------------------------------------------------
+
+       public static String generateDynamicConfigsStr(final 
TaskExecutorResourceSpec taskExecutorResourceSpec) {
+               final Map<String, String> configs = new HashMap<>();
+               configs.put(TaskManagerOptions.FRAMEWORK_HEAP_MEMORY.key(), 
taskExecutorResourceSpec.getFrameworkHeapSize().getBytes() + "b");
+               configs.put(TaskManagerOptions.TASK_HEAP_MEMORY.key(), 
taskExecutorResourceSpec.getTaskHeapSize().getBytes() + "b");
+               configs.put(TaskManagerOptions.TASK_OFF_HEAP_MEMORY.key(), 
taskExecutorResourceSpec.getTaskOffHeapSize().getBytes() + "b");
+               configs.put(TaskManagerOptions.SHUFFLE_MEMORY_MIN.key(), 
taskExecutorResourceSpec.getShuffleMemSize().getBytes() + "b");
+               configs.put(TaskManagerOptions.SHUFFLE_MEMORY_MAX.key(), 
taskExecutorResourceSpec.getShuffleMemSize().getBytes() + "b");
+               configs.put(TaskManagerOptions.MANAGED_MEMORY_SIZE.key(), 
taskExecutorResourceSpec.getManagedMemorySize().getBytes() + "b");
+               
configs.put(TaskManagerOptions.MANAGED_MEMORY_OFFHEAP_SIZE.key(), 
taskExecutorResourceSpec.getOffHeapManagedMemorySize().getBytes() + "b");
+               return assembleDynamicConfigsStr(configs);
+       }
+
+       private static String assembleDynamicConfigsStr(final Map<String, 
String> configs) {
+               final StringBuilder sb = new StringBuilder();
+               for (Map.Entry<String, String> entry : configs.entrySet()) {
+                       sb.append("-D 
").append(entry.getKey()).append("=").append(entry.getValue()).append(" ");
+               }
+               return sb.toString();
+       }
+
+       // 
------------------------------------------------------------------------
+       //  Memory Configuration Calculations
+       // 
------------------------------------------------------------------------
+
+       public static TaskExecutorResourceSpec resourceSpecFromConfig(final 
Configuration config) {
+               if (isTaskHeapMemorySizeExplicitlyConfigured(config) && 
isManagedMemorySizeExplicitlyConfigured(config)) {
+                       // both task heap memory and managed memory are 
configured, use these to derive total flink memory
+                       return 
deriveResourceSpecWithExplicitTaskAndManagedMemory(config);
+               } else if (isTotalFlinkMemorySizeExplicitlyConfigured(config)) {
+                       // either of task heap memory and managed memory is not 
configured, total flink memory is configured,
+                       // derive from total flink memory
+                       return deriveResourceSpecWithTotalFlinkMemory(config);
+               } else if 
(isTotalProcessMemorySizeExplicitlyConfigured(config)) {
+                       // total flink memory is not configured, total process 
memory is configured,
+                       // derive from total process memory
+                       return deriveResourceSpecWithTotalProcessMemory(config);
+               } else {
+                       throw new IllegalConfigurationException("Either Task 
Heap Memory size and Managed Memory size, or Total Flink"
+                               + " Memory size, or Total Process Memory size 
need to be configured explicitly.");
+               }
+       }
+
+       private static TaskExecutorResourceSpec 
deriveResourceSpecWithExplicitTaskAndManagedMemory(final Configuration config) {
+               // derive flink internal memory from explicitly configure task 
heap memory size and managed memory size
+
+               final MemorySize taskHeapMemorySize = 
getTaskHeapMemorySize(config);
+               final MemorySize managedMemorySize = 
getManagedMemorySize(config);
+
+               final MemorySize frameworkHeapMemorySize = 
getFrameworkHeapMemorySize(config);
+               final MemorySize taskOffHeapMemorySize = 
getTaskOffHeapMemorySize(config);
+
+               final OnHeapAndOffHeapManagedMemory 
onHeapAndOffHeapManagedMemory = 
deriveOnHeapAndOffHeapMemoryFromManagedMemory(config, managedMemorySize);
+
+               final MemorySize shuffleMemorySize;
+               final MemorySize totalFlinkExcludeShuffleMemorySize =
+                       
frameworkHeapMemorySize.add(taskHeapMemorySize).add(taskOffHeapMemorySize).add(managedMemorySize);
+
+               if (isTotalFlinkMemorySizeExplicitlyConfigured(config)) {
+                       // derive shuffle memory from total flink memory, and 
check against shuffle min/max
+                       final MemorySize totalFlinkMemorySize = 
getTotalFlinkMemorySize(config);
+                       if (totalFlinkExcludeShuffleMemorySize.getBytes() > 
totalFlinkMemorySize.getBytes()) {
+                               throw new IllegalConfigurationException(
+                                       "Sum of configured Framework Heap 
Memory (" + frameworkHeapMemorySize.toString()
+                                       + "), Task Heap Memory (" + 
taskHeapMemorySize.toString()
+                                       + "), Task Off-Heap Memory (" + 
taskOffHeapMemorySize.toString()
+                                       + ") and Managed Memory (" + 
managedMemorySize.toString()
+                                       + ") exceed configured Total Flink 
Memory (" + totalFlinkMemorySize.toString() + ").");
+                       }
+                       shuffleMemorySize = 
totalFlinkMemorySize.subtract(totalFlinkExcludeShuffleMemorySize);
+                       sanityCheckShuffleMemory(config, shuffleMemorySize, 
totalFlinkMemorySize);
+               } else {
+                       // derive shuffle memory from shuffle configs
+                       if (isUsingLegacyShuffleConfigs(config)) {
+                               shuffleMemorySize = 
getShuffleMemorySizeWithLegacyConfig(config);
+                       } else {
+                               shuffleMemorySize = 
deriveShuffleMemoryWithInverseFraction(config, 
totalFlinkExcludeShuffleMemorySize);
+                       }
+               }
 
 Review comment:
   Looking at the code, we could simply remove the `if (...) {}` block because 
we call `sanityCheckTotalFlinkMemory` a couple of lines further down.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to