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

danny0405 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 424e0cea980 [HUDI-7050] Flink HoodieHiveCatalog supports hadoop 
parameters (#10013)
424e0cea980 is described below

commit 424e0cea980122672db310872335a05630243f3f
Author: chao chen <[email protected]>
AuthorDate: Wed Nov 15 12:20:10 2023 +0800

    [HUDI-7050] Flink HoodieHiveCatalog supports hadoop parameters (#10013)
---
 .../main/java/org/apache/hudi/configuration/HadoopConfigurations.java | 2 +-
 .../main/java/org/apache/hudi/table/catalog/HoodieCatalogFactory.java | 3 ++-
 .../main/java/org/apache/hudi/table/catalog/HoodieCatalogUtil.java    | 4 ++--
 .../main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java    | 2 +-
 .../java/org/apache/hudi/table/catalog/HoodieCatalogTestUtils.java    | 1 +
 5 files changed, 7 insertions(+), 5 deletions(-)

diff --git 
a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/HadoopConfigurations.java
 
b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/HadoopConfigurations.java
index 7a9e18b9b6d..1f5715bf688 100644
--- 
a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/HadoopConfigurations.java
+++ 
b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/HadoopConfigurations.java
@@ -29,7 +29,7 @@ import java.util.Map;
  * Utilities for fetching hadoop configurations.
  */
 public class HadoopConfigurations {
-  private static final String HADOOP_PREFIX = "hadoop.";
+  public static final String HADOOP_PREFIX = "hadoop.";
   private static final String PARQUET_PREFIX = "parquet.";
 
   /**
diff --git 
a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogFactory.java
 
b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogFactory.java
index 436b836eff4..b210475c76e 100644
--- 
a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogFactory.java
+++ 
b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogFactory.java
@@ -18,6 +18,7 @@
 
 package org.apache.hudi.table.catalog;
 
+import org.apache.hudi.configuration.HadoopConfigurations;
 import org.apache.hudi.exception.HoodieCatalogException;
 
 import org.apache.flink.configuration.ConfigOption;
@@ -50,7 +51,7 @@ public class HoodieCatalogFactory implements CatalogFactory {
   public Catalog createCatalog(Context context) {
     final FactoryUtil.CatalogFactoryHelper helper =
         FactoryUtil.createCatalogFactoryHelper(this, context);
-    helper.validate();
+    helper.validateExcept(HadoopConfigurations.HADOOP_PREFIX);
     String mode = helper.getOptions().get(CatalogOptions.MODE);
     switch (mode.toLowerCase(Locale.ROOT)) {
       case "hms":
diff --git 
a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogUtil.java
 
b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogUtil.java
index 3b2d697a988..f0ab2ebc755 100644
--- 
a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogUtil.java
+++ 
b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalogUtil.java
@@ -61,9 +61,9 @@ public class HoodieCatalogUtil {
    * @param hiveConfDir Hive conf directory path.
    * @return A HiveConf instance.
    */
-  public static HiveConf createHiveConf(@Nullable String hiveConfDir) {
+  public static HiveConf createHiveConf(@Nullable String hiveConfDir, 
org.apache.flink.configuration.Configuration flinkConf) {
     // create HiveConf from hadoop configuration with hadoop conf directory 
configured.
-    Configuration hadoopConf = HadoopConfigurations.getHadoopConf(new 
org.apache.flink.configuration.Configuration());
+    Configuration hadoopConf = HadoopConfigurations.getHadoopConf(flinkConf);
 
     // ignore all the static conf file URLs that HiveConf may have set
     HiveConf.setHiveSiteLocation(null);
diff --git 
a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java
 
b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java
index 954da0455de..9fb42451e02 100644
--- 
a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java
+++ 
b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieHiveCatalog.java
@@ -131,7 +131,7 @@ public class HoodieHiveCatalog extends AbstractCatalog {
   private final boolean external;
 
   public HoodieHiveCatalog(String catalogName, Configuration options) {
-    this(catalogName, options, 
HoodieCatalogUtil.createHiveConf(options.getString(CatalogOptions.HIVE_CONF_DIR)),
 false);
+    this(catalogName, options, 
HoodieCatalogUtil.createHiveConf(options.getString(CatalogOptions.HIVE_CONF_DIR),
 options), false);
   }
 
   public HoodieHiveCatalog(
diff --git 
a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/HoodieCatalogTestUtils.java
 
b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/HoodieCatalogTestUtils.java
index c98b4ac0da2..d58b0f99886 100644
--- 
a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/HoodieCatalogTestUtils.java
+++ 
b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/HoodieCatalogTestUtils.java
@@ -48,6 +48,7 @@ public class HoodieCatalogTestUtils {
 
   public static HoodieHiveCatalog createHiveCatalog(String name, boolean 
external) {
     Configuration options = new Configuration();
+    
options.setString("hadoop.dfs.client.block.write.replace-datanode-on-failure.enable",
 "true");
     options.setBoolean(CatalogOptions.TABLE_EXTERNAL, external);
     return new HoodieHiveCatalog(
         name,

Reply via email to