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

yihua 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 16e1adb5b3c [HUDI-7822] Resolve the conflicts between mixed hdfs and 
local path in Flink tests (#10931)
16e1adb5b3c is described below

commit 16e1adb5b3c8e3601044deec8e880ac15ccb74c8
Author: hehuiyuan <[email protected]>
AuthorDate: Sat Jun 1 06:34:51 2024 +0800

    [HUDI-7822] Resolve the conflicts between mixed hdfs and local path in 
Flink tests (#10931)
    
    Co-authored-by: Y Ethan Guo <[email protected]>
---
 .../hudi/table/catalog/TestHoodieCatalog.java       | 21 +++++++++++++++++----
 1 file changed, 17 insertions(+), 4 deletions(-)

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 98c98bebcce..f6737128698 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
@@ -28,6 +28,7 @@ import org.apache.hudi.common.testutils.HoodieTestUtils;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.configuration.FlinkOptions;
 import org.apache.hudi.configuration.HadoopConfigurations;
+import org.apache.hudi.exception.HoodieIOException;
 import org.apache.hudi.exception.HoodieValidationException;
 import org.apache.hudi.keygen.ComplexAvroKeyGenerator;
 import org.apache.hudi.keygen.NonpartitionedAvroKeyGenerator;
@@ -66,12 +67,14 @@ import 
org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
 import org.apache.flink.table.catalog.exceptions.TableNotExistException;
 import org.apache.flink.table.types.DataType;
 import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.hadoop.fs.FileSystem;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.io.TempDir;
 
 import java.io.File;
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -173,8 +176,12 @@ public class TestHoodieCatalog {
     streamTableEnv.getConfig().getConfiguration()
         
.setInteger(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 2);
 
-    File catalogPath = new File(tempFile.getPath());
-    catalogPath.mkdir();
+    try {
+      FileSystem fs = FileSystem.get(HadoopConfigurations.getHadoopConf(new 
Configuration()));
+      fs.mkdirs(new org.apache.hadoop.fs.Path(tempFile.getPath()));
+    } catch (IOException e) {
+      throw new HoodieIOException("Failed to create tempFile dir.", e);
+    }
 
     catalog = new HoodieCatalog("hudi", 
Configuration.fromMap(getDefaultCatalogOption()));
     catalog.open();
@@ -266,6 +273,7 @@ public class TestHoodieCatalog {
 
     // validate key generator for partitioned table
     HoodieTableMetaClient metaClient = createMetaClient(
+        new HadoopStorageConfiguration(HadoopConfigurations.getHadoopConf(new 
Configuration())),
         catalog.inferTablePath(catalogPathStr, tablePath));
     String keyGeneratorClassName = 
metaClient.getTableConfig().getKeyGeneratorClassName();
     assertEquals(keyGeneratorClassName, 
SimpleAvroKeyGenerator.class.getName());
@@ -283,6 +291,7 @@ public class TestHoodieCatalog {
 
     catalog.createTable(singleKeyMultiplePartitionPath, 
singleKeyMultiplePartitionTable, false);
     metaClient = createMetaClient(
+        new HadoopStorageConfiguration(HadoopConfigurations.getHadoopConf(new 
Configuration())),
         catalog.inferTablePath(catalogPathStr, 
singleKeyMultiplePartitionPath));
     keyGeneratorClassName = 
metaClient.getTableConfig().getKeyGeneratorClassName();
     assertThat(keyGeneratorClassName, 
is(ComplexAvroKeyGenerator.class.getName()));
@@ -300,6 +309,7 @@ public class TestHoodieCatalog {
 
     catalog.createTable(multipleKeySinglePartitionPath, 
multipleKeySinglePartitionTable, false);
     metaClient = createMetaClient(
+        new HadoopStorageConfiguration(HadoopConfigurations.getHadoopConf(new 
Configuration())),
         catalog.inferTablePath(catalogPathStr, 
singleKeyMultiplePartitionPath));
     keyGeneratorClassName = 
metaClient.getTableConfig().getKeyGeneratorClassName();
     assertThat(keyGeneratorClassName, 
is(ComplexAvroKeyGenerator.class.getName()));
@@ -317,7 +327,9 @@ public class TestHoodieCatalog {
 
     catalog.createTable(nonPartitionPath, nonPartitionCatalogTable, false);
 
-    metaClient = createMetaClient(catalog.inferTablePath(catalogPathStr, 
nonPartitionPath));
+    metaClient = createMetaClient(
+        new HadoopStorageConfiguration(HadoopConfigurations.getHadoopConf(new 
Configuration())),
+        catalog.inferTablePath(catalogPathStr, nonPartitionPath));
     keyGeneratorClassName = 
metaClient.getTableConfig().getKeyGeneratorClassName();
     assertEquals(keyGeneratorClassName, 
NonpartitionedAvroKeyGenerator.class.getName());
   }
@@ -426,7 +438,8 @@ public class TestHoodieCatalog {
     String tablePathStr = catalog.inferTablePath(catalogPathStr, tablePath);
     Configuration flinkConf = TestConfigurations.getDefaultConf(tablePathStr);
     HoodieTableMetaClient metaClient = HoodieTestUtils
-        .createMetaClient(new 
HadoopStorageConfiguration(HadoopConfigurations.getHadoopConf(flinkConf)), 
tablePathStr);
+        .createMetaClient(
+            new 
HadoopStorageConfiguration(HadoopConfigurations.getHadoopConf(flinkConf)), 
tablePathStr);
     TestData.writeData(TestData.DATA_SET_INSERT, flinkConf);
     assertTrue(catalog.partitionExists(tablePath, partitionSpec));
 

Reply via email to