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

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


The following commit(s) were added to refs/heads/master by this push:
     new c8dddc0f3b69 fix(utilities): Use passed-in configs when propsFilePath 
is null or empty in HoodieStreamer (#17467)
c8dddc0f3b69 is described below

commit c8dddc0f3b6928ed132e5c3caf8c7cb33de1e41d
Author: Prashant Wason <[email protected]>
AuthorDate: Sun Feb 1 09:38:07 2026 -0800

    fix(utilities): Use passed-in configs when propsFilePath is null or empty 
in HoodieStreamer (#17467)
    
    
    Co-Authored-By: Claude (claude-opus-4-5) <[email protected]>
    ---------
    
    Co-authored-by: Claude <[email protected]>
---
 .../hudi/utilities/streamer/HoodieStreamer.java    |  3 +-
 .../streamer/TestHoodieStreamerUtils.java          | 46 ++++++++++++++++++++++
 2 files changed, 48 insertions(+), 1 deletion(-)

diff --git 
a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/HoodieStreamer.java
 
b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/HoodieStreamer.java
index 226134f31103..84929ca69556 100644
--- 
a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/HoodieStreamer.java
+++ 
b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/HoodieStreamer.java
@@ -186,7 +186,8 @@ public class HoodieStreamer implements Serializable {
     //   3. Otherwise, parse provided specified props file (merging in CLI 
overrides)
     if (propsOverride.isPresent()) {
       hoodieConfig.setAll(propsOverride.get());
-    } else if (cfg.propsFilePath.equals(Config.DEFAULT_DFS_SOURCE_PROPERTIES)) 
{
+    } else if (StringUtils.isNullOrEmpty(cfg.propsFilePath)
+        || cfg.propsFilePath.equals(Config.DEFAULT_DFS_SOURCE_PROPERTIES)) {
       hoodieConfig.setAll(UtilHelpers.getConfig(cfg.configs).getProps());
     } else {
       hoodieConfig.setAll(readConfig(hadoopConf, new Path(cfg.propsFilePath), 
cfg.configs).getProps());
diff --git 
a/hudi-utilities/src/test/java/org/apache/hudi/utilities/streamer/TestHoodieStreamerUtils.java
 
b/hudi-utilities/src/test/java/org/apache/hudi/utilities/streamer/TestHoodieStreamerUtils.java
index f9868258ca19..46c0d6b8fe11 100644
--- 
a/hudi-utilities/src/test/java/org/apache/hudi/utilities/streamer/TestHoodieStreamerUtils.java
+++ 
b/hudi-utilities/src/test/java/org/apache/hudi/utilities/streamer/TestHoodieStreamerUtils.java
@@ -23,6 +23,7 @@ import org.apache.hudi.common.config.TypedProperties;
 import org.apache.hudi.common.model.DefaultHoodieRecordPayload;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
+import org.apache.hudi.common.model.HoodieTableType;
 import org.apache.hudi.common.schema.HoodieSchema;
 import org.apache.hudi.common.table.HoodieTableConfig;
 import org.apache.hudi.common.util.Option;
@@ -36,10 +37,12 @@ import 
org.apache.hudi.utilities.testutils.UtilitiesTestBase;
 import org.apache.avro.generic.GenericData;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.avro.util.Utf8;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.spark.SparkException;
 import org.apache.spark.api.java.JavaRDD;
 import org.apache.spark.unsafe.types.UTF8String;
 import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 import org.junit.jupiter.params.ParameterizedTest;
 import org.junit.jupiter.params.provider.Arguments;
 import org.junit.jupiter.params.provider.MethodSource;
@@ -192,4 +195,47 @@ public class TestHoodieStreamerUtils extends 
UtilitiesTestBase {
       }
     }
   }
+
+  @Test
+  void testCombinePropertiesWithNullPropsFilePath() {
+    HoodieStreamer.Config cfg = new HoodieStreamer.Config();
+    cfg.propsFilePath = null;
+    cfg.configs.add("hoodie.test.key=testValue");
+    cfg.tableType = HoodieTableType.COPY_ON_WRITE.name();
+
+    Configuration hadoopConf = new Configuration();
+    TypedProperties result = HoodieStreamer.combineProperties(cfg, 
Option.empty(), hadoopConf);
+
+    assertEquals("testValue", result.getString("hoodie.test.key"));
+  }
+
+  @Test
+  void testCombinePropertiesWithEmptyPropsFilePath() {
+    HoodieStreamer.Config cfg = new HoodieStreamer.Config();
+    cfg.propsFilePath = "";
+    cfg.configs.add("hoodie.another.key=anotherValue");
+    cfg.tableType = HoodieTableType.COPY_ON_WRITE.name();
+
+    Configuration hadoopConf = new Configuration();
+    TypedProperties result = HoodieStreamer.combineProperties(cfg, 
Option.empty(), hadoopConf);
+
+    assertEquals("anotherValue", result.getString("hoodie.another.key"));
+  }
+
+  @Test
+  void testCombinePropertiesWithPropsOverride() {
+    HoodieStreamer.Config cfg = new HoodieStreamer.Config();
+    cfg.propsFilePath = "";
+    cfg.configs.add("hoodie.overridden.key=cliValue");
+    cfg.tableType = HoodieTableType.COPY_ON_WRITE.name();
+
+    TypedProperties overrideProps = new TypedProperties();
+    overrideProps.put("hoodie.overridden.key", "overrideValue");
+
+    Configuration hadoopConf = new Configuration();
+    TypedProperties result = HoodieStreamer.combineProperties(cfg, 
Option.of(overrideProps), hadoopConf);
+
+    // propsOverride takes precedence
+    assertEquals("overrideValue", result.getString("hoodie.overridden.key"));
+  }
 }

Reply via email to