This is an automated email from the ASF dual-hosted git repository. vhs pushed a commit to branch release-1.0.2 in repository https://gitbox.apache.org/repos/asf/hudi.git
commit 02bbef7e44214f7d669fb891be0bfbf10f79212a Author: Lokesh Jain <[email protected]> AuthorDate: Fri Apr 4 21:03:54 2025 +0530 [HUDI-9247] Flip default value of reuse of TimeGenerator instance (#13077) (cherry picked from commit 08d534b0aa064e414766995a809f6fb9f1484836) --- .../common/config/HoodieTimeGeneratorConfig.java | 2 +- .../table/timeline/TestWaitBasedTimeGenerator.java | 24 +++++++++++++--------- 2 files changed, 15 insertions(+), 11 deletions(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieTimeGeneratorConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieTimeGeneratorConfig.java index 16ebff9135b..1dce6f71c09 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieTimeGeneratorConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieTimeGeneratorConfig.java @@ -58,7 +58,7 @@ public class HoodieTimeGeneratorConfig extends HoodieConfig { public static final ConfigProperty<Boolean> TIME_GENERATOR_REUSE_ENABLE = ConfigProperty .key("_hoodie.time.generator.reuse.enable") - .defaultValue(true) + .defaultValue(false) .sinceVersion("1.0.1") .markAdvanced() .withDocumentation("Used only for internal purposes. TimeGeneratos are cached per table base path and re-used across invocations. " diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/timeline/TestWaitBasedTimeGenerator.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/timeline/TestWaitBasedTimeGenerator.java index d776229e63e..99b6eaa93d3 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/timeline/TestWaitBasedTimeGenerator.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/timeline/TestWaitBasedTimeGenerator.java @@ -152,21 +152,16 @@ public class TestWaitBasedTimeGenerator { @Test public void testTimeGeneratorCache() { - TimeGenerator timeGenerator1 = TimeGenerators.getTimeGenerator(timeGeneratorConfig, storageConf); - TimeGenerator timeGenerator2 = TimeGenerators.getTimeGenerator(timeGeneratorConfig, storageConf); - TimeGenerator timeGenerator3 = TimeGenerators.getTimeGenerator(timeGeneratorConfig, storageConf); + HoodieTimeGeneratorConfig timeGeneratorConfigWithReuse = getWaitBasedTimeGenerator(true); + TimeGenerator timeGenerator1 = TimeGenerators.getTimeGenerator(timeGeneratorConfigWithReuse, storageConf); + TimeGenerator timeGenerator2 = TimeGenerators.getTimeGenerator(timeGeneratorConfigWithReuse, storageConf); + TimeGenerator timeGenerator3 = TimeGenerators.getTimeGenerator(timeGeneratorConfigWithReuse, storageConf); assertEquals(timeGenerator1, timeGenerator2); assertEquals(timeGenerator1, timeGenerator3); // disable reuse - HoodieTimeGeneratorConfig timeGeneratorConfigWithNoReuse = HoodieTimeGeneratorConfig.newBuilder() - .withPath("test_wait_based") - .withMaxExpectedClockSkewMs(25L) - .withReuseTimeGenerator(false) - .withTimeGeneratorType(TimeGeneratorType.WAIT_TO_ADJUST_SKEW) - .build(); - + HoodieTimeGeneratorConfig timeGeneratorConfigWithNoReuse = getWaitBasedTimeGenerator(false); TimeGenerator timeGenerator4 = TimeGenerators.getTimeGenerator(timeGeneratorConfigWithNoReuse, storageConf); assertNotEquals(timeGenerator1, timeGenerator4); // how many ever times we call, we should get new time generator @@ -174,4 +169,13 @@ public class TestWaitBasedTimeGenerator { assertNotEquals(timeGenerator4, timeGenerator5); } + private static HoodieTimeGeneratorConfig getWaitBasedTimeGenerator(boolean reuse) { + return HoodieTimeGeneratorConfig.newBuilder() + .withPath("test_wait_based") + .withMaxExpectedClockSkewMs(25L) + .withReuseTimeGenerator(reuse) + .withTimeGeneratorType(TimeGeneratorType.WAIT_TO_ADJUST_SKEW) + .build(); + } + }
