diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ExportCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ExportCommand.java index b196c62d0fba1..effa096bfa9fc 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ExportCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ExportCommand.java @@ -18,12 +18,6 @@ package org.apache.hudi.cli.commands; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; -import org.apache.avro.specific.SpecificData; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieArchivedMetaEntry; import org.apache.hudi.avro.model.HoodieCleanMetadata; @@ -44,8 +38,14 @@ import org.apache.hudi.common.util.collection.ClosableIterator; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.hadoop.fs.HadoopFSUtils; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.avro.specific.SpecificData; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.shell.standard.ShellComponent; @@ -169,7 +169,7 @@ private int copyArchivedInstants(List statuses, Set actionSe LOG.error("Could not load metadata for action " + action + " at instant time " + instantTime); continue; } - final String outPath = localFolder + HoodieLocation.SEPARATOR + instantTime + "." + action; + final String outPath = localFolder + StoragePath.SEPARATOR + instantTime + "." + action; writeToFile(outPath, HoodieAvroUtils.avroToJson(metadata, true)); } } @@ -191,7 +191,7 @@ private int copyNonArchivedInstants(List instants, int limit, Str final HoodieTableMetaClient metaClient = HoodieCLI.getTableMetaClient(); final HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); for (HoodieInstant instant : instants) { - String localPath = localFolder + HoodieLocation.SEPARATOR + instant.getFileName(); + String localPath = localFolder + StoragePath.SEPARATOR + instant.getFileName(); byte[] data = null; switch (instant.getAction()) { diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestHoodieLogFileCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestHoodieLogFileCommand.java index bb35509235b33..c45e86f8e2199 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestHoodieLogFileCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestHoodieLogFileCommand.java @@ -44,7 +44,7 @@ import org.apache.hudi.common.util.FileIOUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.hadoop.fs.HadoopFSUtils; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; @@ -202,7 +202,7 @@ public void testShowLogFileRecordsWithMerge() throws IOException, InterruptedExc // write to path '2015/03/16'. Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); - partitionPath = tablePath + HoodieLocation.SEPARATOR + HoodieTestCommitMetadataGenerator.DEFAULT_SECOND_PARTITION_PATH; + partitionPath = tablePath + StoragePath.SEPARATOR + HoodieTestCommitMetadataGenerator.DEFAULT_SECOND_PARTITION_PATH; Files.createDirectories(Paths.get(partitionPath)); HoodieLogFormat.Writer writer = null; diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestTableCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestTableCommand.java index 22d108241c6cb..5b6abf25f60da 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestTableCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestTableCommand.java @@ -32,7 +32,7 @@ import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.util.Option; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.avro.Schema; import org.apache.hadoop.fs.FileSystem; @@ -147,7 +147,7 @@ public void testCreateWithSpecifiedValues() { assertTrue(ShellEvaluationResultUtil.isSuccess(result)); assertEquals("Metadata for table " + tableName + " loaded", result.toString()); HoodieTableMetaClient client = HoodieCLI.getTableMetaClient(); - assertEquals(metaPath + HoodieLocation.SEPARATOR + "archive", client.getArchivePath()); + assertEquals(metaPath + StoragePath.SEPARATOR + "archive", client.getArchivePath()); assertEquals(tablePath, client.getBasePath()); assertEquals(metaPath, client.getMetaPath()); assertEquals(HoodieTableType.MERGE_ON_READ, client.getTableType()); @@ -186,7 +186,7 @@ public void testRefresh() throws IOException { private void testRefreshCommand(String command) throws IOException { // clean table matedata FileSystem fs = FileSystem.get(hadoopConf()); - fs.delete(new Path(tablePath + HoodieLocation.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME), true); + fs.delete(new Path(tablePath + StoragePath.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME), true); // Create table assertTrue(prepareTable()); diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestBootstrapCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestBootstrapCommand.java index 4e7a9c68a1e80..2d73eb02e46d7 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestBootstrapCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestBootstrapCommand.java @@ -26,7 +26,7 @@ import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.functional.TestBootstrap; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -65,8 +65,8 @@ public class ITTestBootstrapCommand extends HoodieCLIIntegrationTestBase { public void init() { String srcName = "source"; tableName = "test-table"; - sourcePath = basePath + HoodieLocation.SEPARATOR + srcName; - tablePath = basePath + HoodieLocation.SEPARATOR + tableName; + sourcePath = basePath + StoragePath.SEPARATOR + srcName; + tablePath = basePath + StoragePath.SEPARATOR + tableName; // generate test data partitions = Arrays.asList("2018", "2019", "2020"); @@ -74,7 +74,7 @@ public void init() { for (int i = 0; i < partitions.size(); i++) { Dataset df = TestBootstrap.generateTestRawTripDataset(timestamp, i * NUM_OF_RECORDS, i * NUM_OF_RECORDS + NUM_OF_RECORDS, null, jsc, sqlContext); - df.write().parquet(sourcePath + HoodieLocation.SEPARATOR + PARTITION_FIELD + "=" + partitions.get(i)); + df.write().parquet(sourcePath + StoragePath.SEPARATOR + PARTITION_FIELD + "=" + partitions.get(i)); } } diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestHDFSParquetImportCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestHDFSParquetImportCommand.java index 34becfa0de323..3575b85344e05 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestHDFSParquetImportCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestHDFSParquetImportCommand.java @@ -26,7 +26,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.hudi.testutils.HoodieClientTestUtils; import org.apache.hudi.utilities.HDFSParquetImporter; import org.apache.hudi.utilities.functional.TestHDFSParquetImporter; @@ -77,7 +77,7 @@ public class ITTestHDFSParquetImportCommand extends HoodieCLIIntegrationTestBase @BeforeEach public void init() throws IOException, ParseException { tableName = "test_table"; - tablePath = basePath + HoodieLocation.SEPARATOR + tableName; + tablePath = basePath + StoragePath.SEPARATOR + tableName; sourcePath = new Path(basePath, "source"); targetPath = new Path(tablePath); schemaFile = new Path(basePath, "file.schema").toString(); @@ -109,7 +109,7 @@ public void testConvertWithInsert() throws IOException { () -> assertEquals("Table imported to hoodie format", result.toString())); // Check hudi table exist - String metaPath = targetPath + HoodieLocation.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME; + String metaPath = targetPath + StoragePath.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME; assertTrue(Files.exists(Paths.get(metaPath)), "Hoodie table not exist."); // Load meta data diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestMarkersCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestMarkersCommand.java index 194c0b498895e..25dd3c2152cde 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestMarkersCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestMarkersCommand.java @@ -25,7 +25,7 @@ import org.apache.hudi.common.model.IOType; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.testutils.FileCreateUtils; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -54,7 +54,7 @@ public class ITTestMarkersCommand extends HoodieCLIIntegrationTestBase { @BeforeEach public void init() throws IOException { String tableName = "test_table"; - tablePath = basePath + HoodieLocation.SEPARATOR + tableName; + tablePath = basePath + StoragePath.SEPARATOR + tableName; // Create table and connect new TableCommand().createTable( diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestSavepointsCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestSavepointsCommand.java index 3aebd6a483ffc..06a9662b1a126 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestSavepointsCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestSavepointsCommand.java @@ -33,7 +33,7 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.hadoop.fs.Path; import org.junit.jupiter.api.BeforeEach; @@ -66,7 +66,7 @@ public class ITTestSavepointsCommand extends HoodieCLIIntegrationTestBase { @BeforeEach public void init() throws IOException { String tableName = "test_table"; - tablePath = basePath + HoodieLocation.SEPARATOR + tableName; + tablePath = basePath + StoragePath.SEPARATOR + tableName; // Create table and connect new TableCommand().createTable( diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestTableCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestTableCommand.java index 260df2b532aee..a2833ee3c0ee7 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestTableCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestTableCommand.java @@ -37,7 +37,7 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.index.HoodieIndex; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -72,7 +72,7 @@ public class ITTestTableCommand extends HoodieCLIIntegrationTestBase { @Test public void testChangeTableCOW2MOR() throws IOException { - tablePath = basePath + HoodieLocation.SEPARATOR + tableName + "_cow2mor"; + tablePath = basePath + StoragePath.SEPARATOR + tableName + "_cow2mor"; // Create table and connect new TableCommand().createTable( tablePath, "test_table", HoodieTableType.COPY_ON_WRITE.name(), @@ -89,7 +89,7 @@ public void testChangeTableCOW2MOR() throws IOException { @Test public void testChangeTableMOR2COW() throws IOException { - tablePath = basePath + HoodieLocation.SEPARATOR + tableName + "_mor2cow"; + tablePath = basePath + StoragePath.SEPARATOR + tableName + "_mor2cow"; // Create table and connect new TableCommand().createTable( tablePath, "test_table", HoodieTableType.MERGE_ON_READ.name(), @@ -104,7 +104,7 @@ public void testChangeTableMOR2COW() throws IOException { @Test public void testChangeTableMOR2COW_withPendingCompactions() throws Exception { - tablePath = basePath + HoodieLocation.SEPARATOR + tableName + "_cow2mor"; + tablePath = basePath + StoragePath.SEPARATOR + tableName + "_cow2mor"; // Create table and connect new TableCommand().createTable( tablePath, "test_table", HoodieTableType.MERGE_ON_READ.name(), @@ -136,7 +136,7 @@ public void testChangeTableMOR2COW_withPendingCompactions() throws Exception { @Test public void testChangeTableMOR2COW_withFullCompaction() throws Exception { - tablePath = basePath + HoodieLocation.SEPARATOR + tableName + "_cow2mor"; + tablePath = basePath + StoragePath.SEPARATOR + tableName + "_cow2mor"; // Create table and connect new TableCommand().createTable( tablePath, "test_table", HoodieTableType.MERGE_ON_READ.name(), @@ -161,7 +161,7 @@ public void testChangeTableMOR2COW_withFullCompaction() throws Exception { @Test public void testChangeTableMOR2COW_withoutCompaction() throws Exception { - tablePath = basePath + HoodieLocation.SEPARATOR + tableName + "_cow2mor"; + tablePath = basePath + StoragePath.SEPARATOR + tableName + "_cow2mor"; // Create table and connect new TableCommand().createTable( tablePath, "test_table", HoodieTableType.MERGE_ON_READ.name(), diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HeartbeatUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HeartbeatUtils.java index 40e08275b29e2..de54d880632a8 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HeartbeatUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HeartbeatUtils.java @@ -22,7 +22,7 @@ import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.hudi.table.HoodieTable; import org.apache.hadoop.fs.FileSystem; @@ -52,7 +52,7 @@ public static boolean deleteHeartbeatFile(FileSystem fs, String basePath, String boolean deleted = false; try { String heartbeatFolderPath = HoodieTableMetaClient.getHeartbeatFolderPath(basePath); - deleted = fs.delete(new Path(heartbeatFolderPath + HoodieLocation.SEPARATOR + instantTime), false); + deleted = fs.delete(new Path(heartbeatFolderPath + StoragePath.SEPARATOR + instantTime), false); if (!deleted) { LOG.error("Failed to delete heartbeat for instant " + instantTime); } else { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HoodieHeartbeatClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HoodieHeartbeatClient.java index bb08ae997d990..0b1c607c51f05 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HoodieHeartbeatClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/heartbeat/HoodieHeartbeatClient.java @@ -22,7 +22,7 @@ import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieHeartbeatException; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -228,7 +228,7 @@ private void stopHeartbeatTimer(Heartbeat heartbeat) { public static Boolean heartbeatExists(FileSystem fs, String basePath, String instantTime) throws IOException { Path heartbeatFilePath = new Path(HoodieTableMetaClient.getHeartbeatFolderPath(basePath) - + HoodieLocation.SEPARATOR + instantTime); + + StoragePath.SEPARATOR + instantTime); return fs.exists(heartbeatFilePath); } @@ -255,7 +255,7 @@ private void updateHeartbeat(String instantTime) throws HoodieHeartbeatException try { Long newHeartbeatTime = System.currentTimeMillis(); OutputStream outputStream = - this.fs.create(new Path(heartbeatFolderPath + HoodieLocation.SEPARATOR + instantTime), true); + this.fs.create(new Path(heartbeatFolderPath + StoragePath.SEPARATOR + instantTime), true); outputStream.close(); Heartbeat heartbeat = instantToHeartbeatMap.get(instantTime); if (heartbeat.getLastHeartbeatTime() != null && isHeartbeatExpired(instantTime)) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/FileSystemBasedLockProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/FileSystemBasedLockProvider.java index 7dec8684d4365..cf8ffb7186a6a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/FileSystemBasedLockProvider.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/FileSystemBasedLockProvider.java @@ -33,7 +33,7 @@ import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieLockException; import org.apache.hudi.hadoop.fs.HadoopFSUtils; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.hudi.storage.StorageSchemes; import org.apache.hadoop.conf.Configuration; @@ -78,10 +78,10 @@ public FileSystemBasedLockProvider(final LockConfiguration lockConfiguration, fi String lockDirectory = lockConfiguration.getConfig().getString(FILESYSTEM_LOCK_PATH_PROP_KEY, null); if (StringUtils.isNullOrEmpty(lockDirectory)) { lockDirectory = lockConfiguration.getConfig().getString(HoodieWriteConfig.BASE_PATH.key()) - + HoodieLocation.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME; + + StoragePath.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME; } this.lockTimeoutMinutes = lockConfiguration.getConfig().getInteger(FILESYSTEM_LOCK_EXPIRE_PROP_KEY); - this.lockFile = new Path(lockDirectory + HoodieLocation.SEPARATOR + LOCK_FILE_NAME); + this.lockFile = new Path(lockDirectory + StoragePath.SEPARATOR + LOCK_FILE_NAME); this.lockInfo = new LockInfo(); this.sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS"); this.fs = HadoopFSUtils.getFs(this.lockFile.toString(), configuration); @@ -221,6 +221,6 @@ public static TypedProperties getLockConfig(String tablePath) { *

IMPORTANT: this path should be shared especially when there is engine cooperation. */ private static String defaultLockPath(String tablePath) { - return tablePath + HoodieLocation.SEPARATOR + AUXILIARYFOLDER_NAME; + return tablePath + StoragePath.SEPARATOR + AUXILIARYFOLDER_NAME; } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/functional/BaseHoodieFunctionalIndexClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/functional/BaseHoodieFunctionalIndexClient.java index 693699d59d7f4..f8a2bf64f6c6b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/functional/BaseHoodieFunctionalIndexClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/functional/BaseHoodieFunctionalIndexClient.java @@ -21,7 +21,7 @@ import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.hadoop.fs.Path; import org.slf4j.Logger; @@ -47,8 +47,8 @@ public void register(HoodieTableMetaClient metaClient, String indexName, String LOG.info("Registering index {} of using {}", indexName, indexType); String indexMetaPath = metaClient.getTableConfig().getIndexDefinitionPath() .orElseGet(() -> metaClient.getMetaPath() - + HoodieLocation.SEPARATOR + HoodieTableMetaClient.INDEX_DEFINITION_FOLDER_NAME - + HoodieLocation.SEPARATOR + HoodieTableMetaClient.INDEX_DEFINITION_FILE_NAME); + + StoragePath.SEPARATOR + HoodieTableMetaClient.INDEX_DEFINITION_FOLDER_NAME + + StoragePath.SEPARATOR + HoodieTableMetaClient.INDEX_DEFINITION_FILE_NAME); // build HoodieFunctionalIndexMetadata and then add to index definition file metaClient.buildFunctionalIndexDefinition(indexMetaPath, indexName, indexType, columns, options); // update table config if necessary diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/timeline/TestCompletionTimeQueryView.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/timeline/TestCompletionTimeQueryView.java index 3122473565e61..35a7a7be0b45e 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/timeline/TestCompletionTimeQueryView.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/timeline/TestCompletionTimeQueryView.java @@ -35,7 +35,7 @@ import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.index.HoodieIndex; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.hudi.table.HoodieTable; import org.apache.hadoop.conf.Configuration; @@ -66,7 +66,7 @@ public class TestCompletionTimeQueryView { @Test void testReadCompletionTime() throws Exception { String tableName = "testTable"; - String tablePath = tempFile.getAbsolutePath() + HoodieLocation.SEPARATOR + tableName; + String tablePath = tempFile.getAbsolutePath() + StoragePath.SEPARATOR + tableName; HoodieTableMetaClient metaClient = HoodieTestUtils.init(new Configuration(), tablePath, HoodieTableType.COPY_ON_WRITE, tableName); prepareTimeline(tablePath, metaClient); try (CompletionTimeQueryView view = new CompletionTimeQueryView(metaClient, String.format("%08d", 3))) { @@ -95,7 +95,7 @@ void testReadCompletionTime() throws Exception { @Test void testReadStartTime() throws Exception { String tableName = "testTable"; - String tablePath = tempFile.getAbsolutePath() + HoodieLocation.SEPARATOR + tableName; + String tablePath = tempFile.getAbsolutePath() + StoragePath.SEPARATOR + tableName; HoodieTableMetaClient metaClient = HoodieTestUtils.init(new Configuration(), tablePath, HoodieTableType.COPY_ON_WRITE, tableName); prepareTimeline(tablePath, metaClient); try (CompletionTimeQueryView view = new CompletionTimeQueryView(metaClient, String.format("%08d", 3))) { diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/utils/TestLegacyArchivedMetaEntryReader.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/utils/TestLegacyArchivedMetaEntryReader.java index c3ab604adcc23..91831adb36cd5 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/utils/TestLegacyArchivedMetaEntryReader.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/utils/TestLegacyArchivedMetaEntryReader.java @@ -39,7 +39,7 @@ import org.apache.hudi.common.util.collection.ClosableIterator; import org.apache.hudi.exception.HoodieCommitException; import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; @@ -74,7 +74,7 @@ public class TestLegacyArchivedMetaEntryReader { @Test void testReadLegacyArchivedTimeline() throws Exception { String tableName = "testTable"; - String tablePath = tempFile.getAbsolutePath() + HoodieLocation.SEPARATOR + tableName; + String tablePath = tempFile.getAbsolutePath() + StoragePath.SEPARATOR + tableName; HoodieTableMetaClient metaClient = HoodieTestUtils.init(new Configuration(), tablePath, HoodieTableType.COPY_ON_WRITE, tableName); prepareLegacyArchivedTimeline(metaClient); LegacyArchivedMetaEntryReader reader = new LegacyArchivedMetaEntryReader(metaClient); diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/TestJavaHoodieBackedMetadata.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/TestJavaHoodieBackedMetadata.java index 0871df0a3a3e6..d4df65270a865 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/TestJavaHoodieBackedMetadata.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/TestJavaHoodieBackedMetadata.java @@ -98,7 +98,7 @@ import org.apache.hudi.metadata.HoodieTableMetadataUtil; import org.apache.hudi.metadata.JavaHoodieBackedTableMetadataWriter; import org.apache.hudi.metadata.MetadataPartitionType; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.hudi.table.HoodieJavaTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; @@ -1230,7 +1230,7 @@ public void testFailedBootstrap() throws Exception { // remove the MDT partition from dataset to simulate failed bootstrap Properties updateProperties = new Properties(); updateProperties.setProperty(HoodieTableConfig.TABLE_METADATA_PARTITIONS.key(), ""); - HoodieTableConfig.update(fs, new Path(basePath + HoodieLocation.SEPARATOR + METAFOLDER_NAME), + HoodieTableConfig.update(fs, new Path(basePath + StoragePath.SEPARATOR + METAFOLDER_NAME), updateProperties); metaClient = HoodieTableMetaClient.reload(metaClient); @@ -2180,7 +2180,7 @@ public void testRollbackOfPartiallyFailedCommitWithNewPartitions() throws Except // There is no way to simulate failed commit on the main dataset, hence we simply delete the completed // instant so that only the inflight is left over. String commitInstantFileName = metaClient.getActiveTimeline().getReverseOrderedInstants().findFirst().get().getFileName(); - assertTrue(fs.delete(new Path(basePath + HoodieLocation.SEPARATOR + METAFOLDER_NAME, + assertTrue(fs.delete(new Path(basePath + StoragePath.SEPARATOR + METAFOLDER_NAME, commitInstantFileName), false)); } @@ -2280,7 +2280,7 @@ public void testErrorCases() throws Exception { // There is no way to simulate failed commit on the main dataset, hence we simply delete the completed // instant so that only the inflight is left over. String commitInstantFileName = metaClient.getActiveTimeline().getReverseOrderedInstants().findFirst().get().getFileName(); - assertTrue(fs.delete(new Path(basePath + HoodieLocation.SEPARATOR + METAFOLDER_NAME, + assertTrue(fs.delete(new Path(basePath + StoragePath.SEPARATOR + METAFOLDER_NAME, commitInstantFileName), false)); } @@ -2423,7 +2423,7 @@ public void testRepeatedActionWithSameInstantTime() throws Exception { // To simulate failed clean on the main dataset, we will delete the completed clean instant String cleanInstantFileName = metaClient.reloadActiveTimeline().getCleanerTimeline().filterCompletedInstants() .getReverseOrderedInstants().findFirst().get().getFileName(); - assertTrue(fs.delete(new Path(basePath + HoodieLocation.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME, + assertTrue(fs.delete(new Path(basePath + StoragePath.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME, cleanInstantFileName), false)); assertEquals(metaClient.reloadActiveTimeline().getCleanerTimeline().filterInflights().countInstants(), 1); assertEquals(metaClient.reloadActiveTimeline().getCleanerTimeline().filterCompletedInstants().countInstants(), 0); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMetadataWriterUtils.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMetadataWriterUtils.java index 589c0152c7cbd..3f10bde6f46cc 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMetadataWriterUtils.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMetadataWriterUtils.java @@ -31,7 +31,7 @@ import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.index.functional.HoodieFunctionalIndex; import org.apache.hudi.io.storage.HoodieFileWriterFactory; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.avro.Schema; import org.apache.hadoop.fs.Path; @@ -245,7 +245,7 @@ private static Path filePath(String basePath, String partition, String filename) if (partition.isEmpty()) { return new Path(basePath, filename); } else { - return new Path(basePath, partition + HoodieLocation.SEPARATOR + filename); + return new Path(basePath, partition + StoragePath.SEPARATOR + filename); } } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java index 1006d9e2fa268..5abccdbf850ea 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java @@ -47,7 +47,7 @@ import org.apache.hudi.config.HoodieLockConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieWriteConflictException; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.marker.SimpleDirectMarkerBasedDetectionStrategy; import org.apache.hudi.table.marker.SimpleTransactionDirectMarkerBasedDetectionStrategy; @@ -256,7 +256,7 @@ private void testHoodieClientBasicMultiWriterWithEarlyConflictDetection(String t HoodieWriteConfig config4 = HoodieWriteConfig.newBuilder().withProperties(writeConfig.getProps()).withHeartbeatIntervalInMs(heartBeatIntervalForCommit4).build(); final SparkRDDWriteClient client4 = getHoodieWriteClient(config4); - Path heartbeatFilePath = new Path(HoodieTableMetaClient.getHeartbeatFolderPath(basePath) + HoodieLocation.SEPARATOR + nextCommitTime3); + Path heartbeatFilePath = new Path(HoodieTableMetaClient.getHeartbeatFolderPath(basePath) + StoragePath.SEPARATOR + nextCommitTime3); fs.create(heartbeatFilePath, true); // Wait for heart beat expired for failed commitTime3 "003" diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java index c55d715dc4112..32264bbf35fa8 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java @@ -103,7 +103,7 @@ import org.apache.hudi.metadata.HoodieTableMetadataUtil; import org.apache.hudi.metadata.MetadataPartitionType; import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; @@ -1647,7 +1647,7 @@ public void testFailedBootstrap() throws Exception { // remove the MDT partition from dataset to simulate failed bootstrap Properties updateProperties = new Properties(); updateProperties.setProperty(HoodieTableConfig.TABLE_METADATA_PARTITIONS.key(), ""); - HoodieTableConfig.update(fs, new Path(basePath + HoodieLocation.SEPARATOR + METAFOLDER_NAME), + HoodieTableConfig.update(fs, new Path(basePath + StoragePath.SEPARATOR + METAFOLDER_NAME), updateProperties); metaClient = HoodieTableMetaClient.reload(metaClient); @@ -2643,7 +2643,7 @@ public void testRollbackOfPartiallyFailedCommitWithNewPartitions() throws Except // There is no way to simulate failed commit on the main dataset, hence we simply delete the completed // instant so that only the inflight is left over. String commitInstantFileName = metaClient.getActiveTimeline().getReverseOrderedInstants().findFirst().get().getFileName(); - assertTrue(fs.delete(new Path(basePath + HoodieLocation.SEPARATOR + METAFOLDER_NAME, + assertTrue(fs.delete(new Path(basePath + StoragePath.SEPARATOR + METAFOLDER_NAME, commitInstantFileName), false)); } @@ -2695,9 +2695,9 @@ public void testRollbackPendingCommitWithRecordIndex(boolean performUpsert) thro // metadata table partitions are rebootstrapped. metadataWriter.dropMetadataPartitions(Arrays.asList(MetadataPartitionType.RECORD_INDEX, FILES)); assertFalse(fs.exists(new Path(getMetadataTableBasePath(basePath) - + HoodieLocation.SEPARATOR + FILES.getPartitionPath()))); + + StoragePath.SEPARATOR + FILES.getPartitionPath()))); assertFalse(fs.exists(new Path(getMetadataTableBasePath(basePath) - + HoodieLocation.SEPARATOR + MetadataPartitionType.RECORD_INDEX.getPartitionPath()))); + + StoragePath.SEPARATOR + MetadataPartitionType.RECORD_INDEX.getPartitionPath()))); metaClient = HoodieTableMetaClient.reload(metaClient); // Insert/upsert third batch of records @@ -2714,14 +2714,14 @@ public void testRollbackPendingCommitWithRecordIndex(boolean performUpsert) thro writeStatuses = client.insert(jsc.parallelize(records, 1), commitTime).collect(); } assertNoWriteErrors(writeStatuses); - assertTrue(fs.exists(new Path(basePath + HoodieLocation.SEPARATOR + METAFOLDER_NAME))); + assertTrue(fs.exists(new Path(basePath + StoragePath.SEPARATOR + METAFOLDER_NAME))); metaClient = HoodieTableMetaClient.reload(metaClient); assertFalse(metaClient.getActiveTimeline().filterCompletedInstants().filterCompletedInstants().findInstantsAfterOrEquals(commitTime, 1).empty()); assertTrue(fs.exists(new Path(getMetadataTableBasePath(basePath) - + HoodieLocation.SEPARATOR + FILES.getPartitionPath()))); + + StoragePath.SEPARATOR + FILES.getPartitionPath()))); assertTrue(fs.exists(new Path(getMetadataTableBasePath(basePath) - + HoodieLocation.SEPARATOR + MetadataPartitionType.RECORD_INDEX.getPartitionPath()))); + + StoragePath.SEPARATOR + MetadataPartitionType.RECORD_INDEX.getPartitionPath()))); } /** @@ -2862,7 +2862,7 @@ public void testErrorCases() throws Exception { // There is no way to simulate failed commit on the main dataset, hence we simply delete the completed // instant so that only the inflight is left over. String commitInstantFileName = metaClient.getActiveTimeline().getReverseOrderedInstants().findFirst().get().getFileName(); - assertTrue(fs.delete(new Path(basePath + HoodieLocation.SEPARATOR + METAFOLDER_NAME, + assertTrue(fs.delete(new Path(basePath + StoragePath.SEPARATOR + METAFOLDER_NAME, commitInstantFileName), false)); } @@ -3068,7 +3068,7 @@ public void testRepeatedActionWithSameInstantTime() throws Exception { // To simulate failed clean on the main dataset, we will delete the completed clean instant String cleanInstantFileName = metaClient.reloadActiveTimeline().getCleanerTimeline().filterCompletedInstants() .getReverseOrderedInstants().findFirst().get().getFileName(); - assertTrue(fs.delete(new Path(basePath + HoodieLocation.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME, + assertTrue(fs.delete(new Path(basePath + StoragePath.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME, cleanInstantFileName), false)); assertEquals(metaClient.reloadActiveTimeline().getCleanerTimeline().filterInflights().countInstants(), 1); assertEquals(metaClient.reloadActiveTimeline().getCleanerTimeline().filterCompletedInstants().countInstants(), 0); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/DirectMarkerBasedDetectionStrategy.java b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/DirectMarkerBasedDetectionStrategy.java index ea08456d16e3a..a6ab1640c9bb6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/DirectMarkerBasedDetectionStrategy.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/conflict/detection/DirectMarkerBasedDetectionStrategy.java @@ -27,7 +27,7 @@ import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.hadoop.fs.HoodieWrapperFileSystem; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -78,7 +78,7 @@ public DirectMarkerBasedDetectionStrategy(HoodieWrapperFileSystem fs, String par * @throws IOException upon errors. */ public boolean checkMarkerConflict(String basePath, long maxAllowableHeartbeatIntervalInMs) throws IOException { - String tempFolderPath = basePath + HoodieLocation.SEPARATOR + HoodieTableMetaClient.TEMPFOLDER_NAME; + String tempFolderPath = basePath + StoragePath.SEPARATOR + HoodieTableMetaClient.TEMPFOLDER_NAME; List candidateInstants = MarkerUtils.getCandidateInstants(activeTimeline, Arrays.stream(fs.listStatus(new Path(tempFolderPath))).map(FileStatus::getPath).collect(Collectors.toList()), instantTime, maxAllowableHeartbeatIntervalInMs, fs, basePath); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java index 5880d7f906891..d84c677e3418d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java @@ -42,8 +42,8 @@ import org.apache.hudi.hadoop.fs.inline.InLineFSUtils; import org.apache.hudi.hadoop.fs.inline.InLineFileSystem; import org.apache.hudi.metadata.HoodieTableMetadata; -import org.apache.hudi.storage.HoodieLocation; import org.apache.hudi.storage.HoodieStorage; +import org.apache.hudi.storage.StoragePath; import org.apache.hudi.storage.StorageSchemes; import org.apache.hadoop.conf.Configuration; @@ -131,7 +131,7 @@ public static Path makeQualified(FileSystem fs, Path path) { * @param location to be qualified. * @return qualified location, prefixed with the URI of the target HoodieStorage object provided. */ - public static HoodieLocation makeQualified(HoodieStorage storage, HoodieLocation location) { + public static StoragePath makeQualified(HoodieStorage storage, StoragePath location) { return location.makeQualified(storage.getUri()); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/heartbeat/HoodieHeartbeatUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/heartbeat/HoodieHeartbeatUtils.java index f7af86f79542d..57317a831a014 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/heartbeat/HoodieHeartbeatUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/heartbeat/HoodieHeartbeatUtils.java @@ -20,7 +20,7 @@ package org.apache.hudi.common.heartbeat; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -46,7 +46,7 @@ public class HoodieHeartbeatUtils { */ public static Long getLastHeartbeatTime(FileSystem fs, String basePath, String instantTime) throws IOException { Path heartbeatFilePath = new Path(HoodieTableMetaClient.getHeartbeatFolderPath(basePath) - + HoodieLocation.SEPARATOR + instantTime); + + StoragePath.SEPARATOR + instantTime); if (fs.exists(heartbeatFilePath)) { return fs.getFileStatus(heartbeatFilePath).getModificationTime(); } else { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java index 9d451893a630e..a2455e08356b2 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java @@ -58,7 +58,7 @@ import org.apache.hudi.hadoop.fs.SerializablePath; import org.apache.hudi.keygen.constant.KeyGeneratorType; import org.apache.hudi.metadata.HoodieTableMetadata; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; @@ -102,18 +102,18 @@ public class HoodieTableMetaClient implements Serializable { private static final long serialVersionUID = 1L; private static final Logger LOG = LoggerFactory.getLogger(HoodieTableMetaClient.class); public static final String METAFOLDER_NAME = ".hoodie"; - public static final String TEMPFOLDER_NAME = METAFOLDER_NAME + HoodieLocation.SEPARATOR + ".temp"; - public static final String AUXILIARYFOLDER_NAME = METAFOLDER_NAME + HoodieLocation.SEPARATOR + ".aux"; - public static final String BOOTSTRAP_INDEX_ROOT_FOLDER_PATH = AUXILIARYFOLDER_NAME + HoodieLocation.SEPARATOR + ".bootstrap"; - public static final String SAMPLE_WRITES_FOLDER_PATH = AUXILIARYFOLDER_NAME + HoodieLocation.SEPARATOR + ".sample_writes"; - public static final String HEARTBEAT_FOLDER_NAME = METAFOLDER_NAME + HoodieLocation.SEPARATOR + ".heartbeat"; - public static final String METADATA_TABLE_FOLDER_PATH = METAFOLDER_NAME + HoodieLocation.SEPARATOR + "metadata"; + public static final String TEMPFOLDER_NAME = METAFOLDER_NAME + StoragePath.SEPARATOR + ".temp"; + public static final String AUXILIARYFOLDER_NAME = METAFOLDER_NAME + StoragePath.SEPARATOR + ".aux"; + public static final String BOOTSTRAP_INDEX_ROOT_FOLDER_PATH = AUXILIARYFOLDER_NAME + StoragePath.SEPARATOR + ".bootstrap"; + public static final String SAMPLE_WRITES_FOLDER_PATH = AUXILIARYFOLDER_NAME + StoragePath.SEPARATOR + ".sample_writes"; + public static final String HEARTBEAT_FOLDER_NAME = METAFOLDER_NAME + StoragePath.SEPARATOR + ".heartbeat"; + public static final String METADATA_TABLE_FOLDER_PATH = METAFOLDER_NAME + StoragePath.SEPARATOR + "metadata"; public static final String HASHING_METADATA_FOLDER_NAME = - ".bucket_index" + HoodieLocation.SEPARATOR + "consistent_hashing_metadata"; + ".bucket_index" + StoragePath.SEPARATOR + "consistent_hashing_metadata"; public static final String BOOTSTRAP_INDEX_BY_PARTITION_FOLDER_PATH = BOOTSTRAP_INDEX_ROOT_FOLDER_PATH - + HoodieLocation.SEPARATOR + ".partitions"; + + StoragePath.SEPARATOR + ".partitions"; public static final String BOOTSTRAP_INDEX_BY_FILE_ID_FOLDER_PATH = - BOOTSTRAP_INDEX_ROOT_FOLDER_PATH + HoodieLocation.SEPARATOR + ".fileids"; + BOOTSTRAP_INDEX_ROOT_FOLDER_PATH + StoragePath.SEPARATOR + ".fileids"; public static final String SCHEMA_FOLDER_NAME = ".schema"; @@ -324,7 +324,7 @@ public String getHashingMetadataPath() { * @return Temp Folder path */ public String getTempFolderPath() { - return basePath + HoodieLocation.SEPARATOR + TEMPFOLDER_NAME; + return basePath + StoragePath.SEPARATOR + TEMPFOLDER_NAME; } /** @@ -334,35 +334,35 @@ public String getTempFolderPath() { * @return */ public String getMarkerFolderPath(String instantTs) { - return String.format("%s%s%s", getTempFolderPath(), HoodieLocation.SEPARATOR, instantTs); + return String.format("%s%s%s", getTempFolderPath(), StoragePath.SEPARATOR, instantTs); } /** * @return Auxiliary Meta path */ public String getMetaAuxiliaryPath() { - return basePath + HoodieLocation.SEPARATOR + AUXILIARYFOLDER_NAME; + return basePath + StoragePath.SEPARATOR + AUXILIARYFOLDER_NAME; } /** * @return Heartbeat folder path. */ public static String getHeartbeatFolderPath(String basePath) { - return String.format("%s%s%s", basePath, HoodieLocation.SEPARATOR, HEARTBEAT_FOLDER_NAME); + return String.format("%s%s%s", basePath, StoragePath.SEPARATOR, HEARTBEAT_FOLDER_NAME); } /** * @return Bootstrap Index By Partition Folder */ public String getBootstrapIndexByPartitionFolderPath() { - return basePath + HoodieLocation.SEPARATOR + BOOTSTRAP_INDEX_BY_PARTITION_FOLDER_PATH; + return basePath + StoragePath.SEPARATOR + BOOTSTRAP_INDEX_BY_PARTITION_FOLDER_PATH; } /** * @return Bootstrap Index By Hudi File Id Folder */ public String getBootstrapIndexByFileIdFolderNameFolderPath() { - return basePath + HoodieLocation.SEPARATOR + BOOTSTRAP_INDEX_BY_FILE_ID_FOLDER_PATH; + return basePath + StoragePath.SEPARATOR + BOOTSTRAP_INDEX_BY_FILE_ID_FOLDER_PATH; } /** @@ -370,7 +370,7 @@ public String getBootstrapIndexByFileIdFolderNameFolderPath() { */ public String getArchivePath() { String archiveFolder = tableConfig.getArchivelogFolder(); - return getMetaPath() + HoodieLocation.SEPARATOR + archiveFolder; + return getMetaPath() + StoragePath.SEPARATOR + archiveFolder; } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/AbstractHoodieTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/AbstractHoodieTableMetadata.java index 96d93d01bf5a7..2efbfcfa97d9f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/AbstractHoodieTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/AbstractHoodieTableMetadata.java @@ -27,7 +27,7 @@ import org.apache.hudi.hadoop.fs.SerializablePath; import org.apache.hudi.internal.schema.Type; import org.apache.hudi.internal.schema.Types; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import java.util.Collections; import java.util.List; @@ -57,14 +57,14 @@ protected static int getPathPartitionLevel(Types.RecordType partitionFields, Str int level = 1; for (int i = 1; i < path.length() - 1; i++) { - if (path.charAt(i) == HoodieLocation.SEPARATOR_CHAR) { + if (path.charAt(i) == StoragePath.SEPARATOR_CHAR) { level++; } } - if (path.startsWith(HoodieLocation.SEPARATOR)) { + if (path.startsWith(StoragePath.SEPARATOR)) { level--; } - if (path.endsWith(HoodieLocation.SEPARATOR)) { + if (path.endsWith(StoragePath.SEPARATOR)) { level--; } return level; diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java index 40c0debf28e95..10c094fdfb68f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java @@ -39,7 +39,7 @@ import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.hadoop.fs.CachingPath; import org.apache.hudi.io.storage.HoodieAvroHFileReaderImplBase; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.hudi.util.Lazy; import org.apache.avro.Schema; @@ -356,7 +356,7 @@ public static HoodieRecord createBloomFilterMetadataRecor final String bloomFilterType, final ByteBuffer bloomFilter, final boolean isDeleted) { - checkArgument(!baseFileName.contains(HoodieLocation.SEPARATOR) + checkArgument(!baseFileName.contains(StoragePath.SEPARATOR) && FSUtils.isBaseFile(new Path(baseFileName)), "Invalid base file '" + baseFileName + "' for MetaIndexBloomFilter!"); final String bloomFilterIndexKey = getBloomFilterRecordKey(partitionName, baseFileName); diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java index c19b2b62cd7c5..a0551dad725b1 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java @@ -32,7 +32,7 @@ import org.apache.hudi.exception.HoodieMetadataException; import org.apache.hudi.expression.Expression; import org.apache.hudi.internal.schema.Types; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; @@ -69,7 +69,7 @@ public interface HoodieTableMetadata extends Serializable, AutoCloseable { * Return the base-path of the Metadata Table for the given Dataset identified by base-path */ static String getMetadataTableBasePath(String dataTableBasePath) { - return dataTableBasePath + HoodieLocation.SEPARATOR + HoodieTableMetaClient.METADATA_TABLE_FOLDER_PATH; + return dataTableBasePath + StoragePath.SEPARATOR + HoodieTableMetaClient.METADATA_TABLE_FOLDER_PATH; } /** @@ -94,7 +94,7 @@ static String getDataTableBasePathFromMetadataTable(String metadataTableBasePath * @param metadataTableBasePath The base path of the metadata table */ static String getDatasetBasePath(String metadataTableBasePath) { - int endPos = metadataTableBasePath.lastIndexOf(HoodieLocation.SEPARATOR + HoodieTableMetaClient.METADATA_TABLE_FOLDER_PATH); + int endPos = metadataTableBasePath.lastIndexOf(StoragePath.SEPARATOR + HoodieTableMetaClient.METADATA_TABLE_FOLDER_PATH); checkState(endPos != -1, metadataTableBasePath + " should be base path of the metadata table"); return metadataTableBasePath.substring(0, endPos); } @@ -108,7 +108,7 @@ static boolean isMetadataTable(String basePath) { if (basePath == null || basePath.isEmpty()) { return false; } - if (basePath.endsWith(HoodieLocation.SEPARATOR)) { + if (basePath.endsWith(StoragePath.SEPARATOR)) { basePath = basePath.substring(0, basePath.length() - 1); } return basePath.endsWith(HoodieTableMetaClient.METADATA_TABLE_FOLDER_PATH); diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java index e9eba01bf83b8..e195704430544 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java @@ -82,7 +82,7 @@ import org.apache.hudi.hadoop.fs.HadoopFSUtils; import org.apache.hudi.io.storage.HoodieFileReader; import org.apache.hudi.io.storage.HoodieFileReaderFactory; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.hudi.util.Lazy; import org.apache.avro.AvroTypeException; @@ -1912,7 +1912,7 @@ private static Path filePath(String basePath, String partition, String filename) if (partition.isEmpty()) { return new Path(basePath, filename); } else { - return new Path(basePath, partition + HoodieLocation.SEPARATOR + filename); + return new Path(basePath, partition + StoragePath.SEPARATOR + filename); } } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtils.java index ab638bb37a957..350d1a02072c4 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtils.java @@ -34,8 +34,8 @@ import org.apache.hudi.hadoop.fs.HoodieWrapperFileSystem; import org.apache.hudi.hadoop.fs.NoOpConsistencyGuard; import org.apache.hudi.hadoop.fs.inline.InLineFSUtils; -import org.apache.hudi.storage.HoodieLocation; import org.apache.hudi.storage.HoodieStorage; +import org.apache.hudi.storage.StoragePath; import org.apache.hudi.storage.hadoop.HoodieHadoopStorage; import org.apache.hadoop.conf.Configuration; @@ -587,14 +587,14 @@ public void testMakeQualified() { FileSystem wrapperFs = new HoodieWrapperFileSystem(fs, new NoOpConsistencyGuard()); HoodieStorage storage = new HoodieHadoopStorage(fs); HoodieStorage wrapperStorage = new HoodieHadoopStorage(wrapperFs); - assertEquals(new HoodieLocation("file:///x/y"), - FSUtils.makeQualified(storage, new HoodieLocation("/x/y"))); - assertEquals(new HoodieLocation("file:///x/y"), - FSUtils.makeQualified(wrapperStorage, new HoodieLocation("/x/y"))); - assertEquals(new HoodieLocation("s3://x/y"), - FSUtils.makeQualified(storage, new HoodieLocation("s3://x/y"))); - assertEquals(new HoodieLocation("s3://x/y"), - FSUtils.makeQualified(wrapperStorage, new HoodieLocation("s3://x/y"))); + assertEquals(new StoragePath("file:///x/y"), + FSUtils.makeQualified(storage, new StoragePath("/x/y"))); + assertEquals(new StoragePath("file:///x/y"), + FSUtils.makeQualified(wrapperStorage, new StoragePath("/x/y"))); + assertEquals(new StoragePath("s3://x/y"), + FSUtils.makeQualified(storage, new StoragePath("s3://x/y"))); + assertEquals(new StoragePath("s3://x/y"), + FSUtils.makeQualified(wrapperStorage, new StoragePath("s3://x/y"))); } private Path getHoodieTempDir() { diff --git a/hudi-common/src/test/java/org/apache/hudi/common/fs/TestHoodieWrapperFileSystem.java b/hudi-common/src/test/java/org/apache/hudi/common/fs/TestHoodieWrapperFileSystem.java index dc9fdf3674098..dc6bd6f0135fa 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/fs/TestHoodieWrapperFileSystem.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/fs/TestHoodieWrapperFileSystem.java @@ -24,7 +24,7 @@ import org.apache.hudi.hadoop.fs.HadoopFSUtils; import org.apache.hudi.hadoop.fs.HoodieWrapperFileSystem; import org.apache.hudi.hadoop.fs.NoOpConsistencyGuard; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -71,7 +71,7 @@ public static void cleanUp() { public void testCreateImmutableFileInPath() throws IOException { HoodieWrapperFileSystem fs = new HoodieWrapperFileSystem(HadoopFSUtils.getFs(basePath, new Configuration()), new NoOpConsistencyGuard()); String testContent = "test content"; - Path testFile = new Path(basePath + HoodieLocation.SEPARATOR + "clean.00000001"); + Path testFile = new Path(basePath + StoragePath.SEPARATOR + "clean.00000001"); // create same commit twice fs.createImmutableFileInPath(testFile, Option.of(getUTF8Bytes(testContent))); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/meta/CkpMetadata.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/meta/CkpMetadata.java index 32baf6986a526..c193abaa6915b 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/meta/CkpMetadata.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/meta/CkpMetadata.java @@ -23,7 +23,7 @@ import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.VisibleForTesting; import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -220,8 +220,8 @@ public String lastCompleteInstant() { protected static String ckpMetaPath(String basePath, String uniqueId) { // .hoodie/.aux/ckp_meta - String metaPath = basePath + HoodieLocation.SEPARATOR + HoodieTableMetaClient.AUXILIARYFOLDER_NAME - + HoodieLocation.SEPARATOR + CKP_META; + String metaPath = basePath + StoragePath.SEPARATOR + HoodieTableMetaClient.AUXILIARYFOLDER_NAME + + StoragePath.SEPARATOR + CKP_META; return StringUtils.isNullOrEmpty(uniqueId) ? metaPath : metaPath + "_" + uniqueId; } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/FileIndex.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/FileIndex.java index 68c2a05fccd49..c1d4fe1b92496 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/FileIndex.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/FileIndex.java @@ -29,7 +29,7 @@ import org.apache.hudi.source.prune.PartitionPruners; import org.apache.hudi.source.prune.PrimaryKeyPruners; import org.apache.hudi.source.stats.ColumnStatsIndices; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.hudi.util.DataTypeUtils; import org.apache.hudi.util.StreamerUtil; @@ -121,7 +121,7 @@ public List> getPartitions( } List> partitions = new ArrayList<>(); for (String partitionPath : partitionPaths) { - String[] paths = partitionPath.split(HoodieLocation.SEPARATOR); + String[] paths = partitionPath.split(StoragePath.SEPARATOR); Map partitionMapping = new LinkedHashMap<>(); if (hivePartition) { Arrays.stream(paths).forEach(p -> { diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/TableOptionProperties.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/TableOptionProperties.java index bc9c9a3f7b7b7..f451bfce64e4f 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/TableOptionProperties.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/catalog/TableOptionProperties.java @@ -25,7 +25,7 @@ import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieValidationException; import org.apache.hudi.hadoop.fs.HadoopFSUtils; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.hudi.sync.common.util.SparkDataSourceTableUtils; import org.apache.hudi.util.AvroSchemaConverter; @@ -156,7 +156,7 @@ public static Map loadFromProperties(String basePath, Configurat } private static Path getPropertiesFilePath(String basePath) { - String auxPath = basePath + HoodieLocation.SEPARATOR + AUXILIARYFOLDER_NAME; + String auxPath = basePath + StoragePath.SEPARATOR + AUXILIARYFOLDER_NAME; return new Path(auxPath, FILE_NAME); } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FilePathUtils.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FilePathUtils.java index 78467abe9dc07..48f50b69f6610 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FilePathUtils.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FilePathUtils.java @@ -20,7 +20,7 @@ import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.hadoop.fs.HadoopFSUtils; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.hudi.util.DataTypeUtils; import org.apache.flink.api.java.tuple.Tuple2; @@ -99,7 +99,7 @@ public static String generatePartitionPath( int i = 0; for (Map.Entry e : partitionKVs.entrySet()) { if (i > 0) { - suffixBuf.append(HoodieLocation.SEPARATOR); + suffixBuf.append(StoragePath.SEPARATOR); } if (hivePartition) { suffixBuf.append(escapePathName(e.getKey())); @@ -109,7 +109,7 @@ public static String generatePartitionPath( i++; } if (sepSuffix) { - suffixBuf.append(HoodieLocation.SEPARATOR); + suffixBuf.append(StoragePath.SEPARATOR); } return suffixBuf.toString(); } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/ClientIds.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/ClientIds.java index 82350a3b85bce..affea2e5d435f 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/ClientIds.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/ClientIds.java @@ -24,7 +24,7 @@ import org.apache.hudi.configuration.HadoopConfigurations; import org.apache.hudi.exception.HoodieHeartbeatException; import org.apache.hudi.hadoop.fs.HadoopFSUtils; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.flink.configuration.Configuration; import org.apache.hadoop.fs.FileStatus; @@ -149,7 +149,7 @@ public static boolean isHeartbeatExpired(FileSystem fs, Path path, long timeoutT // Utilities // ------------------------------------------------------------------------- private String getHeartbeatFolderPath(String basePath) { - return basePath + HoodieLocation.SEPARATOR + AUXILIARYFOLDER_NAME + HoodieLocation.SEPARATOR + HEARTBEAT_FOLDER_NAME; + return basePath + StoragePath.SEPARATOR + AUXILIARYFOLDER_NAME + StoragePath.SEPARATOR + HEARTBEAT_FOLDER_NAME; } private Path getHeartbeatFilePath(String basePath, String uniqueId) { diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/ViewStorageProperties.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/ViewStorageProperties.java index 1c13e20241513..a4cef4b7d342f 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/ViewStorageProperties.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/ViewStorageProperties.java @@ -24,7 +24,7 @@ import org.apache.hudi.configuration.HadoopConfigurations; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.hadoop.fs.HadoopFSUtils; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.flink.configuration.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -83,7 +83,7 @@ public static FileSystemViewStorageConfig loadFromProperties(String basePath, Co } private static Path getPropertiesFilePath(String basePath, String uniqueId) { - String auxPath = basePath + HoodieLocation.SEPARATOR + AUXILIARYFOLDER_NAME; + String auxPath = basePath + StoragePath.SEPARATOR + AUXILIARYFOLDER_NAME; String fileName = StringUtils.isNullOrEmpty(uniqueId) ? FILE_NAME : FILE_NAME + "_" + uniqueId; return new Path(auxPath, fileName); } diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/ITTestDataStreamWrite.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/ITTestDataStreamWrite.java index 8995d0247bc9a..fea986885f8c2 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/ITTestDataStreamWrite.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/ITTestDataStreamWrite.java @@ -28,7 +28,7 @@ import org.apache.hudi.sink.transform.ChainedTransformer; import org.apache.hudi.sink.transform.Transformer; import org.apache.hudi.sink.utils.Pipelines; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.hudi.table.catalog.HoodieCatalog; import org.apache.hudi.table.catalog.TableOptionProperties; import org.apache.hudi.util.AvroSchemaConverter; @@ -441,7 +441,7 @@ public void testHoodiePipelineBuilderSourceWithSchemaSet() throws Exception { // create table dir final String dbName = DEFAULT_DATABASE.defaultValue(); final String tableName = "t1"; - File testTable = new File(tempFile, dbName + HoodieLocation.SEPARATOR + tableName); + File testTable = new File(tempFile, dbName + StoragePath.SEPARATOR + tableName); testTable.mkdir(); Configuration conf = TestConfigurations.getDefaultConf(testTable.toURI().toString()); diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/bucket/ITTestBucketStreamWrite.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/bucket/ITTestBucketStreamWrite.java index 8301b2ae99a6b..5fc877f212a65 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/bucket/ITTestBucketStreamWrite.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/bucket/ITTestBucketStreamWrite.java @@ -27,7 +27,7 @@ import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.hadoop.fs.HadoopFSUtils; import org.apache.hudi.index.HoodieIndex.IndexType; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.hudi.util.StreamerUtil; import org.apache.hudi.utils.FlinkMiniCluster; import org.apache.hudi.utils.TestConfigurations; @@ -110,7 +110,7 @@ private static void doDeleteCommit(String tablePath, boolean isCow) throws Excep // delete successful commit to simulate an unsuccessful write FileSystem fs = metaClient.getFs(); - Path path = new Path(metaClient.getMetaPath() + HoodieLocation.SEPARATOR + filename); + Path path = new Path(metaClient.getMetaPath() + StoragePath.SEPARATOR + filename); fs.delete(path); commitMetadata.getFileIdAndRelativePaths().forEach((fileId, relativePath) -> { diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/TestWriteBase.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/TestWriteBase.java index b31cc0f8b1971..f40bc9c365aae 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/TestWriteBase.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/TestWriteBase.java @@ -34,7 +34,7 @@ import org.apache.hudi.sink.event.WriteMetadataEvent; import org.apache.hudi.sink.meta.CkpMetadata; import org.apache.hudi.sink.meta.CkpMetadataFactory; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.hudi.util.StreamerUtil; import org.apache.hudi.utils.TestData; import org.apache.hudi.utils.TestUtils; @@ -499,7 +499,7 @@ public TestHarness rollbackLastCompleteInstantToInflight() throws Exception { // refresh the heartbeat in case it is timed out. OutputStream outputStream = metaClient.getFs().create(new Path(HoodieTableMetaClient.getHeartbeatFolderPath(basePath) - + HoodieLocation.SEPARATOR + this.lastComplete), true); + + StoragePath.SEPARATOR + this.lastComplete), true); outputStream.close(); this.lastPending = this.lastComplete; this.lastComplete = lastCompleteInstant(); diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestUtils.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestUtils.java index f4549eb669d70..d1e3172f590aa 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestUtils.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestUtils.java @@ -28,7 +28,7 @@ import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.configuration.HadoopConfigurations; import org.apache.hudi.source.StreamReadMonitoringFunction; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.hudi.table.format.mor.MergeOnReadInputSplit; import org.apache.hudi.util.StreamerUtil; @@ -108,7 +108,7 @@ public static String getNthArchivedInstant(String basePath, int n) { public static String getSplitPartitionPath(MergeOnReadInputSplit split) { assertTrue(split.getLogPaths().isPresent()); final String logPath = split.getLogPaths().get().get(0); - String[] paths = logPath.split(HoodieLocation.SEPARATOR); + String[] paths = logPath.split(StoragePath.SEPARATOR); return paths[paths.length - 2]; } diff --git a/hudi-hadoop-common/src/main/java/org/apache/hudi/hadoop/fs/inline/InLineFSUtils.java b/hudi-hadoop-common/src/main/java/org/apache/hudi/hadoop/fs/inline/InLineFSUtils.java index b7c043f39cfe3..96dfc53a99d60 100644 --- a/hudi-hadoop-common/src/main/java/org/apache/hudi/hadoop/fs/inline/InLineFSUtils.java +++ b/hudi-hadoop-common/src/main/java/org/apache/hudi/hadoop/fs/inline/InLineFSUtils.java @@ -19,7 +19,7 @@ package org.apache.hudi.hadoop.fs.inline; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.hadoop.fs.Path; @@ -36,7 +36,7 @@ public class InLineFSUtils { private static final String START_OFFSET_STR = "start_offset"; private static final String LENGTH_STR = "length"; - private static final String SCHEME_SEPARATOR = "" + HoodieLocation.COLON_CHAR; + private static final String SCHEME_SEPARATOR = "" + StoragePath.COLON_CHAR; private static final String EQUALS_STR = "="; private static final String LOCAL_FILESYSTEM_SCHEME = "file"; @@ -57,8 +57,8 @@ public static Path getInlineFilePath(Path outerPath, String origScheme, long inL final String subPath = new File(outerPath.toString().substring(outerPath.toString().indexOf(":") + 1)).getPath(); return new Path( InLineFileSystem.SCHEME + SCHEME_SEPARATOR - + HoodieLocation.SEPARATOR + subPath + HoodieLocation.SEPARATOR + origScheme - + HoodieLocation.SEPARATOR + "?" + START_OFFSET_STR + EQUALS_STR + inLineStartOffset + + StoragePath.SEPARATOR + subPath + StoragePath.SEPARATOR + origScheme + + StoragePath.SEPARATOR + "?" + START_OFFSET_STR + EQUALS_STR + inLineStartOffset + "&" + LENGTH_STR + EQUALS_STR + inLineLength ); } @@ -87,7 +87,7 @@ public static Path getOuterFilePathFromInlinePath(Path inlineFSPath) { final String pathExceptScheme = basePath.toString().substring(basePath.toString().indexOf(SCHEME_SEPARATOR) + 1); final String fullPath = outerFileScheme + SCHEME_SEPARATOR - + (outerFileScheme.equals(LOCAL_FILESYSTEM_SCHEME) ? HoodieLocation.SEPARATOR : "") + + (outerFileScheme.equals(LOCAL_FILESYSTEM_SCHEME) ? StoragePath.SEPARATOR : "") + pathExceptScheme; return new Path(fullPath); } diff --git a/hudi-hadoop-common/src/main/java/org/apache/hudi/storage/hadoop/HoodieHadoopStorage.java b/hudi-hadoop-common/src/main/java/org/apache/hudi/storage/hadoop/HoodieHadoopStorage.java index c11531aca4b2a..87d4d9667e630 100644 --- a/hudi-hadoop-common/src/main/java/org/apache/hudi/storage/hadoop/HoodieHadoopStorage.java +++ b/hudi-hadoop-common/src/main/java/org/apache/hudi/storage/hadoop/HoodieHadoopStorage.java @@ -19,10 +19,10 @@ package org.apache.hudi.storage.hadoop; -import org.apache.hudi.storage.HoodieFileStatus; -import org.apache.hudi.storage.HoodieLocation; -import org.apache.hudi.storage.HoodieLocationFilter; import org.apache.hudi.storage.HoodieStorage; +import org.apache.hudi.storage.StoragePath; +import org.apache.hudi.storage.StoragePathFilter; +import org.apache.hudi.storage.StoragePathInfo; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -60,108 +60,108 @@ public URI getUri() { } @Override - public OutputStream create(HoodieLocation location, boolean overwrite) throws IOException { - return fs.create(convertHoodieLocationToPath(location), overwrite); + public OutputStream create(StoragePath path, boolean overwrite) throws IOException { + return fs.create(convertToHadoopPath(path), overwrite); } @Override - public InputStream open(HoodieLocation location) throws IOException { - return fs.open(convertHoodieLocationToPath(location)); + public InputStream open(StoragePath path) throws IOException { + return fs.open(convertToHadoopPath(path)); } @Override - public OutputStream append(HoodieLocation location) throws IOException { - return fs.append(convertHoodieLocationToPath(location)); + public OutputStream append(StoragePath path) throws IOException { + return fs.append(convertToHadoopPath(path)); } @Override - public boolean exists(HoodieLocation location) throws IOException { - return fs.exists(convertHoodieLocationToPath(location)); + public boolean exists(StoragePath path) throws IOException { + return fs.exists(convertToHadoopPath(path)); } @Override - public HoodieFileStatus getFileStatus(HoodieLocation location) throws IOException { - return convertToHoodieFileStatus(fs.getFileStatus(convertHoodieLocationToPath(location))); + public StoragePathInfo getPathInfo(StoragePath path) throws IOException { + return convertToStoragePathInfo(fs.getFileStatus(convertToHadoopPath(path))); } @Override - public boolean createDirectory(HoodieLocation location) throws IOException { - return fs.mkdirs(convertHoodieLocationToPath(location)); + public boolean createDirectory(StoragePath path) throws IOException { + return fs.mkdirs(convertToHadoopPath(path)); } @Override - public List listDirectEntries(HoodieLocation location) throws IOException { - return Arrays.stream(fs.listStatus(convertHoodieLocationToPath(location))) - .map(this::convertToHoodieFileStatus) + public List listDirectEntries(StoragePath path) throws IOException { + return Arrays.stream(fs.listStatus(convertToHadoopPath(path))) + .map(this::convertToStoragePathInfo) .collect(Collectors.toList()); } @Override - public List listFiles(HoodieLocation location) throws IOException { - List result = new ArrayList<>(); - RemoteIterator iterator = fs.listFiles(convertHoodieLocationToPath(location), true); + public List listFiles(StoragePath path) throws IOException { + List result = new ArrayList<>(); + RemoteIterator iterator = fs.listFiles(convertToHadoopPath(path), true); while (iterator.hasNext()) { - result.add(convertToHoodieFileStatus(iterator.next())); + result.add(convertToStoragePathInfo(iterator.next())); } return result; } @Override - public List listDirectEntries(List locationList) throws IOException { - return Arrays.stream(fs.listStatus(locationList.stream() - .map(this::convertHoodieLocationToPath) + public List listDirectEntries(List pathList) throws IOException { + return Arrays.stream(fs.listStatus(pathList.stream() + .map(this::convertToHadoopPath) .toArray(Path[]::new))) - .map(this::convertToHoodieFileStatus) + .map(this::convertToStoragePathInfo) .collect(Collectors.toList()); } @Override - public List listDirectEntries(HoodieLocation location, - HoodieLocationFilter filter) + public List listDirectEntries(StoragePath path, + StoragePathFilter filter) throws IOException { return Arrays.stream(fs.listStatus( - convertHoodieLocationToPath(location), path -> - filter.accept(convertPathToHoodieLocation(path)))) - .map(this::convertToHoodieFileStatus) + convertToHadoopPath(path), e -> + filter.accept(convertToStoragePath(e)))) + .map(this::convertToStoragePathInfo) .collect(Collectors.toList()); } @Override - public List globEntries(HoodieLocation locationPattern) + public List globEntries(StoragePath pathPattern) throws IOException { - return Arrays.stream(fs.globStatus(convertHoodieLocationToPath(locationPattern))) - .map(this::convertToHoodieFileStatus) + return Arrays.stream(fs.globStatus(convertToHadoopPath(pathPattern))) + .map(this::convertToStoragePathInfo) .collect(Collectors.toList()); } @Override - public List globEntries(HoodieLocation locationPattern, HoodieLocationFilter filter) + public List globEntries(StoragePath pathPattern, StoragePathFilter filter) throws IOException { - return Arrays.stream(fs.globStatus(convertHoodieLocationToPath(locationPattern), path -> - filter.accept(convertPathToHoodieLocation(path)))) - .map(this::convertToHoodieFileStatus) + return Arrays.stream(fs.globStatus(convertToHadoopPath(pathPattern), path -> + filter.accept(convertToStoragePath(path)))) + .map(this::convertToStoragePathInfo) .collect(Collectors.toList()); } @Override - public boolean rename(HoodieLocation oldLocation, HoodieLocation newLocation) throws IOException { - return fs.rename(convertHoodieLocationToPath(oldLocation), convertHoodieLocationToPath(newLocation)); + public boolean rename(StoragePath oldPath, StoragePath newPath) throws IOException { + return fs.rename(convertToHadoopPath(oldPath), convertToHadoopPath(newPath)); } @Override - public boolean deleteDirectory(HoodieLocation location) throws IOException { - return fs.delete(convertHoodieLocationToPath(location), true); + public boolean deleteDirectory(StoragePath path) throws IOException { + return fs.delete(convertToHadoopPath(path), true); } @Override - public boolean deleteFile(HoodieLocation location) throws IOException { - return fs.delete(convertHoodieLocationToPath(location), false); + public boolean deleteFile(StoragePath path) throws IOException { + return fs.delete(convertToHadoopPath(path), false); } @Override - public HoodieLocation makeQualified(HoodieLocation location) { - return convertPathToHoodieLocation( - fs.makeQualified(convertHoodieLocationToPath(location))); + public StoragePath makeQualified(StoragePath path) { + return convertToStoragePath( + fs.makeQualified(convertToHadoopPath(path))); } @Override @@ -175,26 +175,26 @@ public Object getConf() { } @Override - public OutputStream create(HoodieLocation location) throws IOException { - return fs.create(convertHoodieLocationToPath(location)); + public OutputStream create(StoragePath path) throws IOException { + return fs.create(convertToHadoopPath(path)); } @Override - public boolean createNewFile(HoodieLocation location) throws IOException { - return fs.createNewFile(convertHoodieLocationToPath(location)); + public boolean createNewFile(StoragePath path) throws IOException { + return fs.createNewFile(convertToHadoopPath(path)); } - private Path convertHoodieLocationToPath(HoodieLocation loc) { + private Path convertToHadoopPath(StoragePath loc) { return new Path(loc.toUri()); } - private HoodieLocation convertPathToHoodieLocation(Path path) { - return new HoodieLocation(path.toUri()); + private StoragePath convertToStoragePath(Path path) { + return new StoragePath(path.toUri()); } - private HoodieFileStatus convertToHoodieFileStatus(FileStatus fileStatus) { - return new HoodieFileStatus( - convertPathToHoodieLocation(fileStatus.getPath()), + private StoragePathInfo convertToStoragePathInfo(FileStatus fileStatus) { + return new StoragePathInfo( + convertToStoragePath(fileStatus.getPath()), fileStatus.getLen(), fileStatus.isDirectory(), fileStatus.getModificationTime()); diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java index 505acccee8734..8922b837871fd 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java @@ -43,7 +43,7 @@ import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat; import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit; import org.apache.hudi.hadoop.realtime.HoodieRealtimePath; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; @@ -245,7 +245,7 @@ public static Option getAffectedPartitions(List commitsTo return Option.empty(); } String incrementalInputPaths = partitionsToList.stream() - .map(s -> StringUtils.isNullOrEmpty(s) ? tableMetaClient.getBasePath() : tableMetaClient.getBasePath() + HoodieLocation.SEPARATOR + s) + .map(s -> StringUtils.isNullOrEmpty(s) ? tableMetaClient.getBasePath() : tableMetaClient.getBasePath() + StoragePath.SEPARATOR + s) .filter(s -> { /* * Ensure to return only results from the original input path that has incremental changes diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestInputPathHandler.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestInputPathHandler.java index b88b58f1ad984..902e61ca12ca3 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestInputPathHandler.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestInputPathHandler.java @@ -25,7 +25,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.testutils.minicluster.HdfsTestService; import org.apache.hudi.hadoop.utils.HoodieHiveUtils; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -168,11 +168,11 @@ static HoodieTableMetaClient initTableType(Configuration hadoopConf, String base static List generatePartitions(DistributedFileSystem dfs, String basePath) throws IOException { List paths = new ArrayList<>(); - paths.add(new Path(basePath + HoodieLocation.SEPARATOR + "2019/05/21")); - paths.add(new Path(basePath + HoodieLocation.SEPARATOR + "2019/05/22")); - paths.add(new Path(basePath + HoodieLocation.SEPARATOR + "2019/05/23")); - paths.add(new Path(basePath + HoodieLocation.SEPARATOR + "2019/05/24")); - paths.add(new Path(basePath + HoodieLocation.SEPARATOR + "2019/05/25")); + paths.add(new Path(basePath + StoragePath.SEPARATOR + "2019/05/21")); + paths.add(new Path(basePath + StoragePath.SEPARATOR + "2019/05/22")); + paths.add(new Path(basePath + StoragePath.SEPARATOR + "2019/05/23")); + paths.add(new Path(basePath + StoragePath.SEPARATOR + "2019/05/24")); + paths.add(new Path(basePath + StoragePath.SEPARATOR + "2019/05/25")); for (Path path : paths) { dfs.mkdirs(path); } diff --git a/hudi-io/src/main/java/org/apache/hudi/storage/HoodieStorage.java b/hudi-io/src/main/java/org/apache/hudi/storage/HoodieStorage.java index 75d7dc28defd1..9ab5e9f9e086b 100644 --- a/hudi-io/src/main/java/org/apache/hudi/storage/HoodieStorage.java +++ b/hudi-io/src/main/java/org/apache/hudi/storage/HoodieStorage.java @@ -61,157 +61,157 @@ public abstract class HoodieStorage implements Closeable { public abstract URI getUri(); /** - * Creates an OutputStream at the indicated location. + * Creates an OutputStream at the indicated path. * - * @param location the file to create. + * @param path the file to create. * @param overwrite if a file with this name already exists, then if {@code true}, * the file will be overwritten, and if {@code false} an exception will be thrown. * @return the OutputStream to write to. * @throws IOException IO error. */ @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) - public abstract OutputStream create(HoodieLocation location, boolean overwrite) throws IOException; + public abstract OutputStream create(StoragePath path, boolean overwrite) throws IOException; /** - * Opens an InputStream at the indicated location. + * Opens an InputStream at the indicated path. * - * @param location the file to open. + * @param path the file to open. * @return the InputStream to read from. * @throws IOException IO error. */ @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) - public abstract InputStream open(HoodieLocation location) throws IOException; + public abstract InputStream open(StoragePath path) throws IOException; /** * Appends to an existing file (optional operation). * - * @param location the file to append. + * @param path the file to append. * @return the OutputStream to write to. * @throws IOException IO error. */ @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) - public abstract OutputStream append(HoodieLocation location) throws IOException; + public abstract OutputStream append(StoragePath path) throws IOException; /** - * Checks if a location exists. + * Checks if a path exists. * - * @param location location to check. - * @return {@code true} if the location exists. + * @param path to check. + * @return {@code true} if the path exists. * @throws IOException IO error. */ @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) - public abstract boolean exists(HoodieLocation location) throws IOException; + public abstract boolean exists(StoragePath path) throws IOException; /** - * Returns a file status object that represents the location. + * Returns a {@link StoragePathInfo} object that represents the path. * - * @param location location to check. - * @return a {@link HoodieFileStatus} object. + * @param path to check. + * @return a {@link StoragePathInfo} object. * @throws FileNotFoundException when the path does not exist. * @throws IOException IO error. */ @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) - public abstract HoodieFileStatus getFileStatus(HoodieLocation location) throws IOException; + public abstract StoragePathInfo getPathInfo(StoragePath path) throws IOException; /** * Creates the directory and non-existent parent directories. * - * @param location location to create. + * @param path to create. * @return {@code true} if the directory was created. * @throws IOException IO error. */ @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) - public abstract boolean createDirectory(HoodieLocation location) throws IOException; + public abstract boolean createDirectory(StoragePath path) throws IOException; /** - * Lists the statuses of the direct files/directories in the given location if the path is a directory. + * Lists the path info of the direct files/directories in the given path if the path is a directory. * - * @param location given location. - * @return the statuses of the files/directories in the given location. - * @throws FileNotFoundException when the location does not exist. + * @param path given path. + * @return the list of path info of the files/directories in the given path. + * @throws FileNotFoundException when the path does not exist. * @throws IOException IO error. */ @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) - public abstract List listDirectEntries(HoodieLocation location) throws IOException; + public abstract List listDirectEntries(StoragePath path) throws IOException; /** - * Lists the statuses of all files under the give location recursively. + * Lists the path info of all files under the give path recursively. * - * @param location given location. - * @return the statuses of the files under the given location. - * @throws FileNotFoundException when the location does not exist. + * @param path given path. + * @return the list of path info of the files under the given path. + * @throws FileNotFoundException when the path does not exist. * @throws IOException IO error. */ @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) - public abstract List listFiles(HoodieLocation location) throws IOException; + public abstract List listFiles(StoragePath path) throws IOException; /** - * Lists the statuses of the direct files/directories in the given location + * Lists the path info of the direct files/directories in the given path * and filters the results, if the path is a directory. * - * @param location given location. - * @param filter filter to apply. - * @return the statuses of the files/directories in the given location. - * @throws FileNotFoundException when the location does not exist. + * @param path given path. + * @param filter filter to apply. + * @return the list of path info of the files/directories in the given path. + * @throws FileNotFoundException when the path does not exist. * @throws IOException IO error. */ @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) - public abstract List listDirectEntries(HoodieLocation location, - HoodieLocationFilter filter) throws IOException; + public abstract List listDirectEntries(StoragePath path, + StoragePathFilter filter) throws IOException; /** - * Returns all the files that match the locationPattern and are not checksum files, + * Returns all the files that match the pathPattern and are not checksum files, * and filters the results. * - * @param locationPattern given pattern. - * @param filter filter to apply. - * @return the statuses of the files. + * @param pathPattern given pattern. + * @param filter filter to apply. + * @return the list of path info of the files. * @throws IOException IO error. */ @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) - public abstract List globEntries(HoodieLocation locationPattern, - HoodieLocationFilter filter) throws IOException; + public abstract List globEntries(StoragePath pathPattern, + StoragePathFilter filter) throws IOException; /** - * Renames the location from old to new. + * Renames the path from old to new. * - * @param oldLocation source location. - * @param newLocation destination location. + * @param oldPath source path. + * @param newPath destination path. * @return {@true} if rename is successful. * @throws IOException IO error. */ @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) - public abstract boolean rename(HoodieLocation oldLocation, - HoodieLocation newLocation) throws IOException; + public abstract boolean rename(StoragePath oldPath, + StoragePath newPath) throws IOException; /** - * Deletes a directory at location. + * Deletes a directory at path. * - * @param location directory to delete. + * @param path directory to delete. * @return {@code true} if successful. * @throws IOException IO error. */ @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) - public abstract boolean deleteDirectory(HoodieLocation location) throws IOException; + public abstract boolean deleteDirectory(StoragePath path) throws IOException; /** - * Deletes a file at location. + * Deletes a file at path. * - * @param location file to delete. + * @param path file to delete. * @return {@code true} if successful. * @throws IOException IO error. */ @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) - public abstract boolean deleteFile(HoodieLocation location) throws IOException; + public abstract boolean deleteFile(StoragePath path) throws IOException; /** * Qualifies a path to one which uses this storage and, if relative, made absolute. * - * @param location to qualify. - * @return Qualified location. + * @param path to qualify. + * @return Qualified path. */ @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) - public abstract HoodieLocation makeQualified(HoodieLocation location); + public abstract StoragePath makeQualified(StoragePath path); /** * @return the underlying file system instance if exists. @@ -231,35 +231,35 @@ public abstract boolean rename(HoodieLocation oldLocation, * empty, will first write the content to a temp file if {needCreateTempFile} is * true, and then rename it back after the content is written. * - * @param location file Path. - * @param content content to be stored. + * @param path file path. + * @param content content to be stored. */ @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) - public final void createImmutableFileInPath(HoodieLocation location, + public final void createImmutableFileInPath(StoragePath path, Option content) throws IOException { OutputStream fsout = null; - HoodieLocation tmpLocation = null; + StoragePath tmpPath = null; boolean needTempFile = needCreateTempFile(); try { if (!content.isPresent()) { - fsout = create(location, false); + fsout = create(path, false); } if (content.isPresent() && needTempFile) { - HoodieLocation parent = location.getParent(); - tmpLocation = new HoodieLocation(parent, location.getName() + TMP_PATH_POSTFIX); - fsout = create(tmpLocation, false); + StoragePath parent = path.getParent(); + tmpPath = new StoragePath(parent, path.getName() + TMP_PATH_POSTFIX); + fsout = create(tmpPath, false); fsout.write(content.get()); } if (content.isPresent() && !needTempFile) { - fsout = create(location, false); + fsout = create(path, false); fsout.write(content.get()); } } catch (IOException e) { - String errorMsg = "Failed to create file " + (tmpLocation != null ? tmpLocation : location); + String errorMsg = "Failed to create file " + (tmpPath != null ? tmpPath : path); throw new HoodieIOException(errorMsg, e); } finally { try { @@ -267,27 +267,27 @@ public final void createImmutableFileInPath(HoodieLocation location, fsout.close(); } } catch (IOException e) { - String errorMsg = "Failed to close file " + (needTempFile ? tmpLocation : location); + String errorMsg = "Failed to close file " + (needTempFile ? tmpPath : path); throw new HoodieIOException(errorMsg, e); } boolean renameSuccess = false; try { - if (null != tmpLocation) { - renameSuccess = rename(tmpLocation, location); + if (null != tmpPath) { + renameSuccess = rename(tmpPath, path); } } catch (IOException e) { throw new HoodieIOException( - "Failed to rename " + tmpLocation + " to the target " + location, + "Failed to rename " + tmpPath + " to the target " + path, e); } finally { - if (!renameSuccess && null != tmpLocation) { + if (!renameSuccess && null != tmpPath) { try { - deleteFile(tmpLocation); - LOG.warn("Fail to rename " + tmpLocation + " to " + location - + ", target file exists: " + exists(location)); + deleteFile(tmpPath); + LOG.warn("Fail to rename " + tmpPath + " to " + path + + ", target file exists: " + exists(path)); } catch (IOException e) { - throw new HoodieIOException("Failed to delete tmp file " + tmpLocation, e); + throw new HoodieIOException("Failed to delete tmp file " + tmpPath, e); } } } @@ -303,62 +303,62 @@ public final boolean needCreateTempFile() { } /** - * Create an OutputStream at the indicated location. + * Create an OutputStream at the indicated path. * The file is overwritten by default. * - * @param location the file to create. + * @param path the file to create. * @return the OutputStream to write to. * @throws IOException IO error. */ @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) - public OutputStream create(HoodieLocation location) throws IOException { - return create(location, true); + public OutputStream create(StoragePath path) throws IOException { + return create(path, true); } /** - * Creates an empty new file at the indicated location. + * Creates an empty new file at the indicated path. * - * @param location the file to create. + * @param path the file to create. * @return {@code true} if successfully created; {@code false} if already exists. * @throws IOException IO error. */ @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) - public boolean createNewFile(HoodieLocation location) throws IOException { - if (exists(location)) { + public boolean createNewFile(StoragePath path) throws IOException { + if (exists(path)) { return false; } else { - create(location, false).close(); + create(path, false).close(); return true; } } /** - * Lists the statuses of the direct files/directories in the given list of locations, - * if the locations are directory. + * Lists the file info of the direct files/directories in the given list of paths, + * if the paths are directory. * - * @param locationList given location list. - * @return the statuses of the files/directories in the given locations. - * @throws FileNotFoundException when the location does not exist. + * @param pathList given path list. + * @return the list of path info of the files/directories in the given paths. + * @throws FileNotFoundException when the path does not exist. * @throws IOException IO error. */ @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) - public List listDirectEntries(List locationList) throws IOException { - List result = new ArrayList<>(); - for (HoodieLocation location : locationList) { - result.addAll(listDirectEntries(location)); + public List listDirectEntries(List pathList) throws IOException { + List result = new ArrayList<>(); + for (StoragePath path : pathList) { + result.addAll(listDirectEntries(path)); } return result; } /** - * Returns all the files that match the locationPattern and are not checksum files. + * Returns all the files that match the pathPattern and are not checksum files. * - * @param locationPattern given pattern. - * @return the statuses of the files. + * @param pathPattern given pattern. + * @return the list of file info of the files. * @throws IOException IO error. */ @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) - public List globEntries(HoodieLocation locationPattern) throws IOException { - return globEntries(locationPattern, e -> true); + public List globEntries(StoragePath pathPattern) throws IOException { + return globEntries(pathPattern, e -> true); } } diff --git a/hudi-io/src/main/java/org/apache/hudi/storage/HoodieLocation.java b/hudi-io/src/main/java/org/apache/hudi/storage/StoragePath.java similarity index 84% rename from hudi-io/src/main/java/org/apache/hudi/storage/HoodieLocation.java rename to hudi-io/src/main/java/org/apache/hudi/storage/StoragePath.java index 8b51bd07ff944..f3a88f7c89b98 100644 --- a/hudi-io/src/main/java/org/apache/hudi/storage/HoodieLocation.java +++ b/hudi-io/src/main/java/org/apache/hudi/storage/StoragePath.java @@ -29,24 +29,25 @@ /** * Names a file or directory on storage. - * Location strings use slash (`/`) as the directory separator. + * Path strings use slash (`/`) as the directory separator. * The APIs are mainly based on {@code org.apache.hadoop.fs.Path} class. */ @PublicAPIClass(maturity = ApiMaturityLevel.EVOLVING) -public class HoodieLocation implements Comparable, Serializable { +// StoragePath +public class StoragePath implements Comparable, Serializable { public static final char SEPARATOR_CHAR = '/'; public static final char COLON_CHAR = ':'; public static final String SEPARATOR = "" + SEPARATOR_CHAR; private final URI uri; - private transient volatile HoodieLocation cachedParent; + private transient volatile StoragePath cachedParent; private transient volatile String cachedName; private transient volatile String uriString; - public HoodieLocation(URI uri) { + public StoragePath(URI uri) { this.uri = uri.normalize(); } - public HoodieLocation(String path) { + public StoragePath(String path) { try { // This part of parsing is compatible with hadoop's Path // and required for properly handling encoded path with URI @@ -82,11 +83,11 @@ public HoodieLocation(String path) { } } - public HoodieLocation(String parent, String child) { - this(new HoodieLocation(parent), child); + public StoragePath(String parent, String child) { + this(new StoragePath(parent), child); } - public HoodieLocation(HoodieLocation parent, String child) { + public StoragePath(StoragePath parent, String child) { URI parentUri = parent.toUri(); String normalizedChild = normalize(child, false); @@ -127,19 +128,19 @@ public boolean isAbsolute() { } @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) - public HoodieLocation getParent() { + public StoragePath getParent() { // This value could be overwritten concurrently and that's okay, since - // {@code HoodieLocation} is immutable + // {@code StoragePath} is immutable if (cachedParent == null) { String path = uri.getPath(); int lastSlash = path.lastIndexOf(SEPARATOR_CHAR); if (path.isEmpty() || path.equals(SEPARATOR)) { - throw new IllegalStateException("Cannot get parent location of a root location"); + throw new IllegalStateException("Cannot get parent path of a root path"); } String parentPath = lastSlash == -1 ? "" : path.substring(0, lastSlash == 0 ? 1 : lastSlash); try { - cachedParent = new HoodieLocation(new URI( + cachedParent = new StoragePath(new URI( uri.getScheme(), uri.getAuthority(), parentPath, null, uri.getFragment())); } catch (URISyntaxException e) { throw new IllegalArgumentException(e); @@ -151,7 +152,7 @@ public HoodieLocation getParent() { @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) public String getName() { // This value could be overwritten concurrently and that's okay, since - // {@code HoodieLocation} is immutable + // {@code StoragePath} is immutable if (cachedName == null) { String path = uri.getPath(); int slash = path.lastIndexOf(SEPARATOR); @@ -161,9 +162,9 @@ public String getName() { } @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) - public HoodieLocation getLocationWithoutSchemeAndAuthority() { + public StoragePath getPathWithoutSchemeAndAuthority() { try { - return new HoodieLocation( + return new StoragePath( new URI(null, null, uri.getPath(), uri.getQuery(), uri.getFragment())); } catch (URISyntaxException e) { throw new IllegalArgumentException(e); @@ -188,27 +189,27 @@ public URI toUri() { } /** - * Returns a qualified location object. + * Returns a qualified path object. * - * @param defaultUri if this location is missing the scheme or authority + * @param defaultUri if this path is missing the scheme or authority * components, borrow them from this URI. - * @return this location if it contains a scheme and authority, or + * @return this path if it contains a scheme and authority, or * a new path that includes a path and authority and is fully qualified. */ @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) - public HoodieLocation makeQualified(URI defaultUri) { + public StoragePath makeQualified(URI defaultUri) { if (!isAbsolute()) { throw new IllegalStateException("Only an absolute path can be made qualified"); } - HoodieLocation location = this; - URI locationUri = location.toUri(); + StoragePath path = this; + URI pathUri = path.toUri(); - String scheme = locationUri.getScheme(); - String authority = locationUri.getAuthority(); - String fragment = locationUri.getFragment(); + String scheme = pathUri.getScheme(); + String authority = pathUri.getAuthority(); + String fragment = pathUri.getFragment(); if (scheme != null && (authority != null || defaultUri.getAuthority() == null)) { - return location; + return path; } if (scheme == null) { @@ -225,17 +226,17 @@ public HoodieLocation makeQualified(URI defaultUri) { URI newUri; try { newUri = new URI(scheme, authority, - normalize(locationUri.getPath(), true), null, fragment); + normalize(pathUri.getPath(), true), null, fragment); } catch (URISyntaxException e) { throw new IllegalArgumentException(e); } - return new HoodieLocation(newUri); + return new StoragePath(newUri); } @Override public String toString() { // This value could be overwritten concurrently and that's okay, since - // {@code HoodieLocation} is immutable + // {@code StoragePath} is immutable if (uriString == null) { // We can't use uri.toString(), which escapes everything, because we want // illegal characters unescaped in the string, for glob processing, etc. @@ -262,10 +263,10 @@ public String toString() { @Override public boolean equals(Object o) { - if (!(o instanceof HoodieLocation)) { + if (!(o instanceof StoragePath)) { return false; } - return this.uri.equals(((HoodieLocation) o).toUri()); + return this.uri.equals(((StoragePath) o).toUri()); } @Override @@ -274,7 +275,7 @@ public int hashCode() { } @Override - public int compareTo(HoodieLocation o) { + public int compareTo(StoragePath o) { return this.uri.compareTo(o.uri); } diff --git a/hudi-io/src/main/java/org/apache/hudi/storage/HoodieLocationFilter.java b/hudi-io/src/main/java/org/apache/hudi/storage/StoragePathFilter.java similarity index 77% rename from hudi-io/src/main/java/org/apache/hudi/storage/HoodieLocationFilter.java rename to hudi-io/src/main/java/org/apache/hudi/storage/StoragePathFilter.java index d33686c030c09..357a8e6ad3eee 100644 --- a/hudi-io/src/main/java/org/apache/hudi/storage/HoodieLocationFilter.java +++ b/hudi-io/src/main/java/org/apache/hudi/storage/StoragePathFilter.java @@ -26,17 +26,17 @@ import java.io.Serializable; /** - * Filter for {@link HoodieLocation} + * Filter for {@link StoragePath} * The APIs are mainly based on {@code org.apache.hadoop.fs.PathFilter} class. */ @PublicAPIClass(maturity = ApiMaturityLevel.EVOLVING) -public interface HoodieLocationFilter extends Serializable { +public interface StoragePathFilter extends Serializable { /** - * Tests whether the specified location should be included in a location list. + * Tests whether the specified path should be included in a path list. * - * @param location the location to be tested. - * @return {@code true} if and only if location should be included. + * @param path the path to be tested. + * @return {@code true} if and only if path should be included. */ @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) - boolean accept(HoodieLocation location); + boolean accept(StoragePath path); } diff --git a/hudi-io/src/main/java/org/apache/hudi/storage/HoodieFileStatus.java b/hudi-io/src/main/java/org/apache/hudi/storage/StoragePathInfo.java similarity index 77% rename from hudi-io/src/main/java/org/apache/hudi/storage/HoodieFileStatus.java rename to hudi-io/src/main/java/org/apache/hudi/storage/StoragePathInfo.java index 6f033c5bc9541..b4ec8194b4de8 100644 --- a/hudi-io/src/main/java/org/apache/hudi/storage/HoodieFileStatus.java +++ b/hudi-io/src/main/java/org/apache/hudi/storage/StoragePathInfo.java @@ -26,33 +26,33 @@ import java.io.Serializable; /** - * Represents the information of a directory or a file. + * Represents the information of a storage path representing a directory or a file. * The APIs are mainly based on {@code org.apache.hadoop.fs.FileStatus} class * with simplification based on what Hudi needs. */ @PublicAPIClass(maturity = ApiMaturityLevel.EVOLVING) -public class HoodieFileStatus implements Serializable { - private final HoodieLocation location; +public class StoragePathInfo implements Serializable { + private final StoragePath path; private final long length; private final boolean isDirectory; private final long modificationTime; - public HoodieFileStatus(HoodieLocation location, - long length, - boolean isDirectory, - long modificationTime) { - this.location = location; + public StoragePathInfo(StoragePath path, + long length, + boolean isDirectory, + long modificationTime) { + this.path = path; this.length = length; this.isDirectory = isDirectory; this.modificationTime = modificationTime; } /** - * @return the location. + * @return the path. */ @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING) - public HoodieLocation getLocation() { - return location; + public StoragePath getPath() { + return path; } /** @@ -95,23 +95,23 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) { return false; } - HoodieFileStatus that = (HoodieFileStatus) o; + StoragePathInfo that = (StoragePathInfo) o; // PLEASE NOTE that here we follow the same contract hadoop's FileStatus provides, - // i.e., the equality is purely based on the location. - return getLocation().equals(that.getLocation()); + // i.e., the equality is purely based on the path. + return getPath().equals(that.getPath()); } @Override public int hashCode() { // PLEASE NOTE that here we follow the same contract hadoop's FileStatus provides, - // i.e., the hash code is purely based on the location. - return getLocation().hashCode(); + // i.e., the hash code is purely based on the path. + return getPath().hashCode(); } @Override public String toString() { - return "HoodieFileStatus{" - + "location=" + location + return "StoragePathInfo{" + + "path=" + path + ", length=" + length + ", isDirectory=" + isDirectory + ", modificationTime=" + modificationTime diff --git a/hudi-io/src/test/java/org/apache/hudi/io/storage/TestHoodieLocation.java b/hudi-io/src/test/java/org/apache/hudi/io/storage/TestHoodieLocation.java deleted file mode 100644 index caee807a1f609..0000000000000 --- a/hudi-io/src/test/java/org/apache/hudi/io/storage/TestHoodieLocation.java +++ /dev/null @@ -1,219 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.hudi.io.storage; - -import org.apache.hudi.storage.HoodieLocation; - -import org.junit.jupiter.api.Test; - -import java.net.URI; -import java.net.URISyntaxException; -import java.util.Arrays; - -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertNotEquals; -import static org.junit.jupiter.api.Assertions.assertSame; -import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.junit.jupiter.api.Assertions.assertTrue; - -/** - * Tests {@link HoodieLocation} - */ -public class TestHoodieLocation { - @Test - public void testToString() { - Arrays.stream( - new String[] { - "/", - "/foo", - "/foo/bar", - "foo", - "foo/bar", - "/foo/bar#boo", - "foo/bar#boo", - "file:/a/b/c", - "s3://a/b/c"}) - .forEach(this::toStringTest); - } - - @Test - public void testNormalize() throws URISyntaxException { - assertEquals("", new HoodieLocation(".").toString()); - assertEquals("..", new HoodieLocation("..").toString()); - assertEquals("/", new HoodieLocation("/").toString()); - assertEquals("/", new HoodieLocation("//").toString()); - assertEquals("/", new HoodieLocation("///").toString()); - assertEquals("//foo/", new HoodieLocation("//foo/").toString()); - assertEquals("//foo/", new HoodieLocation("//foo//").toString()); - assertEquals("//foo/bar", new HoodieLocation("//foo//bar").toString()); - assertEquals("/foo", new HoodieLocation("/foo/").toString()); - assertEquals("/foo", new HoodieLocation("/foo/").toString()); - assertEquals("foo", new HoodieLocation("foo/").toString()); - assertEquals("foo", new HoodieLocation("foo//").toString()); - assertEquals("foo/bar", new HoodieLocation("foo//bar").toString()); - assertEquals("file:/a/b/c", new HoodieLocation("file:///a/b/c").toString()); - assertEquals("s3://a/b/c/d/e", new HoodieLocation("s3://a/b/c", "d/e").toString()); - assertEquals("s3://a/b/c/d/e", new HoodieLocation("s3://a/b/c/", "d/e").toString()); - assertEquals("s3://a/b/c/d/e", new HoodieLocation("s3://a/b/c/", "d/e/").toString()); - assertEquals("s3://a/b/c", new HoodieLocation("s3://a/b/c/", "/").toString()); - assertEquals("s3://a/b/c", new HoodieLocation("s3://a/b/c/", "").toString()); - assertEquals("s3://a/b/c/d/e", new HoodieLocation(new HoodieLocation("s3://a/b/c"), "d/e").toString()); - assertEquals("s3://a/b/c/d/e", new HoodieLocation(new HoodieLocation("s3://a/b/c/"), "d/e").toString()); - assertEquals("s3://a/b/c/d/e", new HoodieLocation(new HoodieLocation("s3://a/b/c/"), "d/e/").toString()); - assertEquals("s3://a/b/c", new HoodieLocation(new HoodieLocation("s3://a/b/c/"), "/").toString()); - assertEquals("s3://a/b/c", new HoodieLocation(new HoodieLocation("s3://a/b/c/"), "").toString()); - assertEquals("hdfs://foo/foo2/bar/baz/", new HoodieLocation(new URI("hdfs://foo//foo2///bar/baz///")).toString()); - } - - @Test - public void testIsAbsolute() { - assertTrue(new HoodieLocation("/").isAbsolute()); - assertTrue(new HoodieLocation("/foo").isAbsolute()); - assertFalse(new HoodieLocation("foo").isAbsolute()); - assertFalse(new HoodieLocation("foo/bar").isAbsolute()); - assertFalse(new HoodieLocation(".").isAbsolute()); - } - - @Test - public void testGetParent() { - assertEquals(new HoodieLocation("/foo"), new HoodieLocation("/foo/bar").getParent()); - assertEquals(new HoodieLocation("foo"), new HoodieLocation("foo/bar").getParent()); - assertEquals(new HoodieLocation("/"), new HoodieLocation("/foo").getParent()); - assertEquals(new HoodieLocation("/foo/bar/x"), new HoodieLocation("/foo/bar", "x/y").getParent()); - assertEquals(new HoodieLocation("/foo/bar"), new HoodieLocation("/foo/bar/", "y").getParent()); - assertEquals(new HoodieLocation("/foo"), new HoodieLocation("/foo/bar/", "/").getParent()); - assertThrows(IllegalStateException.class, () -> new HoodieLocation("/").getParent()); - } - - @Test - public void testURI() throws URISyntaxException { - URI uri = new URI("file:///bar#baz"); - HoodieLocation location = new HoodieLocation(uri); - assertEquals(uri, new URI(location.toString())); - assertEquals("foo://bar/baz#boo", new HoodieLocation("foo://bar/", "/baz#boo").toString()); - assertEquals("foo://bar/baz/fud#boo", - new HoodieLocation(new HoodieLocation(new URI("foo://bar/baz#bud")), "fud#boo").toString()); - assertEquals("foo://bar/fud#boo", - new HoodieLocation(new HoodieLocation(new URI("foo://bar/baz#bud")), "/fud#boo").toString()); - } - - @Test - public void testEncoded() { - // encoded character like `%2F` should be kept as is - assertEquals(new HoodieLocation("s3://foo/bar/1%2F2%2F3"), new HoodieLocation("s3://foo/bar", "1%2F2%2F3")); - assertEquals("s3://foo/bar/1%2F2%2F3", new HoodieLocation("s3://foo/bar", "1%2F2%2F3").toString()); - assertEquals(new HoodieLocation("s3://foo/bar/1%2F2%2F3"), - new HoodieLocation(new HoodieLocation("s3://foo/bar"), "1%2F2%2F3")); - assertEquals("s3://foo/bar/1%2F2%2F3", - new HoodieLocation(new HoodieLocation("s3://foo/bar"), "1%2F2%2F3").toString()); - assertEquals("s3://foo/bar/1%2F2%2F3", new HoodieLocation("s3://foo/bar/1%2F2%2F3").toString()); - } - - @Test - public void testPathToUriConversion() throws URISyntaxException { - assertEquals(new URI(null, null, "/foo?bar", null, null), - new HoodieLocation("/foo?bar").toUri()); - assertEquals(new URI(null, null, "/foo\"bar", null, null), - new HoodieLocation("/foo\"bar").toUri()); - assertEquals(new URI(null, null, "/foo bar", null, null), - new HoodieLocation("/foo bar").toUri()); - assertEquals("/foo?bar", new HoodieLocation("http://localhost/foo?bar").toUri().getPath()); - assertEquals("/foo", new URI("http://localhost/foo?bar").getPath()); - assertEquals((new URI("/foo;bar")).getPath(), new HoodieLocation("/foo;bar").toUri().getPath()); - assertEquals(new URI("/foo;bar"), new HoodieLocation("/foo;bar").toUri()); - assertEquals(new URI("/foo+bar"), new HoodieLocation("/foo+bar").toUri()); - assertEquals(new URI("/foo-bar"), new HoodieLocation("/foo-bar").toUri()); - assertEquals(new URI("/foo=bar"), new HoodieLocation("/foo=bar").toUri()); - assertEquals(new URI("/foo,bar"), new HoodieLocation("/foo,bar").toUri()); - } - - @Test - public void testGetName() { - assertEquals("", new HoodieLocation("/").getName()); - assertEquals("foo", new HoodieLocation("foo").getName()); - assertEquals("foo", new HoodieLocation("/foo").getName()); - assertEquals("foo", new HoodieLocation("/foo/").getName()); - assertEquals("bar", new HoodieLocation("/foo/bar").getName()); - assertEquals("bar", new HoodieLocation("hdfs://host/foo/bar").getName()); - assertEquals("bar", new HoodieLocation("hdfs://host", "foo/bar").getName()); - assertEquals("bar", new HoodieLocation("hdfs://host/foo/", "bar").getName()); - } - - @Test - public void testGetLocationWithoutSchemeAndAuthority() { - assertEquals( - new HoodieLocation("/foo/bar/boo"), - new HoodieLocation("/foo/bar/boo").getLocationWithoutSchemeAndAuthority()); - assertEquals( - new HoodieLocation("/foo/bar/boo"), - new HoodieLocation("file:///foo/bar/boo").getLocationWithoutSchemeAndAuthority()); - assertEquals( - new HoodieLocation("/bar/boo"), - new HoodieLocation("s3://foo/bar/boo").getLocationWithoutSchemeAndAuthority()); - } - - @Test - public void testDepth() throws URISyntaxException { - assertEquals(0, new HoodieLocation("/").depth()); - assertEquals(0, new HoodieLocation("///").depth()); - assertEquals(0, new HoodieLocation("//foo/").depth()); - assertEquals(1, new HoodieLocation("//foo//bar").depth()); - assertEquals(5, new HoodieLocation("/a/b/c/d/e").depth()); - assertEquals(4, new HoodieLocation("s3://a/b/c", "d/e").depth()); - assertEquals(2, new HoodieLocation("s3://a/b/c/", "").depth()); - assertEquals(4, new HoodieLocation(new HoodieLocation("s3://a/b/c"), "d/e").depth()); - } - - @Test - public void testMakeQualified() throws URISyntaxException { - URI defaultUri = new URI("hdfs://host1/dir1"); - assertEquals(new HoodieLocation("hdfs://host1/a/b/c"), - new HoodieLocation("/a/b/c").makeQualified(defaultUri)); - assertEquals(new HoodieLocation("hdfs://host2/a/b/c"), - new HoodieLocation("hdfs://host2/a/b/c").makeQualified(defaultUri)); - assertEquals(new HoodieLocation("hdfs://host1/a/b/c"), - new HoodieLocation("hdfs:/a/b/c").makeQualified(defaultUri)); - assertEquals(new HoodieLocation("s3://a/b/c"), - new HoodieLocation("s3://a/b/c/").makeQualified(defaultUri)); - assertThrows(IllegalStateException.class, - () -> new HoodieLocation("a").makeQualified(defaultUri)); - } - - @Test - public void testEquals() { - assertEquals(new HoodieLocation("/foo"), new HoodieLocation("/foo")); - assertEquals(new HoodieLocation("/foo"), new HoodieLocation("/foo/")); - assertEquals(new HoodieLocation("/foo/bar"), new HoodieLocation("/foo//bar/")); - assertNotEquals(new HoodieLocation("/"), new HoodieLocation("/foo")); - } - - @Test - public void testCachedResults() { - HoodieLocation location = new HoodieLocation("s3://x/y/z/"); - assertSame(location.getParent(), location.getParent()); - assertSame(location.getName(), location.getName()); - assertSame(location.toString(), location.toString()); - } - - private void toStringTest(String pathString) { - assertEquals(pathString, new HoodieLocation(pathString).toString()); - } -} diff --git a/hudi-io/src/test/java/org/apache/hudi/io/storage/TestHoodieStorageBase.java b/hudi-io/src/test/java/org/apache/hudi/io/storage/TestHoodieStorageBase.java index 6c7fc2f4dd5bd..a6a0efee6dc09 100644 --- a/hudi-io/src/test/java/org/apache/hudi/io/storage/TestHoodieStorageBase.java +++ b/hudi-io/src/test/java/org/apache/hudi/io/storage/TestHoodieStorageBase.java @@ -21,9 +21,9 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.io.util.IOUtils; -import org.apache.hudi.storage.HoodieFileStatus; -import org.apache.hudi.storage.HoodieLocation; import org.apache.hudi.storage.HoodieStorage; +import org.apache.hudi.storage.StoragePath; +import org.apache.hudi.storage.StoragePathInfo; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; @@ -83,12 +83,12 @@ public abstract class TestHoodieStorageBase { public void cleanUpTempDir() { HoodieStorage storage = getHoodieStorage(); try { - for (HoodieFileStatus status : storage.listDirectEntries(new HoodieLocation(getTempDir()))) { - HoodieLocation location = status.getLocation(); - if (status.isDirectory()) { - storage.deleteDirectory(location); + for (StoragePathInfo pathInfo : storage.listDirectEntries(new StoragePath(getTempDir()))) { + StoragePath path = pathInfo.getPath(); + if (pathInfo.isDirectory()) { + storage.deleteDirectory(path); } else { - storage.deleteFile(location); + storage.deleteFile(path); } } } catch (IOException e) { @@ -110,42 +110,42 @@ public void testGetUri() throws URISyntaxException { public void testCreateWriteAndRead() throws IOException { HoodieStorage storage = getHoodieStorage(); - HoodieLocation location = new HoodieLocation(getTempDir(), "testCreateAppendAndRead/1.file"); - assertFalse(storage.exists(location)); - storage.create(location).close(); - validateFileStatus(storage, location, EMPTY_BYTES, false); + StoragePath path = new StoragePath(getTempDir(), "testCreateAppendAndRead/1.file"); + assertFalse(storage.exists(path)); + storage.create(path).close(); + validatePathInfo(storage, path, EMPTY_BYTES, false); byte[] data = new byte[] {2, 42, 49, (byte) 158, (byte) 233, 66, 9}; // By default, create overwrites the file - try (OutputStream stream = storage.create(location)) { + try (OutputStream stream = storage.create(path)) { stream.write(data); stream.flush(); } - validateFileStatus(storage, location, data, false); - - assertThrows(IOException.class, () -> storage.create(location, false)); - validateFileStatus(storage, location, data, false); - - assertThrows(IOException.class, () -> storage.create(location, false)); - validateFileStatus(storage, location, data, false); - - HoodieLocation location2 = new HoodieLocation(getTempDir(), "testCreateAppendAndRead/2.file"); - assertFalse(storage.exists(location2)); - assertTrue(storage.createNewFile(location2)); - validateFileStatus(storage, location2, EMPTY_BYTES, false); - assertFalse(storage.createNewFile(location2)); - - HoodieLocation location3 = new HoodieLocation(getTempDir(), "testCreateAppendAndRead/3.file"); - assertFalse(storage.exists(location3)); - storage.createImmutableFileInPath(location3, Option.of(data)); - validateFileStatus(storage, location3, data, false); - - HoodieLocation location4 = new HoodieLocation(getTempDir(), "testCreateAppendAndRead/4"); - assertFalse(storage.exists(location4)); - assertTrue(storage.createDirectory(location4)); - validateFileStatus(storage, location4, EMPTY_BYTES, true); - assertTrue(storage.createDirectory(location4)); + validatePathInfo(storage, path, data, false); + + assertThrows(IOException.class, () -> storage.create(path, false)); + validatePathInfo(storage, path, data, false); + + assertThrows(IOException.class, () -> storage.create(path, false)); + validatePathInfo(storage, path, data, false); + + StoragePath path2 = new StoragePath(getTempDir(), "testCreateAppendAndRead/2.file"); + assertFalse(storage.exists(path2)); + assertTrue(storage.createNewFile(path2)); + validatePathInfo(storage, path2, EMPTY_BYTES, false); + assertFalse(storage.createNewFile(path2)); + + StoragePath path3 = new StoragePath(getTempDir(), "testCreateAppendAndRead/3.file"); + assertFalse(storage.exists(path3)); + storage.createImmutableFileInPath(path3, Option.of(data)); + validatePathInfo(storage, path3, data, false); + + StoragePath path4 = new StoragePath(getTempDir(), "testCreateAppendAndRead/4"); + assertFalse(storage.exists(path4)); + assertTrue(storage.createDirectory(path4)); + validatePathInfo(storage, path4, EMPTY_BYTES, true); + assertTrue(storage.createDirectory(path4)); } @Test @@ -162,68 +162,68 @@ public void testListing() throws IOException { // x/z/2.file prepareFilesOnStorage(storage); - validateHoodieFileStatusList( - Arrays.stream(new HoodieFileStatus[] { - new HoodieFileStatus(new HoodieLocation(getTempDir(), "x/1.file"), 0, false, 0), - new HoodieFileStatus(new HoodieLocation(getTempDir(), "x/2.file"), 0, false, 0), - new HoodieFileStatus(new HoodieLocation(getTempDir(), "x/y"), 0, true, 0), - new HoodieFileStatus(new HoodieLocation(getTempDir(), "x/z"), 0, true, 0), + validatePathInfoList( + Arrays.stream(new StoragePathInfo[] { + new StoragePathInfo(new StoragePath(getTempDir(), "x/1.file"), 0, false, 0), + new StoragePathInfo(new StoragePath(getTempDir(), "x/2.file"), 0, false, 0), + new StoragePathInfo(new StoragePath(getTempDir(), "x/y"), 0, true, 0), + new StoragePathInfo(new StoragePath(getTempDir(), "x/z"), 0, true, 0), }).collect(Collectors.toList()), - storage.listDirectEntries(new HoodieLocation(getTempDir(), "x"))); - - validateHoodieFileStatusList( - Arrays.stream(new HoodieFileStatus[] { - new HoodieFileStatus(new HoodieLocation(getTempDir(), "x/1.file"), 0, false, 0), - new HoodieFileStatus(new HoodieLocation(getTempDir(), "x/2.file"), 0, false, 0), - new HoodieFileStatus(new HoodieLocation(getTempDir(), "x/y/1.file"), 0, false, 0), - new HoodieFileStatus(new HoodieLocation(getTempDir(), "x/y/2.file"), 0, false, 0), - new HoodieFileStatus(new HoodieLocation(getTempDir(), "x/z/1.file"), 0, false, 0), - new HoodieFileStatus(new HoodieLocation(getTempDir(), "x/z/2.file"), 0, false, 0) + storage.listDirectEntries(new StoragePath(getTempDir(), "x"))); + + validatePathInfoList( + Arrays.stream(new StoragePathInfo[] { + new StoragePathInfo(new StoragePath(getTempDir(), "x/1.file"), 0, false, 0), + new StoragePathInfo(new StoragePath(getTempDir(), "x/2.file"), 0, false, 0), + new StoragePathInfo(new StoragePath(getTempDir(), "x/y/1.file"), 0, false, 0), + new StoragePathInfo(new StoragePath(getTempDir(), "x/y/2.file"), 0, false, 0), + new StoragePathInfo(new StoragePath(getTempDir(), "x/z/1.file"), 0, false, 0), + new StoragePathInfo(new StoragePath(getTempDir(), "x/z/2.file"), 0, false, 0) }).collect(Collectors.toList()), - storage.listFiles(new HoodieLocation(getTempDir(), "x"))); + storage.listFiles(new StoragePath(getTempDir(), "x"))); - validateHoodieFileStatusList( - Arrays.stream(new HoodieFileStatus[] { - new HoodieFileStatus(new HoodieLocation(getTempDir(), "x/2.file"), 0, false, 0) + validatePathInfoList( + Arrays.stream(new StoragePathInfo[] { + new StoragePathInfo(new StoragePath(getTempDir(), "x/2.file"), 0, false, 0) }).collect(Collectors.toList()), storage.listDirectEntries( - new HoodieLocation(getTempDir(), "x"), e -> e.getName().contains("2"))); - - validateHoodieFileStatusList( - Arrays.stream(new HoodieFileStatus[] { - new HoodieFileStatus(new HoodieLocation(getTempDir(), "w/1.file"), 0, false, 0), - new HoodieFileStatus(new HoodieLocation(getTempDir(), "w/2.file"), 0, false, 0), - new HoodieFileStatus(new HoodieLocation(getTempDir(), "x/z/1.file"), 0, false, 0), - new HoodieFileStatus(new HoodieLocation(getTempDir(), "x/z/2.file"), 0, false, 0) + new StoragePath(getTempDir(), "x"), e -> e.getName().contains("2"))); + + validatePathInfoList( + Arrays.stream(new StoragePathInfo[] { + new StoragePathInfo(new StoragePath(getTempDir(), "w/1.file"), 0, false, 0), + new StoragePathInfo(new StoragePath(getTempDir(), "w/2.file"), 0, false, 0), + new StoragePathInfo(new StoragePath(getTempDir(), "x/z/1.file"), 0, false, 0), + new StoragePathInfo(new StoragePath(getTempDir(), "x/z/2.file"), 0, false, 0) }).collect(Collectors.toList()), - storage.listDirectEntries(Arrays.stream(new HoodieLocation[] { - new HoodieLocation(getTempDir(), "w"), - new HoodieLocation(getTempDir(), "x/z") + storage.listDirectEntries(Arrays.stream(new StoragePath[] { + new StoragePath(getTempDir(), "w"), + new StoragePath(getTempDir(), "x/z") }).collect(Collectors.toList()))); assertThrows(FileNotFoundException.class, - () -> storage.listDirectEntries(new HoodieLocation(getTempDir(), "*"))); + () -> storage.listDirectEntries(new StoragePath(getTempDir(), "*"))); - validateHoodieFileStatusList( - Arrays.stream(new HoodieFileStatus[] { - new HoodieFileStatus(new HoodieLocation(getTempDir(), "x/y/1.file"), 0, false, 0), - new HoodieFileStatus(new HoodieLocation(getTempDir(), "x/z/1.file"), 0, false, 0) + validatePathInfoList( + Arrays.stream(new StoragePathInfo[] { + new StoragePathInfo(new StoragePath(getTempDir(), "x/y/1.file"), 0, false, 0), + new StoragePathInfo(new StoragePath(getTempDir(), "x/z/1.file"), 0, false, 0) }).collect(Collectors.toList()), - storage.globEntries(new HoodieLocation(getTempDir(), "x/*/1.file"))); + storage.globEntries(new StoragePath(getTempDir(), "x/*/1.file"))); - validateHoodieFileStatusList( - Arrays.stream(new HoodieFileStatus[] { - new HoodieFileStatus(new HoodieLocation(getTempDir(), "x/1.file"), 0, false, 0), - new HoodieFileStatus(new HoodieLocation(getTempDir(), "x/2.file"), 0, false, 0), + validatePathInfoList( + Arrays.stream(new StoragePathInfo[] { + new StoragePathInfo(new StoragePath(getTempDir(), "x/1.file"), 0, false, 0), + new StoragePathInfo(new StoragePath(getTempDir(), "x/2.file"), 0, false, 0), }).collect(Collectors.toList()), - storage.globEntries(new HoodieLocation(getTempDir(), "x/*.file"))); + storage.globEntries(new StoragePath(getTempDir(), "x/*.file"))); - validateHoodieFileStatusList( - Arrays.stream(new HoodieFileStatus[] { - new HoodieFileStatus(new HoodieLocation(getTempDir(), "x/y/1.file"), 0, false, 0), + validatePathInfoList( + Arrays.stream(new StoragePathInfo[] { + new StoragePathInfo(new StoragePath(getTempDir(), "x/y/1.file"), 0, false, 0), }).collect(Collectors.toList()), storage.globEntries( - new HoodieLocation(getTempDir(), "x/*/*.file"), + new StoragePath(getTempDir(), "x/*/*.file"), e -> e.getParent().getName().equals("y") && e.getName().contains("1"))); } @@ -231,63 +231,63 @@ public void testListing() throws IOException { public void testFileNotFound() throws IOException { HoodieStorage storage = getHoodieStorage(); - HoodieLocation fileLocation = new HoodieLocation(getTempDir(), "testFileNotFound/1.file"); - HoodieLocation dirLocation = new HoodieLocation(getTempDir(), "testFileNotFound/2"); - assertFalse(storage.exists(fileLocation)); - assertThrows(FileNotFoundException.class, () -> storage.open(fileLocation)); - assertThrows(FileNotFoundException.class, () -> storage.getFileStatus(fileLocation)); - assertThrows(FileNotFoundException.class, () -> storage.listDirectEntries(fileLocation)); - assertThrows(FileNotFoundException.class, () -> storage.listDirectEntries(dirLocation)); - assertThrows(FileNotFoundException.class, () -> storage.listDirectEntries(dirLocation, e -> true)); + StoragePath filePath = new StoragePath(getTempDir(), "testFileNotFound/1.file"); + StoragePath dirPath = new StoragePath(getTempDir(), "testFileNotFound/2"); + assertFalse(storage.exists(filePath)); + assertThrows(FileNotFoundException.class, () -> storage.open(filePath)); + assertThrows(FileNotFoundException.class, () -> storage.getPathInfo(filePath)); + assertThrows(FileNotFoundException.class, () -> storage.listDirectEntries(filePath)); + assertThrows(FileNotFoundException.class, () -> storage.listDirectEntries(dirPath)); + assertThrows(FileNotFoundException.class, () -> storage.listDirectEntries(dirPath, e -> true)); assertThrows(FileNotFoundException.class, () -> storage.listDirectEntries( - Arrays.stream(new HoodieLocation[] {dirLocation}).collect(Collectors.toList()))); + Arrays.stream(new StoragePath[] {dirPath}).collect(Collectors.toList()))); } @Test public void testRename() throws IOException { HoodieStorage storage = getHoodieStorage(); - HoodieLocation location = new HoodieLocation(getTempDir(), "testRename/1.file"); - assertFalse(storage.exists(location)); - storage.create(location).close(); - validateFileStatus(storage, location, EMPTY_BYTES, false); + StoragePath path = new StoragePath(getTempDir(), "testRename/1.file"); + assertFalse(storage.exists(path)); + storage.create(path).close(); + validatePathInfo(storage, path, EMPTY_BYTES, false); - HoodieLocation newLocation = new HoodieLocation(getTempDir(), "testRename/1_renamed.file"); - assertTrue(storage.rename(location, newLocation)); - assertFalse(storage.exists(location)); - validateFileStatus(storage, newLocation, EMPTY_BYTES, false); + StoragePath newPath = new StoragePath(getTempDir(), "testRename/1_renamed.file"); + assertTrue(storage.rename(path, newPath)); + assertFalse(storage.exists(path)); + validatePathInfo(storage, newPath, EMPTY_BYTES, false); } @Test public void testDelete() throws IOException { HoodieStorage storage = getHoodieStorage(); - HoodieLocation location = new HoodieLocation(getTempDir(), "testDelete/1.file"); - assertFalse(storage.exists(location)); - storage.create(location).close(); - assertTrue(storage.exists(location)); + StoragePath path = new StoragePath(getTempDir(), "testDelete/1.file"); + assertFalse(storage.exists(path)); + storage.create(path).close(); + assertTrue(storage.exists(path)); - assertTrue(storage.deleteFile(location)); - assertFalse(storage.exists(location)); - assertFalse(storage.deleteFile(location)); + assertTrue(storage.deleteFile(path)); + assertFalse(storage.exists(path)); + assertFalse(storage.deleteFile(path)); - HoodieLocation location2 = new HoodieLocation(getTempDir(), "testDelete/2"); - assertFalse(storage.exists(location2)); - assertTrue(storage.createDirectory(location2)); - assertTrue(storage.exists(location2)); + StoragePath path2 = new StoragePath(getTempDir(), "testDelete/2"); + assertFalse(storage.exists(path2)); + assertTrue(storage.createDirectory(path2)); + assertTrue(storage.exists(path2)); - assertTrue(storage.deleteDirectory(location2)); - assertFalse(storage.exists(location2)); - assertFalse(storage.deleteDirectory(location2)); + assertTrue(storage.deleteDirectory(path2)); + assertFalse(storage.exists(path2)); + assertFalse(storage.deleteDirectory(path2)); } @Test public void testMakeQualified() { HoodieStorage storage = getHoodieStorage(); - HoodieLocation location = new HoodieLocation("/tmp/testMakeQualified/1.file"); + StoragePath path = new StoragePath("/tmp/testMakeQualified/1.file"); assertEquals( - new HoodieLocation("file:/tmp/testMakeQualified/1.file"), - storage.makeQualified(location)); + new StoragePath("file:/tmp/testMakeQualified/1.file"), + storage.makeQualified(path)); } @Test @@ -310,7 +310,7 @@ protected String getTempDir() { private void prepareFilesOnStorage(HoodieStorage storage) throws IOException { String dir = getTempDir(); for (String relativePath : RELATIVE_FILE_PATHS) { - storage.create(new HoodieLocation(dir, relativePath)).close(); + storage.create(new StoragePath(dir, relativePath)).close(); } } @@ -319,36 +319,36 @@ private HoodieStorage getHoodieStorage() { return getHoodieStorage(getFileSystem(conf), conf); } - private void validateFileStatus(HoodieStorage storage, - HoodieLocation location, - byte[] data, - boolean isDirectory) throws IOException { - assertTrue(storage.exists(location)); - HoodieFileStatus fileStatus = storage.getFileStatus(location); - assertEquals(location, fileStatus.getLocation()); - assertEquals(isDirectory, fileStatus.isDirectory()); - assertEquals(!isDirectory, fileStatus.isFile()); + private void validatePathInfo(HoodieStorage storage, + StoragePath path, + byte[] data, + boolean isDirectory) throws IOException { + assertTrue(storage.exists(path)); + StoragePathInfo pathInfo = storage.getPathInfo(path); + assertEquals(path, pathInfo.getPath()); + assertEquals(isDirectory, pathInfo.isDirectory()); + assertEquals(!isDirectory, pathInfo.isFile()); if (!isDirectory) { - assertEquals(data.length, fileStatus.getLength()); - try (InputStream stream = storage.open(location)) { + assertEquals(data.length, pathInfo.getLength()); + try (InputStream stream = storage.open(path)) { assertArrayEquals(data, IOUtils.readAsByteArray(stream, data.length)); } } - assertTrue(fileStatus.getModificationTime() > 0); + assertTrue(pathInfo.getModificationTime() > 0); } - private void validateHoodieFileStatusList(List expected, - List actual) { + private void validatePathInfoList(List expected, + List actual) { assertEquals(expected.size(), actual.size()); - List sortedExpected = expected.stream() - .sorted(Comparator.comparing(HoodieFileStatus::getLocation)) + List sortedExpected = expected.stream() + .sorted(Comparator.comparing(StoragePathInfo::getPath)) .collect(Collectors.toList()); - List sortedActual = actual.stream() - .sorted(Comparator.comparing(HoodieFileStatus::getLocation)) + List sortedActual = actual.stream() + .sorted(Comparator.comparing(StoragePathInfo::getPath)) .collect(Collectors.toList()); for (int i = 0; i < expected.size(); i++) { - // We cannot use HoodieFileStatus#equals as that only compares the location - assertEquals(sortedExpected.get(i).getLocation(), sortedActual.get(i).getLocation()); + // We cannot use StoragePathInfo#equals as that only compares the path + assertEquals(sortedExpected.get(i).getPath(), sortedActual.get(i).getPath()); assertEquals(sortedExpected.get(i).isDirectory(), sortedActual.get(i).isDirectory()); assertEquals(sortedExpected.get(i).isFile(), sortedActual.get(i).isFile()); if (sortedExpected.get(i).isFile()) { diff --git a/hudi-io/src/test/java/org/apache/hudi/io/storage/TestStoragePath.java b/hudi-io/src/test/java/org/apache/hudi/io/storage/TestStoragePath.java new file mode 100644 index 0000000000000..9195ebec9fdf3 --- /dev/null +++ b/hudi-io/src/test/java/org/apache/hudi/io/storage/TestStoragePath.java @@ -0,0 +1,219 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.io.storage; + +import org.apache.hudi.storage.StoragePath; + +import org.junit.jupiter.api.Test; + +import java.net.URI; +import java.net.URISyntaxException; +import java.util.Arrays; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertSame; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Tests {@link StoragePath} + */ +public class TestStoragePath { + @Test + public void testToString() { + Arrays.stream( + new String[] { + "/", + "/foo", + "/foo/bar", + "foo", + "foo/bar", + "/foo/bar#boo", + "foo/bar#boo", + "file:/a/b/c", + "s3://a/b/c"}) + .forEach(this::toStringTest); + } + + @Test + public void testNormalize() throws URISyntaxException { + assertEquals("", new StoragePath(".").toString()); + assertEquals("..", new StoragePath("..").toString()); + assertEquals("/", new StoragePath("/").toString()); + assertEquals("/", new StoragePath("//").toString()); + assertEquals("/", new StoragePath("///").toString()); + assertEquals("//foo/", new StoragePath("//foo/").toString()); + assertEquals("//foo/", new StoragePath("//foo//").toString()); + assertEquals("//foo/bar", new StoragePath("//foo//bar").toString()); + assertEquals("/foo", new StoragePath("/foo/").toString()); + assertEquals("/foo", new StoragePath("/foo/").toString()); + assertEquals("foo", new StoragePath("foo/").toString()); + assertEquals("foo", new StoragePath("foo//").toString()); + assertEquals("foo/bar", new StoragePath("foo//bar").toString()); + assertEquals("file:/a/b/c", new StoragePath("file:///a/b/c").toString()); + assertEquals("s3://a/b/c/d/e", new StoragePath("s3://a/b/c", "d/e").toString()); + assertEquals("s3://a/b/c/d/e", new StoragePath("s3://a/b/c/", "d/e").toString()); + assertEquals("s3://a/b/c/d/e", new StoragePath("s3://a/b/c/", "d/e/").toString()); + assertEquals("s3://a/b/c", new StoragePath("s3://a/b/c/", "/").toString()); + assertEquals("s3://a/b/c", new StoragePath("s3://a/b/c/", "").toString()); + assertEquals("s3://a/b/c/d/e", new StoragePath(new StoragePath("s3://a/b/c"), "d/e").toString()); + assertEquals("s3://a/b/c/d/e", new StoragePath(new StoragePath("s3://a/b/c/"), "d/e").toString()); + assertEquals("s3://a/b/c/d/e", new StoragePath(new StoragePath("s3://a/b/c/"), "d/e/").toString()); + assertEquals("s3://a/b/c", new StoragePath(new StoragePath("s3://a/b/c/"), "/").toString()); + assertEquals("s3://a/b/c", new StoragePath(new StoragePath("s3://a/b/c/"), "").toString()); + assertEquals("hdfs://foo/foo2/bar/baz/", new StoragePath(new URI("hdfs://foo//foo2///bar/baz///")).toString()); + } + + @Test + public void testIsAbsolute() { + assertTrue(new StoragePath("/").isAbsolute()); + assertTrue(new StoragePath("/foo").isAbsolute()); + assertFalse(new StoragePath("foo").isAbsolute()); + assertFalse(new StoragePath("foo/bar").isAbsolute()); + assertFalse(new StoragePath(".").isAbsolute()); + } + + @Test + public void testGetParent() { + assertEquals(new StoragePath("/foo"), new StoragePath("/foo/bar").getParent()); + assertEquals(new StoragePath("foo"), new StoragePath("foo/bar").getParent()); + assertEquals(new StoragePath("/"), new StoragePath("/foo").getParent()); + assertEquals(new StoragePath("/foo/bar/x"), new StoragePath("/foo/bar", "x/y").getParent()); + assertEquals(new StoragePath("/foo/bar"), new StoragePath("/foo/bar/", "y").getParent()); + assertEquals(new StoragePath("/foo"), new StoragePath("/foo/bar/", "/").getParent()); + assertThrows(IllegalStateException.class, () -> new StoragePath("/").getParent()); + } + + @Test + public void testURI() throws URISyntaxException { + URI uri = new URI("file:///bar#baz"); + StoragePath path = new StoragePath(uri); + assertEquals(uri, new URI(path.toString())); + assertEquals("foo://bar/baz#boo", new StoragePath("foo://bar/", "/baz#boo").toString()); + assertEquals("foo://bar/baz/fud#boo", + new StoragePath(new StoragePath(new URI("foo://bar/baz#bud")), "fud#boo").toString()); + assertEquals("foo://bar/fud#boo", + new StoragePath(new StoragePath(new URI("foo://bar/baz#bud")), "/fud#boo").toString()); + } + + @Test + public void testEncoded() { + // encoded character like `%2F` should be kept as is + assertEquals(new StoragePath("s3://foo/bar/1%2F2%2F3"), new StoragePath("s3://foo/bar", "1%2F2%2F3")); + assertEquals("s3://foo/bar/1%2F2%2F3", new StoragePath("s3://foo/bar", "1%2F2%2F3").toString()); + assertEquals(new StoragePath("s3://foo/bar/1%2F2%2F3"), + new StoragePath(new StoragePath("s3://foo/bar"), "1%2F2%2F3")); + assertEquals("s3://foo/bar/1%2F2%2F3", + new StoragePath(new StoragePath("s3://foo/bar"), "1%2F2%2F3").toString()); + assertEquals("s3://foo/bar/1%2F2%2F3", new StoragePath("s3://foo/bar/1%2F2%2F3").toString()); + } + + @Test + public void testPathToUriConversion() throws URISyntaxException { + assertEquals(new URI(null, null, "/foo?bar", null, null), + new StoragePath("/foo?bar").toUri()); + assertEquals(new URI(null, null, "/foo\"bar", null, null), + new StoragePath("/foo\"bar").toUri()); + assertEquals(new URI(null, null, "/foo bar", null, null), + new StoragePath("/foo bar").toUri()); + assertEquals("/foo?bar", new StoragePath("http://localhost/foo?bar").toUri().getPath()); + assertEquals("/foo", new URI("http://localhost/foo?bar").getPath()); + assertEquals((new URI("/foo;bar")).getPath(), new StoragePath("/foo;bar").toUri().getPath()); + assertEquals(new URI("/foo;bar"), new StoragePath("/foo;bar").toUri()); + assertEquals(new URI("/foo+bar"), new StoragePath("/foo+bar").toUri()); + assertEquals(new URI("/foo-bar"), new StoragePath("/foo-bar").toUri()); + assertEquals(new URI("/foo=bar"), new StoragePath("/foo=bar").toUri()); + assertEquals(new URI("/foo,bar"), new StoragePath("/foo,bar").toUri()); + } + + @Test + public void testGetName() { + assertEquals("", new StoragePath("/").getName()); + assertEquals("foo", new StoragePath("foo").getName()); + assertEquals("foo", new StoragePath("/foo").getName()); + assertEquals("foo", new StoragePath("/foo/").getName()); + assertEquals("bar", new StoragePath("/foo/bar").getName()); + assertEquals("bar", new StoragePath("hdfs://host/foo/bar").getName()); + assertEquals("bar", new StoragePath("hdfs://host", "foo/bar").getName()); + assertEquals("bar", new StoragePath("hdfs://host/foo/", "bar").getName()); + } + + @Test + public void testGetPathWithoutSchemeAndAuthority() { + assertEquals( + new StoragePath("/foo/bar/boo"), + new StoragePath("/foo/bar/boo").getPathWithoutSchemeAndAuthority()); + assertEquals( + new StoragePath("/foo/bar/boo"), + new StoragePath("file:///foo/bar/boo").getPathWithoutSchemeAndAuthority()); + assertEquals( + new StoragePath("/bar/boo"), + new StoragePath("s3://foo/bar/boo").getPathWithoutSchemeAndAuthority()); + } + + @Test + public void testDepth() throws URISyntaxException { + assertEquals(0, new StoragePath("/").depth()); + assertEquals(0, new StoragePath("///").depth()); + assertEquals(0, new StoragePath("//foo/").depth()); + assertEquals(1, new StoragePath("//foo//bar").depth()); + assertEquals(5, new StoragePath("/a/b/c/d/e").depth()); + assertEquals(4, new StoragePath("s3://a/b/c", "d/e").depth()); + assertEquals(2, new StoragePath("s3://a/b/c/", "").depth()); + assertEquals(4, new StoragePath(new StoragePath("s3://a/b/c"), "d/e").depth()); + } + + @Test + public void testMakeQualified() throws URISyntaxException { + URI defaultUri = new URI("hdfs://host1/dir1"); + assertEquals(new StoragePath("hdfs://host1/a/b/c"), + new StoragePath("/a/b/c").makeQualified(defaultUri)); + assertEquals(new StoragePath("hdfs://host2/a/b/c"), + new StoragePath("hdfs://host2/a/b/c").makeQualified(defaultUri)); + assertEquals(new StoragePath("hdfs://host1/a/b/c"), + new StoragePath("hdfs:/a/b/c").makeQualified(defaultUri)); + assertEquals(new StoragePath("s3://a/b/c"), + new StoragePath("s3://a/b/c/").makeQualified(defaultUri)); + assertThrows(IllegalStateException.class, + () -> new StoragePath("a").makeQualified(defaultUri)); + } + + @Test + public void testEquals() { + assertEquals(new StoragePath("/foo"), new StoragePath("/foo")); + assertEquals(new StoragePath("/foo"), new StoragePath("/foo/")); + assertEquals(new StoragePath("/foo/bar"), new StoragePath("/foo//bar/")); + assertNotEquals(new StoragePath("/"), new StoragePath("/foo")); + } + + @Test + public void testCachedResults() { + StoragePath path = new StoragePath("s3://x/y/z/"); + assertSame(path.getParent(), path.getParent()); + assertSame(path.getName(), path.getName()); + assertSame(path.toString(), path.toString()); + } + + private void toStringTest(String pathString) { + assertEquals(pathString, new StoragePath(pathString).toString()); + } +} diff --git a/hudi-io/src/test/java/org/apache/hudi/io/storage/TestHoodieLocationFilter.java b/hudi-io/src/test/java/org/apache/hudi/io/storage/TestStoragePathFilter.java similarity index 58% rename from hudi-io/src/test/java/org/apache/hudi/io/storage/TestHoodieLocationFilter.java rename to hudi-io/src/test/java/org/apache/hudi/io/storage/TestStoragePathFilter.java index 2d66cc23f87ea..7290a6632c784 100644 --- a/hudi-io/src/test/java/org/apache/hudi/io/storage/TestHoodieLocationFilter.java +++ b/hudi-io/src/test/java/org/apache/hudi/io/storage/TestStoragePathFilter.java @@ -19,8 +19,8 @@ package org.apache.hudi.io.storage; -import org.apache.hudi.storage.HoodieLocation; -import org.apache.hudi.storage.HoodieLocationFilter; +import org.apache.hudi.storage.StoragePath; +import org.apache.hudi.storage.StoragePathFilter; import org.junit.jupiter.api.Test; @@ -31,39 +31,39 @@ import static org.junit.jupiter.api.Assertions.assertEquals; /** - * Tests {@link HoodieLocationFilter} + * Tests {@link StoragePathFilter} */ -public class TestHoodieLocationFilter { +public class TestStoragePathFilter { @Test public void testFilter() { - HoodieLocation location1 = new HoodieLocation("/x/y/1"); - HoodieLocation location2 = new HoodieLocation("/x/y/2"); - HoodieLocation location3 = new HoodieLocation("/x/z/1"); - HoodieLocation location4 = new HoodieLocation("/x/z/2"); + StoragePath path1 = new StoragePath("/x/y/1"); + StoragePath path2 = new StoragePath("/x/y/2"); + StoragePath path3 = new StoragePath("/x/z/1"); + StoragePath path4 = new StoragePath("/x/z/2"); - List locationList = Arrays.stream( - new HoodieLocation[] {location1, location2, location3, location4} + List pathList = Arrays.stream( + new StoragePath[] {path1, path2, path3, path4} ).collect(Collectors.toList()); - List expected = Arrays.stream( - new HoodieLocation[] {location1, location2} + List expected = Arrays.stream( + new StoragePath[] {path1, path2} ).collect(Collectors.toList()); assertEquals(expected.stream().sorted().collect(Collectors.toList()), - locationList.stream() - .filter(e -> new HoodieLocationFilter() { + pathList.stream() + .filter(e -> new StoragePathFilter() { @Override - public boolean accept(HoodieLocation location) { - return location.getParent().equals(new HoodieLocation("/x/y")); + public boolean accept(StoragePath path) { + return path.getParent().equals(new StoragePath("/x/y")); } }.accept(e)) .sorted() .collect(Collectors.toList())); - assertEquals(locationList, - locationList.stream() - .filter(e -> new HoodieLocationFilter() { + assertEquals(pathList, + pathList.stream() + .filter(e -> new StoragePathFilter() { @Override - public boolean accept(HoodieLocation location) { + public boolean accept(StoragePath path) { return true; } }.accept(e)) diff --git a/hudi-io/src/test/java/org/apache/hudi/io/storage/TestHoodieFileStatus.java b/hudi-io/src/test/java/org/apache/hudi/io/storage/TestStoragePathInfo.java similarity index 56% rename from hudi-io/src/test/java/org/apache/hudi/io/storage/TestHoodieFileStatus.java rename to hudi-io/src/test/java/org/apache/hudi/io/storage/TestStoragePathInfo.java index 903fc4b4e3ad1..1d92fa075d0fd 100644 --- a/hudi-io/src/test/java/org/apache/hudi/io/storage/TestHoodieFileStatus.java +++ b/hudi-io/src/test/java/org/apache/hudi/io/storage/TestStoragePathInfo.java @@ -19,8 +19,8 @@ package org.apache.hudi.io.storage; -import org.apache.hudi.storage.HoodieFileStatus; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; +import org.apache.hudi.storage.StoragePathInfo; import org.junit.jupiter.api.Test; import org.slf4j.Logger; @@ -36,34 +36,34 @@ import static org.junit.jupiter.api.Assertions.assertFalse; /** - * Tests {@link HoodieFileStatus} + * Tests {@link StoragePathInfo} */ -public class TestHoodieFileStatus { - private static final Logger LOG = LoggerFactory.getLogger(TestHoodieFileStatus.class); +public class TestStoragePathInfo { + private static final Logger LOG = LoggerFactory.getLogger(TestStoragePathInfo.class); private static final long LENGTH = 100; private static final long MODIFICATION_TIME = System.currentTimeMillis(); private static final String PATH1 = "/abc/xyz1"; private static final String PATH2 = "/abc/xyz2"; - private static final HoodieLocation LOCATION1 = new HoodieLocation(PATH1); - private static final HoodieLocation LOCATION2 = new HoodieLocation(PATH2); + private static final StoragePath STORAGE_PATH1 = new StoragePath(PATH1); + private static final StoragePath STORAGE_PATH2 = new StoragePath(PATH2); @Test public void testConstructor() { - HoodieFileStatus fileStatus = new HoodieFileStatus(LOCATION1, LENGTH, false, MODIFICATION_TIME); - validateAccessors(fileStatus, PATH1, LENGTH, false, MODIFICATION_TIME); - fileStatus = new HoodieFileStatus(LOCATION2, -1, true, MODIFICATION_TIME + 2L); - validateAccessors(fileStatus, PATH2, -1, true, MODIFICATION_TIME + 2L); + StoragePathInfo pathInfo = new StoragePathInfo(STORAGE_PATH1, LENGTH, false, MODIFICATION_TIME); + validateAccessors(pathInfo, PATH1, LENGTH, false, MODIFICATION_TIME); + pathInfo = new StoragePathInfo(STORAGE_PATH2, -1, true, MODIFICATION_TIME + 2L); + validateAccessors(pathInfo, PATH2, -1, true, MODIFICATION_TIME + 2L); } @Test public void testSerializability() throws IOException, ClassNotFoundException { - HoodieFileStatus fileStatus = new HoodieFileStatus(LOCATION1, LENGTH, false, MODIFICATION_TIME); + StoragePathInfo pathInfo = new StoragePathInfo(STORAGE_PATH1, LENGTH, false, MODIFICATION_TIME); try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); ObjectOutputStream oos = new ObjectOutputStream(baos)) { - oos.writeObject(fileStatus); + oos.writeObject(pathInfo); try (ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray()); ObjectInputStream ois = new ObjectInputStream(bais)) { - HoodieFileStatus deserialized = (HoodieFileStatus) ois.readObject(); + StoragePathInfo deserialized = (StoragePathInfo) ois.readObject(); validateAccessors(deserialized, PATH1, LENGTH, false, MODIFICATION_TIME); } } @@ -71,32 +71,32 @@ public void testSerializability() throws IOException, ClassNotFoundException { @Test public void testEquals() { - HoodieFileStatus fileStatus1 = new HoodieFileStatus( - new HoodieLocation(PATH1), LENGTH, false, MODIFICATION_TIME); - HoodieFileStatus fileStatus2 = new HoodieFileStatus( - new HoodieLocation(PATH1), LENGTH + 2, false, MODIFICATION_TIME + 2L); - assertEquals(fileStatus1, fileStatus2); + StoragePathInfo pathInfo1 = new StoragePathInfo( + new StoragePath(PATH1), LENGTH, false, MODIFICATION_TIME); + StoragePathInfo pathInfo2 = new StoragePathInfo( + new StoragePath(PATH1), LENGTH + 2, false, MODIFICATION_TIME + 2L); + assertEquals(pathInfo1, pathInfo2); } @Test public void testNotEquals() { - HoodieFileStatus fileStatus1 = new HoodieFileStatus( - LOCATION1, LENGTH, false, MODIFICATION_TIME); - HoodieFileStatus fileStatus2 = new HoodieFileStatus( - LOCATION2, LENGTH, false, MODIFICATION_TIME + 2L); - assertFalse(fileStatus1.equals(fileStatus2)); - assertFalse(fileStatus2.equals(fileStatus1)); + StoragePathInfo pathInfo1 = new StoragePathInfo( + STORAGE_PATH1, LENGTH, false, MODIFICATION_TIME); + StoragePathInfo pathInfo2 = new StoragePathInfo( + STORAGE_PATH2, LENGTH, false, MODIFICATION_TIME + 2L); + assertFalse(pathInfo1.equals(pathInfo2)); + assertFalse(pathInfo2.equals(pathInfo1)); } - private void validateAccessors(HoodieFileStatus fileStatus, - String location, + private void validateAccessors(StoragePathInfo pathInfo, + String path, long length, boolean isDirectory, long modificationTime) { - assertEquals(new HoodieLocation(location), fileStatus.getLocation()); - assertEquals(length, fileStatus.getLength()); - assertEquals(isDirectory, fileStatus.isDirectory()); - assertEquals(!isDirectory, fileStatus.isFile()); - assertEquals(modificationTime, fileStatus.getModificationTime()); + assertEquals(new StoragePath(path), pathInfo.getPath()); + assertEquals(length, pathInfo.getLength()); + assertEquals(isDirectory, pathInfo.isDirectory()); + assertEquals(!isDirectory, pathInfo.isFile()); + assertEquals(modificationTime, pathInfo.getModificationTime()); } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ExportInstantsProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ExportInstantsProcedure.scala index c03ac3e13762d..5c8c4618c25c7 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ExportInstantsProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ExportInstantsProcedure.scala @@ -28,7 +28,7 @@ import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline, TimelineMetadataUtils} import org.apache.hudi.exception.HoodieException import org.apache.hudi.hadoop.fs.HadoopFSUtils -import org.apache.hudi.storage.HoodieLocation +import org.apache.hudi.storage.StoragePath import org.apache.avro.generic.GenericRecord import org.apache.avro.specific.SpecificData @@ -159,7 +159,7 @@ class ExportInstantsProcedure extends BaseProcedure with ProcedureBuilder with L null } val instantTime = archiveEntryRecord.get("commitTime").toString - val outPath = localFolder + HoodieLocation.SEPARATOR + instantTime + "." + action + val outPath = localFolder + StoragePath.SEPARATOR + instantTime + "." + action if (metadata != null) writeToFile(fileSystem, outPath, HoodieAvroUtils.avroToJson(metadata, true)) if ( { copyCount += 1; @@ -182,7 +182,7 @@ class ExportInstantsProcedure extends BaseProcedure with ProcedureBuilder with L val timeline = metaClient.getActiveTimeline val fileSystem = HadoopFSUtils.getFs(metaClient.getBasePath, jsc.hadoopConfiguration()) for (instant <- instants) { - val localPath = localFolder + HoodieLocation.SEPARATOR + instant.getFileName + val localPath = localFolder + StoragePath.SEPARATOR + instant.getFileName val data: Array[Byte] = instant.getAction match { case HoodieTimeline.CLEAN_ACTION => val metadata = TimelineMetadataUtils.deserializeHoodieCleanMetadata(timeline.getInstantDetails(instant).get) diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/testutils/DataSourceTestUtils.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/testutils/DataSourceTestUtils.java index 95d6249a37521..10642318325a3 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/testutils/DataSourceTestUtils.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/testutils/DataSourceTestUtils.java @@ -22,7 +22,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.util.FileIOUtils; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.avro.Schema; import org.apache.hadoop.conf.Configuration; @@ -138,7 +138,7 @@ public static boolean isLogFileOnly(String basePath) throws IOException { while (files.hasNext()) { LocatedFileStatus file = files.next(); // skip meta folder - if (file.isFile() && !file.getPath().toString().contains(HoodieTableMetaClient.METAFOLDER_NAME + HoodieLocation.SEPARATOR)) { + if (file.isFile() && !file.getPath().toString().contains(HoodieTableMetaClient.METAFOLDER_NAME + StoragePath.SEPARATOR)) { if (FSUtils.isBaseFile(file.getPath())) { return false; } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala index df07c72f09072..04488eb8793a3 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala @@ -39,7 +39,7 @@ import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.exception.HoodieException import org.apache.hudi.keygen.TimestampBasedAvroKeyGenerator.TimestampType import org.apache.hudi.metadata.HoodieTableMetadata -import org.apache.hudi.storage.HoodieLocation +import org.apache.hudi.storage.StoragePath import org.apache.hudi.testutils.HoodieSparkClientTestBase import org.apache.hudi.util.JFunction @@ -816,9 +816,9 @@ class TestHoodieFileIndex extends HoodieSparkClientTestBase with ScalaAssertionS if (hiveStylePartitioning) { partitionNames.zip(partitionValues).map { case (name, value) => s"$name=$value" - }.mkString(HoodieLocation.SEPARATOR) + }.mkString(StoragePath.SEPARATOR) } else { - partitionValues.mkString(HoodieLocation.SEPARATOR) + partitionValues.mkString(StoragePath.SEPARATOR) } } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestBootstrapProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestBootstrapProcedure.scala index fc45509190ccb..90ed0906b1cb8 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestBootstrapProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestBootstrapProcedure.scala @@ -21,7 +21,7 @@ import org.apache.hudi.common.model.HoodieTableType import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.functional.TestBootstrap import org.apache.hudi.keygen.constant.KeyGeneratorOptions -import org.apache.hudi.storage.HoodieLocation +import org.apache.hudi.storage.StoragePath import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.sql.{Dataset, Row} @@ -41,8 +41,8 @@ class TestBootstrapProcedure extends HoodieSparkProcedureTestBase { val basePath = s"${tmp.getCanonicalPath}" val srcName: String = "source" - val sourcePath = basePath + HoodieLocation.SEPARATOR + srcName - val tablePath = basePath + HoodieLocation.SEPARATOR + tableName + val sourcePath = basePath + StoragePath.SEPARATOR + srcName + val tablePath = basePath + StoragePath.SEPARATOR + tableName val jsc = new JavaSparkContext(spark.sparkContext) // generate test data @@ -50,7 +50,7 @@ class TestBootstrapProcedure extends HoodieSparkProcedureTestBase { val timestamp: Long = Instant.now.toEpochMilli for (i <- 0 until partitions.size) { val df: Dataset[Row] = TestBootstrap.generateTestRawTripDataset(timestamp, i * NUM_OF_RECORDS, i * NUM_OF_RECORDS + NUM_OF_RECORDS, null, jsc, spark.sqlContext) - df.write.parquet(sourcePath + HoodieLocation.SEPARATOR + PARTITION_FIELD + "=" + partitions.get(i)) + df.write.parquet(sourcePath + StoragePath.SEPARATOR + PARTITION_FIELD + "=" + partitions.get(i)) } spark.sql("set hoodie.bootstrap.parallelism = 20") @@ -106,8 +106,8 @@ class TestBootstrapProcedure extends HoodieSparkProcedureTestBase { val basePath = s"${tmp.getCanonicalPath}" val srcName: String = "source" - val sourcePath = basePath + HoodieLocation.SEPARATOR + srcName - val tablePath = basePath + HoodieLocation.SEPARATOR + tableName + val sourcePath = basePath + StoragePath.SEPARATOR + srcName + val tablePath = basePath + StoragePath.SEPARATOR + tableName val jsc = new JavaSparkContext(spark.sparkContext) // generate test data @@ -115,7 +115,7 @@ class TestBootstrapProcedure extends HoodieSparkProcedureTestBase { val timestamp: Long = Instant.now.toEpochMilli for (i <- 0 until partitions.size) { val df: Dataset[Row] = TestBootstrap.generateTestRawTripDataset(timestamp, i * NUM_OF_RECORDS, i * NUM_OF_RECORDS + NUM_OF_RECORDS, null, jsc, spark.sqlContext) - df.write.parquet(sourcePath + HoodieLocation.SEPARATOR + PARTITION_FIELD + "=" + partitions.get(i)) + df.write.parquet(sourcePath + StoragePath.SEPARATOR + PARTITION_FIELD + "=" + partitions.get(i)) } spark.sql("set hoodie.bootstrap.parallelism = 20") @@ -172,8 +172,8 @@ class TestBootstrapProcedure extends HoodieSparkProcedureTestBase { val basePath = s"${tmp.getCanonicalPath}" val srcName: String = "source" - val sourcePath = basePath + HoodieLocation.SEPARATOR + srcName - val tablePath = basePath + HoodieLocation.SEPARATOR + tableName + val sourcePath = basePath + StoragePath.SEPARATOR + srcName + val tablePath = basePath + StoragePath.SEPARATOR + tableName val jsc = new JavaSparkContext(spark.sparkContext) // generate test data @@ -228,8 +228,8 @@ class TestBootstrapProcedure extends HoodieSparkProcedureTestBase { val basePath = s"${tmp.getCanonicalPath}" val srcName: String = "source" - val sourcePath = basePath + HoodieLocation.SEPARATOR + srcName - val tablePath = basePath + HoodieLocation.SEPARATOR + tableName + val sourcePath = basePath + StoragePath.SEPARATOR + srcName + val tablePath = basePath + StoragePath.SEPARATOR + tableName val jsc = new JavaSparkContext(spark.sparkContext) // generate test data @@ -237,7 +237,7 @@ class TestBootstrapProcedure extends HoodieSparkProcedureTestBase { val timestamp: Long = Instant.now.toEpochMilli for (i <- 0 until partitions.size) { val df: Dataset[Row] = TestBootstrap.generateTestRawTripDataset(timestamp, i * NUM_OF_RECORDS, i * NUM_OF_RECORDS + NUM_OF_RECORDS, null, jsc, spark.sqlContext) - df.write.parquet(sourcePath + HoodieLocation.SEPARATOR + PARTITION_FIELD + "=" + partitions.get(i)) + df.write.parquet(sourcePath + StoragePath.SEPARATOR + PARTITION_FIELD + "=" + partitions.get(i)) } spark.sql("set hoodie.bootstrap.parallelism = 20") diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestHdfsParquetImportProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestHdfsParquetImportProcedure.scala index 9ca3ff0719be9..47cd95f56f8e6 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestHdfsParquetImportProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestHdfsParquetImportProcedure.scala @@ -17,18 +17,16 @@ package org.apache.spark.sql.hudi.procedure -import org.apache.hudi.common.fs.FSUtils - -import org.apache.avro.generic.GenericRecord -import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hudi.common.model.HoodieTableType import org.apache.hudi.common.table.timeline.HoodieActiveTimeline import org.apache.hudi.common.testutils.{HoodieTestDataGenerator, HoodieTestUtils} import org.apache.hudi.common.util.StringUtils.getUTF8Bytes import org.apache.hudi.hadoop.fs.HadoopFSUtils -import org.apache.hudi.storage.HoodieLocation +import org.apache.hudi.storage.StoragePath import org.apache.hudi.testutils.HoodieClientTestUtils +import org.apache.avro.generic.GenericRecord +import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.parquet.avro.AvroParquetWriter import org.apache.parquet.hadoop.ParquetWriter import org.apache.spark.api.java.JavaSparkContext @@ -47,7 +45,7 @@ class TestHdfsParquetImportProcedure extends HoodieSparkProcedureTestBase { withTempDir { tmp => val fs: FileSystem = HadoopFSUtils.getFs(tmp.getCanonicalPath, spark.sparkContext.hadoopConfiguration) val tableName = generateTableName - val tablePath = tmp.getCanonicalPath + HoodieLocation.SEPARATOR + tableName + val tablePath = tmp.getCanonicalPath + StoragePath.SEPARATOR + tableName val sourcePath = new Path(tmp.getCanonicalPath, "source") val targetPath = new Path(tablePath) val schemaFile = new Path(tmp.getCanonicalPath, "file.schema").toString @@ -80,7 +78,7 @@ class TestHdfsParquetImportProcedure extends HoodieSparkProcedureTestBase { withTempDir { tmp => val fs: FileSystem = HadoopFSUtils.getFs(tmp.getCanonicalPath, spark.sparkContext.hadoopConfiguration) val tableName = generateTableName - val tablePath = tmp.getCanonicalPath + HoodieLocation.SEPARATOR + tableName + val tablePath = tmp.getCanonicalPath + StoragePath.SEPARATOR + tableName val sourcePath = new Path(tmp.getCanonicalPath, "source") val targetPath = new Path(tablePath) val schemaFile = new Path(tmp.getCanonicalPath, "file.schema").toString diff --git a/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieSpark32PlusAnalysis.scala b/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieSpark32PlusAnalysis.scala index f4de486dff01c..1c422b473de13 100644 --- a/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieSpark32PlusAnalysis.scala +++ b/hudi-spark-datasource/hudi-spark3.2plus-common/src/main/scala/org/apache/spark/sql/hudi/analysis/HoodieSpark32PlusAnalysis.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.hudi.analysis import org.apache.hudi.{DataSourceReadOptions, DefaultSource, SparkAdapterSupport} -import org.apache.hudi.storage.HoodieLocation +import org.apache.hudi.storage.StoragePath import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.HoodieSpark3CatalystPlanUtils.MatchResolvedTable @@ -92,7 +92,7 @@ case class HoodieSpark32PlusResolveReferences(spark: SparkSession) extends Rule[ case HoodieTableChanges(args) => val (tablePath, opts) = HoodieTableChangesOptionsParser.parseOptions(args, HoodieTableChanges.FUNC_NAME) val hoodieDataSource = new DefaultSource - if (tablePath.contains(HoodieLocation.SEPARATOR)) { + if (tablePath.contains(StoragePath.SEPARATOR)) { // the first param is table path val relation = hoodieDataSource.createRelation(spark.sqlContext, opts ++ Map("path" -> tablePath)) LogicalRelation(relation) @@ -107,7 +107,7 @@ case class HoodieSpark32PlusResolveReferences(spark: SparkSession) extends Rule[ case HoodieTimelineTableValuedFunction(args) => val (tablePath, opts) = HoodieTimelineTableValuedFunctionOptionsParser.parseOptions(args, HoodieTimelineTableValuedFunction.FUNC_NAME) val hoodieDataSource = new DefaultSource - if (tablePath.contains(HoodieLocation.SEPARATOR)) { + if (tablePath.contains(StoragePath.SEPARATOR)) { // the first param is table path val relation = hoodieDataSource.createRelation(spark.sqlContext, opts ++ Map("path" -> tablePath)) LogicalRelation(relation) @@ -122,7 +122,7 @@ case class HoodieSpark32PlusResolveReferences(spark: SparkSession) extends Rule[ case HoodieFileSystemViewTableValuedFunction(args) => val (tablePath, opts) = HoodieFileSystemViewTableValuedFunctionOptionsParser.parseOptions(args, HoodieFileSystemViewTableValuedFunction.FUNC_NAME) val hoodieDataSource = new DefaultSource - if (tablePath.contains(HoodieLocation.SEPARATOR)) { + if (tablePath.contains(StoragePath.SEPARATOR)) { // the first param is table path val relation = hoodieDataSource.createRelation(spark.sqlContext, opts ++ Map("path" -> tablePath)) LogicalRelation(relation) @@ -137,7 +137,7 @@ case class HoodieSpark32PlusResolveReferences(spark: SparkSession) extends Rule[ case HoodieMetadataTableValuedFunction(args) => val (tablePath, opts) = HoodieMetadataTableValuedFunction.parseOptions(args, HoodieMetadataTableValuedFunction.FUNC_NAME) val hoodieDataSource = new DefaultSource - if (tablePath.contains(HoodieLocation.SEPARATOR)) { + if (tablePath.contains(StoragePath.SEPARATOR)) { // the first param is table path val relation = hoodieDataSource.createRelation(spark.sqlContext, opts ++ Map("path" -> (tablePath + "/.hoodie/metadata"))) LogicalRelation(relation) diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestService.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestService.java index 29d144005306f..9e6257a553bba 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestService.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestService.java @@ -20,7 +20,7 @@ import org.apache.hudi.common.testutils.NetworkTestUtils; import org.apache.hudi.common.util.FileIOUtils; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.conf.HiveConf; @@ -220,7 +220,7 @@ private void resetSystemProperties() { } private static String getHiveLocation(String baseLocation) { - return baseLocation + HoodieLocation.SEPARATOR + "hive"; + return baseLocation + StoragePath.SEPARATOR + "hive"; } private HiveServer2 startHiveServer(HiveConf serverConf) { diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerBasedEarlyConflictDetectionRunnable.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerBasedEarlyConflictDetectionRunnable.java index 931bd421b39ec..8303c495d4617 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerBasedEarlyConflictDetectionRunnable.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/marker/MarkerBasedEarlyConflictDetectionRunnable.java @@ -25,7 +25,7 @@ import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.MarkerUtils; import org.apache.hudi.exception.HoodieIOException; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.hudi.timeline.service.handlers.MarkerHandler; import org.apache.hadoop.conf.Configuration; @@ -88,7 +88,7 @@ public void run() { // and the markers from the requests pending processing. currentInstantAllMarkers.addAll(markerHandler.getAllMarkers(markerDir)); currentInstantAllMarkers.addAll(pendingMarkers); - Path tempPath = new Path(basePath + HoodieLocation.SEPARATOR + HoodieTableMetaClient.TEMPFOLDER_NAME); + Path tempPath = new Path(basePath + StoragePath.SEPARATOR + HoodieTableMetaClient.TEMPFOLDER_NAME); List instants = MarkerUtils.getAllMarkerDir(tempPath, fs); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/SparkSampleWritesUtils.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/SparkSampleWritesUtils.java index d4fc5e8053a6e..e7dca04bbe783 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/SparkSampleWritesUtils.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/SparkSampleWritesUtils.java @@ -34,7 +34,7 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.hadoop.fs.CachingPath; import org.apache.hudi.hadoop.fs.HadoopFSUtils; -import org.apache.hudi.storage.HoodieLocation; +import org.apache.hudi.storage.StoragePath; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -139,7 +139,7 @@ private static Pair doSampleWrites(JavaSparkContext jsc, Option } private static String getSampleWritesBasePath(JavaSparkContext jsc, HoodieWriteConfig writeConfig, String instantTime) throws IOException { - Path basePath = new CachingPath(writeConfig.getBasePath(), SAMPLE_WRITES_FOLDER_PATH + HoodieLocation.SEPARATOR + instantTime); + Path basePath = new CachingPath(writeConfig.getBasePath(), SAMPLE_WRITES_FOLDER_PATH + StoragePath.SEPARATOR + instantTime); FileSystem fs = HadoopFSUtils.getFs(basePath, jsc.hadoopConfiguration()); if (fs.exists(basePath)) { fs.delete(basePath, true);