From f435958d7206acad81db2e499e28de4c8b118d20 Mon Sep 17 00:00:00 2001 From: David Phillips Date: Mon, 1 May 2023 18:44:24 -0700 Subject: [PATCH 1/3] Rename Location.parse() to Location.of() --- .../java/io/trino/filesystem/Location.java | 2 +- .../filesystem/local/LocalFileSystem.java | 3 +- .../filesystem/memory/MemoryFileSystem.java | 3 +- .../io/trino/filesystem/TestLocation.java | 131 +++++++++--------- .../hudi/partition/HiveHudiPartitionInfo.java | 4 +- 5 files changed, 70 insertions(+), 73 deletions(-) diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/Location.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/Location.java index 89fc10ba790f..c2dc2c22befd 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/Location.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/Location.java @@ -56,7 +56,7 @@ public final class Location private final OptionalInt port; private final String path; - public static Location parse(String location) + public static Location of(String location) { requireNonNull(location, "location is null"); checkArgument(!location.isEmpty(), "location is empty"); diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalFileSystem.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalFileSystem.java index b8fe1fd12425..aadd87f64fa1 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalFileSystem.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalFileSystem.java @@ -28,7 +28,6 @@ import java.util.Optional; import static com.google.common.base.Preconditions.checkArgument; -import static io.trino.filesystem.Location.parse; import static io.trino.filesystem.local.LocalUtils.handleException; /** @@ -173,7 +172,7 @@ private Path toDirectoryPath(String directoryLocation) private static Location parseLocalLocation(String locationString) { - Location location = parse(locationString); + Location location = Location.of(locationString); checkArgument(location.scheme().equals(Optional.of("local")), "Only 'local' scheme is supported: %s", locationString); checkArgument(location.userInfo().isEmpty(), "Local location cannot contain user info: %s", locationString); checkArgument(location.host().isEmpty(), "Local location cannot contain a host: %s", locationString); diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryFileSystem.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryFileSystem.java index 5eb75ed2ccc8..2958a7b436e7 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryFileSystem.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryFileSystem.java @@ -32,7 +32,6 @@ import java.util.concurrent.ConcurrentMap; import static com.google.common.base.Preconditions.checkArgument; -import static io.trino.filesystem.Location.parse; /** * A blob file system for testing. @@ -173,7 +172,7 @@ private static String toBlobPrefix(String location) private static Location parseMemoryLocation(String locationString) { - Location location = parse(locationString); + Location location = Location.of(locationString); checkArgument(location.scheme().equals(Optional.of("memory")), "Only 'memory' scheme is supported: %s", locationString); checkArgument(location.userInfo().isEmpty(), "Memory location cannot contain user info: %s", locationString); checkArgument(location.host().isEmpty(), "Memory location cannot contain a host: %s", locationString); diff --git a/lib/trino-filesystem/src/test/java/io/trino/filesystem/TestLocation.java b/lib/trino-filesystem/src/test/java/io/trino/filesystem/TestLocation.java index 394535a9a555..f853f9f1cd6a 100644 --- a/lib/trino-filesystem/src/test/java/io/trino/filesystem/TestLocation.java +++ b/lib/trino-filesystem/src/test/java/io/trino/filesystem/TestLocation.java @@ -18,7 +18,6 @@ import java.util.Optional; import java.util.OptionalInt; -import static io.trino.filesystem.Location.parse; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -82,76 +81,76 @@ void testParse() assertLocation("file:/some@what/path", "file", "some@what/path"); // invalid locations - assertThatThrownBy(() -> parse(null)) + assertThatThrownBy(() -> Location.of(null)) .isInstanceOf(NullPointerException.class); - assertThatThrownBy(() -> parse("")) + assertThatThrownBy(() -> Location.of("")) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("location is empty"); - assertThatThrownBy(() -> parse(" ")) + assertThatThrownBy(() -> Location.of(" ")) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("location is blank"); - assertThatThrownBy(() -> parse("x")) + assertThatThrownBy(() -> Location.of("x")) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("scheme"); - assertThatThrownBy(() -> parse("scheme://host:invalid/path")) + assertThatThrownBy(() -> Location.of("scheme://host:invalid/path")) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("port"); - assertThatThrownBy(() -> parse("scheme:/path")) + assertThatThrownBy(() -> Location.of("scheme:/path")) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("scheme"); // fragment is not allowed - assertThatThrownBy(() -> parse("scheme://userInfo@host/some/path#fragement")) + assertThatThrownBy(() -> Location.of("scheme://userInfo@host/some/path#fragement")) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Fragment"); - assertThatThrownBy(() -> parse("scheme://userInfo@ho#st/some/path")) + assertThatThrownBy(() -> Location.of("scheme://userInfo@ho#st/some/path")) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Fragment"); - assertThatThrownBy(() -> parse("scheme://user#Info@host/some/path")) + assertThatThrownBy(() -> Location.of("scheme://user#Info@host/some/path")) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Fragment"); - assertThatThrownBy(() -> parse("sc#heme://userInfo@host/some/path")) + assertThatThrownBy(() -> Location.of("sc#heme://userInfo@host/some/path")) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Fragment"); // query component is not allowed - assertThatThrownBy(() -> parse("scheme://userInfo@host/some/path?fragement")) + assertThatThrownBy(() -> Location.of("scheme://userInfo@host/some/path?fragement")) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("query"); - assertThatThrownBy(() -> parse("scheme://userInfo@ho?st/some/path")) + assertThatThrownBy(() -> Location.of("scheme://userInfo@ho?st/some/path")) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("query"); - assertThatThrownBy(() -> parse("scheme://user?Info@host/some/path")) + assertThatThrownBy(() -> Location.of("scheme://user?Info@host/some/path")) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("query"); - assertThatThrownBy(() -> parse("sc?heme://userInfo@host/some/path")) + assertThatThrownBy(() -> Location.of("sc?heme://userInfo@host/some/path")) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("query"); } private static void assertLocation(String locationString, String scheme, Optional userInfo, String host, String path) { - Location location = parse(locationString); + Location location = Location.of(locationString); Optional expectedHost = host.isEmpty() ? Optional.empty() : Optional.of(host); assertLocation(location, locationString, Optional.of(scheme), userInfo, expectedHost, OptionalInt.empty(), path); } private static void assertLocation(String locationString, String scheme, String path) { - Location location = parse(locationString); + Location location = Location.of(locationString); assertLocation(location, locationString, Optional.of(scheme), Optional.empty(), Optional.empty(), OptionalInt.empty(), path); } private static void assertLocation(String locationString, String scheme, String host, int port, String path) { - Location location = parse(locationString); + Location location = Location.of(locationString); assertLocation(location, locationString, Optional.of(scheme), Optional.empty(), Optional.of(host), OptionalInt.of(port), path); } private static void assertLocation(String locationString, String path) { - Location location = parse(locationString); + Location location = Location.of(locationString); assertLocation(location, locationString, Optional.empty(), Optional.empty(), Optional.empty(), OptionalInt.empty(), path); } @@ -170,9 +169,9 @@ private static void assertLocation(Location location, String locationString, Opt assertThat(location.path()).isEqualTo(path); assertThat(location).isEqualTo(location); - assertThat(location).isEqualTo(parse(locationString)); + assertThat(location).isEqualTo(Location.of(locationString)); assertThat(location.hashCode()).isEqualTo(location.hashCode()); - assertThat(location.hashCode()).isEqualTo(parse(locationString).hashCode()); + assertThat(location.hashCode()).isEqualTo(Location.of(locationString).hashCode()); assertThat(location.toString()).isEqualTo(locationString); } @@ -180,11 +179,11 @@ private static void assertLocation(Location location, String locationString, Opt @Test void testVerifyFileLocation() { - parse("scheme://userInfo@host/name").verifyValidFileLocation(); - parse("scheme://userInfo@host/path/name").verifyValidFileLocation(); + Location.of("scheme://userInfo@host/name").verifyValidFileLocation(); + Location.of("scheme://userInfo@host/path/name").verifyValidFileLocation(); - parse("/name").verifyValidFileLocation(); - parse("/path/name").verifyValidFileLocation(); + Location.of("/name").verifyValidFileLocation(); + Location.of("/path/name").verifyValidFileLocation(); assertInvalidFileLocation("scheme://userInfo@host", "File location must contain a path"); assertInvalidFileLocation("scheme://userInfo@host/", "File location must contain a path"); @@ -198,7 +197,7 @@ void testVerifyFileLocation() private static void assertInvalidFileLocation(String locationString, String expectedErrorMessage) { - Location location = parse(locationString); + Location location = Location.of(locationString); assertThatThrownBy(location::verifyValidFileLocation) .isInstanceOf(IllegalStateException.class) .hasMessageContaining(locationString) @@ -229,7 +228,7 @@ void testFileName() private static void assertFileName(String locationString, String fileName) { // fileName method only works with valid file locations - Location location = parse(locationString); + Location location = Location.of(locationString); location.verifyValidFileLocation(); assertThat(location.fileName()).isEqualTo(fileName); } @@ -237,19 +236,19 @@ private static void assertFileName(String locationString, String fileName) @Test void testParentDirectory() { - assertParentDirectory("scheme://userInfo@host/path/name", parse("scheme://userInfo@host/path")); - assertParentDirectory("scheme://userInfo@host:1234/name", parse("scheme://userInfo@host:1234")); + assertParentDirectory("scheme://userInfo@host/path/name", Location.of("scheme://userInfo@host/path")); + assertParentDirectory("scheme://userInfo@host:1234/name", Location.of("scheme://userInfo@host:1234")); - assertParentDirectory("scheme://userInfo@host/path//name", parse("scheme://userInfo@host/path/")); - assertParentDirectory("scheme://userInfo@host/path///name", parse("scheme://userInfo@host/path//")); - assertParentDirectory("scheme://userInfo@host/path:/name", parse("scheme://userInfo@host/path:")); + assertParentDirectory("scheme://userInfo@host/path//name", Location.of("scheme://userInfo@host/path/")); + assertParentDirectory("scheme://userInfo@host/path///name", Location.of("scheme://userInfo@host/path//")); + assertParentDirectory("scheme://userInfo@host/path:/name", Location.of("scheme://userInfo@host/path:")); - assertParentDirectory("/path/name", parse("/path")); - assertParentDirectory("/name", parse("/")); + assertParentDirectory("/path/name", Location.of("/path")); + assertParentDirectory("/name", Location.of("/")); - assertParentDirectory("/path//name", parse("/path/")); - assertParentDirectory("/path///name", parse("/path//")); - assertParentDirectory("/path:/name", parse("/path:")); + assertParentDirectory("/path//name", Location.of("/path/")); + assertParentDirectory("/path///name", Location.of("/path//")); + assertParentDirectory("/path:/name", Location.of("/path:")); // all valid file locations must have a parent directory // invalid file locations are tested in testVerifyFileLocation @@ -258,7 +257,7 @@ void testParentDirectory() private static void assertParentDirectory(String locationString, Location parentLocation) { // fileName method only works with valid file locations - Location location = parse(locationString); + Location location = Location.of(locationString); location.verifyValidFileLocation(); Location parentDirectory = location.parentDirectory(); @@ -268,59 +267,59 @@ private static void assertParentDirectory(String locationString, Location parent @Test void testAppendPath() { - assertAppendPath("scheme://userInfo@host", "name", parse("scheme://userInfo@host/name")); - assertAppendPath("scheme://userInfo@host/", "name", parse("scheme://userInfo@host/name")); + assertAppendPath("scheme://userInfo@host", "name", Location.of("scheme://userInfo@host/name")); + assertAppendPath("scheme://userInfo@host/", "name", Location.of("scheme://userInfo@host/name")); - assertAppendPath("scheme://userInfo@host:1234/path", "name", parse("scheme://userInfo@host:1234/path/name")); - assertAppendPath("scheme://userInfo@host/path/", "name", parse("scheme://userInfo@host/path/name")); + assertAppendPath("scheme://userInfo@host:1234/path", "name", Location.of("scheme://userInfo@host:1234/path/name")); + assertAppendPath("scheme://userInfo@host/path/", "name", Location.of("scheme://userInfo@host/path/name")); - assertAppendPath("scheme://userInfo@host/path//", "name", parse("scheme://userInfo@host/path//name")); - assertAppendPath("scheme://userInfo@host/path:", "name", parse("scheme://userInfo@host/path:/name")); + assertAppendPath("scheme://userInfo@host/path//", "name", Location.of("scheme://userInfo@host/path//name")); + assertAppendPath("scheme://userInfo@host/path:", "name", Location.of("scheme://userInfo@host/path:/name")); - assertAppendPath("scheme://", "name", parse("scheme:///name")); - assertAppendPath("scheme:///", "name", parse("scheme:///name")); + assertAppendPath("scheme://", "name", Location.of("scheme:///name")); + assertAppendPath("scheme:///", "name", Location.of("scheme:///name")); - assertAppendPath("scheme:///path", "name", parse("scheme:///path/name")); - assertAppendPath("scheme:///path/", "name", parse("scheme:///path/name")); + assertAppendPath("scheme:///path", "name", Location.of("scheme:///path/name")); + assertAppendPath("scheme:///path/", "name", Location.of("scheme:///path/name")); - assertAppendPath("/", "name", parse("/name")); - assertAppendPath("/path", "name", parse("/path/name")); + assertAppendPath("/", "name", Location.of("/name")); + assertAppendPath("/path", "name", Location.of("/path/name")); } private static void assertAppendPath(String locationString, String newPathElement, Location expected) { - Location location = parse(locationString).appendPath(newPathElement); + Location location = Location.of(locationString).appendPath(newPathElement); assertLocation(location, expected); } @Test void testAppendSuffix() { - assertAppendSuffix("scheme://userInfo@host", ".ext", parse("scheme://userInfo@host/.ext")); - assertAppendSuffix("scheme://userInfo@host/", ".ext", parse("scheme://userInfo@host/.ext")); + assertAppendSuffix("scheme://userInfo@host", ".ext", Location.of("scheme://userInfo@host/.ext")); + assertAppendSuffix("scheme://userInfo@host/", ".ext", Location.of("scheme://userInfo@host/.ext")); - assertAppendSuffix("scheme://userInfo@host:1234/path", ".ext", parse("scheme://userInfo@host:1234/path.ext")); - assertAppendSuffix("scheme://userInfo@host/path/", ".ext", parse("scheme://userInfo@host/path/.ext")); + assertAppendSuffix("scheme://userInfo@host:1234/path", ".ext", Location.of("scheme://userInfo@host:1234/path.ext")); + assertAppendSuffix("scheme://userInfo@host/path/", ".ext", Location.of("scheme://userInfo@host/path/.ext")); - assertAppendSuffix("scheme://userInfo@host/path//", ".ext", parse("scheme://userInfo@host/path//.ext")); - assertAppendSuffix("scheme://userInfo@host/path:", ".ext", parse("scheme://userInfo@host/path:.ext")); + assertAppendSuffix("scheme://userInfo@host/path//", ".ext", Location.of("scheme://userInfo@host/path//.ext")); + assertAppendSuffix("scheme://userInfo@host/path:", ".ext", Location.of("scheme://userInfo@host/path:.ext")); - assertAppendSuffix("scheme://", ".ext", parse("scheme:///.ext")); - assertAppendSuffix("scheme:///", ".ext", parse("scheme:///.ext")); + assertAppendSuffix("scheme://", ".ext", Location.of("scheme:///.ext")); + assertAppendSuffix("scheme:///", ".ext", Location.of("scheme:///.ext")); - assertAppendSuffix("scheme:///path", ".ext", parse("scheme:///path.ext")); - assertAppendSuffix("scheme:///path/", ".ext", parse("scheme:///path/.ext")); + assertAppendSuffix("scheme:///path", ".ext", Location.of("scheme:///path.ext")); + assertAppendSuffix("scheme:///path/", ".ext", Location.of("scheme:///path/.ext")); - assertAppendSuffix("scheme:///path", "/foo", parse("scheme:///path/foo")); - assertAppendSuffix("scheme:///path/", "/foo", parse("scheme:///path//foo")); + assertAppendSuffix("scheme:///path", "/foo", Location.of("scheme:///path/foo")); + assertAppendSuffix("scheme:///path/", "/foo", Location.of("scheme:///path//foo")); - assertAppendSuffix("/", ".ext", parse("/.ext")); - assertAppendSuffix("/path", ".ext", parse("/path.ext")); + assertAppendSuffix("/", ".ext", Location.of("/.ext")); + assertAppendSuffix("/path", ".ext", Location.of("/path.ext")); } private static void assertAppendSuffix(String locationString, String suffix, Location expected) { - Location location = parse(locationString).appendSuffix(suffix); + Location location = Location.of(locationString).appendSuffix(suffix); assertLocation(location, expected); } } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/partition/HiveHudiPartitionInfo.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/partition/HiveHudiPartitionInfo.java index afa482d00b62..f8427f7bcdf3 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/partition/HiveHudiPartitionInfo.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/partition/HiveHudiPartitionInfo.java @@ -108,8 +108,8 @@ public void loadPartitionInfo(Optional partition) throw new HoodieIOException(format("Cannot find partition in Hive Metastore: %s", hivePartitionName)); } this.relativePartitionPath = getRelativePartitionPath( - Location.parse(table.getStorage().getLocation()), - Location.parse(partition.get().getStorage().getLocation())); + Location.of(table.getStorage().getLocation()), + Location.of(partition.get().getStorage().getLocation())); this.hivePartitionKeys = buildPartitionKeys(partitionColumns, partition.get().getValues()); } From 12a6f81cc9b973f2483a1739da509ceb1544e2c9 Mon Sep 17 00:00:00 2001 From: David Phillips Date: Mon, 1 May 2023 18:46:21 -0700 Subject: [PATCH 2/3] Replace Location.location() with toString() --- .../src/main/java/io/trino/filesystem/Location.java | 8 +++----- .../src/test/java/io/trino/filesystem/TestLocation.java | 4 ++-- 2 files changed, 5 insertions(+), 7 deletions(-) diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/Location.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/Location.java index c2dc2c22befd..a09862b4221a 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/Location.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/Location.java @@ -121,11 +121,6 @@ private Location withPath(String location, String path) return new Location(location, scheme, userInfo, host, port, path); } - public String location() - { - return location; - } - /** * Returns the scheme of the location, if present. * If the scheme is present, the value will not be an empty string. @@ -287,6 +282,9 @@ public int hashCode() return location.hashCode(); } + /** + * Return the original location string. + */ @Override public String toString() { diff --git a/lib/trino-filesystem/src/test/java/io/trino/filesystem/TestLocation.java b/lib/trino-filesystem/src/test/java/io/trino/filesystem/TestLocation.java index f853f9f1cd6a..b5b13c47d5e8 100644 --- a/lib/trino-filesystem/src/test/java/io/trino/filesystem/TestLocation.java +++ b/lib/trino-filesystem/src/test/java/io/trino/filesystem/TestLocation.java @@ -156,12 +156,12 @@ private static void assertLocation(String locationString, String path) private static void assertLocation(Location actual, Location expected) { - assertLocation(actual, expected.location(), expected.scheme(), expected.userInfo(), expected.host(), expected.port(), expected.path()); + assertLocation(actual, expected.toString(), expected.scheme(), expected.userInfo(), expected.host(), expected.port(), expected.path()); } private static void assertLocation(Location location, String locationString, Optional scheme, Optional userInfo, Optional host, OptionalInt port, String path) { - assertThat(location.location()).isEqualTo(locationString); + assertThat(location.toString()).isEqualTo(locationString); assertThat(location.scheme()).isEqualTo(scheme); assertThat(location.userInfo()).isEqualTo(userInfo); assertThat(location.host()).isEqualTo(host); From 92498d4564eb3454e91e959dfd1a041b7de7e1da Mon Sep 17 00:00:00 2001 From: David Phillips Date: Sun, 30 Apr 2023 12:48:52 -0700 Subject: [PATCH 3/3] Use Location in file system APIs --- .../java/io/trino/filesystem/FileEntry.java | 2 +- .../io/trino/filesystem/TrinoFileSystem.java | 18 +- .../io/trino/filesystem/TrinoInputFile.java | 2 +- .../io/trino/filesystem/TrinoOutputFile.java | 2 +- .../filesystem/local/LocalFileIterator.java | 5 +- .../filesystem/local/LocalFileSystem.java | 43 ++-- .../io/trino/filesystem/local/LocalInput.java | 5 +- .../filesystem/local/LocalInputFile.java | 17 +- .../filesystem/local/LocalInputStream.java | 5 +- .../filesystem/local/LocalOutputFile.java | 15 +- .../filesystem/local/LocalOutputStream.java | 6 +- .../io/trino/filesystem/local/LocalUtils.java | 8 +- .../filesystem/memory/MemoryFileSystem.java | 39 ++-- .../trino/filesystem/memory/MemoryInput.java | 7 +- .../filesystem/memory/MemoryInputFile.java | 13 +- .../filesystem/memory/MemoryInputStream.java | 5 +- .../filesystem/memory/MemoryOutputFile.java | 15 +- .../filesystem/memory/MemoryOutputStream.java | 5 +- .../filesystem/tracing/TracingFileSystem.java | 25 +-- .../filesystem/tracing/TracingInput.java | 13 +- .../filesystem/tracing/TracingInputFile.java | 19 +- .../filesystem/tracing/TracingOutputFile.java | 17 +- .../AbstractTestTrinoFileSystem.java | 204 +++++++++--------- .../io/trino/filesystem/TestFileEntry.java | 17 +- .../filesystem/TrackingFileSystemFactory.java | 52 +++-- .../filesystem/local/TestLocalFileSystem.java | 19 +- .../memory/TestMemoryFileSystem.java | 5 +- .../io/trino/filesystem/hdfs/HadoopPaths.java | 4 +- .../filesystem/hdfs/HdfsFileIterator.java | 32 ++- .../trino/filesystem/hdfs/HdfsFileSystem.java | 21 +- .../io/trino/filesystem/hdfs/HdfsInput.java | 2 +- .../trino/filesystem/hdfs/HdfsInputFile.java | 15 +- .../trino/filesystem/hdfs/HdfsOutputFile.java | 15 +- .../filesystem/hdfs/TestHdfsFileSystem.java | 27 ++- .../line/sequence/SequenceFileReader.java | 13 +- .../line/text/TextLineReaderFactory.java | 2 +- .../hive/formats/rcfile/RcFileReader.java | 5 +- .../formats/TestTrinoDataInputStream.java | 11 +- .../TestSequenceFileReaderWriter.java | 2 +- .../rcfile/TestRcFileReaderManual.java | 3 +- .../deltalake/AbstractDeltaLakePageSink.java | 17 +- .../deltalake/DeltaLakeCdfPageSink.java | 5 +- .../plugin/deltalake/DeltaLakeMergeSink.java | 17 +- .../plugin/deltalake/DeltaLakeMetadata.java | 69 +++--- .../plugin/deltalake/DeltaLakePageSink.java | 3 +- .../deltalake/DeltaLakePageSinkProvider.java | 15 +- .../DeltaLakePageSourceProvider.java | 4 +- .../plugin/deltalake/DeltaLakeWriter.java | 18 +- .../procedure/RegisterTableProcedure.java | 3 +- .../deltalake/procedure/VacuumProcedure.java | 9 +- .../statistics/MetaDirStatisticsAccess.java | 17 +- .../transactionlog/TableSnapshot.java | 16 +- .../transactionlog/TransactionLogAccess.java | 3 +- .../transactionlog/TransactionLogParser.java | 6 +- .../transactionlog/TransactionLogUtil.java | 7 +- .../checkpoint/CheckpointEntryIterator.java | 2 +- .../checkpoint/CheckpointWriterManager.java | 8 +- .../checkpoint/TransactionLogTail.java | 5 +- .../AzureTransactionLogSynchronizer.java | 5 +- .../writer/GcsTransactionLogSynchronizer.java | 7 +- .../writer/NoIsolationSynchronizer.java | 3 +- .../S3NativeTransactionLogSynchronizer.java | 34 ++- .../writer/TransactionLogSynchronizer.java | 3 +- .../writer/TransactionLogWriter.java | 3 +- ...seDeltaLakeRegisterTableProcedureTest.java | 2 +- .../BaseDeltaLakeTableWithCustomLocation.java | 5 +- ...FileTestingTransactionLogSynchronizer.java | 3 +- .../TestDeltaLakeFileOperations.java | 2 +- .../TestDeltaLakeGcsConnectorSmokeTest.java | 9 +- .../deltalake/TestTransactionLogAccess.java | 2 +- .../transactionlog/TestTableSnapshot.java | 2 +- .../TestCheckpointEntryIterator.java | 3 +- .../checkpoint/TestCheckpointWriter.java | 5 +- .../TestDeltaLakeFileStatistics.java | 7 +- .../hive/BackgroundHiveSplitLoader.java | 20 +- .../io/trino/plugin/hive/HiveMetadata.java | 5 +- .../trino/plugin/hive/HiveWriterFactory.java | 3 +- .../plugin/hive/RcFileFileWriterFactory.java | 8 +- .../trino/plugin/hive/SortingFileWriter.java | 55 ++--- .../plugin/hive/fs/MonitoredInputFile.java | 3 +- .../trino/plugin/hive/fs/TrinoFileStatus.java | 2 +- .../hive/line/LineFileWriterFactory.java | 6 +- .../hive/line/LinePageSourceFactory.java | 6 +- .../hive/orc/OrcDeleteDeltaPageSource.java | 4 +- .../trino/plugin/hive/orc/OrcDeletedRows.java | 7 +- .../plugin/hive/orc/OrcFileWriterFactory.java | 15 +- .../plugin/hive/orc/OrcPageSourceFactory.java | 3 +- .../plugin/hive/orc/OriginalFilesUtils.java | 24 +-- .../parquet/ParquetFileWriterFactory.java | 9 +- .../parquet/ParquetPageSourceFactory.java | 4 +- .../hive/parquet/TrinoParquetDataSource.java | 2 +- .../hive/rcfile/RcFilePageSourceFactory.java | 6 +- .../io/trino/plugin/hive/util/AcidTables.java | 29 +-- .../hive/benchmark/StandardFileFormats.java | 4 +- .../orc/TestOrcDeleteDeltaPageSource.java | 5 +- .../plugin/hive/parquet/ParquetTester.java | 8 +- .../hive/parquet/TestBloomFilterStore.java | 12 +- .../plugin/hive/util/TestAcidTables.java | 118 +++++----- .../plugin/hudi/HudiPageSourceProvider.java | 3 +- .../iceberg/IcebergFileWriterFactory.java | 25 ++- .../trino/plugin/iceberg/IcebergMetadata.java | 29 ++- .../trino/plugin/iceberg/IcebergPageSink.java | 3 +- .../iceberg/IcebergPageSourceProvider.java | 21 +- .../iceberg/IcebergParquetFileWriter.java | 10 +- .../plugin/iceberg/IcebergSplitSource.java | 3 +- .../plugin/iceberg/TrinoOrcDataSource.java | 2 +- .../iceberg/catalog/AbstractTrinoCatalog.java | 3 +- .../iceberg/catalog/hms/TrinoHiveCatalog.java | 7 +- .../delete/IcebergPositionDeletePageSink.java | 3 +- .../iceberg/fileio/ForwardingFileIo.java | 9 +- .../iceberg/fileio/ForwardingInputFile.java | 2 +- .../iceberg/fileio/ForwardingOutputFile.java | 5 +- .../iceberg/procedure/MigrateProcedure.java | 19 +- .../procedure/RegisterTableProcedure.java | 16 +- .../BaseIcebergConnectorSmokeTest.java | 27 +-- .../iceberg/BaseIcebergConnectorTest.java | 39 ++-- .../plugin/iceberg/IcebergTestUtils.java | 17 +- .../TestIcebergAbfsConnectorSmokeTest.java | 3 +- .../TestIcebergConnectorSmokeTest.java | 6 +- .../TestIcebergGcsConnectorSmokeTest.java | 9 +- .../TestIcebergMetadataFileOperations.java | 2 +- ...estIcebergMinioAvroConnectorSmokeTest.java | 3 +- .../TestIcebergMinioOrcConnectorTest.java | 7 +- ...IcebergMinioParquetConnectorSmokeTest.java | 3 +- ...stIcebergNodeLocalDynamicSplitPruning.java | 25 +-- .../TestIcebergRegisterTableProcedure.java | 10 +- .../TestIcebergTableWithCustomLocation.java | 15 +- .../TestIcebergTableWithExternalLocation.java | 13 +- ...tIcebergGlueCatalogConnectorSmokeTest.java | 3 +- .../TestIcebergGlueCreateTableFailure.java | 5 +- ...tIcebergJdbcCatalogConnectorSmokeTest.java | 6 +- ...ergTrinoRestCatalogConnectorSmokeTest.java | 6 +- 132 files changed, 924 insertions(+), 847 deletions(-) diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/FileEntry.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/FileEntry.java index 397ea27d1bc1..bfbae5ecb6e6 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/FileEntry.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/FileEntry.java @@ -26,7 +26,7 @@ import static java.util.Comparator.comparing; import static java.util.Objects.requireNonNull; -public record FileEntry(String location, long length, Instant lastModified, Optional> blocks) +public record FileEntry(Location location, long length, Instant lastModified, Optional> blocks) { public FileEntry { diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/TrinoFileSystem.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/TrinoFileSystem.java index b6343a2cb4b4..7dcb61215039 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/TrinoFileSystem.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/TrinoFileSystem.java @@ -54,7 +54,7 @@ public interface TrinoFileSystem * * @throws IllegalArgumentException if location is not valid for this file system */ - TrinoInputFile newInputFile(String location); + TrinoInputFile newInputFile(Location location); /** * Creates a TrinoInputFile with a predeclared length which can be used to read the file data. @@ -64,7 +64,7 @@ public interface TrinoFileSystem * * @throws IllegalArgumentException if location is not valid for this file system */ - TrinoInputFile newInputFile(String location, long length); + TrinoInputFile newInputFile(Location location, long length); /** * Creates a TrinoOutputFile which can be used to create or overwrite the file. The file @@ -72,7 +72,7 @@ public interface TrinoFileSystem * * @throws IllegalArgumentException if location is not valid for this file system */ - TrinoOutputFile newOutputFile(String location); + TrinoOutputFile newOutputFile(Location location); /** * Deletes the specified file. The file location path cannot be empty, and must not end with @@ -81,7 +81,7 @@ public interface TrinoFileSystem * @throws IllegalArgumentException if location is not valid for this file system * @throws IOException if the file does not exist or was not deleted. */ - void deleteFile(String location) + void deleteFile(Location location) throws IOException; /** @@ -92,10 +92,10 @@ void deleteFile(String location) * @throws IllegalArgumentException if location is not valid for this file system * @throws IOException if a file does not exist or was not deleted. */ - default void deleteFiles(Collection locations) + default void deleteFiles(Collection locations) throws IOException { - for (String location : locations) { + for (var location : locations) { deleteFile(location); } } @@ -117,7 +117,7 @@ default void deleteFiles(Collection locations) * @param location the directory to delete * @throws IllegalArgumentException if location is not valid for this file system */ - void deleteDirectory(String location) + void deleteDirectory(Location location) throws IOException; /** @@ -128,7 +128,7 @@ void deleteDirectory(String location) * * @throws IllegalArgumentException if either location is not valid for this file system */ - void renameFile(String source, String target) + void renameFile(Location source, Location target) throws IOException; /** @@ -148,6 +148,6 @@ void renameFile(String source, String target) * @param location the directory to list * @throws IllegalArgumentException if location is not valid for this file system */ - FileIterator listFiles(String location) + FileIterator listFiles(Location location) throws IOException; } diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/TrinoInputFile.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/TrinoInputFile.java index eaefaaccd664..2ec77045a9f8 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/TrinoInputFile.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/TrinoInputFile.java @@ -33,5 +33,5 @@ Instant lastModified() boolean exists() throws IOException; - String location(); + Location location(); } diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/TrinoOutputFile.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/TrinoOutputFile.java index e14dd9244dc9..2737f4b22a22 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/TrinoOutputFile.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/TrinoOutputFile.java @@ -41,5 +41,5 @@ OutputStream create(AggregatedMemoryContext memoryContext) OutputStream createOrOverwrite(AggregatedMemoryContext memoryContext) throws IOException; - String location(); + Location location(); } diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalFileIterator.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalFileIterator.java index 88545f5c7f46..31de369d43ac 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalFileIterator.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalFileIterator.java @@ -15,6 +15,7 @@ import io.trino.filesystem.FileEntry; import io.trino.filesystem.FileIterator; +import io.trino.filesystem.Location; import java.io.IOException; import java.nio.file.Files; @@ -34,7 +35,7 @@ class LocalFileIterator private final Path rootPath; private final Iterator iterator; - public LocalFileIterator(String location, Path rootPath, Path path) + public LocalFileIterator(Location location, Path rootPath, Path path) throws IOException { this.rootPath = requireNonNull(rootPath, "rootPath is null"); @@ -75,7 +76,7 @@ public FileEntry next() } return new FileEntry( - "local:///" + rootPath.relativize(path), + Location.of("local:///" + rootPath.relativize(path)), Files.size(path), Files.getLastModifiedTime(path).toInstant(), Optional.empty()); diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalFileSystem.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalFileSystem.java index aadd87f64fa1..791cfe75e89d 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalFileSystem.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalFileSystem.java @@ -45,25 +45,25 @@ public LocalFileSystem(Path rootPath) } @Override - public TrinoInputFile newInputFile(String location) + public TrinoInputFile newInputFile(Location location) { return new LocalInputFile(location, toFilePath(location)); } @Override - public TrinoInputFile newInputFile(String location, long length) + public TrinoInputFile newInputFile(Location location, long length) { return new LocalInputFile(location, toFilePath(location), length); } @Override - public TrinoOutputFile newOutputFile(String location) + public TrinoOutputFile newOutputFile(Location location) { return new LocalOutputFile(location, toFilePath(location)); } @Override - public void deleteFile(String location) + public void deleteFile(Location location) throws IOException { Path filePath = toFilePath(location); @@ -76,7 +76,7 @@ public void deleteFile(String location) } @Override - public void deleteDirectory(String location) + public void deleteDirectory(Location location) throws IOException { Path directoryPath = toDirectoryPath(location); @@ -121,7 +121,7 @@ public FileVisitResult postVisitDirectory(Path directory, IOException exception) } @Override - public void renameFile(String source, String target) + public void renameFile(Location source, Location target) throws IOException { Path sourcePath = toFilePath(source); @@ -145,45 +145,42 @@ public void renameFile(String source, String target) } @Override - public FileIterator listFiles(String location) + public FileIterator listFiles(Location location) throws IOException { return new LocalFileIterator(location, rootPath, toDirectoryPath(location)); } - private Path toFilePath(String fileLocation) + private Path toFilePath(Location location) { - Location location = parseLocalLocation(fileLocation); + validateLocalLocation(location); location.verifyValidFileLocation(); - Path localPath = toPath(fileLocation, location); + Path localPath = toPath(location); // local file path can not be empty as this would create a file for the root entry - checkArgument(!localPath.equals(rootPath), "Local file location must contain a path: %s", fileLocation); + checkArgument(!localPath.equals(rootPath), "Local file location must contain a path: %s", localPath); return localPath; } - private Path toDirectoryPath(String directoryLocation) + private Path toDirectoryPath(Location location) { - Location location = parseLocalLocation(directoryLocation); - Path localPath = toPath(directoryLocation, location); - return localPath; + validateLocalLocation(location); + return toPath(location); } - private static Location parseLocalLocation(String locationString) + private static void validateLocalLocation(Location location) { - Location location = Location.of(locationString); - checkArgument(location.scheme().equals(Optional.of("local")), "Only 'local' scheme is supported: %s", locationString); - checkArgument(location.userInfo().isEmpty(), "Local location cannot contain user info: %s", locationString); - checkArgument(location.host().isEmpty(), "Local location cannot contain a host: %s", locationString); - return location; + checkArgument(location.scheme().equals(Optional.of("local")), "Only 'local' scheme is supported: %s", location); + checkArgument(location.userInfo().isEmpty(), "Local location cannot contain user info: %s", location); + checkArgument(location.host().isEmpty(), "Local location cannot contain a host: %s", location); } - private Path toPath(String locationString, Location location) + private Path toPath(Location location) { // ensure path isn't something like '../../data' Path localPath = rootPath.resolve(location.path()).normalize(); - checkArgument(localPath.startsWith(rootPath), "Location references data outside of the root: %s", locationString); + checkArgument(localPath.startsWith(rootPath), "Location references data outside of the root: %s", location); return localPath; } } diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalInput.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalInput.java index dfcae5bb1d65..7d9d3a2a258e 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalInput.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalInput.java @@ -13,6 +13,7 @@ */ package io.trino.filesystem.local; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoInput; import java.io.EOFException; @@ -28,12 +29,12 @@ class LocalInput implements TrinoInput { - private final String location; + private final Location location; private final File file; private final RandomAccessFile input; private boolean closed; - public LocalInput(String location, File file) + public LocalInput(Location location, File file) throws IOException { this.location = requireNonNull(location, "location is null"); diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalInputFile.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalInputFile.java index 8031bf517e29..90c571917900 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalInputFile.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalInputFile.java @@ -13,6 +13,7 @@ */ package io.trino.filesystem.local; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoInput; import io.trino.filesystem.TrinoInputFile; import io.trino.filesystem.TrinoInputStream; @@ -33,18 +34,18 @@ public class LocalInputFile implements TrinoInputFile { - private final String location; + private final Location location; private final Path path; private OptionalLong length = OptionalLong.empty(); private Optional lastModified = Optional.empty(); - public LocalInputFile(String location, Path path) + public LocalInputFile(Location location, Path path) { this.location = requireNonNull(location, "location is null"); this.path = requireNonNull(path, "path is null"); } - public LocalInputFile(String location, Path path, long length) + public LocalInputFile(Location location, Path path, long length) { this.location = requireNonNull(location, "location is null"); this.path = requireNonNull(path, "path is null"); @@ -54,7 +55,7 @@ public LocalInputFile(String location, Path path, long length) public LocalInputFile(File file) { - this(file.getPath(), file.toPath()); + this(Location.of(file.toURI().toString()), file.toPath()); } @Override @@ -65,7 +66,7 @@ public TrinoInput newInput() return new LocalInput(location, path.toFile()); } catch (IOException e) { - throw new FileNotFoundException(location); + throw new FileNotFoundException(location.toString()); } } @@ -77,7 +78,7 @@ public TrinoInputStream newStream() return new LocalInputStream(location, path.toFile()); } catch (FileNotFoundException e) { - throw new FileNotFoundException(location); + throw new FileNotFoundException(location.toString()); } } @@ -119,7 +120,7 @@ public boolean exists() } @Override - public String location() + public Location location() { return location; } @@ -127,6 +128,6 @@ public String location() @Override public String toString() { - return location(); + return location.toString(); } } diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalInputStream.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalInputStream.java index 8d35f54008ff..0107daaae5b0 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalInputStream.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalInputStream.java @@ -15,6 +15,7 @@ import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoInputStream; import java.io.BufferedInputStream; @@ -30,7 +31,7 @@ class LocalInputStream extends TrinoInputStream { - private final String location; + private final Location location; private final File file; private final long fileLength; @@ -38,7 +39,7 @@ class LocalInputStream private long position; private boolean closed; - public LocalInputStream(String location, File file) + public LocalInputStream(Location location, File file) throws FileNotFoundException { this.location = requireNonNull(location, "location is null"); diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalOutputFile.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalOutputFile.java index dfab68d6e583..87deb5fd11ff 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalOutputFile.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalOutputFile.java @@ -13,6 +13,7 @@ */ package io.trino.filesystem.local; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoOutputFile; import io.trino.memory.context.AggregatedMemoryContext; @@ -29,10 +30,10 @@ public class LocalOutputFile implements TrinoOutputFile { - private final String location; + private final Location location; private final Path path; - public LocalOutputFile(String location, Path path) + public LocalOutputFile(Location location, Path path) { this.location = requireNonNull(location, "location is null"); this.path = requireNonNull(path, "path is null"); @@ -40,7 +41,7 @@ public LocalOutputFile(String location, Path path) public LocalOutputFile(File file) { - this(file.getPath(), file.toPath()); + this(Location.of(file.toURI().toString()), file.toPath()); } @Override @@ -72,8 +73,14 @@ public OutputStream createOrOverwrite(AggregatedMemoryContext memoryContext) } @Override - public String location() + public Location location() { return location; } + + @Override + public String toString() + { + return location.toString(); + } } diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalOutputStream.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalOutputStream.java index e22d3df4bbd0..11ba499c0f05 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalOutputStream.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalOutputStream.java @@ -13,6 +13,8 @@ */ package io.trino.filesystem.local; +import io.trino.filesystem.Location; + import java.io.BufferedOutputStream; import java.io.IOException; import java.io.OutputStream; @@ -24,11 +26,11 @@ class LocalOutputStream extends OutputStream { - private final String location; + private final Location location; private final OutputStream stream; private boolean closed; - public LocalOutputStream(String location, OutputStream stream) + public LocalOutputStream(Location location, OutputStream stream) { this.location = requireNonNull(location, "location is null"); this.stream = new BufferedOutputStream(requireNonNull(stream, "stream is null"), 4 * 1024); diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalUtils.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalUtils.java index 3e0cd6974b02..158351e34a46 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalUtils.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/local/LocalUtils.java @@ -13,6 +13,8 @@ */ package io.trino.filesystem.local; +import io.trino.filesystem.Location; + import java.io.IOException; import java.nio.file.FileAlreadyExistsException; import java.nio.file.NoSuchFileException; @@ -21,14 +23,14 @@ final class LocalUtils { private LocalUtils() {} - static IOException handleException(String location, IOException exception) + static IOException handleException(Location location, IOException exception) throws IOException { if (exception instanceof NoSuchFileException) { - throw new NoSuchFileException(location); + throw new NoSuchFileException(location.toString()); } if (exception instanceof FileAlreadyExistsException) { - throw new FileAlreadyExistsException(location); + throw new FileAlreadyExistsException(location.toString()); } throw new IOException(exception.getMessage() + ": " + location, exception); } diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryFileSystem.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryFileSystem.java index 2958a7b436e7..4612be0b790f 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryFileSystem.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryFileSystem.java @@ -47,21 +47,21 @@ boolean isEmpty() } @Override - public TrinoInputFile newInputFile(String location) + public TrinoInputFile newInputFile(Location location) { String key = toBlobKey(location); return new MemoryInputFile(location, () -> blobs.get(key), OptionalLong.empty()); } @Override - public TrinoInputFile newInputFile(String location, long length) + public TrinoInputFile newInputFile(Location location, long length) { String key = toBlobKey(location); return new MemoryInputFile(location, () -> blobs.get(key), OptionalLong.of(length)); } @Override - public TrinoOutputFile newOutputFile(String location) + public TrinoOutputFile newOutputFile(Location location) { String key = toBlobKey(location); OutputBlob outputBlob = new OutputBlob() @@ -77,7 +77,7 @@ public void createBlob(Slice data) throws FileAlreadyExistsException { if (blobs.putIfAbsent(key, new MemoryBlob(data)) != null) { - throw new FileAlreadyExistsException(location); + throw new FileAlreadyExistsException(location.toString()); } } @@ -91,16 +91,16 @@ public void overwriteBlob(Slice data) } @Override - public void deleteFile(String location) + public void deleteFile(Location location) throws IOException { if (blobs.remove(toBlobKey(location)) == null) { - throw new NoSuchFileException(location); + throw new NoSuchFileException(location.toString()); } } @Override - public void deleteDirectory(String location) + public void deleteDirectory(Location location) throws IOException { String prefix = toBlobPrefix(location); @@ -108,7 +108,7 @@ public void deleteDirectory(String location) } @Override - public void renameFile(String source, String target) + public void renameFile(Location source, Location target) throws IOException { String sourceKey = toBlobKey(source); @@ -126,14 +126,14 @@ public void renameFile(String source, String target) } @Override - public FileIterator listFiles(String location) + public FileIterator listFiles(Location location) throws IOException { String prefix = toBlobPrefix(location); Iterator iterator = blobs.entrySet().stream() .filter(entry -> entry.getKey().startsWith(prefix)) .map(entry -> new FileEntry( - "memory:///" + entry.getKey(), + Location.of("memory:///" + entry.getKey()), entry.getValue().data().length(), entry.getValue().lastModified(), Optional.empty())) @@ -154,28 +154,27 @@ public FileEntry next() }; } - private static String toBlobKey(String locationString) + private static String toBlobKey(Location location) { - Location location = parseMemoryLocation(locationString); + validateMemoryLocation(location); location.verifyValidFileLocation(); return location.path(); } - private static String toBlobPrefix(String location) + private static String toBlobPrefix(Location location) { - String directoryPath = parseMemoryLocation(location).path(); + validateMemoryLocation(location); + String directoryPath = location.path(); if (!directoryPath.isEmpty() && !directoryPath.endsWith("/")) { directoryPath += "/"; } return directoryPath; } - private static Location parseMemoryLocation(String locationString) + private static void validateMemoryLocation(Location location) { - Location location = Location.of(locationString); - checkArgument(location.scheme().equals(Optional.of("memory")), "Only 'memory' scheme is supported: %s", locationString); - checkArgument(location.userInfo().isEmpty(), "Memory location cannot contain user info: %s", locationString); - checkArgument(location.host().isEmpty(), "Memory location cannot contain a host: %s", locationString); - return location; + checkArgument(location.scheme().equals(Optional.of("memory")), "Only 'memory' scheme is supported: %s", location); + checkArgument(location.userInfo().isEmpty(), "Memory location cannot contain user info: %s", location); + checkArgument(location.host().isEmpty(), "Memory location cannot contain a host: %s", location); } } diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryInput.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryInput.java index 7deec0e9dc94..029a85b9beab 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryInput.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryInput.java @@ -14,6 +14,7 @@ package io.trino.filesystem.memory; import io.airlift.slice.Slice; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoInput; import java.io.EOFException; @@ -27,11 +28,11 @@ class MemoryInput implements TrinoInput { - private final String location; + private final Location location; private final Slice data; private boolean closed; - public MemoryInput(String location, Slice data) + public MemoryInput(Location location, Slice data) { this.location = requireNonNull(location, "location is null"); this.data = requireNonNull(data, "data is null"); @@ -82,6 +83,6 @@ public void close() @Override public String toString() { - return location; + return location.toString(); } } diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryInputFile.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryInputFile.java index 5702b828e97d..deb0dba92347 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryInputFile.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryInputFile.java @@ -14,6 +14,7 @@ package io.trino.filesystem.memory; import io.airlift.slice.Slice; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoInput; import io.trino.filesystem.TrinoInputFile; import io.trino.filesystem.TrinoInputStream; @@ -30,17 +31,17 @@ public class MemoryInputFile implements TrinoInputFile { - private final String location; + private final Location location; private final Supplier dataSupplier; private OptionalLong length; private Optional lastModified = Optional.empty(); - public MemoryInputFile(String location, Slice data) + public MemoryInputFile(Location location, Slice data) { this(location, () -> new MemoryBlob(data), OptionalLong.of(data.length())); } - public MemoryInputFile(String location, Supplier dataSupplier, OptionalLong length) + public MemoryInputFile(Location location, Supplier dataSupplier, OptionalLong length) { this.location = requireNonNull(location, "location is null"); this.dataSupplier = requireNonNull(dataSupplier, "dataSupplier is null"); @@ -89,7 +90,7 @@ public boolean exists() } @Override - public String location() + public Location location() { return location; } @@ -97,7 +98,7 @@ public String location() @Override public String toString() { - return location(); + return location.toString(); } private MemoryBlob getBlobRequired() @@ -105,7 +106,7 @@ private MemoryBlob getBlobRequired() { MemoryBlob data = dataSupplier.get(); if (data == null) { - throw new NoSuchFileException(location); + throw new NoSuchFileException(toString()); } return data; } diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryInputStream.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryInputStream.java index 487968690897..573093a9a14a 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryInputStream.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryInputStream.java @@ -15,6 +15,7 @@ import io.airlift.slice.Slice; import io.airlift.slice.SliceInput; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoInputStream; import java.io.IOException; @@ -24,12 +25,12 @@ class MemoryInputStream extends TrinoInputStream { - private final String location; + private final Location location; private final SliceInput input; private final int length; private boolean closed; - public MemoryInputStream(String location, Slice data) + public MemoryInputStream(Location location, Slice data) { this.location = requireNonNull(location, "location is null"); this.input = requireNonNull(data, "data is null").getInput(); diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryOutputFile.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryOutputFile.java index b546e0c084d6..fd10e1ecfb25 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryOutputFile.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryOutputFile.java @@ -14,6 +14,7 @@ package io.trino.filesystem.memory; import io.airlift.slice.Slice; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoOutputFile; import io.trino.memory.context.AggregatedMemoryContext; @@ -36,10 +37,10 @@ void createBlob(Slice data) void overwriteBlob(Slice data); } - private final String location; + private final Location location; private final OutputBlob outputBlob; - public MemoryOutputFile(String location, OutputBlob outputBlob) + public MemoryOutputFile(Location location, OutputBlob outputBlob) { this.location = requireNonNull(location, "location is null"); this.outputBlob = requireNonNull(outputBlob, "outputBlob is null"); @@ -50,7 +51,7 @@ public OutputStream create(AggregatedMemoryContext memoryContext) throws IOException { if (outputBlob.exists()) { - throw new FileAlreadyExistsException(location); + throw new FileAlreadyExistsException(toString()); } return new MemoryOutputStream(location, outputBlob::createBlob); } @@ -63,8 +64,14 @@ public OutputStream createOrOverwrite(AggregatedMemoryContext memoryContext) } @Override - public String location() + public Location location() { return location; } + + @Override + public String toString() + { + return location.toString(); + } } diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryOutputStream.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryOutputStream.java index c252f1bf1f9e..1c6b13653fe5 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryOutputStream.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/memory/MemoryOutputStream.java @@ -15,6 +15,7 @@ import io.airlift.slice.Slice; import io.airlift.slice.Slices; +import io.trino.filesystem.Location; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -32,11 +33,11 @@ void onClose(Slice data) throws IOException; } - private final String location; + private final Location location; private final OnStreamClose onStreamClose; private ByteArrayOutputStream stream = new ByteArrayOutputStream(); - public MemoryOutputStream(String location, OnStreamClose onStreamClose) + public MemoryOutputStream(Location location, OnStreamClose onStreamClose) { this.location = requireNonNull(location, "location is null"); this.onStreamClose = requireNonNull(onStreamClose, "onStreamClose is null"); diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/tracing/TracingFileSystem.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/tracing/TracingFileSystem.java index 0dbe7788285b..bd392ed9250d 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/tracing/TracingFileSystem.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/tracing/TracingFileSystem.java @@ -16,6 +16,7 @@ import io.opentelemetry.api.trace.Span; import io.opentelemetry.api.trace.Tracer; import io.trino.filesystem.FileIterator; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoInputFile; import io.trino.filesystem.TrinoOutputFile; @@ -40,35 +41,35 @@ public TracingFileSystem(Tracer tracer, TrinoFileSystem delegate) } @Override - public TrinoInputFile newInputFile(String location) + public TrinoInputFile newInputFile(Location location) { return new TracingInputFile(tracer, delegate.newInputFile(location), Optional.empty()); } @Override - public TrinoInputFile newInputFile(String location, long length) + public TrinoInputFile newInputFile(Location location, long length) { return new TracingInputFile(tracer, delegate.newInputFile(location, length), Optional.of(length)); } @Override - public TrinoOutputFile newOutputFile(String location) + public TrinoOutputFile newOutputFile(Location location) { return new TracingOutputFile(tracer, delegate.newOutputFile(location)); } @Override - public void deleteFile(String location) + public void deleteFile(Location location) throws IOException { Span span = tracer.spanBuilder("FileSystem.deleteFile") - .setAttribute(FileSystemAttributes.FILE_LOCATION, location) + .setAttribute(FileSystemAttributes.FILE_LOCATION, location.toString()) .startSpan(); withTracing(span, () -> delegate.deleteFile(location)); } @Override - public void deleteFiles(Collection locations) + public void deleteFiles(Collection locations) throws IOException { Span span = tracer.spanBuilder("FileSystem.deleteFiles") @@ -78,31 +79,31 @@ public void deleteFiles(Collection locations) } @Override - public void deleteDirectory(String location) + public void deleteDirectory(Location location) throws IOException { Span span = tracer.spanBuilder("FileSystem.deleteDirectory") - .setAttribute(FileSystemAttributes.FILE_LOCATION, location) + .setAttribute(FileSystemAttributes.FILE_LOCATION, location.toString()) .startSpan(); withTracing(span, () -> delegate.deleteDirectory(location)); } @Override - public void renameFile(String source, String target) + public void renameFile(Location source, Location target) throws IOException { Span span = tracer.spanBuilder("FileSystem.renameFile") - .setAttribute(FileSystemAttributes.FILE_LOCATION, source) + .setAttribute(FileSystemAttributes.FILE_LOCATION, source.toString()) .startSpan(); withTracing(span, () -> delegate.renameFile(source, target)); } @Override - public FileIterator listFiles(String location) + public FileIterator listFiles(Location location) throws IOException { Span span = tracer.spanBuilder("FileSystem.listFiles") - .setAttribute(FileSystemAttributes.FILE_LOCATION, location) + .setAttribute(FileSystemAttributes.FILE_LOCATION, location.toString()) .startSpan(); return withTracing(span, () -> delegate.listFiles(location)); } diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/tracing/TracingInput.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/tracing/TracingInput.java index 91e766c0a0ee..dac17b1ac126 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/tracing/TracingInput.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/tracing/TracingInput.java @@ -17,6 +17,7 @@ import io.opentelemetry.api.trace.Span; import io.opentelemetry.api.trace.SpanBuilder; import io.opentelemetry.api.trace.Tracer; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoInput; import java.io.IOException; @@ -31,10 +32,10 @@ final class TracingInput { private final Tracer tracer; private final TrinoInput delegate; - private final String location; + private final Location location; private final Optional fileLength; - public TracingInput(Tracer tracer, TrinoInput delegate, String location, Optional fileLength) + public TracingInput(Tracer tracer, TrinoInput delegate, Location location, Optional fileLength) { this.tracer = requireNonNull(tracer, "tracer is null"); this.delegate = requireNonNull(delegate, "delegate is null"); @@ -87,10 +88,16 @@ public void close() delegate.close(); } + @Override + public String toString() + { + return location.toString(); + } + private SpanBuilder spanBuilder(String name, long readLength) { return tracer.spanBuilder(name) - .setAttribute(FileSystemAttributes.FILE_LOCATION, location) + .setAttribute(FileSystemAttributes.FILE_LOCATION, toString()) .setAllAttributes(attribute(FileSystemAttributes.FILE_SIZE, fileLength)) .setAttribute(FileSystemAttributes.FILE_READ_SIZE, readLength); } diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/tracing/TracingInputFile.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/tracing/TracingInputFile.java index 2a3a0df65108..9e9581082903 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/tracing/TracingInputFile.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/tracing/TracingInputFile.java @@ -15,6 +15,7 @@ import io.opentelemetry.api.trace.Span; import io.opentelemetry.api.trace.Tracer; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoInput; import io.trino.filesystem.TrinoInputFile; import io.trino.filesystem.TrinoInputStream; @@ -46,7 +47,7 @@ public TrinoInput newInput() throws IOException { Span span = tracer.spanBuilder("InputFile.newInput") - .setAttribute(FileSystemAttributes.FILE_LOCATION, location()) + .setAttribute(FileSystemAttributes.FILE_LOCATION, toString()) .setAllAttributes(attribute(FileSystemAttributes.FILE_SIZE, length)) .startSpan(); return withTracing(span, () -> new TracingInput(tracer, delegate.newInput(), location(), length)); @@ -57,7 +58,7 @@ public TrinoInputStream newStream() throws IOException { Span span = tracer.spanBuilder("InputFile.newStream") - .setAttribute(FileSystemAttributes.FILE_LOCATION, location()) + .setAttribute(FileSystemAttributes.FILE_LOCATION, toString()) .setAllAttributes(attribute(FileSystemAttributes.FILE_SIZE, length)) .startSpan(); return withTracing(span, delegate::newStream); @@ -73,7 +74,7 @@ public long length() } Span span = tracer.spanBuilder("InputFile.length") - .setAttribute(FileSystemAttributes.FILE_LOCATION, location()) + .setAttribute(FileSystemAttributes.FILE_LOCATION, toString()) .startSpan(); return withTracing(span, delegate::length); } @@ -83,7 +84,7 @@ public Instant lastModified() throws IOException { Span span = tracer.spanBuilder("InputFile.lastModified") - .setAttribute(FileSystemAttributes.FILE_LOCATION, location()) + .setAttribute(FileSystemAttributes.FILE_LOCATION, toString()) .startSpan(); return withTracing(span, delegate::lastModified); } @@ -93,14 +94,20 @@ public boolean exists() throws IOException { Span span = tracer.spanBuilder("InputFile.exists") - .setAttribute(FileSystemAttributes.FILE_LOCATION, location()) + .setAttribute(FileSystemAttributes.FILE_LOCATION, toString()) .startSpan(); return withTracing(span, delegate::exists); } @Override - public String location() + public Location location() { return delegate.location(); } + + @Override + public String toString() + { + return location().toString(); + } } diff --git a/lib/trino-filesystem/src/main/java/io/trino/filesystem/tracing/TracingOutputFile.java b/lib/trino-filesystem/src/main/java/io/trino/filesystem/tracing/TracingOutputFile.java index f815dc2c6d76..2a3941c55266 100644 --- a/lib/trino-filesystem/src/main/java/io/trino/filesystem/tracing/TracingOutputFile.java +++ b/lib/trino-filesystem/src/main/java/io/trino/filesystem/tracing/TracingOutputFile.java @@ -15,6 +15,7 @@ import io.opentelemetry.api.trace.Span; import io.opentelemetry.api.trace.Tracer; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoOutputFile; import io.trino.memory.context.AggregatedMemoryContext; @@ -41,7 +42,7 @@ public OutputStream create() throws IOException { Span span = tracer.spanBuilder("OutputFile.create") - .setAttribute(FileSystemAttributes.FILE_LOCATION, location()) + .setAttribute(FileSystemAttributes.FILE_LOCATION, toString()) .startSpan(); return withTracing(span, () -> delegate.create()); } @@ -51,7 +52,7 @@ public OutputStream createOrOverwrite() throws IOException { Span span = tracer.spanBuilder("OutputFile.createOrOverwrite") - .setAttribute(FileSystemAttributes.FILE_LOCATION, location()) + .setAttribute(FileSystemAttributes.FILE_LOCATION, toString()) .startSpan(); return withTracing(span, () -> delegate.createOrOverwrite()); } @@ -61,7 +62,7 @@ public OutputStream create(AggregatedMemoryContext memoryContext) throws IOException { Span span = tracer.spanBuilder("OutputFile.create") - .setAttribute(FileSystemAttributes.FILE_LOCATION, location()) + .setAttribute(FileSystemAttributes.FILE_LOCATION, toString()) .startSpan(); return withTracing(span, () -> delegate.create(memoryContext)); } @@ -71,14 +72,20 @@ public OutputStream createOrOverwrite(AggregatedMemoryContext memoryContext) throws IOException { Span span = tracer.spanBuilder("OutputFile.createOrOverwrite") - .setAttribute(FileSystemAttributes.FILE_LOCATION, location()) + .setAttribute(FileSystemAttributes.FILE_LOCATION, toString()) .startSpan(); return withTracing(span, () -> delegate.createOrOverwrite(memoryContext)); } @Override - public String location() + public Location location() { return delegate.location(); } + + @Override + public String toString() + { + return location().toString(); + } } diff --git a/lib/trino-filesystem/src/test/java/io/trino/filesystem/AbstractTestTrinoFileSystem.java b/lib/trino-filesystem/src/test/java/io/trino/filesystem/AbstractTestTrinoFileSystem.java index ec05f1cc2aa3..ca688dc63a09 100644 --- a/lib/trino-filesystem/src/test/java/io/trino/filesystem/AbstractTestTrinoFileSystem.java +++ b/lib/trino-filesystem/src/test/java/io/trino/filesystem/AbstractTestTrinoFileSystem.java @@ -49,16 +49,16 @@ public abstract class AbstractTestTrinoFileSystem protected abstract TrinoFileSystem getFileSystem(); - protected abstract String getRootLocation(); + protected abstract Location getRootLocation(); protected abstract void verifyFileSystemIsEmpty(); - protected String createLocation(String path) + protected Location createLocation(String path) { if (path.isEmpty()) { return getRootLocation(); } - return getRootLocation() + "/" + path; + return getRootLocation().appendPath(path); } @BeforeEach @@ -74,21 +74,21 @@ void testInputFileMetadata() // an input file cannot be created at the root of the file system assertThatThrownBy(() -> getFileSystem().newInputFile(getRootLocation())) .isInstanceOf(IllegalStateException.class) - .hasMessageContaining(getRootLocation()); - assertThatThrownBy(() -> getFileSystem().newInputFile(getRootLocation() + "/")) + .hasMessageContaining(getRootLocation().toString()); + assertThatThrownBy(() -> getFileSystem().newInputFile(Location.of(getRootLocation() + "/"))) .isInstanceOf(IllegalStateException.class) - .hasMessageContaining(getRootLocation() + "/"); + .hasMessageContaining(getRootLocation().toString() + "/"); // an input file location cannot end with a slash assertThatThrownBy(() -> getFileSystem().newInputFile(createLocation("foo/"))) .isInstanceOf(IllegalStateException.class) - .hasMessageContaining(createLocation("foo/")); + .hasMessageContaining(createLocation("foo/").toString()); // an input file location cannot end with whitespace assertThatThrownBy(() -> getFileSystem().newInputFile(createLocation("foo "))) .isInstanceOf(IllegalStateException.class) - .hasMessageContaining(createLocation("foo ")); + .hasMessageContaining(createLocation("foo ").toString()); assertThatThrownBy(() -> getFileSystem().newInputFile(createLocation("foo\t"))) .isInstanceOf(IllegalStateException.class) - .hasMessageContaining(createLocation("foo\t")); + .hasMessageContaining(createLocation("foo\t").toString()); try (TempBlob tempBlob = randomBlobLocation("inputFileMetadata")) { TrinoInputFile inputFile = getFileSystem().newInputFile(tempBlob.location()); @@ -98,10 +98,10 @@ void testInputFileMetadata() // getting length or modified time of non-existent file is an error assertThatThrownBy(inputFile::length) .isInstanceOf(NoSuchFileException.class) - .hasMessageContaining(tempBlob.location()); + .hasMessageContaining(tempBlob.location().toString()); assertThatThrownBy(inputFile::lastModified) .isInstanceOf(NoSuchFileException.class) - .hasMessageContaining(tempBlob.location()); + .hasMessageContaining(tempBlob.location().toString()); tempBlob.createOrOverwrite("123456"); @@ -125,21 +125,21 @@ void testInputFileWithLengthMetadata() // an input file cannot be created at the root of the file system assertThatThrownBy(() -> getFileSystem().newInputFile(getRootLocation(), 22)) .isInstanceOf(IllegalStateException.class) - .hasMessageContaining(getRootLocation()); - assertThatThrownBy(() -> getFileSystem().newInputFile(getRootLocation() + "/", 22)) + .hasMessageContaining(getRootLocation().toString()); + assertThatThrownBy(() -> getFileSystem().newInputFile(Location.of(getRootLocation() + "/"), 22)) .isInstanceOf(IllegalStateException.class) .hasMessageContaining(getRootLocation() + "/"); // an input file location cannot end with a slash assertThatThrownBy(() -> getFileSystem().newInputFile(createLocation("foo/"), 22)) .isInstanceOf(IllegalStateException.class) - .hasMessageContaining(createLocation("foo/")); + .hasMessageContaining(createLocation("foo/").toString()); // an input file location cannot end with whitespace assertThatThrownBy(() -> getFileSystem().newInputFile(createLocation("foo "), 22)) .isInstanceOf(IllegalStateException.class) - .hasMessageContaining(createLocation("foo ")); + .hasMessageContaining(createLocation("foo ").toString()); assertThatThrownBy(() -> getFileSystem().newInputFile(createLocation("foo\t"), 22)) .isInstanceOf(IllegalStateException.class) - .hasMessageContaining(createLocation("foo\t")); + .hasMessageContaining(createLocation("foo\t").toString()); try (TempBlob tempBlob = randomBlobLocation("inputFileWithLengthMetadata")) { TrinoInputFile inputFile = getFileSystem().newInputFile(tempBlob.location(), 22); @@ -150,7 +150,7 @@ void testInputFileWithLengthMetadata() // modified time of non-existent file is an error assertThatThrownBy(inputFile::lastModified) .isInstanceOf(NoSuchFileException.class) - .hasMessageContaining(tempBlob.location()); + .hasMessageContaining(tempBlob.location().toString()); // double-check the length did not change in call above assertThat(inputFile.length()).isEqualTo(22); @@ -263,7 +263,7 @@ void testInputFile() currentPosition++; assertThatThrownBy(() -> inputStream.seek(fileSize + 100)) .isInstanceOf(IOException.class) - .hasMessageContaining(tempBlob.location()); + .hasMessageContaining(tempBlob.location().toString()); assertThat(inputStream.getPosition()).isEqualTo(currentPosition); assertThat(inputStream.read()).isGreaterThanOrEqualTo(0); assertThat(inputStream.getPosition()).isEqualTo(currentPosition + 1); @@ -272,19 +272,19 @@ void testInputFile() inputStream.close(); assertThatThrownBy(inputStream::available) .isInstanceOf(IOException.class) - .hasMessageContaining(tempBlob.location()); + .hasMessageContaining(tempBlob.location().toString()); assertThatThrownBy(() -> inputStream.seek(0)) .isInstanceOf(IOException.class) - .hasMessageContaining(tempBlob.location()); + .hasMessageContaining(tempBlob.location().toString()); assertThatThrownBy(inputStream::read) .isInstanceOf(IOException.class) - .hasMessageContaining(tempBlob.location()); + .hasMessageContaining(tempBlob.location().toString()); assertThatThrownBy(() -> inputStream.read(new byte[10])) .isInstanceOf(IOException.class) - .hasMessageContaining(tempBlob.location()); + .hasMessageContaining(tempBlob.location().toString()); assertThatThrownBy(() -> inputStream.read(new byte[10], 2, 3)) .isInstanceOf(IOException.class) - .hasMessageContaining(tempBlob.location()); + .hasMessageContaining(tempBlob.location().toString()); } try (TrinoInput trinoInput = inputFile.newInput()) { @@ -310,7 +310,7 @@ void testInputFile() assertThat(trinoInput.readFully(MEGABYTE, bytes.length)).isEqualTo(Slices.wrappedBuffer(bytes)); assertThatThrownBy(() -> trinoInput.readFully(fileSize - bytes.length + 1, bytes, 0, bytes.length)) .isInstanceOf(IOException.class) - .hasMessageContaining(tempBlob.location()); + .hasMessageContaining(tempBlob.location().toString()); // tail read trinoInput.readTail(bytes, 0, bytes.length); @@ -330,16 +330,16 @@ void testInputFile() trinoInput.close(); assertThatThrownBy(() -> trinoInput.readFully(0, 10)) .isInstanceOf(IOException.class) - .hasMessageContaining(tempBlob.location()); + .hasMessageContaining(tempBlob.location().toString()); assertThatThrownBy(() -> trinoInput.readFully(0, bytes, 0, 10)) .isInstanceOf(IOException.class) - .hasMessageContaining(tempBlob.location()); + .hasMessageContaining(tempBlob.location().toString()); assertThatThrownBy(() -> trinoInput.readTail(10)) .isInstanceOf(IOException.class) - .hasMessageContaining(tempBlob.location()); + .hasMessageContaining(tempBlob.location().toString()); assertThatThrownBy(() -> trinoInput.readTail(bytes, 0, 10)) .isInstanceOf(IOException.class) - .hasMessageContaining(tempBlob.location()); + .hasMessageContaining(tempBlob.location().toString()); } } } @@ -351,21 +351,21 @@ void testOutputFile() // an output file cannot be created at the root of the file system assertThatThrownBy(() -> getFileSystem().newOutputFile(getRootLocation())) .isInstanceOf(IllegalStateException.class) - .hasMessageContaining(getRootLocation()); - assertThatThrownBy(() -> getFileSystem().newOutputFile(getRootLocation() + "/")) + .hasMessageContaining(getRootLocation().toString()); + assertThatThrownBy(() -> getFileSystem().newOutputFile(Location.of(getRootLocation() + "/"))) .isInstanceOf(IllegalStateException.class) .hasMessageContaining(getRootLocation() + "/"); // an output file location cannot end with a slash assertThatThrownBy(() -> getFileSystem().newOutputFile(createLocation("foo/"))) .isInstanceOf(IllegalStateException.class) - .hasMessageContaining(createLocation("foo/")); + .hasMessageContaining(createLocation("foo/").toString()); // an output file location cannot end with whitespace assertThatThrownBy(() -> getFileSystem().newOutputFile(createLocation("foo "))) .isInstanceOf(IllegalStateException.class) - .hasMessageContaining(createLocation("foo ")); + .hasMessageContaining(createLocation("foo ").toString()); assertThatThrownBy(() -> getFileSystem().newOutputFile(createLocation("foo\t"))) .isInstanceOf(IllegalStateException.class) - .hasMessageContaining(createLocation("foo\t")); + .hasMessageContaining(createLocation("foo\t").toString()); try (TempBlob tempBlob = randomBlobLocation("outputFile")) { TrinoOutputFile outputFile = getFileSystem().newOutputFile(tempBlob.location()); @@ -380,7 +380,7 @@ void testOutputFile() // re-create without overwrite is an error assertThatThrownBy(outputFile::create) .isInstanceOf(FileAlreadyExistsException.class) - .hasMessageContaining(tempBlob.location()); + .hasMessageContaining(tempBlob.location().toString()); // verify nothing changed assertThat(tempBlob.read()).isEqualTo("initial"); @@ -412,16 +412,16 @@ void testOutputStreamByteAtATime() // verify all the methods throw after close assertThatThrownBy(() -> outputStream.write(42)) .isInstanceOf(IOException.class) - .hasMessageContaining(tempBlob.location()); + .hasMessageContaining(tempBlob.location().toString()); assertThatThrownBy(() -> outputStream.write(new byte[10])) .isInstanceOf(IOException.class) - .hasMessageContaining(tempBlob.location()); + .hasMessageContaining(tempBlob.location().toString()); assertThatThrownBy(() -> outputStream.write(new byte[10], 1, 3)) .isInstanceOf(IOException.class) - .hasMessageContaining(tempBlob.location()); + .hasMessageContaining(tempBlob.location().toString()); assertThatThrownBy(outputStream::flush) .isInstanceOf(IOException.class) - .hasMessageContaining(tempBlob.location()); + .hasMessageContaining(tempBlob.location().toString()); } try (TrinoInputStream inputStream = tempBlob.inputFile().newStream()) { @@ -453,15 +453,15 @@ protected void testPathHierarchical() // the check is over the entire statement, because some file system delay path checks until the data is uploaded assertThatThrownBy(() -> getFileSystem().newOutputFile(createLocation("../file")).createOrOverwrite().close()) .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining(createLocation("../file")); + .hasMessageContaining(createLocation("../file").toString()); try (TempBlob absolute = new TempBlob(createLocation("b"))) { try (TempBlob alias = new TempBlob(createLocation("a/../b"))) { - absolute.createOrOverwrite(absolute.location()); + absolute.createOrOverwrite(absolute.location().toString()); assertThat(alias.exists()).isTrue(); assertThat(absolute.exists()).isTrue(); - assertThat(alias.read()).isEqualTo(absolute.location()); + assertThat(alias.read()).isEqualTo(absolute.location().toString()); assertThat(listPath("")).containsExactly(absolute.location()); @@ -480,19 +480,19 @@ protected void testPathBlob() assertThat(inputFile.location()).isEqualTo(tempBlob.location()); assertThat(inputFile.exists()).isFalse(); - tempBlob.createOrOverwrite(tempBlob.location()); - assertThat(inputFile.length()).isEqualTo(tempBlob.location().length()); - assertThat(tempBlob.read()).isEqualTo(tempBlob.location()); + tempBlob.createOrOverwrite(tempBlob.location().toString()); + assertThat(inputFile.length()).isEqualTo(tempBlob.location().toString().length()); + assertThat(tempBlob.read()).isEqualTo(tempBlob.location().toString()); assertThat(listPath("..")).containsExactly(tempBlob.location()); getFileSystem().renameFile(tempBlob.location(), createLocation("file")); assertThat(inputFile.exists()).isFalse(); - assertThat(readLocation(createLocation("file"))).isEqualTo(tempBlob.location()); + assertThat(readLocation(createLocation("file"))).isEqualTo(tempBlob.location().toString()); getFileSystem().renameFile(createLocation("file"), tempBlob.location()); assertThat(inputFile.exists()).isTrue(); - assertThat(tempBlob.read()).isEqualTo(tempBlob.location()); + assertThat(tempBlob.read()).isEqualTo(tempBlob.location().toString()); getFileSystem().deleteFile(tempBlob.location()); assertThat(inputFile.exists()).isFalse(); @@ -506,27 +506,27 @@ void testDeleteFile() // delete file location cannot be the root of the file system assertThatThrownBy(() -> getFileSystem().deleteFile(getRootLocation())) .isInstanceOf(IllegalStateException.class) - .hasMessageContaining(getRootLocation()); - assertThatThrownBy(() -> getFileSystem().deleteFile(getRootLocation() + "/")) + .hasMessageContaining(getRootLocation().toString()); + assertThatThrownBy(() -> getFileSystem().deleteFile(Location.of(getRootLocation() + "/"))) .isInstanceOf(IllegalStateException.class) .hasMessageContaining(getRootLocation() + "/"); // delete file location cannot end with a slash assertThatThrownBy(() -> getFileSystem().deleteFile(createLocation("foo/"))) .isInstanceOf(IllegalStateException.class) - .hasMessageContaining(createLocation("foo/")); + .hasMessageContaining(createLocation("foo/").toString()); // delete file location cannot end with whitespace assertThatThrownBy(() -> getFileSystem().deleteFile(createLocation("foo "))) .isInstanceOf(IllegalStateException.class) - .hasMessageContaining(createLocation("foo ")); + .hasMessageContaining(createLocation("foo ").toString()); assertThatThrownBy(() -> getFileSystem().deleteFile(createLocation("foo\t"))) .isInstanceOf(IllegalStateException.class) - .hasMessageContaining(createLocation("foo\t")); + .hasMessageContaining(createLocation("foo\t").toString()); try (TempBlob tempBlob = randomBlobLocation("delete")) { // deleting a non-existent file is an error assertThatThrownBy(() -> getFileSystem().deleteFile(tempBlob.location())) .isInstanceOf(NoSuchFileException.class) - .hasMessageContaining(tempBlob.location()); + .hasMessageContaining(tempBlob.location().toString()); tempBlob.createOrOverwrite("delete me"); @@ -540,10 +540,10 @@ void testDeleteFiles() throws IOException { try (Closer closer = Closer.create()) { - Set locations = createTestDirectoryStructure(closer); + Set locations = createTestDirectoryStructure(closer); getFileSystem().deleteFiles(locations); - for (String location : locations) { + for (Location location : locations) { assertThat(getFileSystem().newInputFile(location).exists()).isFalse(); } } @@ -557,7 +557,7 @@ void testDeleteDirectory() verifyFileSystemIsEmpty(); try (Closer closer = Closer.create()) { - Set locations = createTestDirectoryStructure(closer); + Set locations = createTestDirectoryStructure(closer); // for safety make sure the verification code is functioning assertThatThrownBy(this::verifyFileSystemIsEmpty) @@ -565,7 +565,7 @@ void testDeleteDirectory() // delete directory on a file is a noop getFileSystem().deleteDirectory(createLocation("unknown")); - for (String location : locations) { + for (Location location : locations) { assertThat(getFileSystem().newInputFile(location).exists()).isTrue(); } @@ -573,17 +573,18 @@ void testDeleteDirectory() // delete directory cannot be called on a file assertThatThrownBy(() -> getFileSystem().deleteDirectory(createLocation("level0-file0"))) .isInstanceOf(IOException.class) - .hasMessageContaining(createLocation("level0-file0")); + .hasMessageContaining(createLocation("level0-file0").toString()); } getFileSystem().deleteDirectory(createLocation("level0")); - String deletedLocationPrefix = createLocation("level0/"); - for (String location : locations) { - assertThat(getFileSystem().newInputFile(location).exists()).isEqualTo(!location.startsWith(deletedLocationPrefix)); + Location deletedLocationPrefix = createLocation("level0/"); + for (Location location : locations) { + assertThat(getFileSystem().newInputFile(location).exists()) + .isEqualTo(!location.toString().startsWith(deletedLocationPrefix.toString())); } getFileSystem().deleteDirectory(getRootLocation()); - for (String location : locations) { + for (Location location : locations) { assertThat(getFileSystem().newInputFile(location).exists()).isFalse(); } } @@ -596,36 +597,36 @@ void testRenameFile() // rename file locations cannot be the root of the file system assertThatThrownBy(() -> getFileSystem().renameFile(getRootLocation(), createLocation("file"))) .isInstanceOf(IllegalStateException.class) - .hasMessageContaining(getRootLocation()); + .hasMessageContaining(getRootLocation().toString()); assertThatThrownBy(() -> getFileSystem().renameFile(createLocation("file"), getRootLocation())) .isInstanceOf(IllegalStateException.class) - .hasMessageContaining(getRootLocation()); - assertThatThrownBy(() -> getFileSystem().renameFile(getRootLocation() + "/", createLocation("file"))) + .hasMessageContaining(getRootLocation().toString()); + assertThatThrownBy(() -> getFileSystem().renameFile(Location.of(getRootLocation() + "/"), createLocation("file"))) .isInstanceOf(IllegalStateException.class) .hasMessageContaining(getRootLocation() + "/"); - assertThatThrownBy(() -> getFileSystem().renameFile(createLocation("file"), getRootLocation() + "/")) + assertThatThrownBy(() -> getFileSystem().renameFile(createLocation("file"), Location.of(getRootLocation() + "/"))) .isInstanceOf(IllegalStateException.class) .hasMessageContaining(getRootLocation() + "/"); // rename file locations cannot end with a slash assertThatThrownBy(() -> getFileSystem().renameFile(createLocation("foo/"), createLocation("file"))) .isInstanceOf(IllegalStateException.class) - .hasMessageContaining(createLocation("foo/")); + .hasMessageContaining(createLocation("foo/").toString()); assertThatThrownBy(() -> getFileSystem().renameFile(createLocation("file"), createLocation("foo/"))) .isInstanceOf(IllegalStateException.class) - .hasMessageContaining(createLocation("foo/")); + .hasMessageContaining(createLocation("foo/").toString()); // rename file locations cannot end with whitespace assertThatThrownBy(() -> getFileSystem().renameFile(createLocation("foo "), createLocation("file"))) .isInstanceOf(IllegalStateException.class) - .hasMessageContaining(createLocation("foo ")); + .hasMessageContaining(createLocation("foo ").toString()); assertThatThrownBy(() -> getFileSystem().renameFile(createLocation("file"), createLocation("foo "))) .isInstanceOf(IllegalStateException.class) - .hasMessageContaining(createLocation("foo ")); + .hasMessageContaining(createLocation("foo ").toString()); assertThatThrownBy(() -> getFileSystem().renameFile(createLocation("foo\t"), createLocation("file"))) .isInstanceOf(IllegalStateException.class) - .hasMessageContaining(createLocation("foo\t")); + .hasMessageContaining(createLocation("foo\t").toString()); assertThatThrownBy(() -> getFileSystem().renameFile(createLocation("file"), createLocation("foo\t"))) .isInstanceOf(IllegalStateException.class) - .hasMessageContaining(createLocation("foo\t")); + .hasMessageContaining(createLocation("foo\t").toString()); // todo rename to existing file name try (TempBlob sourceBlob = randomBlobLocation("renameSource"); @@ -633,8 +634,8 @@ void testRenameFile() // renaming a non-existent file is an error assertThatThrownBy(() -> getFileSystem().renameFile(sourceBlob.location(), targetBlob.location())) .isInstanceOf(IOException.class) - .hasMessageContaining(sourceBlob.location()) - .hasMessageContaining(targetBlob.location()); + .hasMessageContaining(sourceBlob.location().toString()) + .hasMessageContaining(targetBlob.location().toString()); // rename sourceBlob.createOrOverwrite("data"); @@ -647,8 +648,8 @@ void testRenameFile() sourceBlob.createOrOverwrite("new data"); assertThatThrownBy(() -> getFileSystem().renameFile(sourceBlob.location(), targetBlob.location())) .isInstanceOf(IOException.class) - .hasMessageContaining(sourceBlob.location()) - .hasMessageContaining(targetBlob.location()); + .hasMessageContaining(sourceBlob.location().toString()) + .hasMessageContaining(targetBlob.location().toString()); assertThat(sourceBlob.exists()).isTrue(); assertThat(targetBlob.exists()).isTrue(); assertThat(sourceBlob.read()).isEqualTo("new data"); @@ -672,29 +673,29 @@ void testListFiles() throws IOException { try (Closer closer = Closer.create()) { - Set locations = createTestDirectoryStructure(closer); + Set locations = createTestDirectoryStructure(closer); assertThat(listPath("")).containsExactlyInAnyOrderElementsOf(locations); assertThat(listPath("level0")).containsExactlyInAnyOrderElementsOf(locations.stream() - .filter(location -> location.startsWith(createLocation("level0/"))) + .filter(location -> location.toString().startsWith(createLocation("level0/").toString())) .toList()); assertThat(listPath("level0/")).containsExactlyInAnyOrderElementsOf(locations.stream() - .filter(location -> location.startsWith(createLocation("level0/"))) + .filter(location -> location.toString().startsWith(createLocation("level0/").toString())) .toList()); assertThat(listPath("level0/level1/")).containsExactlyInAnyOrderElementsOf(locations.stream() - .filter(location -> location.startsWith(createLocation("level0/level1/"))) + .filter(location -> location.toString().startsWith(createLocation("level0/level1/").toString())) .toList()); assertThat(listPath("level0/level1")).containsExactlyInAnyOrderElementsOf(locations.stream() - .filter(location -> location.startsWith(createLocation("level0/level1/"))) + .filter(location -> location.toString().startsWith(createLocation("level0/level1/").toString())) .toList()); assertThat(listPath("level0/level1/level2/")).containsExactlyInAnyOrderElementsOf(locations.stream() - .filter(location -> location.startsWith(createLocation("level0/level1/level2/"))) + .filter(location -> location.toString().startsWith(createLocation("level0/level1/level2/").toString())) .toList()); assertThat(listPath("level0/level1/level2")).containsExactlyInAnyOrderElementsOf(locations.stream() - .filter(location -> location.startsWith(createLocation("level0/level1/level2/"))) + .filter(location -> location.toString().startsWith(createLocation("level0/level1/level2/").toString())) .toList()); assertThat(listPath("level0/level1/level2/level3")).isEmpty(); @@ -705,7 +706,7 @@ void testListFiles() if (isHierarchical()) { assertThatThrownBy(() -> listPath("level0-file0")) .isInstanceOf(IOException.class) - .hasMessageContaining(createLocation("level0-file0")); + .hasMessageContaining(createLocation("level0-file0").toString()); } else { assertThat(listPath("level0-file0")).isEmpty(); @@ -718,21 +719,30 @@ void testListFiles() } } - private List listPath(String path) + private List listPath(String path) throws IOException { - List locations = new ArrayList<>(); - FileIterator fileIterator = getFileSystem().listFiles(createLocation(path)); + List locations = new ArrayList<>(); + FileIterator fileIterator = getFileSystem().listFiles(createListingLocation(path)); while (fileIterator.hasNext()) { FileEntry fileEntry = fileIterator.next(); - String location = fileEntry.location(); - assertThat(fileEntry.length()).isEqualTo(location.length()); + Location location = fileEntry.location(); + assertThat(fileEntry.length()).isEqualTo(location.toString().length()); locations.add(location); } return locations; } - private String readLocation(String path) + private Location createListingLocation(String path) + { + // allow listing a directory with a trailing slash + if (path.equals("/")) { + return createLocation("").appendSuffix("/"); + } + return createLocation(path); + } + + private String readLocation(Location path) { try (InputStream inputStream = getFileSystem().newInputFile(path).newStream()) { return new String(inputStream.readAllBytes(), UTF_8); @@ -742,10 +752,10 @@ private String readLocation(String path) } } - private String createBlob(Closer closer, String path) + private Location createBlob(Closer closer, String path) { - String location = createLocation(path); - closer.register(new TempBlob(location)).createOrOverwrite(location); + Location location = createLocation(path); + closer.register(new TempBlob(location)).createOrOverwrite(location.toString()); return location; } @@ -756,9 +766,9 @@ private TempBlob randomBlobLocation(String nameHint) return tempBlob; } - private Set createTestDirectoryStructure(Closer closer) + private Set createTestDirectoryStructure(Closer closer) { - Set locations = new HashSet<>(); + Set locations = new HashSet<>(); if (!isHierarchical()) { locations.add(createBlob(closer, "level0")); } @@ -783,16 +793,16 @@ private Set createTestDirectoryStructure(Closer closer) private class TempBlob implements Closeable { - private final String location; + private final Location location; private final TrinoFileSystem fileSystem; - public TempBlob(String location) + public TempBlob(Location location) { this.location = requireNonNull(location, "location is null"); fileSystem = getFileSystem(); } - public String location() + public Location location() { return location; } diff --git a/lib/trino-filesystem/src/test/java/io/trino/filesystem/TestFileEntry.java b/lib/trino-filesystem/src/test/java/io/trino/filesystem/TestFileEntry.java index 24ab1852e228..d095a80d3db9 100644 --- a/lib/trino-filesystem/src/test/java/io/trino/filesystem/TestFileEntry.java +++ b/lib/trino-filesystem/src/test/java/io/trino/filesystem/TestFileEntry.java @@ -25,14 +25,15 @@ public class TestFileEntry { + private static final Location LOCATION = Location.of("/test"); private static final Instant MODIFIED = Instant.ofEpochSecond(1234567890); @Test public void testEmptyBlocks() { - assertThat(new FileEntry("/test", 123, MODIFIED, Optional.empty())) + assertThat(new FileEntry(LOCATION, 123, MODIFIED, Optional.empty())) .satisfies(entry -> { - assertThat(entry.location()).isEqualTo("/test"); + assertThat(entry.location()).isEqualTo(LOCATION); assertThat(entry.length()).isEqualTo(123); assertThat(entry.lastModified()).isEqualTo(MODIFIED); assertThat(entry.blocks()).isEmpty(); @@ -46,14 +47,14 @@ public void testPresentBlocks() new Block(List.of(), 0, 50), new Block(List.of(), 50, 70), new Block(List.of(), 100, 150)); - assertThat(new FileEntry("/test", 200, MODIFIED, Optional.of(locations))) + assertThat(new FileEntry(LOCATION, 200, MODIFIED, Optional.of(locations))) .satisfies(entry -> assertThat(entry.blocks()).contains(locations)); } @Test public void testMissingBlocks() { - assertThatThrownBy(() -> new FileEntry("/test", 0, MODIFIED, Optional.of(List.of()))) + assertThatThrownBy(() -> new FileEntry(LOCATION, 0, MODIFIED, Optional.of(List.of()))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("blocks is empty"); } @@ -62,7 +63,7 @@ public void testMissingBlocks() public void testBlocksEmptyFile() { List locations = List.of(new Block(List.of(), 0, 0)); - assertThat(new FileEntry("/test", 0, MODIFIED, Optional.of(locations))) + assertThat(new FileEntry(LOCATION, 0, MODIFIED, Optional.of(locations))) .satisfies(entry -> assertThat(entry.blocks()).contains(locations)); } @@ -70,7 +71,7 @@ public void testBlocksEmptyFile() public void testBlocksGapAtStart() { List locations = List.of(new Block(List.of(), 50, 50)); - assertThatThrownBy(() -> new FileEntry("/test", 100, MODIFIED, Optional.of(locations))) + assertThatThrownBy(() -> new FileEntry(LOCATION, 100, MODIFIED, Optional.of(locations))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("blocks have a gap"); } @@ -81,7 +82,7 @@ public void testBlocksGapInMiddle() List locations = List.of( new Block(List.of(), 0, 50), new Block(List.of(), 100, 100)); - assertThatThrownBy(() -> new FileEntry("/test", 200, MODIFIED, Optional.of(locations))) + assertThatThrownBy(() -> new FileEntry(LOCATION, 200, MODIFIED, Optional.of(locations))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("blocks have a gap"); } @@ -92,7 +93,7 @@ public void testBlocksGapAtEnd() List locations = List.of( new Block(List.of(), 0, 50), new Block(List.of(), 50, 49)); - assertThatThrownBy(() -> new FileEntry("/test", 100, MODIFIED, Optional.of(locations))) + assertThatThrownBy(() -> new FileEntry(LOCATION, 100, MODIFIED, Optional.of(locations))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("blocks do not cover file"); } diff --git a/lib/trino-filesystem/src/test/java/io/trino/filesystem/TrackingFileSystemFactory.java b/lib/trino-filesystem/src/test/java/io/trino/filesystem/TrackingFileSystemFactory.java index b9f8916b78d7..f9d82143a5d5 100644 --- a/lib/trino-filesystem/src/test/java/io/trino/filesystem/TrackingFileSystemFactory.java +++ b/lib/trino-filesystem/src/test/java/io/trino/filesystem/TrackingFileSystemFactory.java @@ -72,7 +72,7 @@ public void reset() operationCounts.clear(); } - private void increment(String path, int fileId, OperationType operationType) + private void increment(Location path, int fileId, OperationType operationType) { OperationContext context = new OperationContext(path, fileId, operationType); operationCounts.merge(context, 1, Math::addExact); // merge is atomic for ConcurrentHashMap @@ -86,7 +86,7 @@ public TrinoFileSystem create(ConnectorIdentity identity) private interface Tracker { - void track(String path, int fileId, OperationType operationType); + void track(Location path, int fileId, OperationType operationType); } private class TrackingFileSystem @@ -102,7 +102,7 @@ private TrackingFileSystem(TrinoFileSystem delegate, Tracker tracker) } @Override - public TrinoInputFile newInputFile(String location) + public TrinoInputFile newInputFile(Location location) { int nextId = fileId.incrementAndGet(); return new TrackingInputFile( @@ -111,7 +111,7 @@ public TrinoInputFile newInputFile(String location) } @Override - public TrinoInputFile newInputFile(String location, long length) + public TrinoInputFile newInputFile(Location location, long length) { int nextId = fileId.incrementAndGet(); return new TrackingInputFile( @@ -120,7 +120,7 @@ public TrinoInputFile newInputFile(String location, long length) } @Override - public TrinoOutputFile newOutputFile(String location) + public TrinoOutputFile newOutputFile(Location location) { int nextId = fileId.incrementAndGet(); return new TrackingOutputFile( @@ -129,35 +129,35 @@ public TrinoOutputFile newOutputFile(String location) } @Override - public void deleteFile(String location) + public void deleteFile(Location location) throws IOException { delegate.deleteFile(location); } @Override - public void deleteFiles(Collection locations) + public void deleteFiles(Collection locations) throws IOException { delegate.deleteFiles(locations); } @Override - public void deleteDirectory(String location) + public void deleteDirectory(Location location) throws IOException { delegate.deleteDirectory(location); } @Override - public void renameFile(String source, String target) + public void renameFile(Location source, Location target) throws IOException { delegate.renameFile(source, target); } @Override - public FileIterator listFiles(String location) + public FileIterator listFiles(Location location) throws IOException { return delegate.listFiles(location); @@ -216,10 +216,16 @@ public Instant lastModified() } @Override - public String location() + public Location location() { return delegate.location(); } + + @Override + public String toString() + { + return delegate.toString(); + } } private static class TrackingOutputFile @@ -251,30 +257,36 @@ public OutputStream createOrOverwrite(AggregatedMemoryContext memoryContext) } @Override - public String location() + public Location location() { tracker.accept(OUTPUT_FILE_LOCATION); return delegate.location(); } + + @Override + public String toString() + { + return delegate.toString(); + } } @Immutable public static class OperationContext { - private final String filePath; + private final Location location; private final int fileId; private final OperationType operationType; - public OperationContext(String filePath, int fileId, OperationType operationType) + public OperationContext(Location location, int fileId, OperationType operationType) { - this.filePath = requireNonNull(filePath, "filePath is null"); + this.location = requireNonNull(location, "location is null"); this.fileId = fileId; this.operationType = requireNonNull(operationType, "operationType is null"); } - public String getFilePath() + public Location getLocation() { - return filePath; + return location; } public int getFileId() @@ -297,7 +309,7 @@ public boolean equals(Object o) return false; } OperationContext that = (OperationContext) o; - return Objects.equals(filePath, that.filePath) + return Objects.equals(location, that.location) && fileId == that.fileId && operationType == that.operationType; } @@ -305,14 +317,14 @@ public boolean equals(Object o) @Override public int hashCode() { - return Objects.hash(filePath, fileId, operationType); + return Objects.hash(location, fileId, operationType); } @Override public String toString() { return toStringHelper(this) - .add("path", filePath) + .add("path", location) .add("fileId", fileId) .add("operation", operationType) .toString(); diff --git a/lib/trino-filesystem/src/test/java/io/trino/filesystem/local/TestLocalFileSystem.java b/lib/trino-filesystem/src/test/java/io/trino/filesystem/local/TestLocalFileSystem.java index c33f2617e5da..b9f4208e6054 100644 --- a/lib/trino-filesystem/src/test/java/io/trino/filesystem/local/TestLocalFileSystem.java +++ b/lib/trino-filesystem/src/test/java/io/trino/filesystem/local/TestLocalFileSystem.java @@ -14,6 +14,7 @@ package io.trino.filesystem.local; import io.trino.filesystem.AbstractTestTrinoFileSystem; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; @@ -88,9 +89,9 @@ protected TrinoFileSystem getFileSystem() } @Override - protected String getRootLocation() + protected Location getRootLocation() { - return "local://"; + return Location.of("local://"); } @Override @@ -111,24 +112,24 @@ void testPathsOutOfBounds() { assertThatThrownBy(() -> getFileSystem().newInputFile(createLocation("../file"))) .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining(createLocation("../file")); + .hasMessageContaining(createLocation("../file").toString()); assertThatThrownBy(() -> getFileSystem().newInputFile(createLocation("../file"), 22)) .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining(createLocation("../file")); + .hasMessageContaining(createLocation("../file").toString()); assertThatThrownBy(() -> getFileSystem().newOutputFile(createLocation("../file"))) .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining(createLocation("../file")); + .hasMessageContaining(createLocation("../file").toString()); assertThatThrownBy(() -> getFileSystem().deleteFile(createLocation("../file"))) .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining(createLocation("../file")); + .hasMessageContaining(createLocation("../file").toString()); assertThatThrownBy(() -> getFileSystem().listFiles(createLocation("../file"))) .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining(createLocation("../file")); + .hasMessageContaining(createLocation("../file").toString()); assertThatThrownBy(() -> getFileSystem().renameFile(createLocation("../file"), createLocation("target"))) .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining(createLocation("../file")); + .hasMessageContaining(createLocation("../file").toString()); assertThatThrownBy(() -> getFileSystem().renameFile(createLocation("source"), createLocation("../file"))) .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining(createLocation("../file")); + .hasMessageContaining(createLocation("../file").toString()); } } diff --git a/lib/trino-filesystem/src/test/java/io/trino/filesystem/memory/TestMemoryFileSystem.java b/lib/trino-filesystem/src/test/java/io/trino/filesystem/memory/TestMemoryFileSystem.java index ae0a7715a32a..a1014dc89809 100644 --- a/lib/trino-filesystem/src/test/java/io/trino/filesystem/memory/TestMemoryFileSystem.java +++ b/lib/trino-filesystem/src/test/java/io/trino/filesystem/memory/TestMemoryFileSystem.java @@ -14,6 +14,7 @@ package io.trino.filesystem.memory; import io.trino.filesystem.AbstractTestTrinoFileSystem; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; @@ -50,9 +51,9 @@ protected TrinoFileSystem getFileSystem() } @Override - protected String getRootLocation() + protected Location getRootLocation() { - return "memory://"; + return Location.of("memory://"); } @Override diff --git a/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HadoopPaths.java b/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HadoopPaths.java index 46a5bddb7eb4..a30bb3a92358 100644 --- a/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HadoopPaths.java +++ b/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HadoopPaths.java @@ -13,6 +13,7 @@ */ package io.trino.filesystem.hdfs; +import io.trino.filesystem.Location; import org.apache.hadoop.fs.Path; import java.net.URI; @@ -24,9 +25,10 @@ public final class HadoopPaths { private HadoopPaths() {} - public static Path hadoopPath(String path) + public static Path hadoopPath(Location location) { // hack to preserve the original path for S3 if necessary + String path = location.toString(); Path hadoopPath = new Path(path); if ("s3".equals(hadoopPath.toUri().getScheme()) && !path.equals(hadoopPath.toString())) { if (hadoopPath.toUri().getFragment() != null) { diff --git a/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HdfsFileIterator.java b/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HdfsFileIterator.java index 422d285199fb..1c30fc2df356 100644 --- a/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HdfsFileIterator.java +++ b/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HdfsFileIterator.java @@ -17,15 +17,14 @@ import io.trino.filesystem.FileEntry; import io.trino.filesystem.FileEntry.Block; import io.trino.filesystem.FileIterator; +import io.trino.filesystem.Location; import org.apache.hadoop.fs.BlockLocation; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; import java.io.IOException; import java.io.UncheckedIOException; -import java.net.URI; import java.time.Instant; import java.util.List; import java.util.Optional; @@ -38,14 +37,14 @@ class HdfsFileIterator implements FileIterator { - private final String listingPath; - private final URI listingUri; + private final Location listingLocation; + private final Path listingPath; private final RemoteIterator iterator; - public HdfsFileIterator(String listingPath, FileSystem fs, RemoteIterator iterator) + public HdfsFileIterator(Location listingLocation, Path listingPath, RemoteIterator iterator) { + this.listingLocation = requireNonNull(listingLocation, "listingPath is null"); this.listingPath = requireNonNull(listingPath, "listingPath is null"); - this.listingUri = new Path(listingPath).makeQualified(fs.getUri(), fs.getWorkingDirectory()).toUri(); this.iterator = requireNonNull(iterator, "iterator is null"); } @@ -64,24 +63,23 @@ public FileEntry next() verify(status.isFile(), "iterator returned a non-file: %s", status); - URI pathUri = status.getPath().toUri(); - URI relativeUri = listingUri.relativize(pathUri); - verify(!relativeUri.equals(pathUri), "cannot relativize [%s] against [%s]", pathUri, listingUri); - - String path = listingPath; - if (!relativeUri.getPath().isEmpty()) { - if (!path.endsWith("/")) { - path += "/"; - } - path += relativeUri.getPath(); + if (status.getPath().equals(listingPath)) { + throw new IOException("Listing location is a file, not a directory: " + listingLocation); } + String root = listingPath.toUri().getPath(); + String path = status.getPath().toUri().getPath(); + + verify(path.startsWith(root), "iterator path [%s] not a child of listing path [%s] for location [%s]", path, root, listingLocation); + + Location location = listingLocation.appendPath(path.substring(root.length() + 1)); + List blocks = Stream.of(status.getBlockLocations()) .map(HdfsFileIterator::toTrinoBlock) .collect(toImmutableList()); return new FileEntry( - path, + location, status.getLen(), Instant.ofEpochMilli(status.getModificationTime()), blocks.isEmpty() ? Optional.empty() : Optional.of(blocks)); diff --git a/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HdfsFileSystem.java b/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HdfsFileSystem.java index 7ef300a9383a..578d19ec6520 100644 --- a/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HdfsFileSystem.java +++ b/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HdfsFileSystem.java @@ -14,6 +14,7 @@ package io.trino.filesystem.hdfs; import io.trino.filesystem.FileIterator; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoInputFile; import io.trino.filesystem.TrinoOutputFile; @@ -51,25 +52,25 @@ public HdfsFileSystem(HdfsEnvironment environment, HdfsContext context) } @Override - public TrinoInputFile newInputFile(String location) + public TrinoInputFile newInputFile(Location location) { return new HdfsInputFile(location, null, environment, context); } @Override - public TrinoInputFile newInputFile(String location, long length) + public TrinoInputFile newInputFile(Location location, long length) { return new HdfsInputFile(location, length, environment, context); } @Override - public TrinoOutputFile newOutputFile(String location) + public TrinoOutputFile newOutputFile(Location location) { return new HdfsOutputFile(location, environment, context); } @Override - public void deleteFile(String location) + public void deleteFile(Location location) throws IOException { Path file = hadoopPath(location); @@ -83,12 +84,12 @@ public void deleteFile(String location) } @Override - public void deleteFiles(Collection locations) + public void deleteFiles(Collection locations) throws IOException { Map> pathsGroupedByDirectory = locations.stream().collect( groupingBy( - path -> hadoopPath(path.replaceFirst("/[^/]*$", "")), + location -> hadoopPath(location.parentDirectory()), mapping(HadoopPaths::hadoopPath, toList()))); for (Entry> directoryWithPaths : pathsGroupedByDirectory.entrySet()) { FileSystem rawFileSystem = getRawFileSystem(environment.getFileSystem(context, directoryWithPaths.getKey())); @@ -107,7 +108,7 @@ public void deleteFiles(Collection locations) } @Override - public void deleteDirectory(String location) + public void deleteDirectory(Location location) throws IOException { Path directory = hadoopPath(location); @@ -121,7 +122,7 @@ public void deleteDirectory(String location) } @Override - public void renameFile(String source, String target) + public void renameFile(Location source, Location target) throws IOException { Path sourcePath = hadoopPath(source); @@ -136,14 +137,14 @@ public void renameFile(String source, String target) } @Override - public FileIterator listFiles(String location) + public FileIterator listFiles(Location location) throws IOException { Path directory = hadoopPath(location); FileSystem fileSystem = environment.getFileSystem(context, directory); return environment.doAs(context.getIdentity(), () -> { try { - return new HdfsFileIterator(location, fileSystem, fileSystem.listFiles(directory, true)); + return new HdfsFileIterator(location, directory, fileSystem.listFiles(directory, true)); } catch (FileNotFoundException e) { return FileIterator.empty(); diff --git a/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HdfsInput.java b/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HdfsInput.java index 4e78476451bb..52fb2e25e2ac 100644 --- a/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HdfsInput.java +++ b/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HdfsInput.java @@ -46,7 +46,7 @@ public void readFully(long position, byte[] buffer, int bufferOffset, int buffer public int readTail(byte[] buffer, int bufferOffset, int bufferLength) throws IOException { - Slice tail = FSDataInputStreamTail.readTail(inputFile.location(), inputFile.length(), stream, bufferLength).getTailSlice(); + Slice tail = FSDataInputStreamTail.readTail(toString(), inputFile.length(), stream, bufferLength).getTailSlice(); tail.getBytes(0, buffer, bufferOffset, tail.length()); return tail.length(); } diff --git a/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HdfsInputFile.java b/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HdfsInputFile.java index a38876e0ec63..a6b2506bd083 100644 --- a/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HdfsInputFile.java +++ b/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HdfsInputFile.java @@ -13,6 +13,7 @@ */ package io.trino.filesystem.hdfs; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoInput; import io.trino.filesystem.TrinoInputFile; import io.trino.filesystem.TrinoInputStream; @@ -33,19 +34,19 @@ class HdfsInputFile implements TrinoInputFile { - private final String path; + private final Location location; private final HdfsEnvironment environment; private final HdfsContext context; private final Path file; private Long length; private FileStatus status; - public HdfsInputFile(String path, Long length, HdfsEnvironment environment, HdfsContext context) + public HdfsInputFile(Location location, Long length, HdfsEnvironment environment, HdfsContext context) { - this.path = requireNonNull(path, "path is null"); + this.location = requireNonNull(location, "location is null"); this.environment = requireNonNull(environment, "environment is null"); this.context = requireNonNull(context, "context is null"); - this.file = hadoopPath(path); + this.file = hadoopPath(location); this.length = length; checkArgument(length == null || length >= 0, "length is negative"); } @@ -90,15 +91,15 @@ public boolean exists() } @Override - public String location() + public Location location() { - return path; + return location; } @Override public String toString() { - return location(); + return location().toString(); } private FSDataInputStream openFile() diff --git a/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HdfsOutputFile.java b/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HdfsOutputFile.java index 644063fd37d7..ccffef59db7a 100644 --- a/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HdfsOutputFile.java +++ b/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HdfsOutputFile.java @@ -13,6 +13,7 @@ */ package io.trino.filesystem.hdfs; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoOutputFile; import io.trino.hdfs.HdfsContext; import io.trino.hdfs.HdfsEnvironment; @@ -31,13 +32,13 @@ class HdfsOutputFile implements TrinoOutputFile { - private final String path; + private final Location location; private final HdfsEnvironment environment; private final HdfsContext context; - public HdfsOutputFile(String path, HdfsEnvironment environment, HdfsContext context) + public HdfsOutputFile(Location location, HdfsEnvironment environment, HdfsContext context) { - this.path = requireNonNull(path, "path is null"); + this.location = requireNonNull(location, "location is null"); this.environment = requireNonNull(environment, "environment is null"); this.context = requireNonNull(context, "context is null"); } @@ -59,7 +60,7 @@ public OutputStream createOrOverwrite(AggregatedMemoryContext memoryContext) private OutputStream create(boolean overwrite, AggregatedMemoryContext memoryContext) throws IOException { - Path file = hadoopPath(path); + Path file = hadoopPath(location); FileSystem fileSystem = environment.getFileSystem(context, file); FileSystem rawFileSystem = getRawFileSystem(fileSystem); if (rawFileSystem instanceof MemoryAwareFileSystem memoryAwareFileSystem) { @@ -69,14 +70,14 @@ private OutputStream create(boolean overwrite, AggregatedMemoryContext memoryCon } @Override - public String location() + public Location location() { - return path; + return location; } @Override public String toString() { - return location(); + return location().toString(); } } diff --git a/lib/trino-hdfs/src/test/java/io/trino/filesystem/hdfs/TestHdfsFileSystem.java b/lib/trino-hdfs/src/test/java/io/trino/filesystem/hdfs/TestHdfsFileSystem.java index f154b93ffb79..c9bf97fac47e 100644 --- a/lib/trino-hdfs/src/test/java/io/trino/filesystem/hdfs/TestHdfsFileSystem.java +++ b/lib/trino-hdfs/src/test/java/io/trino/filesystem/hdfs/TestHdfsFileSystem.java @@ -16,6 +16,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import io.trino.filesystem.FileIterator; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.hdfs.DynamicHdfsConfiguration; @@ -36,6 +37,7 @@ import static java.nio.file.Files.createFile; import static java.nio.file.Files.createTempDirectory; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; public class TestHdfsFileSystem { @@ -51,7 +53,10 @@ public void testListing() TrinoFileSystem fileSystem = factory.create(ConnectorIdentity.ofUser("test")); Path tempDir = createTempDirectory("testListing"); - String root = tempDir.toString(); + + String root = tempDir.toUri().toString(); + assertThat(root).endsWith("/"); + root = root.substring(0, root.length() - 1); assertThat(listFiles(fileSystem, root)).isEmpty(); @@ -65,10 +70,18 @@ public void testListing() root + "/e f", root + "/xyz"); - assertThat(listFiles(fileSystem, root + "/abc")).containsExactly(root + "/abc"); - assertThat(listFiles(fileSystem, root + "/abc/")).containsExactly(root + "/abc/"); - assertThat(listFiles(fileSystem, root + "/abc//")).containsExactly(root + "/abc//"); - assertThat(listFiles(fileSystem, root + "///abc")).containsExactly(root + "///abc"); + for (String path : List.of("/abc", "/abc/", "/abc//", "///abc")) { + String directory = root + path; + assertThatThrownBy(() -> listFiles(fileSystem, directory)) + .isInstanceOf(IOException.class) + .hasMessage("Listing location is a file, not a directory: %s", directory); + } + + String rootPath = tempDir.toAbsolutePath().toString(); + assertThat(listFiles(fileSystem, rootPath)).containsExactlyInAnyOrder( + rootPath + "/abc", + rootPath + "/e f", + rootPath + "/xyz"); createFile(tempDir.resolve("mydir").resolve("qqq")); @@ -89,10 +102,10 @@ public void testListing() private static List listFiles(TrinoFileSystem fileSystem, String path) throws IOException { - FileIterator iterator = fileSystem.listFiles(path); + FileIterator iterator = fileSystem.listFiles(Location.of(path)); ImmutableList.Builder files = ImmutableList.builder(); while (iterator.hasNext()) { - files.add(iterator.next().location()); + files.add(iterator.next().location().toString()); } return files.build(); } diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/sequence/SequenceFileReader.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/sequence/SequenceFileReader.java index f779b417c648..1634e7c4308e 100644 --- a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/sequence/SequenceFileReader.java +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/sequence/SequenceFileReader.java @@ -20,6 +20,7 @@ import io.airlift.slice.Slice; import io.airlift.slice.SliceInput; import io.airlift.slice.Slices; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoInputFile; import io.trino.hive.formats.FileCorruptionException; import io.trino.hive.formats.TrinoDataInputStream; @@ -60,7 +61,7 @@ public final class SequenceFileReader private static final int MAX_METADATA_ENTRIES = 500_000; private static final int MAX_METADATA_STRING_LENGTH = 1024 * 1024; - private final String location; + private final Location location; private final TrinoDataInputStream input; private final String keyClassName; @@ -170,7 +171,7 @@ public SequenceFileReader(TrinoInputFile inputFile, long offset, long length) } } - public String getFileLocation() + public Location getFileLocation() { return location; } @@ -280,7 +281,7 @@ private static class SingleValueReader { private static final int INSTANCE_SIZE = instanceSize(SingleValueReader.class); - private final String location; + private final Location location; private final long fileSize; private final TrinoDataInputStream input; private final ValueDecompressor decompressor; @@ -293,7 +294,7 @@ private static class SingleValueReader private final DynamicSliceOutput uncompressedBuffer = new DynamicSliceOutput(0); public SingleValueReader( - String location, + Location location, long fileSize, TrinoDataInputStream input, ValueDecompressor decompressor, @@ -393,7 +394,7 @@ private static class BlockCompressedValueReader { private static final int INSTANCE_SIZE = instanceSize(BlockCompressedValueReader.class); - private final String location; + private final Location location; private final long fileSize; private final TrinoDataInputStream input; private final long end; @@ -407,7 +408,7 @@ private static class BlockCompressedValueReader private ValuesBlock valuesBlock = ValuesBlock.EMPTY_VALUES_BLOCK; public BlockCompressedValueReader( - String location, + Location location, long fileSize, TrinoDataInputStream input, ValueDecompressor decompressor, diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/text/TextLineReaderFactory.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/text/TextLineReaderFactory.java index 593fa6b23f2b..9d14de9c4833 100644 --- a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/text/TextLineReaderFactory.java +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/line/text/TextLineReaderFactory.java @@ -65,7 +65,7 @@ public LineReader createLineReader( { InputStream inputStream = inputFile.newStream(); try { - Optional codec = CompressionKind.forFile(inputFile.location()) + Optional codec = CompressionKind.forFile(inputFile.location().fileName()) .map(CompressionKind::createCodec); if (codec.isPresent()) { checkArgument(start == 0, "Compressed files are not splittable"); diff --git a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/rcfile/RcFileReader.java b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/rcfile/RcFileReader.java index cab6337cd1fd..508b28fb4347 100644 --- a/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/rcfile/RcFileReader.java +++ b/lib/trino-hive-formats/src/main/java/io/trino/hive/formats/rcfile/RcFileReader.java @@ -17,6 +17,7 @@ import io.airlift.slice.BasicSliceInput; import io.airlift.slice.Slice; import io.airlift.slice.Slices; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoInputFile; import io.trino.hive.formats.FileCorruptionException; import io.trino.hive.formats.ReadWriteUtils; @@ -73,7 +74,7 @@ public class RcFileReader private static final String COLUMN_COUNT_METADATA_KEY = "hive.io.rcfile.column.number"; - private final String location; + private final Location location; private final long fileSize; private final Map readColumns; private final TrinoDataInputStream input; @@ -438,7 +439,7 @@ public Block readBlock(int columnIndex) return columns[columnIndex].readBlock(rowGroupPosition, currentChunkRowCount); } - public String getFileLocation() + public Location getFileLocation() { return location; } diff --git a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/TestTrinoDataInputStream.java b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/TestTrinoDataInputStream.java index 27289d19f414..f05584f6fab5 100644 --- a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/TestTrinoDataInputStream.java +++ b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/TestTrinoDataInputStream.java @@ -18,6 +18,8 @@ import com.google.common.io.ByteStreams; import io.airlift.slice.Slice; import io.airlift.slice.Slices; +import io.trino.filesystem.Location; +import io.trino.filesystem.TrinoInputFile; import io.trino.filesystem.TrinoInputStream; import io.trino.filesystem.memory.MemoryInputFile; import org.testng.annotations.Test; @@ -591,7 +593,7 @@ public void testRetainedSize() throws IOException { int bufferSize = 1024; - TrinoInputStream inputStream = new MemoryInputFile("test", Slices.wrappedBuffer(new byte[] {0, 1})).newStream(); + TrinoInputStream inputStream = getMemoryInputFile(new byte[] {0, 1}).newStream(); TrinoDataInputStream input = new TrinoDataInputStream(inputStream, bufferSize); assertEquals(input.getRetainedSize(), instanceSize(TrinoDataInputStream.class) + sizeOfByteArray(bufferSize)); } @@ -729,7 +731,12 @@ protected abstract String readActual(TrinoDataInputStream input) private static TrinoDataInputStream createTrinoDataInputStream(byte[] bytes) throws IOException { - TrinoInputStream inputStream = new MemoryInputFile("test", Slices.wrappedBuffer(bytes)).newStream(); + TrinoInputStream inputStream = getMemoryInputFile(bytes).newStream(); return new TrinoDataInputStream(inputStream, 16 * 1024); } + + private static TrinoInputFile getMemoryInputFile(byte[] bytes) + { + return new MemoryInputFile(Location.of("memory:///test"), Slices.wrappedBuffer(bytes)); + } } diff --git a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/line/sequence/TestSequenceFileReaderWriter.java b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/line/sequence/TestSequenceFileReaderWriter.java index 852cc3b9e0e1..9fd950739e01 100644 --- a/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/line/sequence/TestSequenceFileReaderWriter.java +++ b/lib/trino-hive-formats/src/test/java/io/trino/hive/formats/line/sequence/TestSequenceFileReaderWriter.java @@ -132,7 +132,7 @@ private static void assertNew(File inputFile, List values, Map readValues(Slice data, int offset, int length) } RcFileReader reader = new RcFileReader( - new MemoryInputFile("test", data), + new MemoryInputFile(Location.of("memory:///test"), data), new BinaryColumnEncodingFactory(DateTimeZone.UTC), ImmutableMap.of(0, SMALLINT), offset, diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/AbstractDeltaLakePageSink.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/AbstractDeltaLakePageSink.java index f92f8d9f68a3..fbf09711f474 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/AbstractDeltaLakePageSink.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/AbstractDeltaLakePageSink.java @@ -21,6 +21,7 @@ import io.airlift.json.JsonCodec; import io.airlift.log.Logger; import io.airlift.slice.Slice; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.parquet.writer.ParquetSchemaConverter; @@ -95,8 +96,8 @@ public abstract class AbstractDeltaLakePageSink private final List writers = new ArrayList<>(); - private final String tableLocation; - protected final String outputPathDirectory; + private final Location tableLocation; + protected final Location outputPathDirectory; private final ConnectorSession session; private final DeltaLakeWriterStats stats; private final String trinoVersion; @@ -116,8 +117,8 @@ public AbstractDeltaLakePageSink( TrinoFileSystemFactory fileSystemFactory, int maxOpenWriters, JsonCodec dataFileInfoCodec, - String tableLocation, - String outputPathDirectory, + Location tableLocation, + Location outputPathDirectory, ConnectorSession session, DeltaLakeWriterStats stats, String trinoVersion) @@ -350,18 +351,18 @@ private int[] getWriterIndexes(Page page) closeWriter(writerIndex); } - String filePath = outputPathDirectory; + Location filePath = outputPathDirectory; List partitionValues = createPartitionValues(partitionColumnTypes, partitionColumns, position); Optional partitionName = Optional.empty(); if (!originalPartitionColumnNames.isEmpty()) { String partName = makePartName(originalPartitionColumnNames, partitionValues); - filePath = appendPath(outputPathDirectory, partName); + filePath = filePath.appendPath(partName); partitionName = Optional.of(partName); } String fileName = session.getQueryId() + "-" + randomUUID(); - filePath = appendPath(filePath, fileName); + filePath = filePath.appendPath(fileName); FileWriter fileWriter = createParquetFileWriter(filePath); @@ -440,7 +441,7 @@ public static List createPartitionValues(List partitionColumnTypes .collect(toList()); } - private FileWriter createParquetFileWriter(String path) + private FileWriter createParquetFileWriter(Location path) { ParquetWriterOptions parquetWriterOptions = ParquetWriterOptions.builder() .setMaxBlockSize(getParquetWriterBlockSize(session)) diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeCdfPageSink.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeCdfPageSink.java index 840b72a804ca..54d473103332 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeCdfPageSink.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeCdfPageSink.java @@ -14,6 +14,7 @@ package io.trino.plugin.deltalake; import io.airlift.json.JsonCodec; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.spi.PageIndexerFactory; import io.trino.spi.connector.ConnectorSession; @@ -37,8 +38,8 @@ public DeltaLakeCdfPageSink( TrinoFileSystemFactory fileSystemFactory, int maxOpenWriters, JsonCodec dataFileInfoCodec, - String outputPath, - String tableLocation, + Location tableLocation, + Location outputPath, ConnectorSession session, DeltaLakeWriterStats stats, String trinoVersion) diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMergeSink.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMergeSink.java index 0df7f5f9de71..72eb7645c4d5 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMergeSink.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMergeSink.java @@ -18,6 +18,7 @@ import io.airlift.json.JsonCodec; import io.airlift.slice.Slice; import io.airlift.slice.Slices; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.TrinoInputFile; @@ -101,7 +102,7 @@ public class DeltaLakeMergeSink private final JsonCodec dataFileInfoCodec; private final JsonCodec mergeResultJsonCodec; private final DeltaLakeWriterStats writerStats; - private final String rootTableLocation; + private final Location rootTableLocation; private final ConnectorPageSink insertPageSink; private final List dataColumns; private final List nonSynthesizedColumns; @@ -125,7 +126,7 @@ public DeltaLakeMergeSink( JsonCodec dataFileInfoCodec, JsonCodec mergeResultJsonCodec, DeltaLakeWriterStats writerStats, - String rootTableLocation, + Location rootTableLocation, ConnectorPageSink insertPageSink, List tableColumns, int domainCompactionThreshold, @@ -324,12 +325,12 @@ private List rewriteFile(String sourceLocationPath, FileDeletion deletion { try { Path sourcePath = new Path(sourceLocationPath); - Path rootTablePath = new Path(rootTableLocation); + Path rootTablePath = new Path(rootTableLocation.toString()); String sourceRelativePath = rootTablePath.toUri().relativize(sourcePath.toUri()).toString(); Path targetPath = new Path(sourcePath.getParent(), session.getQueryId() + "_" + randomUUID()); String targetRelativePath = rootTablePath.toUri().relativize(targetPath.toUri()).toString(); - FileWriter fileWriter = createParquetFileWriter(targetPath.toString(), dataColumns); + FileWriter fileWriter = createParquetFileWriter(Location.of(targetPath.toString()), dataColumns); DeltaLakeWriter writer = new DeltaLakeWriter( fileSystem, @@ -341,7 +342,7 @@ private List rewriteFile(String sourceLocationPath, FileDeletion deletion dataColumns, DATA); - Optional newFileInfo = rewriteParquetFile(sourcePath.toString(), deletion, writer); + Optional newFileInfo = rewriteParquetFile(Location.of(sourcePath.toString()), deletion, writer); DeltaLakeMergeResult result = new DeltaLakeMergeResult(Optional.of(sourceRelativePath), newFileInfo); return ImmutableList.of(utf8Slice(mergeResultJsonCodec.toJson(result))); @@ -351,7 +352,7 @@ private List rewriteFile(String sourceLocationPath, FileDeletion deletion } } - private FileWriter createParquetFileWriter(String path, List dataColumns) + private FileWriter createParquetFileWriter(Location path, List dataColumns) { ParquetWriterOptions parquetWriterOptions = ParquetWriterOptions.builder() .setMaxBlockSize(getParquetWriterBlockSize(session)) @@ -394,7 +395,7 @@ private FileWriter createParquetFileWriter(String path, List rewriteParquetFile(String path, FileDeletion deletion, DeltaLakeWriter fileWriter) + private Optional rewriteParquetFile(Location path, FileDeletion deletion, DeltaLakeWriter fileWriter) throws IOException { LongBitmapDataProvider rowsDeletedByDelete = deletion.rowsDeletedByDelete(); @@ -495,7 +496,7 @@ private void storeCdfEntries(Page page, int[] deleted, int deletedCount, FileDel } } - private ReaderPageSource createParquetPageSource(String path) + private ReaderPageSource createParquetPageSource(Location path) throws IOException { TrinoInputFile inputFile = fileSystem.newInputFile(path); diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMetadata.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMetadata.java index d06f35d19a3e..d095f30b59f4 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMetadata.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMetadata.java @@ -27,8 +27,8 @@ import io.airlift.slice.Slice; import io.airlift.stats.cardinality.HyperLogLog; import io.airlift.units.DataSize; -import io.trino.filesystem.FileEntry; import io.trino.filesystem.FileIterator; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.plugin.base.classloader.ClassLoaderSafeSystemTable; @@ -164,7 +164,6 @@ import static com.google.common.collect.Sets.difference; import static com.google.common.primitives.Ints.max; import static io.trino.filesystem.Locations.appendPath; -import static io.trino.filesystem.Locations.getFileName; import static io.trino.filesystem.Locations.getParent; import static io.trino.plugin.deltalake.DataFileInfo.DataFileType.DATA; import static io.trino.plugin.deltalake.DeltaLakeAnalyzeProperties.getColumnNames; @@ -684,9 +683,9 @@ public void createSchema(ConnectorSession session, String schemaName, Map location = DeltaLakeSchemaProperties.getLocation(properties).map(locationUri -> { try { - fileSystemFactory.create(session).newInputFile(locationUri).exists(); + fileSystemFactory.create(session).newInputFile(Location.of(locationUri)).exists(); } - catch (IOException e) { + catch (IOException | IllegalArgumentException e) { throw new TrinoException(INVALID_SCHEMA_PROPERTY, "Invalid location URI: " + locationUri, e); } return locationUri; @@ -733,7 +732,7 @@ public void dropSchema(ConnectorSession session, String schemaName) // If we see no files or can't see the location at all, use fallback. boolean deleteData = location.map(path -> { try { - return !fileSystemFactory.create(session).listFiles(path).hasNext(); + return !fileSystemFactory.create(session).listFiles(Location.of(path)).hasNext(); } catch (IOException | RuntimeException e) { LOG.warn(e, "Could not check schema directory '%s'", path); @@ -763,10 +762,10 @@ public void createTable(ConnectorSession session, ConnectorTableMetadata tableMe tableNameForLocation += "-" + randomUUID().toString().replace("-", ""); } location = appendPath(schemaLocation, tableNameForLocation); - checkPathContainsNoFiles(session, location); + checkPathContainsNoFiles(session, Location.of(location)); external = false; } - String deltaLogDirectory = getTransactionLogDir(location); + Location deltaLogDirectory = Location.of(getTransactionLogDir(location)); Optional checkpointInterval = getCheckpointInterval(tableMetadata.getProperties()); Optional changeDataFeedEnabled = getChangeDataFeedEnabled(tableMetadata.getProperties()); @@ -897,9 +896,9 @@ public DeltaLakeOutputTableHandle beginCreateTable(ConnectorSession session, Con location = appendPath(schemaLocation, tableNameForLocation); external = false; } - checkPathContainsNoFiles(session, location); - String finalLocation = location; + Location finalLocation = Location.of(location); + checkPathContainsNoFiles(session, finalLocation); setRollback(() -> deleteRecursivelyIfExists(fileSystemFactory.create(session), finalLocation)); return new DeltaLakeOutputTableHandle( @@ -924,7 +923,7 @@ private Optional getSchemaLocation(Database database) return schemaLocation; } - private void checkPathContainsNoFiles(ConnectorSession session, String targetPath) + private void checkPathContainsNoFiles(ConnectorSession session, Location targetPath) { try { TrinoFileSystem fileSystem = fileSystemFactory.create(session); @@ -971,7 +970,7 @@ private void checkColumnTypes(List columnMetadata) } } - private static void deleteRecursivelyIfExists(TrinoFileSystem fileSystem, String path) + private static void deleteRecursivelyIfExists(TrinoFileSystem fileSystem, Location path) { try { fileSystem.deleteDirectory(path); @@ -1065,9 +1064,8 @@ public Optional finishCreateTable( catch (Exception e) { // Remove the transaction log entry if the table creation fails try { - String transactionLogLocation = getTransactionLogDir(handle.getLocation()); - TrinoFileSystem fileSystem = fileSystemFactory.create(session); - fileSystem.deleteDirectory(transactionLogLocation); + Location transactionLogDir = Location.of(getTransactionLogDir(location)); + fileSystemFactory.create(session).deleteDirectory(transactionLogDir); } catch (IOException ioException) { // Nothing to do, the IOException is probably the same reason why the initial write failed @@ -1911,8 +1909,10 @@ private void writeCheckpointIfNeeded(ConnectorSession session, SchemaTableName t private void cleanupFailedWrite(ConnectorSession session, String tableLocation, List dataFiles) { - List filesToDelete = dataFiles.stream() - .map(dataFile -> appendPath(tableLocation, dataFile.getPath())) + Location location = Location.of(tableLocation); + List filesToDelete = dataFiles.stream() + .map(DataFileInfo::getPath) + .map(location::appendPath) .collect(toImmutableList()); try { TrinoFileSystem fileSystem = fileSystemFactory.create(session); @@ -1941,7 +1941,7 @@ public void dropTable(ConnectorSession session, ConnectorTableHandle tableHandle metastore.dropTable(session, handle.schemaTableName(), handle.location(), deleteData); if (deleteData) { try { - fileSystemFactory.create(session).deleteDirectory(handle.location()); + fileSystemFactory.create(session).deleteDirectory(Location.of(handle.location())); } catch (IOException e) { throw new TrinoException(DELTA_LAKE_FILESYSTEM_ERROR, format("Failed to delete directory %s of the table %s", handle.location(), handle.schemaTableName()), e); @@ -2572,13 +2572,13 @@ private void cleanExtraOutputFiles(ConnectorSession session, Set validWr .collect(toImmutableSet()); for (String location : fileLocations) { - cleanExtraOutputFiles(session, session.getQueryId(), location, validWrittenFilePaths); + cleanExtraOutputFiles(session, session.getQueryId(), Location.of(location), validWrittenFilePaths); } } - private void cleanExtraOutputFiles(ConnectorSession session, String queryId, String location, Set filesToKeep) + private void cleanExtraOutputFiles(ConnectorSession session, String queryId, Location location, Set filesToKeep) { - Deque filesToDelete = new ArrayDeque<>(); + Deque filesToDelete = new ArrayDeque<>(); try { LOG.debug("Deleting failed attempt files from %s for query %s", location, queryId); TrinoFileSystem fileSystem = fileSystemFactory.create(session); @@ -2590,10 +2590,9 @@ private void cleanExtraOutputFiles(ConnectorSession session, String queryId, Str // files within given partition are written flat into location; we need to list recursively FileIterator iterator = fileSystem.listFiles(location); while (iterator.hasNext()) { - FileEntry file = iterator.next(); - String fileName = getFileName(file.location()); - if (isFileCreatedByQuery(fileName, queryId) && !filesToKeep.contains(location + "/" + fileName)) { - filesToDelete.add(fileName); + Location file = iterator.next().location(); + if (isFileCreatedByQuery(file, queryId) && !filesToKeep.contains(file.toString())) { + filesToDelete.add(file); } } @@ -2602,31 +2601,17 @@ private void cleanExtraOutputFiles(ConnectorSession session, String queryId, Str } LOG.info("Found %s files to delete and %s to retain in location %s for query %s", filesToDelete.size(), filesToKeep.size(), location, queryId); - ImmutableList.Builder filesToDeleteBuilder = ImmutableList.builder(); - Iterator filesToDeleteIterator = filesToDelete.iterator(); - while (filesToDeleteIterator.hasNext()) { - String fileName = filesToDeleteIterator.next(); - LOG.debug("Going to delete failed attempt file %s/%s for query %s", location, fileName, queryId); - filesToDeleteBuilder.add(fileName); - filesToDeleteIterator.remove(); - } - - List deletedFiles = filesToDeleteBuilder.build(); - if (!deletedFiles.isEmpty()) { - fileSystem.deleteFiles(deletedFiles); - LOG.info("Deleted failed attempt files %s from %s for query %s", deletedFiles, location, queryId); - } + fileSystem.deleteFiles(filesToDelete); } catch (IOException e) { - throw new TrinoException(GENERIC_INTERNAL_ERROR, - format("Could not clean up extraneous output files; remaining files: %s", filesToDelete), e); + throw new TrinoException(DELTA_LAKE_FILESYSTEM_ERROR, "Failed to clean up extraneous output files", e); } } - private boolean isFileCreatedByQuery(String fileName, String queryId) + private static boolean isFileCreatedByQuery(Location file, String queryId) { verify(!queryId.contains("-"), "queryId(%s) should not contain hyphens", queryId); - return fileName.startsWith(queryId + "-"); + return file.fileName().startsWith(queryId + "-"); } @Override diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakePageSink.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakePageSink.java index 337c5cf4ce43..cedaca32a157 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakePageSink.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakePageSink.java @@ -14,6 +14,7 @@ package io.trino.plugin.deltalake; import io.airlift.json.JsonCodec; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.spi.PageIndexerFactory; import io.trino.spi.connector.ConnectorSession; @@ -34,7 +35,7 @@ public DeltaLakePageSink( TrinoFileSystemFactory fileSystemFactory, int maxOpenWriters, JsonCodec dataFileInfoCodec, - String tableLocation, + Location tableLocation, ConnectorSession session, DeltaLakeWriterStats stats, String trinoVersion) diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakePageSinkProvider.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakePageSinkProvider.java index 95a92142b0bd..90428a29a751 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakePageSinkProvider.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakePageSinkProvider.java @@ -15,6 +15,7 @@ import com.google.common.collect.ImmutableList; import io.airlift.json.JsonCodec; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.plugin.deltalake.procedure.DeltaLakeTableExecuteHandle; import io.trino.plugin.deltalake.procedure.DeltaTableOptimizeHandle; @@ -49,7 +50,6 @@ import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.changeDataFeedEnabled; import static io.trino.plugin.deltalake.transactionlog.DeltaLakeSchemaSupport.extractSchema; import static io.trino.spi.type.VarcharType.VARCHAR; -import static java.lang.String.format; import static java.util.Objects.requireNonNull; public class DeltaLakePageSinkProvider @@ -101,7 +101,7 @@ public ConnectorPageSink createPageSink(ConnectorTransactionHandle transactionHa fileSystemFactory, maxPartitionsPerWriter, dataFileInfoCodec, - tableHandle.getLocation(), + Location.of(tableHandle.getLocation()), session, stats, trinoVersion); @@ -119,7 +119,7 @@ public ConnectorPageSink createPageSink(ConnectorTransactionHandle transactionHa fileSystemFactory, maxPartitionsPerWriter, dataFileInfoCodec, - tableHandle.getLocation(), + Location.of(tableHandle.getLocation()), session, stats, trinoVersion); @@ -140,7 +140,7 @@ public ConnectorPageSink createPageSink(ConnectorTransactionHandle transactionHa fileSystemFactory, maxPartitionsPerWriter, dataFileInfoCodec, - executeHandle.getTableLocation(), + Location.of(executeHandle.getTableLocation()), session, stats, trinoVersion); @@ -165,7 +165,7 @@ public ConnectorMergeSink createMergeSink(ConnectorTransactionHandle transaction dataFileInfoCodec, mergeResultJsonCodec, stats, - tableHandle.getLocation(), + Location.of(tableHandle.getLocation()), pageSink, tableHandle.getInputColumns(), domainCompactionThreshold, @@ -198,6 +198,7 @@ private DeltaLakeCdfPageSink createCdfPageSink( VARCHAR, REGULAR)) .build(); + Location tableLocation = Location.of(mergeTableHandle.getTableHandle().getLocation()); return new DeltaLakeCdfPageSink( typeManager.getTypeOperators(), @@ -207,8 +208,8 @@ private DeltaLakeCdfPageSink createCdfPageSink( fileSystemFactory, maxPartitionsPerWriter, dataFileInfoCodec, - format("%s/%s/", mergeTableHandle.getTableHandle().getLocation(), CHANGE_DATA_FOLDER_NAME), - mergeTableHandle.getTableHandle().getLocation(), + tableLocation, + tableLocation.appendPath(CHANGE_DATA_FOLDER_NAME), session, stats, trinoVersion); diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakePageSourceProvider.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakePageSourceProvider.java index 32e59a8be78e..fd9b818fa2f0 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakePageSourceProvider.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakePageSourceProvider.java @@ -17,6 +17,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.TrinoInputFile; import io.trino.parquet.ParquetDataSource; @@ -176,7 +177,8 @@ public ConnectorPageSource createPageSource( split.getFileModifiedTime()); } - TrinoInputFile inputFile = fileSystemFactory.create(session).newInputFile(split.getPath(), split.getFileSize()); + Location location = Location.of(split.getPath()); + TrinoInputFile inputFile = fileSystemFactory.create(session).newInputFile(location, split.getFileSize()); ParquetReaderOptions options = parquetReaderOptions.withMaxReadBlockSize(getParquetMaxReadBlockSize(session)) .withMaxReadBlockRowCount(getParquetMaxReadBlockRowCount(session)) .withUseColumnIndex(isParquetUseColumnIndex(session)) diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeWriter.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeWriter.java index 6dab9ef9e8f9..2058933adc3f 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeWriter.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeWriter.java @@ -17,6 +17,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMultimap; import com.google.common.collect.Multimap; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoInputFile; import io.trino.parquet.ParquetReaderOptions; @@ -60,7 +61,6 @@ import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.ImmutableMap.toImmutableMap; import static com.google.common.collect.Iterables.getOnlyElement; -import static io.trino.filesystem.Locations.appendPath; import static io.trino.plugin.deltalake.transactionlog.DeltaLakeParquetStatisticsUtils.hasInvalidStatistics; import static io.trino.plugin.deltalake.transactionlog.DeltaLakeParquetStatisticsUtils.jsonEncodeMax; import static io.trino.plugin.deltalake.transactionlog.DeltaLakeParquetStatisticsUtils.jsonEncodeMin; @@ -78,7 +78,7 @@ public class DeltaLakeWriter { private final TrinoFileSystem fileSystem; private final FileWriter fileWriter; - private final String rootTableLocation; + private final Location rootTableLocation; private final String relativeFilePath; private final List partitionValues; private final DeltaLakeWriterStats stats; @@ -93,7 +93,7 @@ public class DeltaLakeWriter public DeltaLakeWriter( TrinoFileSystem fileSystem, FileWriter fileWriter, - String rootTableLocation, + Location rootTableLocation, String relativeFilePath, List partitionValues, DeltaLakeWriterStats stats, @@ -185,6 +185,7 @@ public long getRowCount() public DataFileInfo getDataFileInfo() throws IOException { + TrinoInputFile inputFile = fileSystem.newInputFile(rootTableLocation.appendPath(relativeFilePath)); List dataColumnNames = columnHandles.stream().map(DeltaLakeColumnHandle::getPhysicalName).collect(toImmutableList()); List dataColumnTypes = columnHandles.stream().map(DeltaLakeColumnHandle::getPhysicalType).collect(toImmutableList()); return new DataFileInfo( @@ -193,16 +194,10 @@ public DataFileInfo getDataFileInfo() creationTime, dataFileType, partitionValues, - readStatistics(fileSystem, rootTableLocation, dataColumnNames, dataColumnTypes, relativeFilePath, rowCount)); + readStatistics(inputFile, dataColumnNames, dataColumnTypes, rowCount)); } - private static DeltaLakeJsonFileStatistics readStatistics( - TrinoFileSystem fileSystem, - String tableLocation, - List dataColumnNames, - List dataColumnTypes, - String relativeFilePath, - Long rowCount) + private static DeltaLakeJsonFileStatistics readStatistics(TrinoInputFile inputFile, List dataColumnNames, List dataColumnTypes, long rowCount) throws IOException { ImmutableMap.Builder typeForColumn = ImmutableMap.builder(); @@ -210,7 +205,6 @@ private static DeltaLakeJsonFileStatistics readStatistics( typeForColumn.put(dataColumnNames.get(i), dataColumnTypes.get(i)); } - TrinoInputFile inputFile = fileSystem.newInputFile(appendPath(tableLocation, relativeFilePath)); try (TrinoParquetDataSource trinoParquetDataSource = new TrinoParquetDataSource( inputFile, new ParquetReaderOptions(), diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/procedure/RegisterTableProcedure.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/procedure/RegisterTableProcedure.java index ef7e40ef008c..b568f7b942e6 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/procedure/RegisterTableProcedure.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/procedure/RegisterTableProcedure.java @@ -14,6 +14,7 @@ package io.trino.plugin.deltalake.procedure; import com.google.common.collect.ImmutableList; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.plugin.deltalake.DeltaLakeConfig; @@ -143,7 +144,7 @@ private void doRegisterTable( TrinoFileSystem fileSystem = fileSystemFactory.create(session); try { - String transactionLogDir = getTransactionLogDir(tableLocation); + Location transactionLogDir = Location.of(getTransactionLogDir(tableLocation)); if (!fileSystem.listFiles(transactionLogDir).hasNext()) { throw new TrinoException(GENERIC_USER_ERROR, format("No transaction log found in location %s", transactionLogDir)); } diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/procedure/VacuumProcedure.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/procedure/VacuumProcedure.java index 37185a5db2a3..19c962cba5ea 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/procedure/VacuumProcedure.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/procedure/VacuumProcedure.java @@ -18,6 +18,7 @@ import io.airlift.units.Duration; import io.trino.filesystem.FileEntry; import io.trino.filesystem.FileIterator; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.plugin.base.CatalogName; @@ -213,11 +214,11 @@ private void doVacuum( long retainedUnknownFiles = 0; long removedFiles = 0; - List filesToDelete = new ArrayList<>(); - FileIterator listing = fileSystem.listFiles(tableLocation); + List filesToDelete = new ArrayList<>(); + FileIterator listing = fileSystem.listFiles(Location.of(tableLocation)); while (listing.hasNext()) { FileEntry entry = listing.next(); - String location = entry.location(); + String location = entry.location().toString(); checkState( location.startsWith(commonPathPrefix), "Unexpected path [%s] returned when listing files under [%s]", @@ -253,7 +254,7 @@ private void doVacuum( } log.debug("[%s] deleting file [%s] with modification time %s", queryId, location, modificationTime); - filesToDelete.add(location); + filesToDelete.add(entry.location()); if (filesToDelete.size() == DELETE_BATCH_SIZE) { fileSystem.deleteFiles(filesToDelete); removedFiles += filesToDelete.size(); diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/statistics/MetaDirStatisticsAccess.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/statistics/MetaDirStatisticsAccess.java index a336c5509f21..1043d67fbecd 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/statistics/MetaDirStatisticsAccess.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/statistics/MetaDirStatisticsAccess.java @@ -15,6 +15,7 @@ import com.google.inject.Inject; import io.airlift.json.JsonCodec; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.TrinoInputFile; @@ -26,7 +27,6 @@ import java.io.OutputStream; import java.util.Optional; -import static io.trino.filesystem.Locations.appendPath; import static io.trino.plugin.deltalake.transactionlog.TransactionLogUtil.TRANSACTION_LOG_DIRECTORY; import static io.trino.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; import static java.lang.String.format; @@ -58,14 +58,15 @@ public Optional readExtendedStatistics( ConnectorSession session, String tableLocation) { - return readExtendedStatistics(session, tableLocation, STATISTICS_META_DIR, STATISTICS_FILE) - .or(() -> readExtendedStatistics(session, tableLocation, STARBURST_META_DIR, STARBURST_STATISTICS_FILE)); + Location location = Location.of(tableLocation); + return readExtendedStatistics(session, location, STATISTICS_META_DIR, STATISTICS_FILE) + .or(() -> readExtendedStatistics(session, location, STARBURST_META_DIR, STARBURST_STATISTICS_FILE)); } - private Optional readExtendedStatistics(ConnectorSession session, String tableLocation, String statisticsDirectory, String statisticsFile) + private Optional readExtendedStatistics(ConnectorSession session, Location tableLocation, String statisticsDirectory, String statisticsFile) { try { - String statisticsPath = appendPath(tableLocation, appendPath(statisticsDirectory, statisticsFile)); + Location statisticsPath = tableLocation.appendPath(statisticsDirectory).appendPath(statisticsFile); TrinoInputFile inputFile = fileSystemFactory.create(session).newInputFile(statisticsPath); if (!inputFile.exists()) { return Optional.empty(); @@ -87,7 +88,7 @@ public void updateExtendedStatistics( ExtendedStatistics statistics) { try { - String statisticsPath = appendPath(tableLocation, appendPath(STATISTICS_META_DIR, STATISTICS_FILE)); + Location statisticsPath = Location.of(tableLocation).appendPath(STATISTICS_META_DIR).appendPath(STATISTICS_FILE); TrinoFileSystem fileSystem = fileSystemFactory.create(session); try (OutputStream outputStream = fileSystem.newOutputFile(statisticsPath).createOrOverwrite()) { @@ -95,7 +96,7 @@ public void updateExtendedStatistics( } // Remove outdated Starburst stats file, if it exists. - String starburstStatisticsPath = appendPath(tableLocation, appendPath(STARBURST_META_DIR, STARBURST_STATISTICS_FILE)); + Location starburstStatisticsPath = Location.of(tableLocation).appendPath(STARBURST_META_DIR).appendPath(STARBURST_STATISTICS_FILE); if (fileSystem.newInputFile(starburstStatisticsPath).exists()) { fileSystem.deleteFile(starburstStatisticsPath); } @@ -108,7 +109,7 @@ public void updateExtendedStatistics( @Override public void deleteExtendedStatistics(ConnectorSession session, String tableLocation) { - String statisticsPath = appendPath(tableLocation, appendPath(STATISTICS_META_DIR, STATISTICS_FILE)); + Location statisticsPath = Location.of(tableLocation).appendPath(STATISTICS_META_DIR).appendPath(STATISTICS_FILE); try { TrinoFileSystem fileSystem = fileSystemFactory.create(session); if (fileSystem.newInputFile(statisticsPath).exists()) { diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/TableSnapshot.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/TableSnapshot.java index 167c34a4fd38..e89c149a28a2 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/TableSnapshot.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/TableSnapshot.java @@ -15,6 +15,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoInputFile; import io.trino.parquet.ParquetReaderOptions; @@ -36,7 +37,6 @@ import java.util.stream.Stream; import static com.google.common.collect.Streams.stream; -import static io.trino.filesystem.Locations.appendPath; import static io.trino.plugin.deltalake.DeltaLakeErrorCode.DELTA_LAKE_BAD_DATA; import static io.trino.plugin.deltalake.DeltaLakeErrorCode.DELTA_LAKE_INVALID_SCHEMA; import static io.trino.plugin.deltalake.transactionlog.TransactionLogParser.readLastCheckpoint; @@ -185,7 +185,7 @@ public Stream getCheckpointTransactionLogEntries( Optional.empty(); Stream resultStream = Stream.empty(); - for (String checkpointPath : getCheckpointPartPaths(checkpoint)) { + for (Location checkpointPath : getCheckpointPartPaths(checkpoint)) { TrinoInputFile checkpointFile = fileSystem.newInputFile(checkpointPath); resultStream = Stream.concat( resultStream, @@ -248,7 +248,7 @@ private MetadataEntry getCheckpointMetadataEntry( LastCheckpoint checkpoint) throws IOException { - for (String checkpointPath : getCheckpointPartPaths(checkpoint)) { + for (Location checkpointPath : getCheckpointPartPaths(checkpoint)) { TrinoInputFile checkpointFile = fileSystem.newInputFile(checkpointPath); Stream metadataEntries = getCheckpointTransactionLogEntries( session, @@ -267,17 +267,17 @@ private MetadataEntry getCheckpointMetadataEntry( throw new TrinoException(DELTA_LAKE_BAD_DATA, "Checkpoint found without metadata entry: " + checkpoint); } - private List getCheckpointPartPaths(LastCheckpoint checkpoint) + private List getCheckpointPartPaths(LastCheckpoint checkpoint) { - String transactionLogDir = getTransactionLogDir(tableLocation); - ImmutableList.Builder paths = ImmutableList.builder(); + Location transactionLogDir = Location.of(getTransactionLogDir(tableLocation)); + ImmutableList.Builder paths = ImmutableList.builder(); if (checkpoint.getParts().isEmpty()) { - paths.add(appendPath(transactionLogDir, format("%020d.checkpoint.parquet", checkpoint.getVersion()))); + paths.add(transactionLogDir.appendPath("%020d.checkpoint.parquet".formatted(checkpoint.getVersion()))); } else { int partsCount = checkpoint.getParts().get(); for (int i = 1; i <= partsCount; i++) { - paths.add(appendPath(transactionLogDir, format("%020d.checkpoint.%010d.%010d.parquet", checkpoint.getVersion(), i, partsCount))); + paths.add(transactionLogDir.appendPath("%020d.checkpoint.%010d.%010d.parquet".formatted(checkpoint.getVersion(), i, partsCount))); } } return paths.build(); diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/TransactionLogAccess.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/TransactionLogAccess.java index 8e0e5f9ddeda..a1903d939264 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/TransactionLogAccess.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/TransactionLogAccess.java @@ -22,6 +22,7 @@ import io.airlift.jmx.CacheStatsMBean; import io.airlift.log.Logger; import io.trino.collect.cache.EvictableCacheBuilder; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.TrinoInputFile; @@ -414,7 +415,7 @@ public List getPastTableVersions(TrinoFileSystem fileSystem, String transa { ImmutableList.Builder result = ImmutableList.builder(); for (long version = lastVersion; version >= 0; version--) { - String entryPath = getTransactionLogJsonEntryPath(transactionLogDir, version); + Location entryPath = getTransactionLogJsonEntryPath(transactionLogDir, version); TrinoInputFile inputFile = fileSystem.newInputFile(entryPath); try { if (inputFile.lastModified().isBefore(startAt)) { diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/TransactionLogParser.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/TransactionLogParser.java index bf0053a8998b..89c5325cc215 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/TransactionLogParser.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/TransactionLogParser.java @@ -21,6 +21,7 @@ import dev.failsafe.RetryPolicy; import io.airlift.json.ObjectMapperProvider; import io.airlift.log.Logger; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoInputFile; import io.trino.plugin.base.util.JsonUtils; @@ -53,7 +54,6 @@ import java.util.function.Function; import static io.airlift.slice.Slices.utf8Slice; -import static io.trino.filesystem.Locations.appendPath; import static io.trino.plugin.deltalake.transactionlog.TransactionLogUtil.getTransactionLogDir; import static io.trino.plugin.deltalake.transactionlog.TransactionLogUtil.getTransactionLogJsonEntryPath; import static io.trino.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; @@ -234,7 +234,7 @@ static Optional readLastCheckpoint(TrinoFileSystem fileSystem, S private static Optional tryReadLastCheckpoint(TrinoFileSystem fileSystem, String tableLocation) throws JsonParseException, JsonMappingException { - String checkpointPath = appendPath(getTransactionLogDir(tableLocation), LAST_CHECKPOINT_FILENAME); + Location checkpointPath = Location.of(getTransactionLogDir(tableLocation)).appendPath(LAST_CHECKPOINT_FILENAME); TrinoInputFile inputFile = fileSystem.newInputFile(checkpointPath); try (InputStream lastCheckpointInput = inputFile.newStream()) { // Note: there apparently is 8K buffering applied and _last_checkpoint should be much smaller. @@ -259,7 +259,7 @@ public static long getMandatoryCurrentVersion(TrinoFileSystem fileSystem, String String transactionLogDir = getTransactionLogDir(tableLocation); while (true) { - String entryPath = getTransactionLogJsonEntryPath(transactionLogDir, version + 1); + Location entryPath = getTransactionLogJsonEntryPath(transactionLogDir, version + 1); if (!fileSystem.newInputFile(entryPath).exists()) { return version; } diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/TransactionLogUtil.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/TransactionLogUtil.java index 27c63f881c55..40dc6004e763 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/TransactionLogUtil.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/TransactionLogUtil.java @@ -13,8 +13,9 @@ */ package io.trino.plugin.deltalake.transactionlog; +import io.trino.filesystem.Location; + import static io.trino.filesystem.Locations.appendPath; -import static java.lang.String.format; public final class TransactionLogUtil { @@ -27,8 +28,8 @@ public static String getTransactionLogDir(String tableLocation) return appendPath(tableLocation, TRANSACTION_LOG_DIRECTORY); } - public static String getTransactionLogJsonEntryPath(String transactionLogDir, long entryNumber) + public static Location getTransactionLogJsonEntryPath(String transactionLogDir, long entryNumber) { - return appendPath(transactionLogDir, format("%020d.json", entryNumber)); + return Location.of(transactionLogDir).appendPath("%020d.json".formatted(entryNumber)); } } diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java index 24a698408823..f7a957cc233c 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointEntryIterator.java @@ -146,7 +146,7 @@ public CheckpointEntryIterator( boolean checkpointRowStatisticsWritingEnabled, int domainCompactionThreshold) { - this.checkpointPath = checkpoint.location(); + this.checkpointPath = checkpoint.location().toString(); this.session = requireNonNull(session, "session is null"); this.stringList = (ArrayType) typeManager.getType(TypeSignature.arrayType(VarcharType.VARCHAR.getTypeSignature())); this.stringMap = (MapType) typeManager.getType(TypeSignature.mapType(VarcharType.VARCHAR.getTypeSignature(), VarcharType.VARCHAR.getTypeSignature())); diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointWriterManager.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointWriterManager.java index 49fcd17ff41c..f3425d906c78 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointWriterManager.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/CheckpointWriterManager.java @@ -15,6 +15,7 @@ import com.google.common.collect.ImmutableSet; import io.airlift.json.JsonCodec; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.TrinoOutputFile; @@ -37,7 +38,6 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; import static com.google.common.collect.MoreCollectors.toOptional; -import static io.trino.filesystem.Locations.appendPath; import static io.trino.plugin.deltalake.transactionlog.TransactionLogParser.LAST_CHECKPOINT_FILENAME; import static io.trino.plugin.deltalake.transactionlog.TransactionLogUtil.getTransactionLogDir; import static io.trino.plugin.deltalake.transactionlog.checkpoint.CheckpointEntryIterator.EntryType.ADD; @@ -127,8 +127,8 @@ public void writeCheckpoint(ConnectorSession session, TableSnapshot snapshot) snapshot.getJsonTransactionLogEntries() .forEach(checkpointBuilder::addLogEntry); - String transactionLogDirectory = getTransactionLogDir(snapshot.getTableLocation()); - String targetFile = appendPath(transactionLogDirectory, String.format("%020d.checkpoint.parquet", newCheckpointVersion)); + Location transactionLogDir = Location.of(getTransactionLogDir(snapshot.getTableLocation())); + Location targetFile = transactionLogDir.appendPath("%020d.checkpoint.parquet".formatted(newCheckpointVersion)); CheckpointWriter checkpointWriter = new CheckpointWriter(typeManager, checkpointSchemaManager, trinoVersion); CheckpointEntries checkpointEntries = checkpointBuilder.build(); TrinoOutputFile checkpointFile = fileSystemFactory.create(session).newOutputFile(targetFile); @@ -136,7 +136,7 @@ public void writeCheckpoint(ConnectorSession session, TableSnapshot snapshot) // update last checkpoint file LastCheckpoint newLastCheckpoint = new LastCheckpoint(newCheckpointVersion, checkpointEntries.size(), Optional.empty()); - String checkpointPath = appendPath(transactionLogDirectory, LAST_CHECKPOINT_FILENAME); + Location checkpointPath = transactionLogDir.appendPath(LAST_CHECKPOINT_FILENAME); TrinoOutputFile outputFile = fileSystem.newOutputFile(checkpointPath); try (OutputStream outputStream = outputFile.createOrOverwrite()) { outputStream.write(lastCheckpointCodec.toJsonBytes(newLastCheckpoint)); diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/TransactionLogTail.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/TransactionLogTail.java index c42265d716a9..fbbdc15f93ff 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/TransactionLogTail.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/checkpoint/TransactionLogTail.java @@ -14,6 +14,7 @@ package io.trino.plugin.deltalake.transactionlog.checkpoint; import com.google.common.collect.ImmutableList; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoInputFile; import io.trino.plugin.deltalake.transactionlog.DeltaLakeTransactionLogEntry; @@ -80,7 +81,7 @@ public static TransactionLogTail loadNewTail( } else { if (endVersion.isPresent()) { - throw new MissingTransactionLogException(getTransactionLogJsonEntryPath(transactionLogDir, entryNumber)); + throw new MissingTransactionLogException(getTransactionLogJsonEntryPath(transactionLogDir, entryNumber).toString()); } endOfTail = true; } @@ -126,7 +127,7 @@ public Optional getUpdatedTail(TrinoFileSystem fileSystem, S public static Optional> getEntriesFromJson(long entryNumber, String transactionLogDir, TrinoFileSystem fileSystem) throws IOException { - String transactionLogFilePath = getTransactionLogJsonEntryPath(transactionLogDir, entryNumber); + Location transactionLogFilePath = getTransactionLogJsonEntryPath(transactionLogDir, entryNumber); TrinoInputFile inputFile = fileSystem.newInputFile(transactionLogFilePath); try (BufferedReader reader = new BufferedReader( new InputStreamReader(inputFile.newStream(), UTF_8), diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/writer/AzureTransactionLogSynchronizer.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/writer/AzureTransactionLogSynchronizer.java index d1caf1cac94c..94150e84314e 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/writer/AzureTransactionLogSynchronizer.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/writer/AzureTransactionLogSynchronizer.java @@ -13,6 +13,7 @@ */ package io.trino.plugin.deltalake.transactionlog.writer; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.spi.connector.ConnectorSession; @@ -40,9 +41,9 @@ public AzureTransactionLogSynchronizer(TrinoFileSystemFactory fileSystemFactory) // This approach should be compatible with OSS Delta Lake. // We assume ADLS Gen2 supports atomic renames which will not overwrite existing files @Override - public void write(ConnectorSession session, String clusterId, String newLogEntryPath, byte[] entryContents) + public void write(ConnectorSession session, String clusterId, Location newLogEntryPath, byte[] entryContents) { - String tmpFilePath = newLogEntryPath + "." + UUID.randomUUID() + ".tmp"; + Location tmpFilePath = newLogEntryPath.appendSuffix("." + UUID.randomUUID() + ".tmp"); boolean conflict = false; TrinoFileSystem fileSystem = fileSystemFactory.create(session); diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/writer/GcsTransactionLogSynchronizer.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/writer/GcsTransactionLogSynchronizer.java index 798e1dd11220..bed00dcb7cc1 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/writer/GcsTransactionLogSynchronizer.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/writer/GcsTransactionLogSynchronizer.java @@ -17,6 +17,7 @@ import com.google.cloud.hadoop.repackaged.gcs.com.google.api.services.storage.Storage; import com.google.cloud.hadoop.repackaged.gcs.com.google.api.services.storage.model.StorageObject; import com.google.cloud.hadoop.repackaged.gcs.com.google.cloud.hadoop.gcsio.StorageResourceId; +import io.trino.filesystem.Location; import io.trino.plugin.deltalake.GcsStorageFactory; import io.trino.spi.connector.ConnectorSession; @@ -45,11 +46,11 @@ public GcsTransactionLogSynchronizer(GcsStorageFactory gcsStorageFactory) // in order to avoid leaked output streams in case of I/O exceptions occurring while uploading // the blob content. @Override - public void write(ConnectorSession session, String clusterId, String newLogEntryPath, byte[] entryContents) + public void write(ConnectorSession session, String clusterId, Location newLogEntryPath, byte[] entryContents) { - Storage storage = gcsStorageFactory.create(session, newLogEntryPath); + Storage storage = gcsStorageFactory.create(session, newLogEntryPath.toString()); try { - createStorageObjectExclusively(newLogEntryPath, entryContents, storage); + createStorageObjectExclusively(newLogEntryPath.toString(), entryContents, storage); } catch (IOException e) { throw new UncheckedIOException(e); diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/writer/NoIsolationSynchronizer.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/writer/NoIsolationSynchronizer.java index 15dce8180d3b..ca824b85eaf6 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/writer/NoIsolationSynchronizer.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/writer/NoIsolationSynchronizer.java @@ -13,6 +13,7 @@ */ package io.trino.plugin.deltalake.transactionlog.writer; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.TrinoOutputFile; @@ -38,7 +39,7 @@ public NoIsolationSynchronizer(TrinoFileSystemFactory fileSystemFactory) } @Override - public void write(ConnectorSession session, String clusterId, String newLogEntryPath, byte[] entryContents) + public void write(ConnectorSession session, String clusterId, Location newLogEntryPath, byte[] entryContents) throws UncheckedIOException { TrinoFileSystem fileSystem = fileSystemFactory.create(session); diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/writer/S3NativeTransactionLogSynchronizer.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/writer/S3NativeTransactionLogSynchronizer.java index 56028acd2a51..6e82c5240168 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/writer/S3NativeTransactionLogSynchronizer.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/writer/S3NativeTransactionLogSynchronizer.java @@ -20,8 +20,10 @@ import io.airlift.log.Logger; import io.trino.filesystem.FileEntry; import io.trino.filesystem.FileIterator; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.filesystem.TrinoInputFile; import io.trino.filesystem.TrinoOutputFile; import io.trino.spi.connector.ConnectorSession; @@ -40,9 +42,6 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; -import static io.trino.filesystem.Locations.appendPath; -import static io.trino.filesystem.Locations.getFileName; -import static io.trino.filesystem.Locations.getParent; import static java.lang.String.format; import static java.time.temporal.ChronoUnit.MINUTES; import static java.util.Objects.requireNonNull; @@ -79,11 +78,11 @@ public boolean isUnsafe() } @Override - public void write(ConnectorSession session, String clusterId, String newLogEntryPath, byte[] entryContents) + public void write(ConnectorSession session, String clusterId, Location newLogEntryPath, byte[] entryContents) { TrinoFileSystem fileSystem = fileSystemFactory.create(session); - String locksDirectory = appendPath(getParent(newLogEntryPath), LOCK_DIRECTORY); - String newEntryFilename = getFileName(newLogEntryPath); + Location locksDirectory = newLogEntryPath.parentDirectory().appendPath(LOCK_DIRECTORY); + String newEntryFilename = newLogEntryPath.fileName(); Optional myLockInfo = Optional.empty(); try { @@ -162,13 +161,13 @@ public void write(ConnectorSession session, String clusterId, String newLogEntry } } - private LockInfo writeNewLockInfo(TrinoFileSystem fileSystem, String lockDirectory, String logEntryFilename, String clusterId, String queryId) + private LockInfo writeNewLockInfo(TrinoFileSystem fileSystem, Location lockDirectory, String logEntryFilename, String clusterId, String queryId) throws IOException { String lockFilename = logEntryFilename + "." + LOCK_INFIX + queryId; Instant expiration = Instant.now().plus(EXPIRATION_DURATION); LockFileContents contents = new LockFileContents(clusterId, queryId, expiration.toEpochMilli()); - String lockPath = appendPath(lockDirectory, lockFilename); + Location lockPath = lockDirectory.appendPath(lockFilename); TrinoOutputFile lockFile = fileSystem.newOutputFile(lockPath); byte[] contentsBytes = lockFileContentsJsonCodec.toJsonBytes(contents); try (OutputStream outputStream = lockFile.create()) { @@ -177,14 +176,13 @@ private LockInfo writeNewLockInfo(TrinoFileSystem fileSystem, String lockDirecto return new LockInfo(lockFilename, contents); } - private static void deleteLock(TrinoFileSystem fileSystem, String lockDirectoryPath, LockInfo lockInfo) + private static void deleteLock(TrinoFileSystem fileSystem, Location lockDirectoryPath, LockInfo lockInfo) throws IOException { - String lockPath = appendPath(lockDirectoryPath, lockInfo.getLockFilename()); - fileSystem.deleteFile(lockPath); + fileSystem.deleteFile(lockDirectoryPath.appendPath(lockInfo.getLockFilename())); } - private List listLockInfos(TrinoFileSystem fileSystem, String lockDirectoryPath) + private List listLockInfos(TrinoFileSystem fileSystem, Location lockDirectoryPath) throws IOException { FileIterator files = fileSystem.listFiles(lockDirectoryPath); @@ -192,21 +190,21 @@ private List listLockInfos(TrinoFileSystem fileSystem, String lockDire while (files.hasNext()) { FileEntry entry = files.next(); - String name = entry.location().substring(entry.location().lastIndexOf('/') + 1); + String name = entry.location().fileName(); if (LOCK_FILENAME_PATTERN.matcher(name).matches()) { - Optional lockInfo = parseLockFile(fileSystem, entry.location(), name); - lockInfo.ifPresent(lockInfos::add); + TrinoInputFile file = fileSystem.newInputFile(entry.location()); + parseLockFile(file, name).ifPresent(lockInfos::add); } } return lockInfos.build(); } - private Optional parseLockFile(TrinoFileSystem fileSystem, String path, String name) + private Optional parseLockFile(TrinoInputFile file, String name) throws IOException { byte[] bytes = null; - try (InputStream inputStream = fileSystem.newInputFile(path).newStream()) { + try (InputStream inputStream = file.newStream()) { bytes = inputStream.readAllBytes(); LockFileContents lockFileContents = lockFileContentsJsonCodec.fromJson(bytes); return Optional.of(new LockInfo(name, lockFileContents)); @@ -216,7 +214,7 @@ private Optional parseLockFile(TrinoFileSystem fileSystem, String path if (bytes != null) { content = Base64.getEncoder().encodeToString(bytes); } - LOG.warn(e, "Could not parse lock file: %s; contents=%s", path, content); + LOG.warn(e, "Could not parse lock file: %s; contents=%s", file.location(), content); return Optional.empty(); } catch (FileNotFoundException e) { diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/writer/TransactionLogSynchronizer.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/writer/TransactionLogSynchronizer.java index ef287290d9c6..74032c514cb6 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/writer/TransactionLogSynchronizer.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/writer/TransactionLogSynchronizer.java @@ -13,6 +13,7 @@ */ package io.trino.plugin.deltalake.transactionlog.writer; +import io.trino.filesystem.Location; import io.trino.spi.connector.ConnectorSession; public interface TransactionLogSynchronizer @@ -23,7 +24,7 @@ public interface TransactionLogSynchronizer * @throws TransactionConflictException If file cannot be written because of conflict with other transaction * @throws RuntimeException If some other unexpected error occurs */ - void write(ConnectorSession session, String clusterId, String newLogEntryPath, byte[] entryContents); + void write(ConnectorSession session, String clusterId, Location newLogEntryPath, byte[] entryContents); /** * Whether or not writes using this Synchronizer need to be enabled with the "delta.enable-non-concurrent-writes" config property. diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/writer/TransactionLogWriter.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/writer/TransactionLogWriter.java index 9aa673bccb0e..e16f87e372c1 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/writer/TransactionLogWriter.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/transactionlog/writer/TransactionLogWriter.java @@ -15,6 +15,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import io.airlift.json.ObjectMapperProvider; +import io.trino.filesystem.Location; import io.trino.plugin.deltalake.transactionlog.AddFileEntry; import io.trino.plugin.deltalake.transactionlog.CdfFileEntry; import io.trino.plugin.deltalake.transactionlog.CommitInfoEntry; @@ -97,7 +98,7 @@ public void flush() String transactionLogLocation = getTransactionLogDir(tableLocation); CommitInfoEntry commitInfo = requireNonNull(commitInfoEntry.get().getCommitInfo(), "commitInfoEntry.get().getCommitInfo() is null"); - String logEntry = getTransactionLogJsonEntryPath(transactionLogLocation, commitInfo.getVersion()); + Location logEntry = getTransactionLogJsonEntryPath(transactionLogLocation, commitInfo.getVersion()); ByteArrayOutputStream bos = new ByteArrayOutputStream(); writeEntry(bos, commitInfoEntry.get()); diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeRegisterTableProcedureTest.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeRegisterTableProcedureTest.java index e3de2c814af6..e6cb69f32ddb 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeRegisterTableProcedureTest.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeRegisterTableProcedureTest.java @@ -228,7 +228,7 @@ public void testRegisterTableWithInvalidDeltaTable() // Delete files under transaction log directory and put an invalid log file to verify register_table call fails String transactionLogDir = new URI(getTransactionLogDir(tableLocation)).getPath(); deleteDirectoryContents(Path.of(transactionLogDir), ALLOW_INSECURE); - new File(getTransactionLogJsonEntryPath(transactionLogDir, 0)).createNewFile(); + new File("/" + getTransactionLogJsonEntryPath(transactionLogDir, 0).path()).createNewFile(); assertQueryFails(format("CALL system.register_table('%s', '%s', '%s')", SCHEMA, tableNameNew, tableLocation), ".*Metadata not found in transaction log for (.*)"); diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeTableWithCustomLocation.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeTableWithCustomLocation.java index ddf2ab7e80ad..e0d05e44588b 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeTableWithCustomLocation.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/BaseDeltaLakeTableWithCustomLocation.java @@ -13,6 +13,7 @@ */ package io.trino.plugin.deltalake; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.plugin.hive.metastore.HiveMetastore; import io.trino.plugin.hive.metastore.Table; @@ -62,12 +63,12 @@ public void testCreateAndDrop() Table table = metastore.getTable(SCHEMA, tableName).orElseThrow(); assertThat(table.getTableType()).isEqualTo(MANAGED_TABLE.name()); - String tableLocation = table.getStorage().getLocation(); + Location tableLocation = Location.of(table.getStorage().getLocation()); TrinoFileSystem fileSystem = HDFS_FILE_SYSTEM_FACTORY.create(getSession().toConnectorSession()); assertTrue(fileSystem.listFiles(tableLocation).hasNext(), "The directory corresponding to the table storage location should exist"); List materializedRows = computeActual("SELECT \"$path\" FROM " + tableName).getMaterializedRows(); assertEquals(materializedRows.size(), 1); - String filePath = (String) materializedRows.get(0).getField(0); + Location filePath = Location.of((String) materializedRows.get(0).getField(0)); assertTrue(fileSystem.listFiles(filePath).hasNext(), "The data file should exist"); assertQuerySucceeds(format("DROP TABLE %s", tableName)); assertFalse(metastore.getTable(SCHEMA, tableName).isPresent(), "Table should be dropped"); diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/FileTestingTransactionLogSynchronizer.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/FileTestingTransactionLogSynchronizer.java index 5ecffc4c7caa..bfee345fd25d 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/FileTestingTransactionLogSynchronizer.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/FileTestingTransactionLogSynchronizer.java @@ -13,6 +13,7 @@ */ package io.trino.plugin.deltalake; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoOutputFile; import io.trino.plugin.deltalake.transactionlog.writer.TransactionLogSynchronizer; @@ -34,7 +35,7 @@ public boolean isUnsafe() } @Override - public void write(ConnectorSession session, String clusterId, String newLogEntryPath, byte[] entryContents) + public void write(ConnectorSession session, String clusterId, Location newLogEntryPath, byte[] entryContents) { try { TrinoFileSystem fileSystem = HDFS_FILE_SYSTEM_FACTORY.create(session); diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeFileOperations.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeFileOperations.java index 2bc9aba57314..aea285b290b9 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeFileOperations.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeFileOperations.java @@ -170,7 +170,7 @@ private Multiset getOperations() return trackingFileSystemFactory.getOperationCounts() .entrySet().stream() .flatMap(entry -> nCopies(entry.getValue(), FileOperation.create( - entry.getKey().getFilePath(), + entry.getKey().getLocation().path(), entry.getKey().getOperationType())).stream()) .collect(toCollection(HashMultiset::create)); } diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeGcsConnectorSmokeTest.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeGcsConnectorSmokeTest.java index 6d6094cb7789..74270f4cc2c0 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeGcsConnectorSmokeTest.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeGcsConnectorSmokeTest.java @@ -21,6 +21,7 @@ import com.google.common.reflect.ClassPath; import io.airlift.log.Logger; import io.trino.filesystem.FileIterator; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoOutputFile; import io.trino.hadoop.ConfigurationInstantiator; @@ -106,7 +107,7 @@ public void removeTestData() { if (fileSystem != null) { try { - fileSystem.deleteDirectory(bucketUrl()); + fileSystem.deleteDirectory(Location.of(bucketUrl())); } catch (IOException e) { // The GCS bucket should be configured to expire objects automatically. Clean up issues do not need to fail the test. @@ -171,7 +172,7 @@ protected void registerTableFromResources(String table, String resourcePath, Que for (ClassPath.ResourceInfo resourceInfo : resources) { String fileName = resourceInfo.getResourceName().replaceFirst("^" + Pattern.quote(resourcePath), quoteReplacement(targetDirectory)); ByteSource byteSource = resourceInfo.asByteSource(); - TrinoOutputFile trinoOutputFile = fileSystem.newOutputFile(fileName); + TrinoOutputFile trinoOutputFile = fileSystem.newOutputFile(Location.of(fileName)); try (OutputStream fileStream = trinoOutputFile.createOrOverwrite()) { ByteStreams.copy(byteSource.openBufferedStream(), fileStream); } @@ -208,9 +209,9 @@ private List listAllFilesRecursive(String directory) { ImmutableList.Builder locations = ImmutableList.builder(); try { - FileIterator files = fileSystem.listFiles(bucketUrl() + directory); + FileIterator files = fileSystem.listFiles(Location.of(bucketUrl()).appendPath(directory)); while (files.hasNext()) { - locations.add(files.next().location()); + locations.add(files.next().location().toString()); } return locations.build(); } diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestTransactionLogAccess.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestTransactionLogAccess.java index 55b35dccab2f..a03ef6b22514 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestTransactionLogAccess.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestTransactionLogAccess.java @@ -878,7 +878,7 @@ private Multiset getOperations() return trackingFileSystemFactory.getOperationCounts() .entrySet().stream() .flatMap(entry -> nCopies(entry.getValue(), new FileOperation( - entry.getKey().getFilePath().replaceFirst(".*/_delta_log/", ""), + entry.getKey().getLocation().toString().replaceFirst(".*/_delta_log/", ""), entry.getKey().getOperationType())).stream()) .collect(toCollection(HashMultiset::create)); } diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/transactionlog/TestTableSnapshot.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/transactionlog/TestTableSnapshot.java index e946b503da90..aa0c86382f65 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/transactionlog/TestTableSnapshot.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/transactionlog/TestTableSnapshot.java @@ -211,7 +211,7 @@ private Multiset getOperations() return trackingFileSystemFactory.getOperationCounts() .entrySet().stream() .flatMap(entry -> nCopies(entry.getValue(), new FileOperation( - entry.getKey().getFilePath().replaceFirst(".*/_delta_log/", ""), + entry.getKey().getLocation().toString().replaceFirst(".*/_delta_log/", ""), entry.getKey().getOperationType())).stream()) .collect(toCollection(HashMultiset::create)); } diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/transactionlog/checkpoint/TestCheckpointEntryIterator.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/transactionlog/checkpoint/TestCheckpointEntryIterator.java index a9260e3a5a87..d461ed238b92 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/transactionlog/checkpoint/TestCheckpointEntryIterator.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/transactionlog/checkpoint/TestCheckpointEntryIterator.java @@ -16,6 +16,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterators; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoInputFile; import io.trino.filesystem.hdfs.HdfsFileSystemFactory; @@ -216,7 +217,7 @@ private CheckpointEntryIterator createCheckpointEntryIterator(URI checkpointUri, throws IOException { TrinoFileSystem fileSystem = new HdfsFileSystemFactory(HDFS_ENVIRONMENT).create(SESSION); - TrinoInputFile checkpointFile = fileSystem.newInputFile(checkpointUri.toString()); + TrinoInputFile checkpointFile = fileSystem.newInputFile(Location.of(checkpointUri.toString())); return new CheckpointEntryIterator( checkpointFile, diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/transactionlog/checkpoint/TestCheckpointWriter.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/transactionlog/checkpoint/TestCheckpointWriter.java index 18dde1331b29..211afc4c0001 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/transactionlog/checkpoint/TestCheckpointWriter.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/transactionlog/checkpoint/TestCheckpointWriter.java @@ -17,6 +17,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import io.airlift.slice.Slice; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoInputFile; import io.trino.filesystem.TrinoOutputFile; @@ -479,7 +480,7 @@ private CheckpointEntries readCheckpoint(String checkpointPath, MetadataEntry me throws IOException { TrinoFileSystem fileSystem = new HdfsFileSystemFactory(HDFS_ENVIRONMENT).create(SESSION); - TrinoInputFile checkpointFile = fileSystem.newInputFile(checkpointPath); + TrinoInputFile checkpointFile = fileSystem.newInputFile(Location.of(checkpointPath)); Iterator checkpointEntryIterator = new CheckpointEntryIterator( checkpointFile, @@ -505,6 +506,6 @@ private CheckpointEntries readCheckpoint(String checkpointPath, MetadataEntry me private static TrinoOutputFile createOutputFile(String path) { - return new HdfsFileSystemFactory(HDFS_ENVIRONMENT).create(SESSION).newOutputFile(path); + return new HdfsFileSystemFactory(HDFS_ENVIRONMENT).create(SESSION).newOutputFile(Location.of(path)); } } diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/transactionlog/statistics/TestDeltaLakeFileStatistics.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/transactionlog/statistics/TestDeltaLakeFileStatistics.java index b29539e002e2..b29445a9e3f5 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/transactionlog/statistics/TestDeltaLakeFileStatistics.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/transactionlog/statistics/TestDeltaLakeFileStatistics.java @@ -16,9 +16,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableSet; import io.airlift.json.ObjectMapperProvider; -import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoInputFile; -import io.trino.filesystem.hdfs.HdfsFileSystemFactory; +import io.trino.filesystem.local.LocalInputFile; import io.trino.plugin.deltalake.DeltaLakeColumnHandle; import io.trino.plugin.deltalake.DeltaLakeConfig; import io.trino.plugin.deltalake.transactionlog.DeltaLakeTransactionLogEntry; @@ -49,7 +48,6 @@ import static io.airlift.slice.Slices.utf8Slice; import static io.trino.plugin.deltalake.DeltaLakeColumnType.REGULAR; import static io.trino.plugin.deltalake.transactionlog.checkpoint.CheckpointEntryIterator.EntryType.METADATA; -import static io.trino.plugin.hive.HiveTestUtils.HDFS_ENVIRONMENT; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.BooleanType.BOOLEAN; import static io.trino.spi.type.DateTimeEncoding.packDateTimeWithZone; @@ -93,8 +91,7 @@ public void testParseParquetStatistics() TypeManager typeManager = TESTING_TYPE_MANAGER; CheckpointSchemaManager checkpointSchemaManager = new CheckpointSchemaManager(typeManager); - TrinoFileSystem fileSystem = new HdfsFileSystemFactory(HDFS_ENVIRONMENT).create(SESSION); - TrinoInputFile checkpointFile = fileSystem.newInputFile(statsFile.toURI().toString()); + TrinoInputFile checkpointFile = new LocalInputFile(statsFile); CheckpointEntryIterator metadataEntryIterator = new CheckpointEntryIterator( checkpointFile, diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/BackgroundHiveSplitLoader.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/BackgroundHiveSplitLoader.java index 36a8a488bd14..53300de2b77d 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/BackgroundHiveSplitLoader.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/BackgroundHiveSplitLoader.java @@ -24,6 +24,7 @@ import com.google.common.util.concurrent.ListenableFuture; import io.airlift.units.Duration; import io.trino.filesystem.FileEntry; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.hdfs.HdfsContext; @@ -678,15 +679,16 @@ private ListenableFuture getTransactionalSplits(Path path, boolean splitta { TrinoFileSystem fileSystem = fileSystemFactory.create(session); ValidWriteIdList writeIds = validWriteIds.orElseThrow(() -> new IllegalStateException("No validWriteIds present")); - AcidState acidState = getAcidState(fileSystem, path.toString(), writeIds); + AcidState acidState = getAcidState(fileSystem, Location.of(path.toString()), writeIds); boolean fullAcid = isFullAcidTable(table.getParameters()); AcidInfo.Builder acidInfoBuilder = AcidInfo.builder(path); if (fullAcid) { // From Hive version >= 3.0, delta/base files will always have file '_orc_acid_version' with value >= '2'. - Optional baseOrDeltaPath = acidState.baseDirectory().or(() -> - acidState.deltas().stream().findFirst().map(ParsedDelta::path)); + Optional baseOrDeltaPath = acidState.baseDirectory() + .or(() -> acidState.deltas().stream().findFirst() + .map(delta -> Location.of(delta.path()))); if (baseOrDeltaPath.isPresent() && readAcidVersionFile(fileSystem, baseOrDeltaPath.get()) >= 2) { // Trino cannot read ORC ACID tables with version < 2 (written by Hive older than 3.0) @@ -722,7 +724,7 @@ private ListenableFuture getTransactionalSplits(Path path, boolean splitta for (FileEntry entry : acidState.originalFiles()) { // Hive requires "original" files of transactional tables to conform to the bucketed tables naming pattern, to match them with delete deltas. - acidInfoBuilder.addOriginalFile(new Path(entry.location()), entry.length(), getRequiredBucketNumber(entry.location())); + acidInfoBuilder.addOriginalFile(new Path(entry.location().toString()), entry.length(), getRequiredBucketNumber(entry.location())); } if (tableBucketInfo.isPresent()) { @@ -768,9 +770,9 @@ private static Optional acidInfo(boolean fullAcid, AcidInfo.Builder bu return fullAcid ? builder.build() : Optional.empty(); } - private static Optional acidInfoForOriginalFiles(boolean fullAcid, AcidInfo.Builder builder, String path) + private static Optional acidInfoForOriginalFiles(boolean fullAcid, AcidInfo.Builder builder, Location location) { - return fullAcid ? Optional.of(builder.buildWithRequiredOriginalFiles(getRequiredBucketNumber(path))) : Optional.empty(); + return fullAcid ? Optional.of(builder.buildWithRequiredOriginalFiles(getRequiredBucketNumber(location))) : Optional.empty(); } private ListenableFuture addSplitsToSource(InputSplit[] targetSplits, InternalHiveSplitFactory splitFactory) @@ -931,10 +933,10 @@ static void validateFileBuckets(ListMultimap bucketFil } } - private static int getRequiredBucketNumber(String path) + private static int getRequiredBucketNumber(Location location) { - return getBucketNumber(path.substring(path.lastIndexOf('/') + 1)) - .orElseThrow(() -> new IllegalStateException("Cannot get bucket number from path: " + path)); + return getBucketNumber(location.fileName()) + .orElseThrow(() -> new IllegalStateException("Cannot get bucket number from location: " + location)); } @VisibleForTesting diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadata.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadata.java index 392be5abe4da..0969b290a4b8 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadata.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveMetadata.java @@ -28,6 +28,7 @@ import io.airlift.log.Logger; import io.airlift.slice.Slice; import io.airlift.units.DataSize; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.TrinoOutputFile; @@ -1873,7 +1874,7 @@ private void createEmptyFiles(ConnectorSession session, Path path, Table table, if (format.getOutputFormat().equals(HIVE_IGNORE_KEY_OUTPUT_FORMAT_CLASS)) { TrinoFileSystem fileSystem = new HdfsFileSystemFactory(hdfsEnvironment).create(session.getIdentity()); for (String fileName : fileNames) { - TrinoOutputFile trinoOutputFile = fileSystem.newOutputFile(new Path(path, fileName).toString()); + TrinoOutputFile trinoOutputFile = fileSystem.newOutputFile(Location.of(path.toString()).appendPath(fileName)); try { // create empty file trinoOutputFile.create(newSimpleAggregatedMemoryContext()).close(); @@ -2250,7 +2251,7 @@ private void removeNonCurrentQueryFiles(ConnectorSession session, Path partition private void createOrcAcidVersionFile(ConnectorIdentity identity, String deltaDirectory) { try { - writeAcidVersionFile(fileSystemFactory.create(identity), deltaDirectory); + writeAcidVersionFile(fileSystemFactory.create(identity), Location.of(deltaDirectory)); } catch (IOException e) { throw new TrinoException(HIVE_FILESYSTEM_ERROR, "Exception writing _orc_acid_version file for delta directory: " + deltaDirectory, e); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveWriterFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveWriterFactory.java index 99fb96308172..19984ee86c12 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveWriterFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveWriterFactory.java @@ -21,6 +21,7 @@ import com.google.common.collect.Sets; import io.airlift.event.client.EventClient; import io.airlift.units.DataSize; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.hdfs.HdfsContext; @@ -360,7 +361,7 @@ public HiveWriter createWriter(Page partitionColumns, int position, OptionalInt if (!writeInfo.getWriteMode().isWritePathSameAsTargetPath()) { // When target path is different from write path, // verify that the target directory for the partition does not already exist - String writeInfoTargetPath = writeInfo.getTargetPath().toString(); + Location writeInfoTargetPath = Location.of(writeInfo.getTargetPath().toString()); try { if (fileSystem.newInputFile(writeInfoTargetPath).exists()) { throw new TrinoException(HIVE_PATH_ALREADY_EXISTS, format( diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/RcFileFileWriterFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/RcFileFileWriterFactory.java index dd606b22f624..1c02c24802d8 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/RcFileFileWriterFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/RcFileFileWriterFactory.java @@ -15,6 +15,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoInputFile; import io.trino.filesystem.hdfs.HdfsFileSystemFactory; @@ -133,16 +134,17 @@ else if (COLUMNAR_SERDE_CLASS.equals(storageFormat.getSerde())) { .toArray(); try { + Location location = Location.of(path.toString()); TrinoFileSystem fileSystem = new HdfsFileSystemFactory(hdfsEnvironment).create(session.getIdentity()); AggregatedMemoryContext outputStreamMemoryContext = newSimpleAggregatedMemoryContext(); - OutputStream outputStream = fileSystem.newOutputFile(path.toString()).create(outputStreamMemoryContext); + OutputStream outputStream = fileSystem.newOutputFile(location).create(outputStreamMemoryContext); Optional> validationInputFactory = Optional.empty(); if (isRcfileOptimizedWriterValidate(session)) { - validationInputFactory = Optional.of(() -> fileSystem.newInputFile(path.toString())); + validationInputFactory = Optional.of(() -> fileSystem.newInputFile(location)); } - Closeable rollbackAction = () -> fileSystem.deleteFile(path.toString()); + Closeable rollbackAction = () -> fileSystem.deleteFile(location); return Optional.of(new RcFileFileWriter( outputStream, diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/SortingFileWriter.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/SortingFileWriter.java index 72bd5f247eed..7299c733aec3 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/SortingFileWriter.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/SortingFileWriter.java @@ -17,6 +17,7 @@ import com.google.common.io.Closer; import io.airlift.log.Logger; import io.airlift.units.DataSize; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoInputFile; import io.trino.orc.OrcDataSink; @@ -75,7 +76,7 @@ public class SortingFileWriter private final FileWriter outputWriter; private final SortBuffer sortBuffer; private final TempFileSinkFactory tempFileSinkFactory; - private final Queue tempFiles = new PriorityQueue<>(comparing(TempFile::getSize)); + private final Queue tempFiles = new PriorityQueue<>(comparing(TempFile::size)); private final AtomicLong nextFileId = new AtomicLong(); private final TypeOperators typeOperators; @@ -169,7 +170,7 @@ private static Closeable createRollbackAction(TrinoFileSystem fileSystem, Queue< { return () -> { for (TempFile file : tempFiles) { - cleanupFile(fileSystem, file.getPath()); + cleanupFile(fileSystem, file.location()); } }; } @@ -227,10 +228,9 @@ private void mergeFiles(Iterable files, Consumer consumer) Collection> iterators = new ArrayList<>(); for (TempFile tempFile : files) { - String file = tempFile.getPath(); - TrinoInputFile inputFile = fileSystem.newInputFile(file); + TrinoInputFile inputFile = fileSystem.newInputFile(tempFile.location()); OrcDataSource dataSource = new HdfsOrcDataSource( - new OrcDataSourceId(file), + new OrcDataSourceId(tempFile.location().toString()), inputFile.length(), new OrcReaderOptions(), inputFile, @@ -243,7 +243,7 @@ private void mergeFiles(Iterable files, Consumer consumer) .forEachRemaining(consumer); for (TempFile tempFile : files) { - fileSystem.deleteFile(tempFile.getPath()); + fileSystem.deleteFile(tempFile.location()); } } catch (IOException e) { @@ -253,7 +253,7 @@ private void mergeFiles(Iterable files, Consumer consumer) private void writeTempFile(Consumer consumer) { - String tempFile = getTempFileName(); + Location tempFile = getTempFileName(); try (TempFileWriter writer = new TempFileWriter(types, tempFileSinkFactory.createSink(fileSystem, tempFile))) { consumer.accept(writer); @@ -266,56 +266,33 @@ private void writeTempFile(Consumer consumer) } } - private static void cleanupFile(TrinoFileSystem fileSystem, String file) + private static void cleanupFile(TrinoFileSystem fileSystem, Location location) { try { - fileSystem.deleteFile(file); + fileSystem.deleteFile(location); } catch (IOException e) { - log.warn(e, "Failed to delete temporary file: %s", file); + log.warn(e, "Failed to delete temporary file: %s", location); } } - private String getTempFileName() + private Location getTempFileName() { - return tempFilePrefix + "." + nextFileId.getAndIncrement(); + return Location.of(tempFilePrefix + "." + nextFileId.getAndIncrement()); } - private static class TempFile + private record TempFile(Location location, long size) { - private final String path; - private final long size; - - public TempFile(String path, long size) + public TempFile { checkArgument(size >= 0, "size is negative"); - this.path = requireNonNull(path, "path is null"); - this.size = size; - } - - public String getPath() - { - return path; - } - - public long getSize() - { - return size; - } - - @Override - public String toString() - { - return toStringHelper(this) - .add("path", path) - .add("size", size) - .toString(); + requireNonNull(location, "location is null"); } } public interface TempFileSinkFactory { - OrcDataSink createSink(TrinoFileSystem fileSystem, String path) + OrcDataSink createSink(TrinoFileSystem fileSystem, Location location) throws IOException; } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/fs/MonitoredInputFile.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/fs/MonitoredInputFile.java index e4caf10bdd31..ea593b4bd22e 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/fs/MonitoredInputFile.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/fs/MonitoredInputFile.java @@ -13,6 +13,7 @@ */ package io.trino.plugin.hive.fs; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoInput; import io.trino.filesystem.TrinoInputFile; import io.trino.filesystem.TrinoInputStream; @@ -71,7 +72,7 @@ public boolean exists() } @Override - public String location() + public Location location() { return delegate.location(); } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/fs/TrinoFileStatus.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/fs/TrinoFileStatus.java index 98f57e3ff707..2f7c7f998ae8 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/fs/TrinoFileStatus.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/fs/TrinoFileStatus.java @@ -42,7 +42,7 @@ public TrinoFileStatus(FileEntry entry) .stream() .map(BlockLocation::new) .collect(toImmutableList()), - new Path(entry.location()), + new Path(entry.location().toString()), false, entry.length(), entry.lastModified().toEpochMilli()); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/line/LineFileWriterFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/line/LineFileWriterFactory.java index 0ca22cf1ae30..97d4b335029d 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/line/LineFileWriterFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/line/LineFileWriterFactory.java @@ -16,6 +16,7 @@ import io.airlift.slice.DynamicSliceOutput; import io.airlift.slice.Slice; import io.airlift.slice.Slices; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.hive.formats.compression.CompressionKind; @@ -127,9 +128,10 @@ public Optional createFileWriter( LineSerializer lineSerializer = lineSerializerFactory.create(columns, fromProperties(schema)); try { + Location location = Location.of(path.toString()); TrinoFileSystem fileSystem = fileSystemFactory.create(session.getIdentity()); AggregatedMemoryContext outputStreamMemoryContext = newSimpleAggregatedMemoryContext(); - OutputStream outputStream = fileSystem.newOutputFile(path.toString()).create(outputStreamMemoryContext); + OutputStream outputStream = fileSystem.newOutputFile(location).create(outputStreamMemoryContext); LineWriter lineWriter = lineWriterFactory.createLineWriter(session, outputStream, compressionKind); @@ -140,7 +142,7 @@ public Optional createFileWriter( return Optional.of(new LineFileWriter( lineWriter, lineSerializer, - () -> fileSystem.deleteFile(path.toString()), + () -> fileSystem.deleteFile(location), fileInputColumnIndexes)); } catch (TrinoException e) { diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/line/LinePageSourceFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/line/LinePageSourceFactory.java index e3a25b67889e..3b335c5441ee 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/line/LinePageSourceFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/line/LinePageSourceFactory.java @@ -17,6 +17,7 @@ import io.airlift.slice.Slices; import io.airlift.units.DataSize; import io.airlift.units.DataSize.Unit; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.TrinoInputFile; @@ -141,8 +142,9 @@ public Optional createPageSource( } // buffer file if small + Location location = Location.of(path.toString()); TrinoFileSystem trinoFileSystem = fileSystemFactory.create(session.getIdentity()); - TrinoInputFile inputFile = new MonitoredInputFile(stats, trinoFileSystem.newInputFile(path.toString())); + TrinoInputFile inputFile = new MonitoredInputFile(stats, trinoFileSystem.newInputFile(location)); try { length = min(inputFile.length() - start, length); if (!inputFile.exists()) { @@ -151,7 +153,7 @@ public Optional createPageSource( if (estimatedFileSize < SMALL_FILE_SIZE.toBytes()) { try (InputStream inputStream = inputFile.newStream()) { byte[] data = inputStream.readAllBytes(); - inputFile = new MemoryInputFile(path.toString(), Slices.wrappedBuffer(data)); + inputFile = new MemoryInputFile(location, Slices.wrappedBuffer(data)); } } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcDeleteDeltaPageSource.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcDeleteDeltaPageSource.java index 4d50ce58118c..a4f80feae9aa 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcDeleteDeltaPageSource.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcDeleteDeltaPageSource.java @@ -74,10 +74,10 @@ public static Optional createOrcDeleteDeltaPageSource( FileFormatDataSourceStats stats) { OrcDataSource orcDataSource; - String path = inputFile.location(); + String path = inputFile.location().toString(); try { orcDataSource = new HdfsOrcDataSource( - new OrcDataSourceId(inputFile.location()), + new OrcDataSourceId(path), inputFile.length(), options, inputFile, diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcDeletedRows.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcDeletedRows.java index ed9986ab8fc2..aa169280b8a8 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcDeletedRows.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcDeletedRows.java @@ -14,6 +14,7 @@ package io.trino.plugin.hive.orc; import com.google.common.collect.ImmutableSet; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.TrinoInputFile; @@ -295,7 +296,7 @@ private class Loader @Nullable private ConnectorPageSource currentPageSource; @Nullable - private Path currentPath; + private Location currentPath; @Nullable private Page currentPage; private int currentPagePosition; @@ -312,8 +313,8 @@ public Optional> loadOrYield() try { if (currentPageSource == null) { String deleteDeltaDirectory = deleteDeltaDirectories.next(); - currentPath = createPath(acidInfo, deleteDeltaDirectory, sourceFileName); - TrinoInputFile inputFile = fileSystem.newInputFile(currentPath.toString()); + currentPath = Location.of(createPath(acidInfo, deleteDeltaDirectory, sourceFileName).toString()); + TrinoInputFile inputFile = fileSystem.newInputFile(currentPath); if (inputFile.exists()) { currentPageSource = pageSourceFactory.createPageSource(inputFile).orElseGet(EmptyPageSource::new); } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcFileWriterFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcFileWriterFactory.java index b9275dfe79d1..edb90a3651f1 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcFileWriterFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcFileWriterFactory.java @@ -14,6 +14,7 @@ package io.trino.plugin.hive.orc; import com.google.common.collect.ImmutableMap; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.TrinoInputFile; @@ -155,16 +156,16 @@ public Optional createFileWriter( .toArray(); try { TrinoFileSystem fileSystem = fileSystemFactory.create(session); - String stringPath = path.toString(); - OrcDataSink orcDataSink = createOrcDataSink(fileSystem, stringPath); + Location location = Location.of(path.toString()); + OrcDataSink orcDataSink = createOrcDataSink(fileSystem, location); Optional> validationInputFactory = Optional.empty(); if (isOrcOptimizedWriterValidate(session)) { validationInputFactory = Optional.of(() -> { try { - TrinoInputFile inputFile = fileSystem.newInputFile(stringPath); + TrinoInputFile inputFile = fileSystem.newInputFile(location); return new HdfsOrcDataSource( - new OrcDataSourceId(stringPath), + new OrcDataSourceId(location.toString()), inputFile.length(), new OrcReaderOptions(), inputFile, @@ -176,7 +177,7 @@ public Optional createFileWriter( }); } - Closeable rollbackAction = () -> fileSystem.deleteFile(stringPath); + Closeable rollbackAction = () -> fileSystem.deleteFile(location); if (transaction.isInsert() && useAcidSchema) { // Only add the ACID columns if the request is for insert-type operations - - for delete operations, @@ -219,10 +220,10 @@ public Optional createFileWriter( } } - public static OrcDataSink createOrcDataSink(TrinoFileSystem fileSystem, String path) + public static OrcDataSink createOrcDataSink(TrinoFileSystem fileSystem, Location location) throws IOException { - return OutputStreamOrcDataSink.create(fileSystem.newOutputFile(path)); + return OutputStreamOrcDataSink.create(fileSystem.newOutputFile(location)); } private static CompressionKind getCompression(Properties schema, JobConf configuration) diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcPageSourceFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcPageSourceFactory.java index 69c23a00d688..155c0201081b 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcPageSourceFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OrcPageSourceFactory.java @@ -17,6 +17,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import io.airlift.slice.Slice; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.TrinoInputFile; @@ -260,7 +261,7 @@ private ConnectorPageSource createOrcPageSource( boolean originalFilesPresent = acidInfo.isPresent() && !acidInfo.get().getOriginalFiles().isEmpty(); try { TrinoFileSystem fileSystem = fileSystemFactory.create(session.getIdentity()); - TrinoInputFile inputFile = fileSystem.newInputFile(path.toString()); + TrinoInputFile inputFile = fileSystem.newInputFile(Location.of(path.toString())); orcDataSource = new HdfsOrcDataSource( new OrcDataSourceId(path.toString()), estimatedFileSize, diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OriginalFilesUtils.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OriginalFilesUtils.java index b20eb1b1e002..5c2516b5ecac 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OriginalFilesUtils.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/orc/OriginalFilesUtils.java @@ -13,7 +13,7 @@ */ package io.trino.plugin.hive.orc; -import io.trino.filesystem.TrinoFileSystem; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.TrinoInputFile; import io.trino.orc.OrcDataSource; @@ -56,7 +56,9 @@ public static long getPrecedingRowCount( for (OriginalFileInfo originalFileInfo : originalFileInfos) { Path path = new Path(splitPath.getParent() + "/" + originalFileInfo.getName()); if (path.compareTo(splitPath) < 0) { - rowCount += getRowsInFile(path.toString(), fileSystemFactory, identity, options, stats, originalFileInfo.getFileSize()); + TrinoInputFile inputFile = fileSystemFactory.create(identity) + .newInputFile(Location.of(path.toString()), originalFileInfo.getFileSize()); + rowCount += getRowsInFile(inputFile, options, stats); } } @@ -66,25 +68,17 @@ public static long getPrecedingRowCount( /** * Returns number of rows present in the file, based on the ORC footer. */ - private static Long getRowsInFile( - String splitPath, - TrinoFileSystemFactory fileSystemFactory, - ConnectorIdentity identity, - OrcReaderOptions options, - FileFormatDataSourceStats stats, - long fileSize) + private static Long getRowsInFile(TrinoInputFile inputFile, OrcReaderOptions options, FileFormatDataSourceStats stats) { try { - TrinoFileSystem fileSystem = fileSystemFactory.create(identity); - TrinoInputFile inputFile = fileSystem.newInputFile(splitPath); try (OrcDataSource orcDataSource = new HdfsOrcDataSource( - new OrcDataSourceId(splitPath), - fileSize, + new OrcDataSourceId(inputFile.location().toString()), + inputFile.length(), options, inputFile, stats)) { OrcReader reader = createOrcReader(orcDataSource, options) - .orElseThrow(() -> new TrinoException(HIVE_CANNOT_OPEN_SPLIT, "Could not read ORC footer from empty file: " + splitPath)); + .orElseThrow(() -> new TrinoException(HIVE_CANNOT_OPEN_SPLIT, "Could not read ORC footer from empty file: " + inputFile.location())); return reader.getFooter().getNumberOfRows(); } } @@ -92,7 +86,7 @@ private static Long getRowsInFile( throw e; } catch (Exception e) { - throw new TrinoException(HIVE_CANNOT_OPEN_SPLIT, "Could not read ORC footer from file: " + splitPath, e); + throw new TrinoException(HIVE_CANNOT_OPEN_SPLIT, "Could not read ORC footer from file: " + inputFile.location(), e); } } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetFileWriterFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetFileWriterFactory.java index bb26c25ed7a6..d15c2f1072e5 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetFileWriterFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetFileWriterFactory.java @@ -13,6 +13,7 @@ */ package io.trino.plugin.hive.parquet; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.TrinoInputFile; @@ -128,11 +129,11 @@ public Optional createFileWriter( .mapToInt(inputColumnNames::indexOf) .toArray(); - String pathString = path.toString(); + Location location = Location.of(path.toString()); try { TrinoFileSystem fileSystem = fileSystemFactory.create(session); - Closeable rollbackAction = () -> fileSystem.deleteFile(pathString); + Closeable rollbackAction = () -> fileSystem.deleteFile(location); ParquetSchemaConverter schemaConverter = new ParquetSchemaConverter( fileColumnTypes, @@ -144,7 +145,7 @@ public Optional createFileWriter( if (isParquetOptimizedWriterValidate(session)) { validationInputFactory = Optional.of(() -> { try { - TrinoInputFile inputFile = fileSystem.newInputFile(pathString); + TrinoInputFile inputFile = fileSystem.newInputFile(location); return new TrinoParquetDataSource(inputFile, new ParquetReaderOptions(), readStats); } catch (IOException e) { @@ -154,7 +155,7 @@ public Optional createFileWriter( } return Optional.of(new ParquetFileWriter( - fileSystem.newOutputFile(pathString), + fileSystem.newOutputFile(location), rollbackAction, fileColumnTypes, fileColumnNames, diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetPageSourceFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetPageSourceFactory.java index 5ccac2ac366d..bcd9947bf8d8 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetPageSourceFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetPageSourceFactory.java @@ -16,6 +16,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.TrinoInputFile; @@ -181,8 +182,9 @@ public Optional createPageSource( checkArgument(acidInfo.isEmpty(), "Acid is not supported"); + Location location = Location.of(path.toString()); TrinoFileSystem fileSystem = fileSystemFactory.create(session); - TrinoInputFile inputFile = fileSystem.newInputFile(path.toString(), estimatedFileSize); + TrinoInputFile inputFile = fileSystem.newInputFile(location, estimatedFileSize); return Optional.of(createPageSource( inputFile, diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/TrinoParquetDataSource.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/TrinoParquetDataSource.java index 134665394437..99ab9f48d63a 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/TrinoParquetDataSource.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/TrinoParquetDataSource.java @@ -34,7 +34,7 @@ public class TrinoParquetDataSource public TrinoParquetDataSource(TrinoInputFile file, ParquetReaderOptions options, FileFormatDataSourceStats stats) throws IOException { - super(new ParquetDataSourceId(file.location()), file.length(), options); + super(new ParquetDataSourceId(file.location().toString()), file.length(), options); this.stats = requireNonNull(stats, "stats is null"); this.input = file.newInput(); } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/rcfile/RcFilePageSourceFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/rcfile/RcFilePageSourceFactory.java index b650f42821f6..8ad8d650886b 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/rcfile/RcFilePageSourceFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/rcfile/RcFilePageSourceFactory.java @@ -18,6 +18,7 @@ import io.airlift.slice.Slices; import io.airlift.units.DataSize; import io.airlift.units.DataSize.Unit; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoInputFile; import io.trino.filesystem.hdfs.HdfsFileSystemFactory; @@ -141,8 +142,9 @@ else if (deserializerClassName.equals(COLUMNAR_SERDE_CLASS)) { .collect(toImmutableList()); } + Location location = Location.of(path.toString()); TrinoFileSystem trinoFileSystem = new HdfsFileSystemFactory(hdfsEnvironment).create(session.getIdentity()); - TrinoInputFile inputFile = new MonitoredInputFile(stats, trinoFileSystem.newInputFile(path.toString())); + TrinoInputFile inputFile = new MonitoredInputFile(stats, trinoFileSystem.newInputFile(location)); try { length = min(inputFile.length() - start, length); if (!inputFile.exists()) { @@ -151,7 +153,7 @@ else if (deserializerClassName.equals(COLUMNAR_SERDE_CLASS)) { if (estimatedFileSize < BUFFER_SIZE.toBytes()) { try (InputStream inputStream = inputFile.newStream()) { byte[] data = inputStream.readAllBytes(); - inputFile = new MemoryInputFile(path.toString(), Slices.wrappedBuffer(data)); + inputFile = new MemoryInputFile(location, Slices.wrappedBuffer(data)); } } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/AcidTables.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/AcidTables.java index ebb5b7a80047..869dbd52c925 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/AcidTables.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/AcidTables.java @@ -22,6 +22,7 @@ import com.google.common.collect.ListMultimap; import io.trino.filesystem.FileEntry; import io.trino.filesystem.FileIterator; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoInputFile; import io.trino.filesystem.TrinoOutputFile; @@ -82,7 +83,7 @@ public static String deleteDeltaSubdir(long writeId, int statementId) return "delete_" + deltaSubdir(writeId, statementId); } - public static void writeAcidVersionFile(TrinoFileSystem fileSystem, String deltaOrBaseDir) + public static void writeAcidVersionFile(TrinoFileSystem fileSystem, Location deltaOrBaseDir) throws IOException { TrinoOutputFile file = fileSystem.newOutputFile(versionFilePath(deltaOrBaseDir)); @@ -91,7 +92,7 @@ public static void writeAcidVersionFile(TrinoFileSystem fileSystem, String delta } } - public static int readAcidVersionFile(TrinoFileSystem fileSystem, String deltaOrBaseDir) + public static int readAcidVersionFile(TrinoFileSystem fileSystem, Location deltaOrBaseDir) throws IOException { TrinoInputFile file = fileSystem.newInputFile(versionFilePath(deltaOrBaseDir)); @@ -108,12 +109,12 @@ public static int readAcidVersionFile(TrinoFileSystem fileSystem, String deltaOr } } - private static String versionFilePath(String deltaOrBaseDir) + private static Location versionFilePath(Location deltaOrBaseDir) { - return deltaOrBaseDir + "/_orc_acid_version"; + return deltaOrBaseDir.appendPath("_orc_acid_version"); } - public static AcidState getAcidState(TrinoFileSystem fileSystem, String directory, ValidWriteIdList writeIdList) + public static AcidState getAcidState(TrinoFileSystem fileSystem, Location directory, ValidWriteIdList writeIdList) throws IOException { // directory = /hive/data/abc @@ -126,7 +127,7 @@ public static AcidState getAcidState(TrinoFileSystem fileSystem, String director List originalFiles = new ArrayList<>(); for (FileEntry file : listFiles(fileSystem, directory)) { - String suffix = listingSuffix(directory, file.location()); + String suffix = listingSuffix(directory.toString(), file.location().toString()); int slash = suffix.indexOf('/'); String name = (slash == -1) ? "" : suffix.substring(0, slash); @@ -188,7 +189,7 @@ else if (file.length() > 0) { originalFiles.clear(); } - originalFiles.sort(comparing(FileEntry::location)); + originalFiles.sort(comparing(entry -> entry.location().toString())); workingDeltas.sort(null); List deltas = new ArrayList<>(); @@ -217,7 +218,9 @@ else if ((prev != null) && } } - return new AcidState(Optional.ofNullable(bestBasePath), bestBaseFiles, deltas, originalFiles); + Optional baseDirectory = Optional.ofNullable(bestBasePath).map(Location::of); + + return new AcidState(baseDirectory, bestBaseFiles, deltas, originalFiles); } private static boolean isValidBase(ParsedBase base, ValidWriteIdList writeIdList, TrinoFileSystem fileSystem, String baseDir) @@ -237,7 +240,8 @@ private static boolean isValidBase(ParsedBase base, ValidWriteIdList writeIdList private static boolean isCompacted(TrinoFileSystem fileSystem, String baseDir) throws IOException { - TrinoInputFile file = fileSystem.newInputFile(baseDir + "/_metadata_acid"); + Location location = Location.of(baseDir).appendPath("_metadata_acid"); + TrinoInputFile file = fileSystem.newInputFile(location); if (!file.exists()) { return false; } @@ -305,14 +309,15 @@ static ParsedBase parseBase(String name) parseLong(name.substring(index + 2))); } - private static List listFiles(TrinoFileSystem fileSystem, String directory) + private static List listFiles(TrinoFileSystem fileSystem, Location directory) throws IOException { List files = new ArrayList<>(); FileIterator iterator = fileSystem.listFiles(directory); while (iterator.hasNext()) { FileEntry file = iterator.next(); - if (!file.location().contains("/_") && !file.location().contains("/.")) { + String path = file.location().path(); + if (!path.contains("/_") && !path.contains("/.")) { files.add(file); } } @@ -329,7 +334,7 @@ private static String listingSuffix(String directory, String file) } public record AcidState( - Optional baseDirectory, + Optional baseDirectory, List baseFiles, List deltas, List originalFiles) diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/benchmark/StandardFileFormats.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/benchmark/StandardFileFormats.java index adc6a8de93ca..b343418e37d4 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/benchmark/StandardFileFormats.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/benchmark/StandardFileFormats.java @@ -15,6 +15,7 @@ import com.google.common.collect.ImmutableMap; import io.trino.filesystem.hdfs.HdfsFileSystemFactory; +import io.trino.filesystem.local.LocalOutputFile; import io.trino.hdfs.HdfsEnvironment; import io.trino.hive.formats.encodings.ColumnEncodingFactory; import io.trino.hive.formats.encodings.binary.BinaryColumnEncodingFactory; @@ -57,7 +58,6 @@ import static io.trino.parquet.writer.ParquetSchemaConverter.HIVE_PARQUET_USE_INT96_TIMESTAMP_ENCODING; import static io.trino.parquet.writer.ParquetSchemaConverter.HIVE_PARQUET_USE_LEGACY_DECIMAL_ENCODING; import static io.trino.plugin.hive.HiveTestUtils.HDFS_FILE_SYSTEM_FACTORY; -import static io.trino.plugin.hive.HiveTestUtils.SESSION; import static io.trino.plugin.hive.HiveTestUtils.createGenericHiveRecordCursorProvider; import static io.trino.plugin.hive.benchmark.AbstractFileFormat.createSchema; import static io.trino.plugin.hive.metastore.StorageFormat.fromHiveStorageFormat; @@ -347,7 +347,7 @@ public PrestoOrcFormatWriter(File targetFile, List columnNames, List files = state.originalFiles(); assertEquals(files.size(), 7); - assertEquals(files.get(0).location(), "mock:/tbl/part1/000000_0"); - assertEquals(files.get(1).location(), "mock:/tbl/part1/000000_0_copy_1"); - assertEquals(files.get(2).location(), "mock:/tbl/part1/000000_0_copy_2"); - assertEquals(files.get(3).location(), "mock:/tbl/part1/000001_1"); - assertEquals(files.get(4).location(), "mock:/tbl/part1/000002_0"); - assertEquals(files.get(5).location(), "mock:/tbl/part1/random"); - assertEquals(files.get(6).location(), "mock:/tbl/part1/subdir/000000_0"); + assertEquals(files.get(0).location(), Location.of("mock:///tbl/part1/000000_0")); + assertEquals(files.get(1).location(), Location.of("mock:///tbl/part1/000000_0_copy_1")); + assertEquals(files.get(2).location(), Location.of("mock:///tbl/part1/000000_0_copy_2")); + assertEquals(files.get(3).location(), Location.of("mock:///tbl/part1/000001_1")); + assertEquals(files.get(4).location(), Location.of("mock:///tbl/part1/000002_0")); + assertEquals(files.get(5).location(), Location.of("mock:///tbl/part1/random")); + assertEquals(files.get(6).location(), Location.of("mock:///tbl/part1/subdir/000000_0")); } @Test @@ -159,27 +159,27 @@ public void testOriginalDeltas() new MockFile("mock:/tbl/part1/delta_101_101/bucket_0", 0, FAKE_DATA)); AcidState state = getAcidState( testingTrinoFileSystem(fs), - new MockPath(fs, "mock:/tbl/part1").toString(), + Location.of("mock:///tbl/part1"), new ValidWriteIdList("tbl:100:%d:".formatted(Long.MAX_VALUE))); assertThat(state.baseDirectory()).isEmpty(); List files = state.originalFiles(); assertEquals(files.size(), 5); - assertEquals(files.get(0).location(), "mock:/tbl/part1/000000_0"); - assertEquals(files.get(1).location(), "mock:/tbl/part1/000001_1"); - assertEquals(files.get(2).location(), "mock:/tbl/part1/000002_0"); - assertEquals(files.get(3).location(), "mock:/tbl/part1/random"); - assertEquals(files.get(4).location(), "mock:/tbl/part1/subdir/000000_0"); + assertEquals(files.get(0).location(), Location.of("mock:///tbl/part1/000000_0")); + assertEquals(files.get(1).location(), Location.of("mock:///tbl/part1/000001_1")); + assertEquals(files.get(2).location(), Location.of("mock:///tbl/part1/000002_0")); + assertEquals(files.get(3).location(), Location.of("mock:///tbl/part1/random")); + assertEquals(files.get(4).location(), Location.of("mock:///tbl/part1/subdir/000000_0")); List deltas = state.deltas(); assertEquals(deltas.size(), 2); ParsedDelta delta = deltas.get(0); - assertEquals(delta.path(), "mock:/tbl/part1/delta_025_030"); + assertEquals(delta.path(), "mock:///tbl/part1/delta_025_030"); assertEquals(delta.min(), 25); assertEquals(delta.max(), 30); delta = deltas.get(1); - assertEquals(delta.path(), "mock:/tbl/part1/delta_050_100"); + assertEquals(delta.path(), "mock:///tbl/part1/delta_050_100"); assertEquals(delta.min(), 50); assertEquals(delta.max(), 100); } @@ -201,16 +201,16 @@ public void testBaseDeltas() new MockFile("mock:/tbl/part1/delta_90_120/bucket_0", 0, FAKE_DATA)); AcidState dir = getAcidState( testingTrinoFileSystem(fs), - new MockPath(fs, "mock:/tbl/part1").toString(), + Location.of("mock:///tbl/part1"), new ValidWriteIdList("tbl:100:%d:".formatted(Long.MAX_VALUE))); - assertThat(dir.baseDirectory()).contains("mock:/tbl/part1/base_49"); + assertThat(dir.baseDirectory()).contains(Location.of("mock:///tbl/part1/base_49")); assertEquals(dir.originalFiles().size(), 0); List deltas = dir.deltas(); assertEquals(deltas.size(), 1); ParsedDelta delta = deltas.get(0); - assertEquals(delta.path(), "mock:/tbl/part1/delta_050_105"); + assertEquals(delta.path(), "mock:///tbl/part1/delta_050_105"); assertEquals(delta.min(), 50); assertEquals(delta.max(), 105); } @@ -226,10 +226,10 @@ public void testObsoleteOriginals() new MockFile("mock:/tbl/part1/000001_1", 500, FAKE_DATA)); AcidState state = getAcidState( testingTrinoFileSystem(fs), - new MockPath(fs, "mock:/tbl/part1").toString(), + Location.of("mock:///tbl/part1"), new ValidWriteIdList("tbl:150:%d:".formatted(Long.MAX_VALUE))); - assertThat(state.baseDirectory()).contains("mock:/tbl/part1/base_10"); + assertThat(state.baseDirectory()).contains(Location.of("mock:///tbl/part1/base_10")); } @Test @@ -246,17 +246,17 @@ public void testOverlapingDelta() new MockFile("mock:/tbl/part1/base_50/bucket_0", 500, FAKE_DATA)); AcidState state = getAcidState( testingTrinoFileSystem(fs), - new MockPath(fs, "mock:/tbl/part1").toString(), + Location.of("mock:///tbl/part1"), new ValidWriteIdList("tbl:100:%d:".formatted(Long.MAX_VALUE))); - assertThat(state.baseDirectory()).contains("mock:/tbl/part1/base_50"); + assertThat(state.baseDirectory()).contains(Location.of("mock:///tbl/part1/base_50")); List deltas = state.deltas(); assertEquals(deltas.size(), 4); - assertEquals(deltas.get(0).path(), "mock:/tbl/part1/delta_40_60"); - assertEquals(deltas.get(1).path(), "mock:/tbl/part1/delta_00061_61"); - assertEquals(deltas.get(2).path(), "mock:/tbl/part1/delta_000062_62"); - assertEquals(deltas.get(3).path(), "mock:/tbl/part1/delta_0000063_63"); + assertEquals(deltas.get(0).path(), "mock:///tbl/part1/delta_40_60"); + assertEquals(deltas.get(1).path(), "mock:///tbl/part1/delta_00061_61"); + assertEquals(deltas.get(2).path(), "mock:///tbl/part1/delta_000062_62"); + assertEquals(deltas.get(3).path(), "mock:///tbl/part1/delta_0000063_63"); } @Test @@ -277,18 +277,18 @@ public void testOverlapingDelta2() new MockFile("mock:/tbl/part1/base_50/bucket_0", 500, FAKE_DATA)); AcidState state = getAcidState( testingTrinoFileSystem(fs), - new MockPath(fs, "mock:/tbl/part1").toString(), + Location.of("mock:///tbl/part1"), new ValidWriteIdList("tbl:100:%d:".formatted(Long.MAX_VALUE))); - assertThat(state.baseDirectory()).contains("mock:/tbl/part1/base_50"); + assertThat(state.baseDirectory()).contains(Location.of("mock:///tbl/part1/base_50")); List deltas = state.deltas(); assertEquals(deltas.size(), 5); - assertEquals(deltas.get(0).path(), "mock:/tbl/part1/delta_40_60"); - assertEquals(deltas.get(1).path(), "mock:/tbl/part1/delta_00061_61_0"); - assertEquals(deltas.get(2).path(), "mock:/tbl/part1/delta_000062_62_0"); - assertEquals(deltas.get(3).path(), "mock:/tbl/part1/delta_000062_62_3"); - assertEquals(deltas.get(4).path(), "mock:/tbl/part1/delta_0000063_63_0"); + assertEquals(deltas.get(0).path(), "mock:///tbl/part1/delta_40_60"); + assertEquals(deltas.get(1).path(), "mock:///tbl/part1/delta_00061_61_0"); + assertEquals(deltas.get(2).path(), "mock:///tbl/part1/delta_000062_62_0"); + assertEquals(deltas.get(3).path(), "mock:///tbl/part1/delta_000062_62_3"); + assertEquals(deltas.get(4).path(), "mock:///tbl/part1/delta_0000063_63_0"); } @Test @@ -300,13 +300,13 @@ public void deltasWithOpenTxnInRead() new MockFile("mock:/tbl/part1/delta_2_5/bucket_0", 500, FAKE_DATA)); AcidState state = getAcidState( testingTrinoFileSystem(fs), - new MockPath(fs, "mock:/tbl/part1").toString(), + Location.of("mock:///tbl/part1"), new ValidWriteIdList("tbl:100:4:4")); List deltas = state.deltas(); assertEquals(deltas.size(), 2); - assertEquals(deltas.get(0).path(), "mock:/tbl/part1/delta_1_1"); - assertEquals(deltas.get(1).path(), "mock:/tbl/part1/delta_2_5"); + assertEquals(deltas.get(0).path(), "mock:///tbl/part1/delta_1_1"); + assertEquals(deltas.get(1).path(), "mock:///tbl/part1/delta_2_5"); } @Test @@ -321,13 +321,13 @@ public void deltasWithOpenTxnInRead2() new MockFile("mock:/tbl/part1/delta_101_101_1/bucket_0", 500, FAKE_DATA)); AcidState state = getAcidState( testingTrinoFileSystem(fs), - new MockPath(fs, "mock:/tbl/part1").toString(), + Location.of("mock:///tbl/part1"), new ValidWriteIdList("tbl:100:4:4")); List deltas = state.deltas(); assertEquals(deltas.size(), 2); - assertEquals(deltas.get(0).path(), "mock:/tbl/part1/delta_1_1"); - assertEquals(deltas.get(1).path(), "mock:/tbl/part1/delta_2_5"); + assertEquals(deltas.get(0).path(), "mock:///tbl/part1/delta_1_1"); + assertEquals(deltas.get(1).path(), "mock:///tbl/part1/delta_2_5"); } @Test @@ -348,16 +348,16 @@ public void testBaseWithDeleteDeltas() new MockFile("mock:/tbl/part1/delete_delta_110_110/bucket_0", 0, FAKE_DATA)); AcidState state = getAcidState( testingTrinoFileSystem(fs), - new MockPath(fs, "mock:/tbl/part1").toString(), + Location.of("mock:///tbl/part1"), new ValidWriteIdList("tbl:100:%d:".formatted(Long.MAX_VALUE))); - assertThat(state.baseDirectory()).contains("mock:/tbl/part1/base_49"); + assertThat(state.baseDirectory()).contains(Location.of("mock:///tbl/part1/base_49")); assertThat(state.originalFiles()).isEmpty(); List deltas = state.deltas(); assertEquals(deltas.size(), 2); - assertEquals(deltas.get(0).path(), "mock:/tbl/part1/delete_delta_050_105"); - assertEquals(deltas.get(1).path(), "mock:/tbl/part1/delta_050_105"); + assertEquals(deltas.get(0).path(), "mock:///tbl/part1/delete_delta_050_105"); + assertEquals(deltas.get(1).path(), "mock:///tbl/part1/delta_050_105"); // The delete_delta_110_110 should not be read because it is greater than the high watermark. } @@ -378,19 +378,19 @@ public void testOverlapingDeltaAndDeleteDelta() new MockFile("mock:/tbl/part1/base_50/bucket_0", 500, FAKE_DATA)); AcidState state = getAcidState( testingTrinoFileSystem(fs), - new MockPath(fs, "mock:/tbl/part1").toString(), + Location.of("mock:///tbl/part1"), new ValidWriteIdList("tbl:100:%d:".formatted(Long.MAX_VALUE))); - assertThat(state.baseDirectory()).contains("mock:/tbl/part1/base_50"); + assertThat(state.baseDirectory()).contains(Location.of("mock:///tbl/part1/base_50")); List deltas = state.deltas(); assertEquals(deltas.size(), 6); - assertEquals(deltas.get(0).path(), "mock:/tbl/part1/delete_delta_40_60"); - assertEquals(deltas.get(1).path(), "mock:/tbl/part1/delta_40_60"); - assertEquals(deltas.get(2).path(), "mock:/tbl/part1/delta_00061_61"); - assertEquals(deltas.get(3).path(), "mock:/tbl/part1/delta_000062_62"); - assertEquals(deltas.get(4).path(), "mock:/tbl/part1/delta_0000063_63"); - assertEquals(deltas.get(5).path(), "mock:/tbl/part1/delete_delta_00064_64"); + assertEquals(deltas.get(0).path(), "mock:///tbl/part1/delete_delta_40_60"); + assertEquals(deltas.get(1).path(), "mock:///tbl/part1/delta_40_60"); + assertEquals(deltas.get(2).path(), "mock:///tbl/part1/delta_00061_61"); + assertEquals(deltas.get(3).path(), "mock:///tbl/part1/delta_000062_62"); + assertEquals(deltas.get(4).path(), "mock:///tbl/part1/delta_0000063_63"); + assertEquals(deltas.get(5).path(), "mock:///tbl/part1/delete_delta_00064_64"); } @Test @@ -404,12 +404,12 @@ public void testMinorCompactedDeltaMakesInBetweenDelteDeltaObsolete() new MockFile("mock:/tbl/part1/delete_delta_50_50/bucket_0", 500, FAKE_DATA)); AcidState state = getAcidState( testingTrinoFileSystem(fs), - new MockPath(fs, "mock:/tbl/part1").toString(), + Location.of("mock:///tbl/part1"), new ValidWriteIdList("tbl:100:%d:".formatted(Long.MAX_VALUE))); List deltas = state.deltas(); assertEquals(deltas.size(), 1); - assertEquals(deltas.get(0).path(), "mock:/tbl/part1/delta_40_60"); + assertEquals(deltas.get(0).path(), "mock:///tbl/part1/delta_40_60"); } @Test @@ -426,14 +426,14 @@ public void deleteDeltasWithOpenTxnInRead() new MockFile("mock:/tbl/part1/delta_101_101_1/bucket_0", 500, FAKE_DATA)); AcidState state = getAcidState( testingTrinoFileSystem(fs), - new MockPath(fs, "mock:/tbl/part1").toString(), + Location.of("mock:///tbl/part1"), new ValidWriteIdList("tbl:100:4:4")); List deltas = state.deltas(); assertEquals(deltas.size(), 3); - assertEquals(deltas.get(0).path(), "mock:/tbl/part1/delta_1_1"); - assertEquals(deltas.get(1).path(), "mock:/tbl/part1/delete_delta_2_5"); - assertEquals(deltas.get(2).path(), "mock:/tbl/part1/delta_2_5"); + assertEquals(deltas.get(0).path(), "mock:///tbl/part1/delta_1_1"); + assertEquals(deltas.get(1).path(), "mock:///tbl/part1/delete_delta_2_5"); + assertEquals(deltas.get(2).path(), "mock:///tbl/part1/delta_2_5"); // Note that delete_delta_3_3 should not be read, when a minor compacted // [delete_]delta_2_5 is present. } diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java index 236b41855d8b..28aa012a18af 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java @@ -14,6 +14,7 @@ package io.trino.plugin.hudi; import com.google.common.collect.ImmutableList; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.TrinoInputFile; @@ -160,7 +161,7 @@ public ConnectorPageSource createPageSource( .filter(columnHandle -> !columnHandle.isPartitionKey() && !columnHandle.isHidden()) .collect(Collectors.toList()); TrinoFileSystem fileSystem = fileSystemFactory.create(session); - TrinoInputFile inputFile = fileSystem.newInputFile(path, split.getFileSize()); + TrinoInputFile inputFile = fileSystem.newInputFile(Location.of(path), split.getFileSize()); ConnectorPageSource dataPageSource = createPageSource(session, regularColumns, split, inputFile, dataSourceStats, options, timeZone); return new HudiPageSource( diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergFileWriterFactory.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergFileWriterFactory.java index 416121fabafc..f0fe4af5226f 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergFileWriterFactory.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergFileWriterFactory.java @@ -17,6 +17,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import io.airlift.units.DataSize; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoInputFile; import io.trino.filesystem.TrinoOutputFile; @@ -30,14 +31,13 @@ import io.trino.plugin.hive.FileFormatDataSourceStats; import io.trino.plugin.hive.NodeVersion; import io.trino.plugin.hive.orc.OrcWriterConfig; -import io.trino.plugin.iceberg.fileio.ForwardingFileIo; +import io.trino.plugin.iceberg.fileio.ForwardingOutputFile; import io.trino.spi.TrinoException; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.type.Type; import io.trino.spi.type.TypeManager; import org.apache.iceberg.MetricsConfig; import org.apache.iceberg.Schema; -import org.apache.iceberg.io.FileIO; import org.apache.iceberg.types.Types; import org.weakref.jmx.Managed; @@ -117,7 +117,7 @@ public OrcWriterStats getOrcWriterStats() public IcebergFileWriter createDataFileWriter( TrinoFileSystem fileSystem, - String outputPath, + Location outputPath, Schema icebergSchema, ConnectorSession session, IcebergFileFormat fileFormat, @@ -131,7 +131,7 @@ public IcebergFileWriter createDataFileWriter( case ORC: return createOrcWriter(metricsConfig, fileSystem, outputPath, icebergSchema, session, storageProperties, getOrcStringStatisticsLimit(session)); case AVRO: - return createAvroWriter(new ForwardingFileIo(fileSystem), outputPath, icebergSchema, session); + return createAvroWriter(fileSystem, outputPath, icebergSchema, session); default: throw new TrinoException(NOT_SUPPORTED, "File format not supported: " + fileFormat); } @@ -139,7 +139,7 @@ public IcebergFileWriter createDataFileWriter( public IcebergFileWriter createPositionDeleteWriter( TrinoFileSystem fileSystem, - String outputPath, + Location outputPath, ConnectorSession session, IcebergFileFormat fileFormat, Map storageProperties) @@ -150,7 +150,7 @@ public IcebergFileWriter createPositionDeleteWriter( case ORC: return createOrcWriter(FULL_METRICS_CONFIG, fileSystem, outputPath, POSITION_DELETE_SCHEMA, session, storageProperties, DataSize.ofBytes(Integer.MAX_VALUE)); case AVRO: - return createAvroWriter(new ForwardingFileIo(fileSystem), outputPath, POSITION_DELETE_SCHEMA, session); + return createAvroWriter(fileSystem, outputPath, POSITION_DELETE_SCHEMA, session); default: throw new TrinoException(NOT_SUPPORTED, "File format not supported: " + fileFormat); } @@ -159,7 +159,7 @@ public IcebergFileWriter createPositionDeleteWriter( private IcebergFileWriter createParquetWriter( MetricsConfig metricsConfig, TrinoFileSystem fileSystem, - String outputPath, + Location outputPath, Schema icebergSchema, ConnectorSession session) { @@ -193,7 +193,6 @@ private IcebergFileWriter createParquetWriter( IntStream.range(0, fileColumnNames.size()).toArray(), getCompressionCodec(session).getParquetCompressionCodec(), nodeVersion.toString(), - outputPath, fileSystem); } catch (IOException e) { @@ -204,7 +203,7 @@ private IcebergFileWriter createParquetWriter( private IcebergFileWriter createOrcWriter( MetricsConfig metricsConfig, TrinoFileSystem fileSystem, - String outputPath, + Location outputPath, Schema icebergSchema, ConnectorSession session, Map storageProperties, @@ -287,19 +286,19 @@ public static OrcWriterOptions withBloomFilterOptions(OrcWriterOptions orcWriter } private IcebergFileWriter createAvroWriter( - FileIO fileIo, - String outputPath, + TrinoFileSystem fileSystem, + Location outputPath, Schema icebergSchema, ConnectorSession session) { - Closeable rollbackAction = () -> fileIo.deleteFile(outputPath); + Closeable rollbackAction = () -> fileSystem.deleteFile(outputPath); List columnTypes = icebergSchema.columns().stream() .map(column -> toTrinoType(column.type(), typeManager)) .collect(toImmutableList()); return new IcebergAvroFileWriter( - fileIo.newOutputFile(outputPath), + new ForwardingOutputFile(fileSystem, outputPath.toString()), rollbackAction, icebergSchema, columnTypes, diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java index b2c032917df1..d0ca6c00812c 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java @@ -29,6 +29,7 @@ import io.airlift.units.Duration; import io.trino.filesystem.FileEntry; import io.trino.filesystem.FileIterator; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.plugin.base.classloader.ClassLoaderSafeSystemTable; @@ -733,7 +734,7 @@ public ConnectorOutputTableHandle beginCreateTable(ConnectorSession session, Con throw new SchemaNotFoundException(schemaName); } transaction = newCreateTableTransaction(catalog, tableMetadata, session); - String location = transaction.table().location(); + Location location = Location.of(transaction.table().location()); TrinoFileSystem fileSystem = fileSystemFactory.create(session); try { if (fileSystem.listFiles(location).hasNext()) { @@ -945,11 +946,11 @@ private void cleanExtraOutputFiles(ConnectorSession session, Set written Set locations = getOutputFilesLocations(writtenFiles); Set fileNames = getOutputFilesFileNames(writtenFiles); for (String location : locations) { - cleanExtraOutputFiles(fileSystem, session.getQueryId(), location, fileNames); + cleanExtraOutputFiles(fileSystem, session.getQueryId(), Location.of(location), fileNames); } } - private static void cleanExtraOutputFiles(TrinoFileSystem fileSystem, String queryId, String location, Set fileNamesToKeep) + private static void cleanExtraOutputFiles(TrinoFileSystem fileSystem, String queryId, Location location, Set fileNamesToKeep) { checkArgument(!queryId.contains("-"), "query ID should not contain hyphens: %s", queryId); @@ -960,7 +961,7 @@ private static void cleanExtraOutputFiles(TrinoFileSystem fileSystem, String que FileIterator iterator = fileSystem.listFiles(location); while (iterator.hasNext()) { FileEntry entry = iterator.next(); - String name = fileName(entry.location()); + String name = entry.location().fileName(); if (name.startsWith(queryId + "-") && !fileNamesToKeep.contains(name)) { filesToDelete.add(name); } @@ -972,14 +973,11 @@ private static void cleanExtraOutputFiles(TrinoFileSystem fileSystem, String que log.info("Found %s files to delete and %s to retain in location %s for query %s", filesToDelete.size(), fileNamesToKeep.size(), location, queryId); ImmutableList.Builder deletedFilesBuilder = ImmutableList.builder(); - Iterator filesToDeleteIterator = filesToDelete.iterator(); - List deleteBatch = new ArrayList<>(); - while (filesToDeleteIterator.hasNext()) { - String fileName = filesToDeleteIterator.next(); + List deleteBatch = new ArrayList<>(); + for (String fileName : filesToDelete) { deletedFilesBuilder.add(fileName); - filesToDeleteIterator.remove(); - deleteBatch.add(location + "/" + fileName); + deleteBatch.add(location.appendPath(fileName)); if (deleteBatch.size() >= DELETE_BATCH_SIZE) { log.debug("Deleting failed attempt files %s for query %s", deleteBatch, queryId); fileSystem.deleteFiles(deleteBatch); @@ -1321,10 +1319,10 @@ private void executeExpireSnapshots(ConnectorSession session, IcebergTableExecut long expireTimestampMillis = session.getStart().toEpochMilli() - retention.toMillis(); TrinoFileSystem fileSystem = fileSystemFactory.create(session); - List pathsToDelete = new ArrayList<>(); + List pathsToDelete = new ArrayList<>(); // deleteFunction is not accessed from multiple threads unless .executeDeleteWith() is used Consumer deleteFunction = path -> { - pathsToDelete.add(path); + pathsToDelete.add(Location.of(path)); if (pathsToDelete.size() == DELETE_BATCH_SIZE) { try { fileSystem.deleteFiles(pathsToDelete); @@ -1461,12 +1459,12 @@ private static ManifestReader> readerForManifest(Table private void scanAndDeleteInvalidFiles(Table table, ConnectorSession session, SchemaTableName schemaTableName, Instant expiration, Set validFiles, String subfolder) { try { - List filesToDelete = new ArrayList<>(); + List filesToDelete = new ArrayList<>(); TrinoFileSystem fileSystem = fileSystemFactory.create(session); - FileIterator allFiles = fileSystem.listFiles(table.location() + "/" + subfolder); + FileIterator allFiles = fileSystem.listFiles(Location.of(table.location()).appendPath(subfolder)); while (allFiles.hasNext()) { FileEntry entry = allFiles.next(); - if (entry.lastModified().isBefore(expiration) && !validFiles.contains(fileName(entry.location()))) { + if (entry.lastModified().isBefore(expiration) && !validFiles.contains(entry.location().fileName())) { filesToDelete.add(entry.location()); if (filesToDelete.size() >= DELETE_BATCH_SIZE) { log.debug("Deleting files while removing orphan files for table %s [%s]", schemaTableName, filesToDelete); @@ -2158,6 +2156,7 @@ private void finishWrite(ConnectorSession session, IcebergTableHandle table, Col fileSystem.deleteFiles(fullyDeletedFiles.values().stream() .flatMap(Collection::stream) .map(CommitTaskData::getPath) + .map(Location::of) .collect(toImmutableSet())); } catch (IOException e) { diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSink.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSink.java index c23a252e7e69..be59bfd35eb3 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSink.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSink.java @@ -18,6 +18,7 @@ import io.airlift.json.JsonCodec; import io.airlift.slice.Slice; import io.airlift.units.DataSize; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.plugin.iceberg.PartitionTransforms.ColumnTransform; import io.trino.spi.Page; @@ -409,7 +410,7 @@ private WriteContext createWriter(String outputPath, Optional par { IcebergFileWriter writer = fileWriterFactory.createDataFileWriter( fileSystem, - outputPath, + Location.of(outputPath), outputSchema, session, fileFormat, diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java index eb1430bb129f..1d84562df9e5 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java @@ -20,6 +20,7 @@ import com.google.common.collect.ImmutableSet; import com.google.common.graph.Traverser; import io.airlift.slice.Slice; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.TrinoInputFile; @@ -58,7 +59,7 @@ import io.trino.plugin.iceberg.delete.DeleteFilter; import io.trino.plugin.iceberg.delete.PositionDeleteFilter; import io.trino.plugin.iceberg.delete.RowPredicate; -import io.trino.plugin.iceberg.fileio.ForwardingFileIo; +import io.trino.plugin.iceberg.fileio.ForwardingInputFile; import io.trino.spi.TrinoException; import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ConnectorPageSource; @@ -291,12 +292,11 @@ else if (identity.getId() == TRINO_MERGE_PARTITION_DATA) { TrinoFileSystem fileSystem = fileSystemFactory.create(session); TrinoInputFile inputfile = isUseFileSizeFromMetadata(session) - ? fileSystem.newInputFile(split.getPath(), split.getFileSize()) - : fileSystem.newInputFile(split.getPath()); + ? fileSystem.newInputFile(Location.of(split.getPath()), split.getFileSize()) + : fileSystem.newInputFile(Location.of(split.getPath())); ReaderPageSourceWithRowPositions readerPageSourceWithRowPositions = createDataPageSource( session, - fileSystem, inputfile, split.getStart(), split.getLength(), @@ -443,8 +443,7 @@ private ConnectorPageSource openDeletes( TrinoFileSystem fileSystem = fileSystemFactory.create(session); return createDataPageSource( session, - fileSystem, - fileSystem.newInputFile(delete.path(), delete.fileSizeInBytes()), + fileSystem.newInputFile(Location.of(delete.path()), delete.fileSizeInBytes()), 0, delete.fileSizeInBytes(), delete.recordCount(), @@ -462,7 +461,6 @@ private ConnectorPageSource openDeletes( public ReaderPageSourceWithRowPositions createDataPageSource( ConnectorSession session, - TrinoFileSystem fileSystem, TrinoInputFile inputFile, long start, long length, @@ -521,7 +519,6 @@ public ReaderPageSourceWithRowPositions createDataPageSource( partitionKeys); case AVRO: return createAvroPageSource( - fileSystem, inputFile, start, length, @@ -600,7 +597,7 @@ else if (partitionKeys.containsKey(column.getId())) { deserializePartitionValue(trinoType, partitionKeys.get(column.getId()).orElse(null), column.getName())))); } else if (column.isPathColumn()) { - columnAdaptations.add(ColumnAdaptation.constantColumn(nativeValueToBlock(FILE_PATH.getType(), utf8Slice(inputFile.location())))); + columnAdaptations.add(ColumnAdaptation.constantColumn(nativeValueToBlock(FILE_PATH.getType(), utf8Slice(inputFile.location().toString())))); } else if (column.isFileModifiedTimeColumn()) { columnAdaptations.add(ColumnAdaptation.constantColumn(nativeValueToBlock(FILE_MODIFIED_TIME.getType(), packDateTimeWithZone(inputFile.lastModified().toEpochMilli(), UTC_KEY)))); @@ -963,7 +960,7 @@ else if (partitionKeys.containsKey(column.getId())) { deserializePartitionValue(trinoType, partitionKeys.get(column.getId()).orElse(null), column.getName()))); } else if (column.isPathColumn()) { - pageSourceBuilder.addConstantColumn(nativeValueToBlock(FILE_PATH.getType(), utf8Slice(inputFile.location()))); + pageSourceBuilder.addConstantColumn(nativeValueToBlock(FILE_PATH.getType(), utf8Slice(inputFile.location().toString()))); } else if (column.isFileModifiedTimeColumn()) { pageSourceBuilder.addConstantColumn(nativeValueToBlock(FILE_MODIFIED_TIME.getType(), packDateTimeWithZone(inputFile.lastModified().toEpochMilli(), UTC_KEY))); @@ -1045,7 +1042,6 @@ else if (column.getId() == TRINO_MERGE_PARTITION_DATA) { } private static ReaderPageSourceWithRowPositions createAvroPageSource( - TrinoFileSystem fileSystem, TrinoInputFile inputFile, long start, long length, @@ -1065,10 +1061,9 @@ private static ReaderPageSourceWithRowPositions createAvroPageSource( .map(readerColumns -> (List) readerColumns.get().stream().map(IcebergColumnHandle.class::cast).collect(toImmutableList())) .orElse(columns); - InputFile file; + InputFile file = new ForwardingInputFile(inputFile); OptionalLong fileModifiedTime = OptionalLong.empty(); try { - file = new ForwardingFileIo(fileSystem).newInputFile(inputFile.location(), inputFile.length()); if (readColumns.stream().anyMatch(IcebergColumnHandle::isFileModifiedTimeColumn)) { fileModifiedTime = OptionalLong.of(inputFile.lastModified().toEpochMilli()); } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergParquetFileWriter.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergParquetFileWriter.java index 7dea195a20d6..104598c8dd0a 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergParquetFileWriter.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergParquetFileWriter.java @@ -17,7 +17,7 @@ import io.trino.filesystem.TrinoOutputFile; import io.trino.parquet.writer.ParquetWriterOptions; import io.trino.plugin.hive.parquet.ParquetFileWriter; -import io.trino.plugin.iceberg.fileio.ForwardingFileIo; +import io.trino.plugin.iceberg.fileio.ForwardingInputFile; import io.trino.spi.type.Type; import org.apache.iceberg.Metrics; import org.apache.iceberg.MetricsConfig; @@ -39,8 +39,7 @@ public class IcebergParquetFileWriter implements IcebergFileWriter { private final MetricsConfig metricsConfig; - private final String outputPath; - private final TrinoFileSystem fileSystem; + private final InputFile inputFile; public IcebergParquetFileWriter( MetricsConfig metricsConfig, @@ -54,7 +53,6 @@ public IcebergParquetFileWriter( int[] fileInputColumnIndexes, CompressionCodec compressionCodec, String trinoVersion, - String outputPath, TrinoFileSystem fileSystem) throws IOException { @@ -72,14 +70,12 @@ public IcebergParquetFileWriter( Optional.empty(), Optional.empty()); this.metricsConfig = requireNonNull(metricsConfig, "metricsConfig is null"); - this.outputPath = requireNonNull(outputPath, "outputPath is null"); - this.fileSystem = requireNonNull(fileSystem, "fileSystem is null"); + this.inputFile = new ForwardingInputFile(fileSystem.newInputFile(outputFile.location())); } @Override public Metrics getMetrics() { - InputFile inputFile = new ForwardingFileIo(fileSystem).newInputFile(outputPath); return fileMetrics(inputFile, metricsConfig); } } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java index bb980218ae35..c6ddeea56d6c 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java @@ -21,6 +21,7 @@ import com.google.common.io.Closer; import io.airlift.units.DataSize; import io.airlift.units.Duration; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.TrinoInputFile; import io.trino.plugin.iceberg.delete.DeleteFile; @@ -276,7 +277,7 @@ public CompletableFuture getNextBatch(int maxSize) private long getModificationTime(String path) { try { - TrinoInputFile inputFile = fileSystemFactory.create(session).newInputFile(path); + TrinoInputFile inputFile = fileSystemFactory.create(session).newInputFile(Location.of(path)); return inputFile.lastModified().toEpochMilli(); } catch (IOException e) { diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/TrinoOrcDataSource.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/TrinoOrcDataSource.java index ce49c2cb7b5c..0dca29a6561d 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/TrinoOrcDataSource.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/TrinoOrcDataSource.java @@ -34,7 +34,7 @@ public class TrinoOrcDataSource public TrinoOrcDataSource(TrinoInputFile file, OrcReaderOptions options, FileFormatDataSourceStats stats) throws IOException { - super(new OrcDataSourceId(file.location()), file.length(), options); + super(new OrcDataSourceId(file.location().toString()), file.length(), options); this.stats = requireNonNull(stats, "stats is null"); this.input = file.newInput(); } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/AbstractTrinoCatalog.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/AbstractTrinoCatalog.java index 852f4561c4a3..58fddc398e66 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/AbstractTrinoCatalog.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/AbstractTrinoCatalog.java @@ -16,6 +16,7 @@ import com.google.common.collect.ImmutableMap; import dev.failsafe.Failsafe; import dev.failsafe.RetryPolicy; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.plugin.base.CatalogName; import io.trino.plugin.hive.HiveMetadata; @@ -212,7 +213,7 @@ protected String createNewTableName(String baseTableName) protected void deleteTableDirectory(TrinoFileSystem fileSystem, SchemaTableName schemaTableName, String tableLocation) { try { - fileSystem.deleteDirectory(tableLocation); + fileSystem.deleteDirectory(Location.of(tableLocation)); } catch (IOException e) { throw new TrinoException(ICEBERG_FILESYSTEM_ERROR, format("Failed to delete directory %s of the table %s", tableLocation, schemaTableName), e); diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/TrinoHiveCatalog.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/TrinoHiveCatalog.java index 95fd22cb9a43..ec91ab3fdecd 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/TrinoHiveCatalog.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/TrinoHiveCatalog.java @@ -16,6 +16,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import io.airlift.log.Logger; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.plugin.base.CatalogName; import io.trino.plugin.hive.HiveSchemaProperties; @@ -197,9 +198,9 @@ public void createNamespace(ConnectorSession session, String namespace, Map { String location = (String) value; try { - fileSystemFactory.create(session).newInputFile(location).exists(); + fileSystemFactory.create(session).newInputFile(Location.of(location)).exists(); } - catch (IOException e) { + catch (IOException | IllegalArgumentException e) { throw new TrinoException(INVALID_SCHEMA_PROPERTY, "Invalid location URI: " + location, e); } database.setLocation(Optional.of(location)); @@ -228,7 +229,7 @@ public void dropNamespace(ConnectorSession session, String namespace) // If we fail to check the schema location, behave according to fallback. boolean deleteData = location.map(path -> { try { - return !fileSystemFactory.create(session).listFiles(path).hasNext(); + return !fileSystemFactory.create(session).listFiles(Location.of(path)).hasNext(); } catch (IOException | RuntimeException e) { log.warn(e, "Could not check schema directory '%s'", path); diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/delete/IcebergPositionDeletePageSink.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/delete/IcebergPositionDeletePageSink.java index b9a16d602218..b8fa505b286b 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/delete/IcebergPositionDeletePageSink.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/delete/IcebergPositionDeletePageSink.java @@ -15,6 +15,7 @@ import io.airlift.json.JsonCodec; import io.airlift.slice.Slice; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.plugin.iceberg.CommitTaskData; import io.trino.plugin.iceberg.IcebergFileFormat; @@ -88,7 +89,7 @@ public IcebergPositionDeletePageSink( this.outputPath = partition .map(partitionData -> locationProvider.newDataLocation(partitionSpec, partitionData, fileName)) .orElseGet(() -> locationProvider.newDataLocation(fileName)); - this.writer = fileWriterFactory.createPositionDeleteWriter(fileSystem, outputPath, session, fileFormat, storageProperties); + this.writer = fileWriterFactory.createPositionDeleteWriter(fileSystem, Location.of(outputPath), session, fileFormat, storageProperties); } @Override diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/fileio/ForwardingFileIo.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/fileio/ForwardingFileIo.java index 620ec646ac2d..03dcb9d109d9 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/fileio/ForwardingFileIo.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/fileio/ForwardingFileIo.java @@ -14,6 +14,7 @@ package io.trino.plugin.iceberg.fileio; import com.google.common.collect.Iterables; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import org.apache.iceberg.io.BulkDeletionFailureException; import org.apache.iceberg.io.InputFile; @@ -44,13 +45,13 @@ public ForwardingFileIo(TrinoFileSystem fileSystem) @Override public InputFile newInputFile(String path) { - return new ForwardingInputFile(fileSystem.newInputFile(path)); + return new ForwardingInputFile(fileSystem.newInputFile(Location.of(path))); } @Override public InputFile newInputFile(String path, long length) { - return new ForwardingInputFile(fileSystem.newInputFile(path, length)); + return new ForwardingInputFile(fileSystem.newInputFile(Location.of(path), length)); } @Override @@ -63,7 +64,7 @@ public OutputFile newOutputFile(String path) public void deleteFile(String path) { try { - fileSystem.deleteFile(path); + fileSystem.deleteFile(Location.of(path)); } catch (IOException e) { throw new UncheckedIOException("Failed to delete file: " + path, e); @@ -81,7 +82,7 @@ public void deleteFiles(Iterable pathsToDelete) private void deleteBatch(List filesToDelete) { try { - fileSystem.deleteFiles(filesToDelete); + fileSystem.deleteFiles(filesToDelete.stream().map(Location::of).toList()); } catch (IOException e) { throw new UncheckedIOException( diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/fileio/ForwardingInputFile.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/fileio/ForwardingInputFile.java index add5c10dcf46..3715f23adb56 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/fileio/ForwardingInputFile.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/fileio/ForwardingInputFile.java @@ -62,7 +62,7 @@ public SeekableInputStream newStream() @Override public String location() { - return inputFile.location(); + return inputFile.location().toString(); } @Override diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/fileio/ForwardingOutputFile.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/fileio/ForwardingOutputFile.java index b03cdb059e9a..40a65d5b7a36 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/fileio/ForwardingOutputFile.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/fileio/ForwardingOutputFile.java @@ -14,6 +14,7 @@ package io.trino.plugin.iceberg.fileio; import com.google.common.io.CountingOutputStream; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoOutputFile; import org.apache.iceberg.io.InputFile; @@ -35,7 +36,7 @@ public class ForwardingOutputFile public ForwardingOutputFile(TrinoFileSystem fileSystem, String path) { this.fileSystem = requireNonNull(fileSystem, "fileSystem is null"); - this.outputFile = fileSystem.newOutputFile(path); + this.outputFile = fileSystem.newOutputFile(Location.of(path)); } @Override @@ -65,7 +66,7 @@ public PositionOutputStream createOrOverwrite() @Override public String location() { - return outputFile.location(); + return outputFile.location().toString(); } @Override diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/procedure/MigrateProcedure.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/procedure/MigrateProcedure.java index dbe3272fe3ce..d8941455bbed 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/procedure/MigrateProcedure.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/procedure/MigrateProcedure.java @@ -19,8 +19,10 @@ import io.airlift.log.Logger; import io.trino.filesystem.FileEntry; import io.trino.filesystem.FileIterator; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.filesystem.TrinoInputFile; import io.trino.plugin.hive.HiveStorageFormat; import io.trino.plugin.hive.metastore.Column; import io.trino.plugin.hive.metastore.HiveMetastore; @@ -313,22 +315,23 @@ private List buildDataFiles(ConnectorSession session, RecursiveDirecto { // TODO: Introduce parallelism TrinoFileSystem fileSystem = fileSystemFactory.create(session); - FileIterator files = fileSystem.listFiles(location); + FileIterator files = fileSystem.listFiles(Location.of(location)); ImmutableList.Builder dataFilesBuilder = ImmutableList.builder(); while (files.hasNext()) { FileEntry file = files.next(); - String relativePath = file.location().substring(location.length()); + String fileLocation = file.location().toString(); + String relativePath = fileLocation.substring(location.length()); if (relativePath.contains("/_") || relativePath.contains("/.")) { continue; } - if (recursive == RecursiveDirectory.FALSE && isRecursive(location, file.location())) { + if (recursive == RecursiveDirectory.FALSE && isRecursive(location, fileLocation)) { continue; } - else if (recursive == RecursiveDirectory.FAIL && isRecursive(location, file.location())) { + if (recursive == RecursiveDirectory.FAIL && isRecursive(location, fileLocation)) { throw new TrinoException(NOT_SUPPORTED, "Recursive directory must not exist when recursive_directory argument is 'fail': " + file.location()); } - Metrics metrics = loadMetrics(fileSystem, format, file.location(), nameMapping); + Metrics metrics = loadMetrics(fileSystem.newInputFile(file.location()), format, nameMapping); DataFile dataFile = buildDataFile(file, partition, partitionSpec, format.name(), metrics); dataFilesBuilder.add(dataFile); } @@ -344,9 +347,9 @@ private static boolean isRecursive(String baseLocation, String location) return suffix.contains("/"); } - private Metrics loadMetrics(TrinoFileSystem fileSystem, HiveStorageFormat storageFormat, String path, NameMapping nameMapping) + private static Metrics loadMetrics(TrinoInputFile file, HiveStorageFormat storageFormat, NameMapping nameMapping) { - InputFile inputFile = new ForwardingInputFile(fileSystem.newInputFile(path)); + InputFile inputFile = new ForwardingInputFile(file); return switch (storageFormat) { case ORC -> OrcMetrics.fromInputFile(inputFile, METRICS_CONFIG, nameMapping); case PARQUET -> ParquetUtil.fileMetrics(inputFile, METRICS_CONFIG, nameMapping); @@ -376,7 +379,7 @@ private static List getPartitionColumnNames(io.trino.plugin.hive.metasto private static DataFile buildDataFile(FileEntry file, StructLike partition, PartitionSpec spec, String format, Metrics metrics) { return DataFiles.builder(spec) - .withPath(file.location()) + .withPath(file.location().toString()) .withFormat(format) .withFileSizeInBytes(file.length()) .withMetrics(metrics) diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/procedure/RegisterTableProcedure.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/procedure/RegisterTableProcedure.java index c274de1bd89c..9eeb27d08ef0 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/procedure/RegisterTableProcedure.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/procedure/RegisterTableProcedure.java @@ -16,6 +16,7 @@ import com.google.common.collect.ImmutableList; import io.trino.filesystem.FileEntry; import io.trino.filesystem.FileIterator; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.plugin.iceberg.IcebergConfig; @@ -143,7 +144,7 @@ private void doRegisterTable( TrinoFileSystem fileSystem = fileSystemFactory.create(clientSession); String metadataLocation = getMetadataLocation(fileSystem, tableLocation, metadataFileName); - validateLocation(fileSystem, metadataLocation); + validateLocation(fileSystem, Location.of(metadataLocation)); try { // Try to read the metadata file. Invalid metadata file will throw the exception. TableMetadataParser.read(new ForwardingFileIo(fileSystem), metadataLocation); @@ -179,20 +180,21 @@ public static String getLatestMetadataLocation(TrinoFileSystem fileSystem, Strin String metadataDirectoryLocation = format("%s/%s", stripTrailingSlash(location), METADATA_FOLDER_NAME); try { int latestMetadataVersion = -1; - FileIterator fileIterator = fileSystem.listFiles(metadataDirectoryLocation); + FileIterator fileIterator = fileSystem.listFiles(Location.of(metadataDirectoryLocation)); while (fileIterator.hasNext()) { FileEntry fileEntry = fileIterator.next(); - if (fileEntry.location().contains(METADATA_FILE_EXTENSION)) { - OptionalInt version = parseVersion(fileEntry.location()); + String fileLocation = fileEntry.location().toString(); + if (fileLocation.contains(METADATA_FILE_EXTENSION)) { + OptionalInt version = parseVersion(fileLocation); if (version.isPresent()) { int versionNumber = version.getAsInt(); if (versionNumber > latestMetadataVersion) { latestMetadataVersion = versionNumber; latestMetadataLocations.clear(); - latestMetadataLocations.add(fileEntry.location()); + latestMetadataLocations.add(fileLocation); } else if (versionNumber == latestMetadataVersion) { - latestMetadataLocations.add(fileEntry.location()); + latestMetadataLocations.add(fileLocation); } } } @@ -213,7 +215,7 @@ else if (versionNumber == latestMetadataVersion) { return getOnlyElement(latestMetadataLocations); } - private static void validateLocation(TrinoFileSystem fileSystem, String location) + private static void validateLocation(TrinoFileSystem fileSystem, Location location) { try { if (!fileSystem.newInputFile(location).exists()) { diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorSmokeTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorSmokeTest.java index d29f806d64c7..9c9bcbe8495c 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorSmokeTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorSmokeTest.java @@ -16,6 +16,7 @@ import com.google.common.collect.ImmutableList; import io.trino.Session; import io.trino.filesystem.FileIterator; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.hdfs.HdfsFileSystemFactory; import io.trino.plugin.iceberg.fileio.ForwardingFileIo; @@ -459,7 +460,7 @@ public void testSortedNationTable() "WITH (sorted_by = ARRAY['comment'], format = '" + format.name() + "') AS SELECT * FROM nation WITH NO DATA")) { assertUpdate(withSmallRowGroups, "INSERT INTO " + table.getName() + " SELECT * FROM nation", 25); for (Object filePath : computeActual("SELECT file_path from \"" + table.getName() + "$files\"").getOnlyColumnAsSet()) { - assertTrue(isFileSorted((String) filePath, "comment")); + assertTrue(isFileSorted(Location.of((String) filePath), "comment")); } assertQuery("SELECT * FROM " + table.getName(), "SELECT * FROM nation"); } @@ -479,7 +480,7 @@ public void testFileSortingWithLargerTable() "INSERT INTO " + table.getName() + " TABLE tpch.tiny.lineitem", "VALUES 60175"); for (Object filePath : computeActual("SELECT file_path from \"" + table.getName() + "$files\"").getOnlyColumnAsSet()) { - assertTrue(isFileSorted((String) filePath, "comment")); + assertTrue(isFileSorted(Location.of((String) filePath), "comment")); } assertQuery("SELECT * FROM " + table.getName(), "SELECT * FROM lineitem"); } @@ -492,8 +493,8 @@ public void testDropTableWithMissingMetadataFile() String tableName = "test_drop_table_with_missing_metadata_file_" + randomNameSuffix(); assertUpdate("CREATE TABLE " + tableName + " AS SELECT 1 x, 'INDIA' y", 1); - String metadataLocation = getMetadataLocation(tableName); - String tableLocation = getTableLocation(tableName); + Location metadataLocation = Location.of(getMetadataLocation(tableName)); + Location tableLocation = Location.of(getTableLocation(tableName)); // Delete current metadata file trinoFileSystem.deleteFile(metadataLocation); @@ -514,8 +515,8 @@ public void testDropTableWithMissingSnapshotFile() String metadataLocation = getMetadataLocation(tableName); TableMetadata tableMetadata = TableMetadataParser.read(new ForwardingFileIo(trinoFileSystem), metadataLocation); - String tableLocation = tableMetadata.location(); - String currentSnapshotFile = tableMetadata.currentSnapshot().manifestListLocation(); + Location tableLocation = Location.of(tableMetadata.location()); + Location currentSnapshotFile = Location.of(tableMetadata.currentSnapshot().manifestListLocation()); // Delete current snapshot file trinoFileSystem.deleteFile(currentSnapshotFile); @@ -537,8 +538,8 @@ public void testDropTableWithMissingManifestListFile() String metadataLocation = getMetadataLocation(tableName); FileIO fileIo = new ForwardingFileIo(trinoFileSystem); TableMetadata tableMetadata = TableMetadataParser.read(fileIo, metadataLocation); - String tableLocation = tableMetadata.location(); - String manifestListFile = tableMetadata.currentSnapshot().allManifests(fileIo).get(0).path(); + Location tableLocation = Location.of(tableMetadata.location()); + Location manifestListFile = Location.of(tableMetadata.currentSnapshot().allManifests(fileIo).get(0).path()); // Delete Manifest List file trinoFileSystem.deleteFile(manifestListFile); @@ -558,11 +559,11 @@ public void testDropTableWithMissingDataFile() assertUpdate("CREATE TABLE " + tableName + " AS SELECT 1 x, 'INDIA' y", 1); assertUpdate("INSERT INTO " + tableName + " VALUES (2, 'POLAND')", 1); - String tableLocation = getTableLocation(tableName); - String tableDataPath = String.format("%s/%s", tableLocation, "data"); + Location tableLocation = Location.of(getTableLocation(tableName)); + Location tableDataPath = tableLocation.appendPath("data"); FileIterator fileIterator = trinoFileSystem.listFiles(tableDataPath); assertTrue(fileIterator.hasNext()); - String dataFile = fileIterator.next().location(); + Location dataFile = fileIterator.next().location(); // Delete data file trinoFileSystem.deleteFile(dataFile); @@ -582,7 +583,7 @@ public void testDropTableWithNonExistentTableLocation() assertUpdate("CREATE TABLE " + tableName + " AS SELECT 1 x, 'INDIA' y", 1); assertUpdate("INSERT INTO " + tableName + " VALUES (2, 'POLAND')", 1); - String tableLocation = getTableLocation(tableName); + Location tableLocation = Location.of(getTableLocation(tableName)); // Delete table location trinoFileSystem.deleteDirectory(tableLocation); @@ -593,7 +594,7 @@ public void testDropTableWithNonExistentTableLocation() assertFalse(getQueryRunner().tableExists(getSession(), tableName)); } - protected abstract boolean isFileSorted(String path, String sortColumnName); + protected abstract boolean isFileSorted(Location path, String sortColumnName); private String getTableLocation(String tableName) { diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java index e93cd9d1a523..6c4f7746cabb 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/BaseIcebergConnectorTest.java @@ -20,6 +20,7 @@ import io.airlift.units.Duration; import io.trino.Session; import io.trino.filesystem.FileIterator; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.hdfs.HdfsFileSystemFactory; @@ -4202,7 +4203,7 @@ protected DataFileReader readManifestFile(String location) throws IOException { Path tempFile = getDistributedQueryRunner().getCoordinator().getBaseDataDir().resolve(randomUUID() + "-manifest-copy"); - try (InputStream inputStream = fileSystemFactory.create(SESSION).newInputFile(location).newStream()) { + try (InputStream inputStream = fileSystemFactory.create(SESSION).newInputFile(Location.of(location)).newStream()) { Files.copy(inputStream, tempFile); } return new DataFileReader<>(tempFile.toFile(), new GenericDatumReader<>()); @@ -6698,7 +6699,7 @@ public void testDropTableWithMissingMetadataFile() TrinoFileSystem trinoFileSystem = fileSystemFactory.create(SESSION); String tableLocation = getTableLocation(tableName); - String metadataLocation = getLatestMetadataLocation(trinoFileSystem, tableLocation); + Location metadataLocation = Location.of(getLatestMetadataLocation(trinoFileSystem, tableLocation)); // Delete current metadata file trinoFileSystem.deleteFile(metadataLocation); @@ -6707,7 +6708,7 @@ public void testDropTableWithMissingMetadataFile() // try to drop table assertUpdate("DROP TABLE " + tableName); assertFalse(getQueryRunner().tableExists(getSession(), tableName)); - assertFalse(trinoFileSystem.listFiles(tableLocation).hasNext(), "Table location should not exist"); + assertFalse(trinoFileSystem.listFiles(Location.of(tableLocation)).hasNext(), "Table location should not exist"); } @Test @@ -6721,7 +6722,7 @@ public void testDropTableWithMissingSnapshotFile() String tableLocation = getTableLocation(tableName); String metadataLocation = getLatestMetadataLocation(trinoFileSystem, tableLocation); TableMetadata tableMetadata = TableMetadataParser.read(new ForwardingFileIo(trinoFileSystem), metadataLocation); - String currentSnapshotFile = tableMetadata.currentSnapshot().manifestListLocation(); + Location currentSnapshotFile = Location.of(tableMetadata.currentSnapshot().manifestListLocation()); // Delete current snapshot file trinoFileSystem.deleteFile(currentSnapshotFile); @@ -6730,7 +6731,7 @@ public void testDropTableWithMissingSnapshotFile() // try to drop table assertUpdate("DROP TABLE " + tableName); assertFalse(getQueryRunner().tableExists(getSession(), tableName)); - assertFalse(trinoFileSystem.listFiles(tableLocation).hasNext(), "Table location should not exist"); + assertFalse(trinoFileSystem.listFiles(Location.of(tableLocation)).hasNext(), "Table location should not exist"); } @Test @@ -6745,7 +6746,7 @@ public void testDropTableWithMissingManifestListFile() String metadataLocation = getLatestMetadataLocation(trinoFileSystem, tableLocation); FileIO fileIo = new ForwardingFileIo(trinoFileSystem); TableMetadata tableMetadata = TableMetadataParser.read(fileIo, metadataLocation); - String manifestListFile = tableMetadata.currentSnapshot().allManifests(fileIo).get(0).path(); + Location manifestListFile = Location.of(tableMetadata.currentSnapshot().allManifests(fileIo).get(0).path()); // Delete Manifest List file trinoFileSystem.deleteFile(manifestListFile); @@ -6754,7 +6755,7 @@ public void testDropTableWithMissingManifestListFile() // try to drop table assertUpdate("DROP TABLE " + tableName); assertFalse(getQueryRunner().tableExists(getSession(), tableName)); - assertFalse(trinoFileSystem.listFiles(tableLocation).hasNext(), "Table location should not exist"); + assertFalse(trinoFileSystem.listFiles(Location.of(tableLocation)).hasNext(), "Table location should not exist"); } @Test @@ -6766,11 +6767,11 @@ public void testDropTableWithMissingDataFile() assertUpdate("INSERT INTO " + tableName + " VALUES (2, 'POLAND')", 1); TrinoFileSystem trinoFileSystem = fileSystemFactory.create(SESSION); - String tableLocation = getTableLocation(tableName); - String tableDataPath = String.format("%s/%s", tableLocation, "data"); + Location tableLocation = Location.of(getTableLocation(tableName)); + Location tableDataPath = tableLocation.appendPath("data"); FileIterator fileIterator = trinoFileSystem.listFiles(tableDataPath); assertTrue(fileIterator.hasNext()); - String dataFile = fileIterator.next().location(); + Location dataFile = fileIterator.next().location(); // Delete data file trinoFileSystem.deleteFile(dataFile); @@ -6791,7 +6792,7 @@ public void testDropTableWithNonExistentTableLocation() assertUpdate("INSERT INTO " + tableName + " VALUES (2, 'POLAND')", 1); TrinoFileSystem trinoFileSystem = fileSystemFactory.create(SESSION); - String tableLocation = getTableLocation(tableName); + Location tableLocation = Location.of(getTableLocation(tableName)); // Delete table location trinoFileSystem.deleteDirectory(tableLocation); @@ -6812,8 +6813,8 @@ public void testCorruptedTableLocation() assertUpdate("INSERT INTO " + tableName + " VALUES (1, 'INDIA', ROW ('Aug', 1947)), (2, 'POLAND', ROW ('Nov', 1918)), (3, 'USA', ROW ('Jul', 1776))", 3); TrinoFileSystem trinoFileSystem = fileSystemFactory.create(SESSION); - String tableLocation = getTableLocation(tableName); - String metadataLocation = tableLocation + "/metadata"; + Location tableLocation = Location.of(getTableLocation(tableName)); + Location metadataLocation = tableLocation.appendPath("metadata"); // break the table by deleting all metadata files trinoFileSystem.deleteDirectory(metadataLocation); @@ -6902,8 +6903,8 @@ public void testDropCorruptedTableWithHiveRedirection() .containsAll(queryRunner.execute("TABLE " + icebergTableName)); TrinoFileSystem trinoFileSystem = fileSystemFactory.create(SESSION); - String tableLocation = (String) queryRunner.execute("SELECT DISTINCT regexp_replace(\"$path\", '/[^/]*/[^/]*$', '') FROM " + tableName).getOnlyValue(); - String metadataLocation = tableLocation + "/metadata"; + Location tableLocation = Location.of((String) queryRunner.execute("SELECT DISTINCT regexp_replace(\"$path\", '/[^/]*/[^/]*$', '') FROM " + tableName).getOnlyValue()); + Location metadataLocation = tableLocation.appendPath("metadata"); // break the table by deleting all metadata files trinoFileSystem.deleteDirectory(metadataLocation); @@ -6972,9 +6973,9 @@ protected List listFiles(String directory) { TrinoFileSystem fileSystem = fileSystemFactory.create(SESSION); ImmutableList.Builder files = ImmutableList.builder(); - FileIterator listing = fileSystem.listFiles(directory); + FileIterator listing = fileSystem.listFiles(Location.of(directory)); while (listing.hasNext()) { - String location = listing.next().location(); + String location = listing.next().location().toString(); if (location.matches(".*/\\..*\\.crc")) { continue; } @@ -6987,14 +6988,14 @@ protected long fileSize(String location) throws IOException { TrinoFileSystem fileSystem = fileSystemFactory.create(SESSION); - return fileSystem.newInputFile(location).length(); + return fileSystem.newInputFile(Location.of(location)).length(); } protected void createFile(String location) throws IOException { TrinoFileSystem fileSystem = fileSystemFactory.create(SESSION); - fileSystem.newOutputFile(location).create().close(); + fileSystem.newOutputFile(Location.of(location)).create().close(); } private List getSnapshotIds(String tableName) diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergTestUtils.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergTestUtils.java index c23260ccc02d..032eb93c1d3f 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergTestUtils.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergTestUtils.java @@ -15,11 +15,11 @@ import io.airlift.slice.Slice; import io.trino.Session; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.TrinoInputFile; import io.trino.filesystem.local.LocalInputFile; -import io.trino.orc.FileOrcDataSource; import io.trino.orc.OrcDataSource; import io.trino.orc.OrcReader; import io.trino.orc.OrcReaderOptions; @@ -35,7 +35,6 @@ import org.apache.parquet.hadoop.metadata.ParquetMetadata; import java.io.File; -import java.io.FileNotFoundException; import java.io.IOException; import java.io.UncheckedIOException; import java.util.List; @@ -63,19 +62,7 @@ public static Session withSmallRowGroups(Session session) .build(); } - public static boolean checkOrcFileSorting(String path, String sortColumnName) - { - return checkOrcFileSorting(() -> { - try { - return new FileOrcDataSource(new File(path), new OrcReaderOptions()); - } - catch (FileNotFoundException e) { - throw new UncheckedIOException(e); - } - }, sortColumnName); - } - - public static boolean checkOrcFileSorting(TrinoFileSystemFactory fileSystemFactory, String path, String sortColumnName) + public static boolean checkOrcFileSorting(TrinoFileSystemFactory fileSystemFactory, Location path, String sortColumnName) { return checkOrcFileSorting(() -> { try { diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergAbfsConnectorSmokeTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergAbfsConnectorSmokeTest.java index 037613a3cb36..7e8245f2cc7c 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergAbfsConnectorSmokeTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergAbfsConnectorSmokeTest.java @@ -16,6 +16,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.io.Resources; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.hdfs.HdfsFileSystemFactory; import io.trino.hdfs.ConfigurationInitializer; @@ -182,7 +183,7 @@ protected void deleteDirectory(String location) } @Override - protected boolean isFileSorted(String path, String sortColumnName) + protected boolean isFileSorted(Location path, String sortColumnName) { return checkOrcFileSorting(fileSystemFactory, path, sortColumnName); } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergConnectorSmokeTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergConnectorSmokeTest.java index 61314d9027e6..4d5fdeafdaf5 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergConnectorSmokeTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergConnectorSmokeTest.java @@ -14,6 +14,7 @@ package io.trino.plugin.iceberg; import com.google.common.collect.ImmutableMap; +import io.trino.filesystem.Location; import io.trino.plugin.hive.metastore.HiveMetastore; import io.trino.testing.QueryRunner; import org.testng.annotations.AfterClass; @@ -26,6 +27,7 @@ import static com.google.common.io.MoreFiles.deleteRecursively; import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; +import static io.trino.plugin.hive.HiveTestUtils.HDFS_FILE_SYSTEM_FACTORY; import static io.trino.plugin.hive.metastore.file.FileHiveMetastore.createTestingFileHiveMetastore; import static io.trino.plugin.iceberg.IcebergTestUtils.checkOrcFileSorting; import static java.lang.String.format; @@ -107,8 +109,8 @@ protected void deleteDirectory(String location) } @Override - protected boolean isFileSorted(String path, String sortColumnName) + protected boolean isFileSorted(Location path, String sortColumnName) { - return checkOrcFileSorting(path, sortColumnName); + return checkOrcFileSorting(HDFS_FILE_SYSTEM_FACTORY, path, sortColumnName); } } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergGcsConnectorSmokeTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergGcsConnectorSmokeTest.java index b9e904352ca6..2777c46e9422 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergGcsConnectorSmokeTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergGcsConnectorSmokeTest.java @@ -17,6 +17,7 @@ import com.google.common.collect.ImmutableSet; import com.google.common.io.Resources; import io.airlift.log.Logger; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.hdfs.HdfsFileSystemFactory; @@ -135,7 +136,7 @@ public void removeTestData() { try { TrinoFileSystem fileSystem = fileSystemFactory.create(SESSION); - fileSystem.deleteDirectory(schemaPath()); + fileSystem.deleteDirectory(Location.of(schemaPath())); } catch (IOException e) { // The GCS bucket should be configured to expire objects automatically. Clean up issues do not need to fail the test. @@ -173,7 +174,7 @@ protected boolean locationExists(String location) { try { TrinoFileSystem fileSystem = fileSystemFactory.create(SESSION); - return fileSystem.newInputFile(location).exists(); + return fileSystem.newInputFile(Location.of(location)).exists(); } catch (IOException e) { throw new UncheckedIOException(e); @@ -218,7 +219,7 @@ protected void deleteDirectory(String location) { try { TrinoFileSystem fileSystem = fileSystemFactory.create(SESSION); - fileSystem.deleteDirectory(location); + fileSystem.deleteDirectory(Location.of(location)); } catch (IOException e) { throw new UncheckedIOException(e); @@ -226,7 +227,7 @@ protected void deleteDirectory(String location) } @Override - protected boolean isFileSorted(String path, String sortColumnName) + protected boolean isFileSorted(Location path, String sortColumnName) { return checkOrcFileSorting(fileSystemFactory, path, sortColumnName); } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataFileOperations.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataFileOperations.java index 683acfbc9aa3..37fbefdb9033 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataFileOperations.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMetadataFileOperations.java @@ -459,7 +459,7 @@ private Multiset getOperations() return trackingFileSystemFactory.getOperationCounts() .entrySet().stream() .flatMap(entry -> nCopies(entry.getValue(), new FileOperation( - fromFilePath(entry.getKey().getFilePath()), + fromFilePath(entry.getKey().getLocation().toString()), entry.getKey().getOperationType())).stream()) .collect(toCollection(HashMultiset::create)); } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMinioAvroConnectorSmokeTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMinioAvroConnectorSmokeTest.java index 1c6ec8dd3fa1..dcfbdfd2e7ea 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMinioAvroConnectorSmokeTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMinioAvroConnectorSmokeTest.java @@ -13,6 +13,7 @@ */ package io.trino.plugin.iceberg; +import io.trino.filesystem.Location; import org.testng.SkipException; import static org.apache.iceberg.FileFormat.AVRO; @@ -38,7 +39,7 @@ public void testFileSortingWithLargerTable() } @Override - protected boolean isFileSorted(String path, String sortColumnName) + protected boolean isFileSorted(Location path, String sortColumnName) { throw new IllegalStateException("File sorting tests should be skipped for Avro"); } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMinioOrcConnectorTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMinioOrcConnectorTest.java index 6fd4dad72261..ce8ed82447c8 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMinioOrcConnectorTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMinioOrcConnectorTest.java @@ -16,6 +16,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import io.trino.Session; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.hdfs.HdfsFileSystemFactory; import io.trino.hdfs.ConfigurationInitializer; @@ -123,7 +124,7 @@ protected boolean supportsRowGroupStatistics(String typeName) @Override protected boolean isFileSorted(String path, String sortColumnName) { - return checkOrcFileSorting(fileSystemFactory, path, sortColumnName); + return checkOrcFileSorting(fileSystemFactory, Location.of(path), sortColumnName); } @Test @@ -147,10 +148,10 @@ private void testReadSingleIntegerColumnOrcFile(String orcFileResourceName, int try (TestTable table = new TestTable(getQueryRunner()::execute, "test_read_as_integer", "(\"_col0\") AS VALUES 0, NULL")) { String orcFilePath = (String) computeScalar(format("SELECT DISTINCT file_path FROM \"%s$files\"", table.getName())); TrinoFileSystem fileSystem = fileSystemFactory.create(SESSION); - try (OutputStream outputStream = fileSystem.newOutputFile(orcFilePath).createOrOverwrite()) { + try (OutputStream outputStream = fileSystem.newOutputFile(Location.of(orcFilePath)).createOrOverwrite()) { Files.copy(new File(getResource(orcFileResourceName).toURI()).toPath(), outputStream); } - fileSystem.deleteFiles(List.of(orcFilePath.replaceAll("/([^/]*)$", ".$1.crc"))); + fileSystem.deleteFiles(List.of(Location.of(orcFilePath.replaceAll("/([^/]*)$", ".$1.crc")))); Session ignoreFileSizeFromMetadata = Session.builder(getSession()) // The replaced and replacing file sizes may be different diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMinioParquetConnectorSmokeTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMinioParquetConnectorSmokeTest.java index 200c5aa67d16..de99bce050a2 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMinioParquetConnectorSmokeTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMinioParquetConnectorSmokeTest.java @@ -13,6 +13,7 @@ */ package io.trino.plugin.iceberg; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import static io.trino.plugin.iceberg.IcebergTestUtils.checkParquetFileSorting; @@ -28,7 +29,7 @@ public TestIcebergMinioParquetConnectorSmokeTest() } @Override - protected boolean isFileSorted(String path, String sortColumnName) + protected boolean isFileSorted(Location path, String sortColumnName) { TrinoFileSystem fileSystem = fileSystemFactory.create(SESSION); return checkParquetFileSorting(fileSystem.newInputFile(path), sortColumnName); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergNodeLocalDynamicSplitPruning.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergNodeLocalDynamicSplitPruning.java index 3acb35d56685..d9ce8c792319 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergNodeLocalDynamicSplitPruning.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergNodeLocalDynamicSplitPruning.java @@ -16,10 +16,12 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; -import io.trino.filesystem.TrinoFileSystem; +import io.airlift.testing.TempFile; import io.trino.filesystem.TrinoInputFile; import io.trino.filesystem.TrinoOutputFile; import io.trino.filesystem.hdfs.HdfsFileSystemFactory; +import io.trino.filesystem.local.LocalInputFile; +import io.trino.filesystem.local.LocalOutputFile; import io.trino.metadata.TableHandle; import io.trino.orc.OrcWriteValidation; import io.trino.orc.OrcWriter; @@ -52,17 +54,14 @@ import java.io.IOException; import java.nio.file.Files; -import java.nio.file.Path; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; -import java.util.UUID; import java.util.concurrent.CompletableFuture; import static io.trino.orc.metadata.CompressionKind.NONE; import static io.trino.plugin.hive.HiveTestUtils.HDFS_ENVIRONMENT; -import static io.trino.plugin.hive.HiveTestUtils.SESSION; import static io.trino.plugin.hive.HiveType.HIVE_INT; import static io.trino.plugin.hive.HiveType.HIVE_STRING; import static io.trino.plugin.iceberg.ColumnIdentity.TypeCategory.PRIMITIVE; @@ -103,11 +102,11 @@ public void testDynamicSplitPruning() { IcebergConfig icebergConfig = new IcebergConfig(); HiveTransactionHandle transaction = new HiveTransactionHandle(false); - String path = "/tmp/" + UUID.randomUUID() + ".tmp"; - try { - TrinoFileSystem fileSystem = new HdfsFileSystemFactory(HDFS_ENVIRONMENT).create(SESSION); - TrinoOutputFile outputFile = fileSystem.newOutputFile(path); - TrinoInputFile inputFile = fileSystem.newInputFile(path); + try (TempFile file = new TempFile()) { + Files.delete(file.path()); + + TrinoOutputFile outputFile = new LocalOutputFile(file.file()); + TrinoInputFile inputFile = new LocalInputFile(file.file()); writeOrcContent(outputFile); try (ConnectorPageSource emptyPageSource = createTestingPageSource(transaction, icebergConfig, inputFile, getDynamicFilter(getTupleDomainForSplitPruning()))) { @@ -123,9 +122,6 @@ public void testDynamicSplitPruning() assertEquals(page.getBlock(1).getSlice(0, 0, page.getBlock(1).getSliceLength(0)).toStringUtf8(), DATA_COLUMN_VALUE); } } - finally { - Files.deleteIfExists(Path.of(path)); - } } private static void writeOrcContent(TrinoOutputFile outputFile) @@ -157,7 +153,7 @@ private static ConnectorPageSource createTestingPageSource(HiveTransactionHandle throws IOException { IcebergSplit split = new IcebergSplit( - "file:///" + inputFile.location(), + inputFile.toString(), 0, inputFile.length(), inputFile.length(), @@ -169,8 +165,7 @@ private static ConnectorPageSource createTestingPageSource(HiveTransactionHandle ImmutableList.of(), SplitWeight.standard()); - String filePath = inputFile.location(); - String tablePath = filePath.substring(0, filePath.lastIndexOf("/")); + String tablePath = inputFile.location().fileName(); TableHandle tableHandle = new TableHandle( TEST_CATALOG_HANDLE, new IcebergTableHandle( diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergRegisterTableProcedure.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergRegisterTableProcedure.java index 841050742272..270af3be7baa 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergRegisterTableProcedure.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergRegisterTableProcedure.java @@ -16,6 +16,7 @@ import com.google.common.collect.ImmutableMap; import io.trino.filesystem.FileEntry; import io.trino.filesystem.FileIterator; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.hdfs.HdfsFileSystemFactory; import io.trino.plugin.hive.metastore.HiveMetastore; @@ -298,17 +299,16 @@ public void testRegisterTableWithInvalidMetadataFile() assertUpdate(format("INSERT INTO %s values(1, 'INDIA', true)", tableName), 1); assertUpdate(format("INSERT INTO %s values(2, 'USA', false)", tableName), 1); - String tableLocation = getTableLocation(tableName); + Location tableLocation = Location.of(getTableLocation(tableName)); String tableNameNew = tableName + "_new"; - String metadataDirectoryLocation = format("%s/%s", tableLocation, METADATA_FOLDER_NAME); + Location metadataDirectoryLocation = tableLocation.appendPath(METADATA_FOLDER_NAME); FileIterator fileIterator = fileSystem.listFiles(metadataDirectoryLocation); // Find one invalid metadata file inside metadata folder String invalidMetadataFileName = "invalid-default.avro"; while (fileIterator.hasNext()) { FileEntry fileEntry = fileIterator.next(); - if (fileEntry.location().endsWith(".avro")) { - String file = fileEntry.location(); - invalidMetadataFileName = file.substring(file.lastIndexOf("/") + 1); + if (fileEntry.location().fileName().endsWith(".avro")) { + invalidMetadataFileName = fileEntry.location().fileName(); break; } } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableWithCustomLocation.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableWithCustomLocation.java index ced2618850fd..80b672ac5459 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableWithCustomLocation.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableWithCustomLocation.java @@ -13,13 +13,13 @@ */ package io.trino.plugin.iceberg; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.plugin.hive.metastore.Table; import io.trino.plugin.hive.metastore.file.FileHiveMetastore; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.DistributedQueryRunner; import io.trino.testing.MaterializedResult; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.metastore.TableType; import org.testng.annotations.AfterClass; import org.testng.annotations.Test; @@ -89,17 +89,20 @@ public void testCreateAndDrop() Table table = metastore.getTable("tpch", tableName).orElseThrow(); assertThat(table.getTableType()).isEqualTo(TableType.EXTERNAL_TABLE.name()); - Path tableLocation = new Path(table.getStorage().getLocation()); + Location tableLocation = Location.of(table.getStorage().getLocation()); TrinoFileSystem fileSystem = HDFS_FILE_SYSTEM_FACTORY.create(SESSION); - assertTrue(fileSystem.newInputFile(tableLocation.toString()).exists(), "The directory corresponding to the table storage location should exist"); + assertTrue(fileSystem.newInputFile(tableLocation).exists(), "The directory corresponding to the table storage location should exist"); + MaterializedResult materializedResult = computeActual("SELECT * FROM \"test_create_and_drop$files\""); assertEquals(materializedResult.getRowCount(), 1); DataFileRecord dataFile = toDataFileRecord(materializedResult.getMaterializedRows().get(0)); - assertTrue(fileSystem.newInputFile(new Path(dataFile.getFilePath()).toString()).exists(), "The data file should exist"); + Location dataFileLocation = Location.of(dataFile.getFilePath()); + assertTrue(fileSystem.newInputFile(dataFileLocation).exists(), "The data file should exist"); + assertQuerySucceeds(format("DROP TABLE %s", tableName)); assertFalse(metastore.getTable("tpch", tableName).isPresent(), "Table should be dropped"); - assertFalse(fileSystem.newInputFile(new Path(dataFile.getFilePath()).toString()).exists(), "The data file should have been removed"); - assertFalse(fileSystem.newInputFile(tableLocation.toString()).exists(), "The directory corresponding to the dropped Iceberg table should not be removed because it may be shared with other tables"); + assertFalse(fileSystem.newInputFile(dataFileLocation).exists(), "The data file should have been removed"); + assertFalse(fileSystem.newInputFile(tableLocation).exists(), "The directory corresponding to the dropped Iceberg table should not be removed because it may be shared with other tables"); } @Test diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableWithExternalLocation.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableWithExternalLocation.java index df21119e58b1..45fd7369b2f7 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableWithExternalLocation.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableWithExternalLocation.java @@ -13,13 +13,13 @@ */ package io.trino.plugin.iceberg; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.plugin.hive.metastore.Table; import io.trino.plugin.hive.metastore.file.FileHiveMetastore; import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.DistributedQueryRunner; import io.trino.testing.MaterializedResult; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.metastore.TableType; import org.testng.annotations.AfterClass; import org.testng.annotations.Test; @@ -78,17 +78,18 @@ public void testCreateAndDrop() Table table = metastore.getTable("tpch", tableName).orElseThrow(); assertThat(table.getTableType()).isEqualTo(TableType.EXTERNAL_TABLE.name()); - Path tableLocation = new Path(table.getStorage().getLocation()); + Location tableLocation = Location.of(table.getStorage().getLocation()); TrinoFileSystem fileSystem = HDFS_FILE_SYSTEM_FACTORY.create(SESSION); - assertTrue(fileSystem.newInputFile(tableLocation.toString()).exists(), "The directory corresponding to the table storage location should exist"); + assertTrue(fileSystem.newInputFile(tableLocation).exists(), "The directory corresponding to the table storage location should exist"); MaterializedResult materializedResult = computeActual("SELECT * FROM \"test_table_external_create_and_drop$files\""); assertEquals(materializedResult.getRowCount(), 1); DataFileRecord dataFile = toDataFileRecord(materializedResult.getMaterializedRows().get(0)); - assertTrue(fileSystem.newInputFile(new Path(dataFile.getFilePath()).toString()).exists(), "The data file should exist"); + Location dataFileLocation = Location.of(dataFile.getFilePath()); + assertTrue(fileSystem.newInputFile(dataFileLocation).exists(), "The data file should exist"); assertQuerySucceeds(format("DROP TABLE %s", tableName)); assertThat(metastore.getTable("tpch", tableName)).as("Table should be dropped").isEmpty(); - assertFalse(fileSystem.newInputFile(new Path(dataFile.getFilePath()).toString()).exists(), "The data file should have been removed"); - assertFalse(fileSystem.newInputFile(tableLocation.toString()).exists(), "The directory corresponding to the dropped Iceberg table should be removed as we don't allow shared locations."); + assertFalse(fileSystem.newInputFile(dataFileLocation).exists(), "The data file should have been removed"); + assertFalse(fileSystem.newInputFile(tableLocation).exists(), "The directory corresponding to the dropped Iceberg table should be removed as we don't allow shared locations."); } } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogConnectorSmokeTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogConnectorSmokeTest.java index 8668c118c1ec..4a1ea3a42a4e 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogConnectorSmokeTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCatalogConnectorSmokeTest.java @@ -25,6 +25,7 @@ import com.amazonaws.services.s3.model.S3ObjectSummary; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.hdfs.HdfsFileSystemFactory; @@ -243,7 +244,7 @@ protected void deleteDirectory(String location) } @Override - protected boolean isFileSorted(String path, String sortColumnName) + protected boolean isFileSorted(Location path, String sortColumnName) { TrinoFileSystem fileSystem = fileSystemFactory.create(SESSION); return checkParquetFileSorting(fileSystem.newInputFile(path), sortColumnName); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCreateTableFailure.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCreateTableFailure.java index 3d6ef0600eb8..f328ba2eed9f 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCreateTableFailure.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestIcebergGlueCreateTableFailure.java @@ -21,6 +21,7 @@ import io.trino.Session; import io.trino.filesystem.FileEntry; import io.trino.filesystem.FileIterator; +import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.hdfs.HdfsFileSystemFactory; import io.trino.metadata.InternalFunctionBundle; @@ -169,12 +170,12 @@ private void testCreateTableFailure(String expectedExceptionMessage, boolean sho protected void assertMetadataLocation(String tableName, boolean shouldMetadataFileExist) throws Exception { - FileIterator fileIterator = fileSystem.listFiles(dataDirectory.toString()); + FileIterator fileIterator = fileSystem.listFiles(Location.of(dataDirectory.toString())); String tableLocationPrefix = Path.of(dataDirectory.toString(), tableName).toString(); boolean metadataFileFound = false; while (fileIterator.hasNext()) { FileEntry fileEntry = fileIterator.next(); - String location = fileEntry.location(); + String location = fileEntry.location().toString(); if (location.startsWith(tableLocationPrefix) && location.endsWith(".metadata.json")) { metadataFileFound = true; break; diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/jdbc/TestIcebergJdbcCatalogConnectorSmokeTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/jdbc/TestIcebergJdbcCatalogConnectorSmokeTest.java index 59d05e3488f6..0e0dae239edb 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/jdbc/TestIcebergJdbcCatalogConnectorSmokeTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/jdbc/TestIcebergJdbcCatalogConnectorSmokeTest.java @@ -14,6 +14,7 @@ package io.trino.plugin.iceberg.catalog.jdbc; import com.google.common.collect.ImmutableMap; +import io.trino.filesystem.Location; import io.trino.hadoop.ConfigurationInstantiator; import io.trino.plugin.iceberg.BaseIcebergConnectorSmokeTest; import io.trino.plugin.iceberg.IcebergConfig; @@ -33,6 +34,7 @@ import static com.google.common.io.MoreFiles.deleteRecursively; import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; +import static io.trino.plugin.hive.HiveTestUtils.HDFS_FILE_SYSTEM_FACTORY; import static io.trino.plugin.iceberg.IcebergTestUtils.checkOrcFileSorting; import static io.trino.plugin.iceberg.catalog.jdbc.TestingIcebergJdbcServer.PASSWORD; import static io.trino.plugin.iceberg.catalog.jdbc.TestingIcebergJdbcServer.USER; @@ -170,8 +172,8 @@ protected void deleteDirectory(String location) } @Override - protected boolean isFileSorted(String path, String sortColumnName) + protected boolean isFileSorted(Location path, String sortColumnName) { - return checkOrcFileSorting(path, sortColumnName); + return checkOrcFileSorting(HDFS_FILE_SYSTEM_FACTORY, path, sortColumnName); } } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/rest/TestIcebergTrinoRestCatalogConnectorSmokeTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/rest/TestIcebergTrinoRestCatalogConnectorSmokeTest.java index 440b5ee91073..510d46738daa 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/rest/TestIcebergTrinoRestCatalogConnectorSmokeTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/rest/TestIcebergTrinoRestCatalogConnectorSmokeTest.java @@ -15,6 +15,7 @@ import com.google.common.collect.ImmutableMap; import io.airlift.http.server.testing.TestingHttpServer; +import io.trino.filesystem.Location; import io.trino.plugin.iceberg.BaseIcebergConnectorSmokeTest; import io.trino.plugin.iceberg.IcebergConfig; import io.trino.plugin.iceberg.IcebergQueryRunner; @@ -34,6 +35,7 @@ import static com.google.common.io.MoreFiles.deleteRecursively; import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; +import static io.trino.plugin.hive.HiveTestUtils.HDFS_FILE_SYSTEM_FACTORY; import static io.trino.plugin.iceberg.IcebergTestUtils.checkOrcFileSorting; import static io.trino.plugin.iceberg.catalog.rest.RestCatalogTestUtils.backendCatalog; import static java.lang.String.format; @@ -258,9 +260,9 @@ public void testDropTableWithNonExistentTableLocation() } @Override - protected boolean isFileSorted(String path, String sortColumnName) + protected boolean isFileSorted(Location path, String sortColumnName) { - return checkOrcFileSorting(path, sortColumnName); + return checkOrcFileSorting(HDFS_FILE_SYSTEM_FACTORY, path, sortColumnName); } @Override