xushiyan commented on code in PR #6284:
URL: https://github.com/apache/hudi/pull/6284#discussion_r996967389
##########
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java:
##########
@@ -219,6 +219,11 @@ public class HoodieTableConfig extends HoodieConfig {
.defaultValue(false)
.withDocumentation("When set to true, will not write the partition
columns into hudi. By default, false.");
+ public static final ConfigProperty<String> SPILLABLE_MAP_BASE_PATH =
ConfigProperty
+ .key("hoodie.memory.spillable.map.path")
Review Comment:
shouldn't this be `hoodie.memory.spillable.map.basepath` ?
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMemoryConfig.java:
##########
@@ -77,10 +80,19 @@ public class HoodieMemoryConfig extends HoodieConfig {
.defaultValue(16 * 1024 * 1024)
.withDocumentation("Property to control the max memory in bytes for dfs
input stream buffer size");
- public static final ConfigProperty<String> SPILLABLE_MAP_BASE_PATH =
ConfigProperty
+ public static ConfigProperty<String> SPILLABLE_MAP_BASE_PATH = ConfigProperty
Review Comment:
why remove `final` ?
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMemoryConfig.java:
##########
@@ -77,10 +80,19 @@ public class HoodieMemoryConfig extends HoodieConfig {
.defaultValue(16 * 1024 * 1024)
.withDocumentation("Property to control the max memory in bytes for dfs
input stream buffer size");
- public static final ConfigProperty<String> SPILLABLE_MAP_BASE_PATH =
ConfigProperty
+ public static ConfigProperty<String> SPILLABLE_MAP_BASE_PATH = ConfigProperty
.key("hoodie.memory.spillable.map.path")
.defaultValue("/tmp/")
- .withDocumentation("Default file path prefix for spillable map");
+ .withInferFunction(cfg -> {
+ // if user doesn't configure it, infer the path from env var
+ // if nothing to infer from, return Option.empty() then it will
fallback to the default /tmp/
+ if (cfg.contains(HoodieTableConfig.SPILLABLE_MAP_BASE_PATH)) {
+ return
Option.of(cfg.getString(HoodieTableConfig.SPILLABLE_MAP_BASE_PATH));
+ }
+ String[] localDirs = FileIOUtils.getConfiguredLocalDirs();
+ return (localDirs != null && localDirs.length > 0) ?
Option.of(localDirs[0]) : Option.of("/tmp/");
Review Comment:
/nit return option.empty means using the default val
```suggestion
return (localDirs != null && localDirs.length > 0) ?
Option.of(localDirs[0]) : Option.empty();
```
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]