You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by la...@apache.org on 2020/04/22 08:33:26 UTC

[incubator-hudi] branch master updated: [HUDI-816] Fixed MAX_MEMORY_FOR_MERGE_PROP and MAX_MEMORY_FOR_COMPACTION_PROP do not work due to HUDI-678 (#1536)

This is an automated email from the ASF dual-hosted git repository.

lamberken pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-hudi.git


The following commit(s) were added to refs/heads/master by this push:
     new 26684f5  [HUDI-816] Fixed MAX_MEMORY_FOR_MERGE_PROP and MAX_MEMORY_FOR_COMPACTION_PROP do not work due to HUDI-678 (#1536)
26684f5 is described below

commit 26684f5984baafdfcbef3718fa5abcbd0983a6a4
Author: leesf <le...@apache.org>
AuthorDate: Wed Apr 22 16:33:18 2020 +0800

    [HUDI-816] Fixed MAX_MEMORY_FOR_MERGE_PROP and MAX_MEMORY_FOR_COMPACTION_PROP do not work due to HUDI-678 (#1536)
---
 .../apache/hudi/client/utils/SparkConfigUtils.java | 10 +++-
 .../hudi/client/utils/TestSparkConfigUtils.java    | 65 ++++++++++++++++++++++
 2 files changed, 74 insertions(+), 1 deletion(-)

diff --git a/hudi-client/src/main/java/org/apache/hudi/client/utils/SparkConfigUtils.java b/hudi-client/src/main/java/org/apache/hudi/client/utils/SparkConfigUtils.java
index f6b8549..604be01 100644
--- a/hudi-client/src/main/java/org/apache/hudi/client/utils/SparkConfigUtils.java
+++ b/hudi-client/src/main/java/org/apache/hudi/client/utils/SparkConfigUtils.java
@@ -30,6 +30,8 @@ import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FOR_S
 import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION;
 import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE;
 import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MIN_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES;
+import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FOR_COMPACTION_PROP;
+import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE_PROP;
 import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP;
 import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FRACTION_FOR_MERGE_PROP;
 import static org.apache.hudi.config.HoodieWriteConfig.WRITE_STATUS_STORAGE_LEVEL;
@@ -83,11 +85,17 @@ public class SparkConfigUtils {
   }
 
   public static long getMaxMemoryPerPartitionMerge(Properties properties) {
+    if (properties.containsKey(MAX_MEMORY_FOR_MERGE_PROP)) {
+      return Long.parseLong(properties.getProperty(MAX_MEMORY_FOR_MERGE_PROP));
+    }
     String fraction = properties.getProperty(MAX_MEMORY_FRACTION_FOR_MERGE_PROP, DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE);
     return getMaxMemoryAllowedForMerge(fraction);
   }
 
-  public static Long getMaxMemoryPerCompaction(Properties properties) {
+  public static long getMaxMemoryPerCompaction(Properties properties) {
+    if (properties.containsKey(MAX_MEMORY_FOR_COMPACTION_PROP)) {
+      return Long.parseLong(properties.getProperty(MAX_MEMORY_FOR_COMPACTION_PROP));
+    }
     String fraction = properties.getProperty(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP, DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION);
     return getMaxMemoryAllowedForMerge(fraction);
   }
diff --git a/hudi-client/src/test/java/org/apache/hudi/client/utils/TestSparkConfigUtils.java b/hudi-client/src/test/java/org/apache/hudi/client/utils/TestSparkConfigUtils.java
new file mode 100644
index 0000000..9463bfb
--- /dev/null
+++ b/hudi-client/src/test/java/org/apache/hudi/client/utils/TestSparkConfigUtils.java
@@ -0,0 +1,65 @@
+/*
+ * 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.hudi.client.utils;
+
+import org.apache.hudi.config.HoodieMemoryConfig;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION;
+import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE;
+import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP;
+import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FRACTION_FOR_MERGE_PROP;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class TestSparkConfigUtils {
+  @TempDir
+  public java.nio.file.Path basePath;
+
+  @Test
+  public void testMaxMemoryPerPartitionMergeWithMaxSizeDefined() {
+    String path = basePath.toString();
+
+    long mergeMaxSize = 1000;
+    long compactionMaxSize = 1000;
+
+    HoodieMemoryConfig memoryConfig = HoodieMemoryConfig.newBuilder().withMaxMemoryMaxSize(mergeMaxSize, compactionMaxSize).build();
+    HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(path).withMemoryConfig(memoryConfig).build();
+
+    assertEquals(mergeMaxSize, SparkConfigUtils.getMaxMemoryPerPartitionMerge(config.getProps()));
+    assertEquals(compactionMaxSize, SparkConfigUtils.getMaxMemoryPerCompaction(config.getProps()));
+  }
+
+  @Test
+  public void testMaxMemoryPerPartitionMergeInDefault() {
+    String path = basePath.toString();
+
+    HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(path).build();
+
+    String compactionFraction = config.getProps().getProperty(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP, DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION);
+    long compactionMaxSize = SparkConfigUtils.getMaxMemoryAllowedForMerge(compactionFraction);
+
+    String mergeFraction = config.getProps().getProperty(MAX_MEMORY_FRACTION_FOR_MERGE_PROP, DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE);
+    long mergeMaxSize = SparkConfigUtils.getMaxMemoryAllowedForMerge(mergeFraction);
+
+    assertEquals(mergeMaxSize, SparkConfigUtils.getMaxMemoryPerPartitionMerge(config.getProps()));
+    assertEquals(compactionMaxSize, SparkConfigUtils.getMaxMemoryPerCompaction(config.getProps()));
+  }
+}