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 43d23a49a02 [HUDI-5577] Validate option catalog.path in dfs mode 
(#7698)
43d23a49a02 is described below

commit 43d23a49a0213da8585aa7240924ac0439cc21f9
Author: Ruguo Yu <[email protected]>
AuthorDate: Thu Jan 19 13:36:49 2023 +0800

    [HUDI-5577] Validate option catalog.path in dfs mode (#7698)
---
 .../src/main/java/org/apache/hudi/table/catalog/HoodieCatalog.java     | 3 +++
 .../src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalog.java | 3 +++
 2 files changed, 6 insertions(+)

diff --git 
a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalog.java
 
b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalog.java
index 9b18e8d90b6..374bad86ab2 100644
--- 
a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalog.java
+++ 
b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalog.java
@@ -34,6 +34,7 @@ import org.apache.hudi.util.StreamerUtil;
 import org.apache.avro.Schema;
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.table.api.ValidationException;
 import org.apache.flink.table.catalog.AbstractCatalog;
 import org.apache.flink.table.catalog.CatalogBaseTable;
 import org.apache.flink.table.catalog.CatalogDatabase;
@@ -99,6 +100,8 @@ public class HoodieCatalog extends AbstractCatalog {
 
   public HoodieCatalog(String name, Configuration options) {
     super(name, options.get(DEFAULT_DATABASE));
+    options.getOptional(CATALOG_PATH).orElseThrow(() ->
+        new ValidationException("Option [catalog.path] should not be empty."));
     this.catalogPathStr = options.get(CATALOG_PATH);
     this.hadoopConf = HadoopConfigurations.getHadoopConf(options);
     this.tableCommonOptions = CatalogOptions.tableCommonOptions(options);
diff --git 
a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalog.java
 
b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalog.java
index e7736321b2e..1246e140a5f 100644
--- 
a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalog.java
+++ 
b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalog.java
@@ -35,6 +35,7 @@ import org.apache.flink.table.api.DataTypes;
 import org.apache.flink.table.api.EnvironmentSettings;
 import org.apache.flink.table.api.Schema;
 import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.ValidationException;
 import org.apache.flink.table.api.config.ExecutionConfigOptions;
 import org.apache.flink.table.api.internal.TableEnvironmentImpl;
 import org.apache.flink.table.catalog.CatalogBaseTable;
@@ -150,6 +151,8 @@ public class TestHoodieCatalog {
     File testDb = new File(tempFile, TEST_DEFAULT_DATABASE);
     testDb.mkdir();
     Map<String, String> catalogOptions = new HashMap<>();
+    assertThrows(ValidationException.class,
+        () -> catalog = new HoodieCatalog("hudi", 
Configuration.fromMap(catalogOptions)));
     catalogPathStr = tempFile.getAbsolutePath();
     catalogOptions.put(CATALOG_PATH.key(), catalogPathStr);
     catalogOptions.put(DEFAULT_DATABASE.key(), TEST_DEFAULT_DATABASE);

Reply via email to