diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index f5a7e99b684e..bc65e90eaeae 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -850,7 +850,7 @@ public FileContent content() { @Override public CharSequence path() { - return deleteFile.path(); + return deleteFile.location(); } @Override diff --git a/core/src/main/java/org/apache/iceberg/V1Metadata.java b/core/src/main/java/org/apache/iceberg/V1Metadata.java index c5319a0bc100..81fd65d99803 100644 --- a/core/src/main/java/org/apache/iceberg/V1Metadata.java +++ b/core/src/main/java/org/apache/iceberg/V1Metadata.java @@ -416,7 +416,7 @@ public FileContent content() { @Override public CharSequence path() { - return wrapped.path(); + return wrapped.location(); } @Override diff --git a/core/src/main/java/org/apache/iceberg/V2Metadata.java b/core/src/main/java/org/apache/iceberg/V2Metadata.java index 18c3b0a40613..2b98cd767c94 100644 --- a/core/src/main/java/org/apache/iceberg/V2Metadata.java +++ b/core/src/main/java/org/apache/iceberg/V2Metadata.java @@ -486,7 +486,7 @@ public FileContent content() { @Override public CharSequence path() { - return wrapped.path(); + return wrapped.location(); } @Override diff --git a/core/src/main/java/org/apache/iceberg/V3Metadata.java b/core/src/main/java/org/apache/iceberg/V3Metadata.java index fc11a7df03de..458796d4752a 100644 --- a/core/src/main/java/org/apache/iceberg/V3Metadata.java +++ b/core/src/main/java/org/apache/iceberg/V3Metadata.java @@ -500,7 +500,7 @@ public FileContent content() { @Override public CharSequence path() { - return wrapped.path(); + return wrapped.location(); } @Override diff --git a/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java b/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java index e96cd5c0cccf..532f51e3589d 100644 --- a/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java +++ b/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java @@ -164,7 +164,7 @@ public void testReplaceManifestsSeparate() { // cluster by path will split the manifest into two - table.rewriteManifests().clusterBy(file -> file.location()).commit(); + table.rewriteManifests().clusterBy(ContentFile::location).commit(); List manifests = table.currentSnapshot().allManifests(table.io()); assertThat(manifests).hasSize(2); diff --git a/data/src/test/java/org/apache/iceberg/data/TestDataFileIndexStatsFilters.java b/data/src/test/java/org/apache/iceberg/data/TestDataFileIndexStatsFilters.java index 2cbf2ad4d1f0..5de7cca92426 100644 --- a/data/src/test/java/org/apache/iceberg/data/TestDataFileIndexStatsFilters.java +++ b/data/src/test/java/org/apache/iceberg/data/TestDataFileIndexStatsFilters.java @@ -451,7 +451,7 @@ private void assertDeletes(FileScanTask task, DeleteFile... expectedDeleteFiles) } private CharSequenceSet deletePaths(FileScanTask task) { - return CharSequenceSet.of(Iterables.transform(task.deletes(), ContentFile::path)); + return CharSequenceSet.of(Iterables.transform(task.deletes(), ContentFile::location)); } private List planTasks() throws IOException { diff --git a/data/src/test/java/org/apache/iceberg/io/TestTaskEqualityDeltaWriter.java b/data/src/test/java/org/apache/iceberg/io/TestTaskEqualityDeltaWriter.java index a1c72ae3b1e0..b1688e6653f2 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestTaskEqualityDeltaWriter.java +++ b/data/src/test/java/org/apache/iceberg/io/TestTaskEqualityDeltaWriter.java @@ -178,7 +178,7 @@ public void testInsertDuplicatedKey() throws IOException { // Check records in the data file. DataFile dataFile = result.dataFiles()[0]; - assertThat(readRecordsAsList(table.schema(), dataFile.path())) + assertThat(readRecordsAsList(table.schema(), dataFile.location())) .isEqualTo( ImmutableList.of( createRecord(1, "aaa"), @@ -192,13 +192,13 @@ public void testInsertDuplicatedKey() throws IOException { // Check records in the pos-delete file. Schema posDeleteSchema = DeleteSchemaUtil.pathPosSchema(); - assertThat(readRecordsAsList(posDeleteSchema, posDeleteFile.path())) + assertThat(readRecordsAsList(posDeleteSchema, posDeleteFile.location())) .isEqualTo( ImmutableList.of( - posRecord.copy("file_path", dataFile.path(), "pos", 0L), - posRecord.copy("file_path", dataFile.path(), "pos", 1L), - posRecord.copy("file_path", dataFile.path(), "pos", 2L), - posRecord.copy("file_path", dataFile.path(), "pos", 3L))); + posRecord.copy("file_path", dataFile.location(), "pos", 0L), + posRecord.copy("file_path", dataFile.location(), "pos", 1L), + posRecord.copy("file_path", dataFile.location(), "pos", 2L), + posRecord.copy("file_path", dataFile.location(), "pos", 3L))); } @TestTemplate @@ -226,13 +226,13 @@ public void testUpsertSameRow() throws IOException { // Check records in the data file. DataFile dataFile = result.dataFiles()[0]; - assertThat(readRecordsAsList(table.schema(), dataFile.path())) + assertThat(readRecordsAsList(table.schema(), dataFile.location())) .isEqualTo(ImmutableList.of(record, record)); // Check records in the pos-delete file. DeleteFile posDeleteFile = result.deleteFiles()[0]; - assertThat(readRecordsAsList(DeleteSchemaUtil.pathPosSchema(), posDeleteFile.path())) - .isEqualTo(ImmutableList.of(posRecord.copy("file_path", dataFile.path(), "pos", 0L))); + assertThat(readRecordsAsList(DeleteSchemaUtil.pathPosSchema(), posDeleteFile.location())) + .isEqualTo(ImmutableList.of(posRecord.copy("file_path", dataFile.location(), "pos", 0L))); deltaWriter = createTaskWriter(eqDeleteFieldIds, eqDeleteRowSchema, DeleteGranularity.PARTITION); @@ -312,7 +312,7 @@ public void testUpsertData() throws IOException { // Check records in the data file. DataFile dataFile = result.dataFiles()[0]; - assertThat(readRecordsAsList(table.schema(), dataFile.path())) + assertThat(readRecordsAsList(table.schema(), dataFile.location())) .isEqualTo( ImmutableList.of( createRecord(5, "aaa"), createRecord(6, "aaa"), createRecord(7, "ccc"))); @@ -320,7 +320,7 @@ public void testUpsertData() throws IOException { // Check records in the eq-delete file. DeleteFile eqDeleteFile = result.deleteFiles()[0]; assertThat(eqDeleteFile.content()).isEqualTo(FileContent.EQUALITY_DELETES); - assertThat(readRecordsAsList(eqDeleteRowSchema, eqDeleteFile.path())) + assertThat(readRecordsAsList(eqDeleteRowSchema, eqDeleteFile.location())) .isEqualTo( ImmutableList.of(keyFunc.apply("aaa"), keyFunc.apply("ccc"), keyFunc.apply("bbb"))); @@ -328,8 +328,8 @@ public void testUpsertData() throws IOException { DeleteFile posDeleteFile = result.deleteFiles()[1]; Schema posDeleteSchema = DeleteSchemaUtil.pathPosSchema(); assertThat(posDeleteFile.content()).isEqualTo(FileContent.POSITION_DELETES); - assertThat(readRecordsAsList(posDeleteSchema, posDeleteFile.path())) - .isEqualTo(ImmutableList.of(posRecord.copy("file_path", dataFile.path(), "pos", 0L))); + assertThat(readRecordsAsList(posDeleteSchema, posDeleteFile.location())) + .isEqualTo(ImmutableList.of(posRecord.copy("file_path", dataFile.location(), "pos", 0L))); } @TestTemplate @@ -397,7 +397,7 @@ public void testUpsertDataWithFullRowSchema() throws IOException { // Check records in the data file. DataFile dataFile = result.dataFiles()[0]; - assertThat(readRecordsAsList(table.schema(), dataFile.path())) + assertThat(readRecordsAsList(table.schema(), dataFile.location())) .isEqualTo( ImmutableList.of( createRecord(5, "aaa"), createRecord(6, "aaa"), createRecord(7, "ccc"))); @@ -405,7 +405,7 @@ public void testUpsertDataWithFullRowSchema() throws IOException { // Check records in the eq-delete file. DeleteFile eqDeleteFile = result.deleteFiles()[0]; assertThat(eqDeleteFile.content()).isEqualTo(FileContent.EQUALITY_DELETES); - assertThat(readRecordsAsList(eqDeleteRowSchema, eqDeleteFile.path())) + assertThat(readRecordsAsList(eqDeleteRowSchema, eqDeleteFile.location())) .isEqualTo( ImmutableList.of( createRecord(3, "aaa"), createRecord(4, "ccc"), createRecord(2, "bbb"))); @@ -414,8 +414,8 @@ public void testUpsertDataWithFullRowSchema() throws IOException { DeleteFile posDeleteFile = result.deleteFiles()[1]; Schema posDeleteSchema = DeleteSchemaUtil.pathPosSchema(); assertThat(posDeleteFile.content()).isEqualTo(FileContent.POSITION_DELETES); - assertThat(readRecordsAsList(posDeleteSchema, posDeleteFile.path())) - .isEqualTo(ImmutableList.of(posRecord.copy("file_path", dataFile.path(), "pos", 0L))); + assertThat(readRecordsAsList(posDeleteSchema, posDeleteFile.location())) + .isEqualTo(ImmutableList.of(posRecord.copy("file_path", dataFile.location(), "pos", 0L))); } @TestTemplate diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java index 9c75a5e0f0fc..bf6f72cc287a 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java @@ -220,7 +220,7 @@ private static class FlinkDeleteFilter extends DeleteFilter { Schema tableSchema, Schema requestedSchema, InputFilesDecryptor inputFilesDecryptor) { - super(task.file().path().toString(), task.deletes(), tableSchema, requestedSchema); + super(task.file().location(), task.deletes(), tableSchema, requestedSchema); this.requiredRowType = FlinkSchemaUtil.convert(requiredSchema()); this.asStructLike = new RowDataWrapper(requiredRowType, requiredSchema().asStruct()); this.inputFilesDecryptor = inputFilesDecryptor; diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java index 344f64833b62..b6d6f60ef673 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java @@ -107,7 +107,7 @@ private String toString(Collection files) { .map( fileScanTask -> MoreObjects.toStringHelper(fileScanTask) - .add("file", fileScanTask.file().path().toString()) + .add("file", fileScanTask.file().location()) .add("start", fileScanTask.start()) .add("length", fileScanTask.length()) .toString()) diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java index ba0ea867ffb7..243c50a72b20 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java @@ -184,7 +184,7 @@ private TableTestStats extractPartitionResults(TableSchemaType tableSchemaType) for (FileScanTask scanTask : fileScanTasks) { long recordCountInFile = scanTask.file().recordCount(); - String[] splitFilePath = scanTask.file().path().toString().split("/"); + String[] splitFilePath = scanTask.file().location().split("/"); // Filename example: 00007-0-a7d3a29a-33e9-4740-88f4-0f494397d60c-00001.parquet // Writer ID: .......^^^^^ String filename = splitFilePath[splitFilePath.length - 1]; diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java index b283b8390a2b..44b1c57eda64 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java @@ -241,6 +241,6 @@ public void testDeleteStats() throws Exception { new String( deleteFile.lowerBounds().get(MetadataColumns.DELETE_FILE_PATH.fieldId()).array()); DataFile dataFile = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); - assumeThat(fromStat).isEqualTo(dataFile.path().toString()); + assumeThat(fromStat).isEqualTo(dataFile.location()); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java index 8bfd6cb3d043..a01926783d05 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java @@ -114,7 +114,7 @@ public void testCloseTwice() throws IOException { FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - assertThat(fs.exists(new Path(dataFile.path().toString()))).isTrue(); + assertThat(fs.exists(new Path(dataFile.location()))).isTrue(); } } } @@ -133,7 +133,7 @@ public void testAbort() throws IOException { FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - assertThat(fs.exists(new Path(dataFile.path().toString()))).isFalse(); + assertThat(fs.exists(new Path(dataFile.location()))).isFalse(); } } } @@ -155,7 +155,7 @@ public void testCompleteFiles() throws IOException { FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - assertThat(fs.exists(new Path(dataFile.path().toString()))).isTrue(); + assertThat(fs.exists(new Path(dataFile.location()))).isTrue(); } AppendFiles appendFiles = table.newAppend(); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java index 5767fa02c822..f9c678f24089 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java @@ -161,10 +161,9 @@ public void testTableScanThenIncrementalWithNonEmptyTable() throws Exception { assertThat(split.task().files()).hasSize(2); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); - Set expectedFiles = - ImmutableSet.of(dataFile1.path().toString(), dataFile2.path().toString()); + Set expectedFiles = ImmutableSet.of(dataFile1.location(), dataFile2.location()); assertThat(discoveredFiles).containsExactlyInAnyOrderElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); @@ -244,10 +243,10 @@ public void testIncrementalFromLatestSnapshotWithNonEmptyTable() throws Exceptio assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); // should discover dataFile2 appended in snapshot2 - Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); + Set expectedFiles = ImmutableSet.of(dataFile2.location()); assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); @@ -316,11 +315,10 @@ public void testIncrementalFromEarliestSnapshotWithNonEmptyTable() throws Except assertThat(split.task().files()).hasSize(2); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); // should discover files appended in both snapshot1 and snapshot2 - Set expectedFiles = - ImmutableSet.of(dataFile1.path().toString(), dataFile2.path().toString()); + Set expectedFiles = ImmutableSet.of(dataFile1.location(), dataFile2.location()); assertThat(discoveredFiles).containsExactlyInAnyOrderElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); @@ -406,10 +404,10 @@ public void testIncrementalFromSnapshotId() throws Exception { assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); // should discover dataFile2 appended in snapshot2 - Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); + Set expectedFiles = ImmutableSet.of(dataFile2.location()); assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); @@ -489,10 +487,10 @@ public void testIncrementalFromSnapshotTimestamp() throws Exception { assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); // should discover dataFile2 appended in snapshot2 - Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); + Set expectedFiles = ImmutableSet.of(dataFile2.location()); assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); @@ -529,12 +527,12 @@ public void testMaxPlanningSnapshotCount() throws Exception { ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); // should discover dataFile1 appended in snapshot1 verifyMaxPlanningSnapshotCountResult( - secondResult, null, snapshot1, ImmutableSet.of(dataFile1.path().toString())); + secondResult, null, snapshot1, ImmutableSet.of(dataFile1.location())); ContinuousEnumerationResult thirdResult = splitPlanner.planSplits(secondResult.toPosition()); // should discover dataFile2 appended in snapshot2 verifyMaxPlanningSnapshotCountResult( - thirdResult, snapshot1, snapshot2, ImmutableSet.of(dataFile2.path().toString())); + thirdResult, snapshot1, snapshot2, ImmutableSet.of(dataFile2.location())); } @Test @@ -670,7 +668,7 @@ private void verifyMaxPlanningSnapshotCountResult( assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java index 9c75a5e0f0fc..bf6f72cc287a 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java @@ -220,7 +220,7 @@ private static class FlinkDeleteFilter extends DeleteFilter { Schema tableSchema, Schema requestedSchema, InputFilesDecryptor inputFilesDecryptor) { - super(task.file().path().toString(), task.deletes(), tableSchema, requestedSchema); + super(task.file().location(), task.deletes(), tableSchema, requestedSchema); this.requiredRowType = FlinkSchemaUtil.convert(requiredSchema()); this.asStructLike = new RowDataWrapper(requiredRowType, requiredSchema().asStruct()); this.inputFilesDecryptor = inputFilesDecryptor; diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java index 344f64833b62..b6d6f60ef673 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java @@ -107,7 +107,7 @@ private String toString(Collection files) { .map( fileScanTask -> MoreObjects.toStringHelper(fileScanTask) - .add("file", fileScanTask.file().path().toString()) + .add("file", fileScanTask.file().location()) .add("start", fileScanTask.start()) .add("length", fileScanTask.length()) .toString()) diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java index 0b9c2193b4d5..04d7b8da6b9c 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java @@ -646,11 +646,11 @@ public void testSetCurrentAndCherryPickSnapshotId() { private void validateTableFiles(Table tbl, DataFile... expectedFiles) { tbl.refresh(); Set expectedFilePaths = - Arrays.stream(expectedFiles).map(DataFile::path).collect(Collectors.toSet()); + Arrays.stream(expectedFiles).map(DataFile::location).collect(Collectors.toSet()); Set actualFilePaths = StreamSupport.stream(tbl.newScan().planFiles().spliterator(), false) .map(FileScanTask::file) - .map(ContentFile::path) + .map(ContentFile::location) .collect(Collectors.toSet()); assertThat(actualFilePaths).as("Files should match").isEqualTo(expectedFilePaths); } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java index 8cebf950c5f0..3cab89e1ac7d 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java @@ -602,7 +602,7 @@ public static void assertEquals(ContentFile expected, ContentFile actual) assertThat(actual).isNotNull(); assertThat(actual.specId()).as("SpecId").isEqualTo(expected.specId()); assertThat(actual.content()).as("Content").isEqualTo(expected.content()); - assertThat(actual.path()).as("Path").isEqualTo(expected.path()); + assertThat(actual.location()).as("Location").isEqualTo(expected.location()); assertThat(actual.format()).as("Format").isEqualTo(expected.format()); assertThat(actual.partition().size()) .as("Partition size") diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java index 6b8399f666d4..aedc6edd6991 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java @@ -383,7 +383,7 @@ public void testRewriteAvoidRepeateCompress() throws IOException { assertThat(dataFilesRewrote).hasSize(2); // the biggest file do not be rewrote List rewroteDataFileNames = - dataFilesRewrote.stream().map(ContentFile::path).collect(Collectors.toList()); + dataFilesRewrote.stream().map(ContentFile::location).collect(Collectors.toList()); assertThat(rewroteDataFileNames).contains(file.getAbsolutePath()); // Assert the table records as expected. diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java index ba0ea867ffb7..243c50a72b20 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java @@ -184,7 +184,7 @@ private TableTestStats extractPartitionResults(TableSchemaType tableSchemaType) for (FileScanTask scanTask : fileScanTasks) { long recordCountInFile = scanTask.file().recordCount(); - String[] splitFilePath = scanTask.file().path().toString().split("/"); + String[] splitFilePath = scanTask.file().location().split("/"); // Filename example: 00007-0-a7d3a29a-33e9-4740-88f4-0f494397d60c-00001.parquet // Writer ID: .......^^^^^ String filename = splitFilePath[splitFilePath.length - 1]; diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java index b283b8390a2b..44b1c57eda64 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java @@ -241,6 +241,6 @@ public void testDeleteStats() throws Exception { new String( deleteFile.lowerBounds().get(MetadataColumns.DELETE_FILE_PATH.fieldId()).array()); DataFile dataFile = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); - assumeThat(fromStat).isEqualTo(dataFile.path().toString()); + assumeThat(fromStat).isEqualTo(dataFile.location()); } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java index 007d5f314946..69f39fba5130 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java @@ -224,7 +224,7 @@ public void testDeleteStats() throws Exception { new String( deleteFile.lowerBounds().get(MetadataColumns.DELETE_FILE_PATH.fieldId()).array()); DataFile dataFile = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); - assumeThat(fromStat).isEqualTo(dataFile.path().toString()); + assumeThat(fromStat).isEqualTo(dataFile.location()); } protected void testChangeLogs( diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java index 8bfd6cb3d043..a01926783d05 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java @@ -114,7 +114,7 @@ public void testCloseTwice() throws IOException { FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - assertThat(fs.exists(new Path(dataFile.path().toString()))).isTrue(); + assertThat(fs.exists(new Path(dataFile.location()))).isTrue(); } } } @@ -133,7 +133,7 @@ public void testAbort() throws IOException { FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - assertThat(fs.exists(new Path(dataFile.path().toString()))).isFalse(); + assertThat(fs.exists(new Path(dataFile.location()))).isFalse(); } } } @@ -155,7 +155,7 @@ public void testCompleteFiles() throws IOException { FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - assertThat(fs.exists(new Path(dataFile.path().toString()))).isTrue(); + assertThat(fs.exists(new Path(dataFile.location()))).isTrue(); } AppendFiles appendFiles = table.newAppend(); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java index 5767fa02c822..f66b9e302924 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java @@ -105,7 +105,8 @@ private CycleResult verifyOneCycle( .hasSize(1) .first() .satisfies( - fileScanTask -> assertThat(fileScanTask.file().path()).isEqualTo(dataFile.path())); + fileScanTask -> + assertThat(fileScanTask.file().location()).isEqualTo(dataFile.location())); return new CycleResult(result.toPosition(), split); } @@ -161,10 +162,9 @@ public void testTableScanThenIncrementalWithNonEmptyTable() throws Exception { assertThat(split.task().files()).hasSize(2); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); - Set expectedFiles = - ImmutableSet.of(dataFile1.path().toString(), dataFile2.path().toString()); + Set expectedFiles = ImmutableSet.of(dataFile1.location(), dataFile2.location()); assertThat(discoveredFiles).containsExactlyInAnyOrderElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); @@ -244,10 +244,10 @@ public void testIncrementalFromLatestSnapshotWithNonEmptyTable() throws Exceptio assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); // should discover dataFile2 appended in snapshot2 - Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); + Set expectedFiles = ImmutableSet.of(dataFile2.location()); assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); @@ -316,11 +316,10 @@ public void testIncrementalFromEarliestSnapshotWithNonEmptyTable() throws Except assertThat(split.task().files()).hasSize(2); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); // should discover files appended in both snapshot1 and snapshot2 - Set expectedFiles = - ImmutableSet.of(dataFile1.path().toString(), dataFile2.path().toString()); + Set expectedFiles = ImmutableSet.of(dataFile1.location(), dataFile2.location()); assertThat(discoveredFiles).containsExactlyInAnyOrderElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); @@ -406,10 +405,10 @@ public void testIncrementalFromSnapshotId() throws Exception { assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); // should discover dataFile2 appended in snapshot2 - Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); + Set expectedFiles = ImmutableSet.of(dataFile2.location()); assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); @@ -489,10 +488,10 @@ public void testIncrementalFromSnapshotTimestamp() throws Exception { assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); // should discover dataFile2 appended in snapshot2 - Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); + Set expectedFiles = ImmutableSet.of(dataFile2.location()); assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); @@ -529,12 +528,12 @@ public void testMaxPlanningSnapshotCount() throws Exception { ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); // should discover dataFile1 appended in snapshot1 verifyMaxPlanningSnapshotCountResult( - secondResult, null, snapshot1, ImmutableSet.of(dataFile1.path().toString())); + secondResult, null, snapshot1, ImmutableSet.of(dataFile1.location())); ContinuousEnumerationResult thirdResult = splitPlanner.planSplits(secondResult.toPosition()); // should discover dataFile2 appended in snapshot2 verifyMaxPlanningSnapshotCountResult( - thirdResult, snapshot1, snapshot2, ImmutableSet.of(dataFile2.path().toString())); + thirdResult, snapshot1, snapshot2, ImmutableSet.of(dataFile2.location())); } @Test @@ -670,7 +669,7 @@ private void verifyMaxPlanningSnapshotCountResult( assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java index 12bacdcd074d..4a21f451e1e5 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java @@ -170,7 +170,7 @@ private void assertSplitEquals(IcebergSourceSplit expected, IcebergSourceSplit a for (int i = 0; i < expectedTasks.size(); ++i) { FileScanTask expectedTask = expectedTasks.get(i); FileScanTask actualTask = actualTasks.get(i); - assertThat(actualTask.file().path()).isEqualTo(expectedTask.file().path()); + assertThat(actualTask.file().location()).isEqualTo(expectedTask.file().location()); assertThat(actualTask.sizeBytes()).isEqualTo(expectedTask.sizeBytes()); assertThat(actualTask.filesCount()).isEqualTo(expectedTask.filesCount()); assertThat(actualTask.start()).isEqualTo(expectedTask.start()); diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java index 9c75a5e0f0fc..bf6f72cc287a 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java @@ -220,7 +220,7 @@ private static class FlinkDeleteFilter extends DeleteFilter { Schema tableSchema, Schema requestedSchema, InputFilesDecryptor inputFilesDecryptor) { - super(task.file().path().toString(), task.deletes(), tableSchema, requestedSchema); + super(task.file().location(), task.deletes(), tableSchema, requestedSchema); this.requiredRowType = FlinkSchemaUtil.convert(requiredSchema()); this.asStructLike = new RowDataWrapper(requiredRowType, requiredSchema().asStruct()); this.inputFilesDecryptor = inputFilesDecryptor; diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java index 344f64833b62..b6d6f60ef673 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java @@ -107,7 +107,7 @@ private String toString(Collection files) { .map( fileScanTask -> MoreObjects.toStringHelper(fileScanTask) - .add("file", fileScanTask.file().path().toString()) + .add("file", fileScanTask.file().location()) .add("start", fileScanTask.start()) .add("length", fileScanTask.length()) .toString()) diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java index 0b9c2193b4d5..04d7b8da6b9c 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java @@ -646,11 +646,11 @@ public void testSetCurrentAndCherryPickSnapshotId() { private void validateTableFiles(Table tbl, DataFile... expectedFiles) { tbl.refresh(); Set expectedFilePaths = - Arrays.stream(expectedFiles).map(DataFile::path).collect(Collectors.toSet()); + Arrays.stream(expectedFiles).map(DataFile::location).collect(Collectors.toSet()); Set actualFilePaths = StreamSupport.stream(tbl.newScan().planFiles().spliterator(), false) .map(FileScanTask::file) - .map(ContentFile::path) + .map(ContentFile::location) .collect(Collectors.toSet()); assertThat(actualFilePaths).as("Files should match").isEqualTo(expectedFilePaths); } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java index 8cebf950c5f0..3cab89e1ac7d 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java @@ -602,7 +602,7 @@ public static void assertEquals(ContentFile expected, ContentFile actual) assertThat(actual).isNotNull(); assertThat(actual.specId()).as("SpecId").isEqualTo(expected.specId()); assertThat(actual.content()).as("Content").isEqualTo(expected.content()); - assertThat(actual.path()).as("Path").isEqualTo(expected.path()); + assertThat(actual.location()).as("Location").isEqualTo(expected.location()); assertThat(actual.format()).as("Format").isEqualTo(expected.format()); assertThat(actual.partition().size()) .as("Partition size") diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java index 085e7e48204c..a4dde8af3c12 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java @@ -390,7 +390,7 @@ public void testRewriteAvoidRepeateCompress() throws IOException { assertThat(dataFilesRewrote).hasSize(2); // the biggest file do not be rewrote List rewroteDataFileNames = - dataFilesRewrote.stream().map(ContentFile::path).collect(Collectors.toList()); + dataFilesRewrote.stream().map(ContentFile::location).collect(Collectors.toList()); assertThat(rewroteDataFileNames).contains(file.getAbsolutePath()); // Assert the table records as expected. diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java index ba0ea867ffb7..243c50a72b20 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java @@ -184,7 +184,7 @@ private TableTestStats extractPartitionResults(TableSchemaType tableSchemaType) for (FileScanTask scanTask : fileScanTasks) { long recordCountInFile = scanTask.file().recordCount(); - String[] splitFilePath = scanTask.file().path().toString().split("/"); + String[] splitFilePath = scanTask.file().location().split("/"); // Filename example: 00007-0-a7d3a29a-33e9-4740-88f4-0f494397d60c-00001.parquet // Writer ID: .......^^^^^ String filename = splitFilePath[splitFilePath.length - 1]; diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java index b283b8390a2b..44b1c57eda64 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java @@ -241,6 +241,6 @@ public void testDeleteStats() throws Exception { new String( deleteFile.lowerBounds().get(MetadataColumns.DELETE_FILE_PATH.fieldId()).array()); DataFile dataFile = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); - assumeThat(fromStat).isEqualTo(dataFile.path().toString()); + assumeThat(fromStat).isEqualTo(dataFile.location()); } } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java index 007d5f314946..69f39fba5130 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java @@ -224,7 +224,7 @@ public void testDeleteStats() throws Exception { new String( deleteFile.lowerBounds().get(MetadataColumns.DELETE_FILE_PATH.fieldId()).array()); DataFile dataFile = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); - assumeThat(fromStat).isEqualTo(dataFile.path().toString()); + assumeThat(fromStat).isEqualTo(dataFile.location()); } protected void testChangeLogs( diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java index 8bfd6cb3d043..a01926783d05 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java @@ -114,7 +114,7 @@ public void testCloseTwice() throws IOException { FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - assertThat(fs.exists(new Path(dataFile.path().toString()))).isTrue(); + assertThat(fs.exists(new Path(dataFile.location()))).isTrue(); } } } @@ -133,7 +133,7 @@ public void testAbort() throws IOException { FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - assertThat(fs.exists(new Path(dataFile.path().toString()))).isFalse(); + assertThat(fs.exists(new Path(dataFile.location()))).isFalse(); } } } @@ -155,7 +155,7 @@ public void testCompleteFiles() throws IOException { FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - assertThat(fs.exists(new Path(dataFile.path().toString()))).isTrue(); + assertThat(fs.exists(new Path(dataFile.location()))).isTrue(); } AppendFiles appendFiles = table.newAppend(); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java index 5767fa02c822..f66b9e302924 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java @@ -105,7 +105,8 @@ private CycleResult verifyOneCycle( .hasSize(1) .first() .satisfies( - fileScanTask -> assertThat(fileScanTask.file().path()).isEqualTo(dataFile.path())); + fileScanTask -> + assertThat(fileScanTask.file().location()).isEqualTo(dataFile.location())); return new CycleResult(result.toPosition(), split); } @@ -161,10 +162,9 @@ public void testTableScanThenIncrementalWithNonEmptyTable() throws Exception { assertThat(split.task().files()).hasSize(2); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); - Set expectedFiles = - ImmutableSet.of(dataFile1.path().toString(), dataFile2.path().toString()); + Set expectedFiles = ImmutableSet.of(dataFile1.location(), dataFile2.location()); assertThat(discoveredFiles).containsExactlyInAnyOrderElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); @@ -244,10 +244,10 @@ public void testIncrementalFromLatestSnapshotWithNonEmptyTable() throws Exceptio assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); // should discover dataFile2 appended in snapshot2 - Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); + Set expectedFiles = ImmutableSet.of(dataFile2.location()); assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); @@ -316,11 +316,10 @@ public void testIncrementalFromEarliestSnapshotWithNonEmptyTable() throws Except assertThat(split.task().files()).hasSize(2); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); // should discover files appended in both snapshot1 and snapshot2 - Set expectedFiles = - ImmutableSet.of(dataFile1.path().toString(), dataFile2.path().toString()); + Set expectedFiles = ImmutableSet.of(dataFile1.location(), dataFile2.location()); assertThat(discoveredFiles).containsExactlyInAnyOrderElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); @@ -406,10 +405,10 @@ public void testIncrementalFromSnapshotId() throws Exception { assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); // should discover dataFile2 appended in snapshot2 - Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); + Set expectedFiles = ImmutableSet.of(dataFile2.location()); assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); @@ -489,10 +488,10 @@ public void testIncrementalFromSnapshotTimestamp() throws Exception { assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); // should discover dataFile2 appended in snapshot2 - Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); + Set expectedFiles = ImmutableSet.of(dataFile2.location()); assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); @@ -529,12 +528,12 @@ public void testMaxPlanningSnapshotCount() throws Exception { ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); // should discover dataFile1 appended in snapshot1 verifyMaxPlanningSnapshotCountResult( - secondResult, null, snapshot1, ImmutableSet.of(dataFile1.path().toString())); + secondResult, null, snapshot1, ImmutableSet.of(dataFile1.location())); ContinuousEnumerationResult thirdResult = splitPlanner.planSplits(secondResult.toPosition()); // should discover dataFile2 appended in snapshot2 verifyMaxPlanningSnapshotCountResult( - thirdResult, snapshot1, snapshot2, ImmutableSet.of(dataFile2.path().toString())); + thirdResult, snapshot1, snapshot2, ImmutableSet.of(dataFile2.location())); } @Test @@ -670,7 +669,7 @@ private void verifyMaxPlanningSnapshotCountResult( assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) + .map(fileScanTask -> fileScanTask.file().location()) .collect(Collectors.toSet()); assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java index 12bacdcd074d..4a21f451e1e5 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java @@ -170,7 +170,7 @@ private void assertSplitEquals(IcebergSourceSplit expected, IcebergSourceSplit a for (int i = 0; i < expectedTasks.size(); ++i) { FileScanTask expectedTask = expectedTasks.get(i); FileScanTask actualTask = actualTasks.get(i); - assertThat(actualTask.file().path()).isEqualTo(expectedTask.file().path()); + assertThat(actualTask.file().location()).isEqualTo(expectedTask.file().location()); assertThat(actualTask.sizeBytes()).isEqualTo(expectedTask.sizeBytes()); assertThat(actualTask.filesCount()).isEqualTo(expectedTask.filesCount()); assertThat(actualTask.start()).isEqualTo(expectedTask.start()); diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableTest.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableTest.java index 13c459128dec..5650c4e82458 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableTest.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableTest.java @@ -497,7 +497,7 @@ public void testRegisterHadoopTableToHiveCatalog() throws IOException, TExceptio String file1Location = appendData(table, "file1"); List tasks = Lists.newArrayList(table.newScan().planFiles()); assertThat(tasks).as("Should scan 1 file").hasSize(1); - assertThat(file1Location).isEqualTo(tasks.get(0).file().path()); + assertThat(file1Location).isEqualTo(tasks.get(0).file().location()); // collect metadata file List metadataFiles = @@ -528,7 +528,7 @@ public void testRegisterHadoopTableToHiveCatalog() throws IOException, TExceptio tasks = Lists.newArrayList(table.newScan().planFiles()); assertThat(tasks).as("Should scan 2 files").hasSize(2); Set files = - tasks.stream().map(task -> task.file().path().toString()).collect(Collectors.toSet()); + tasks.stream().map(task -> task.file().location()).collect(Collectors.toSet()); assertThat(files).contains(file1Location, file2Location); } diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java index b09540fdc9f6..845892b3d817 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java @@ -33,6 +33,7 @@ import java.util.function.Predicate; import java.util.stream.Collectors; import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.RowDelta; @@ -209,7 +210,7 @@ private void commitToTable( .filter(payload -> payload.dataFiles() != null) .flatMap(payload -> payload.dataFiles().stream()) .filter(dataFile -> dataFile.recordCount() > 0) - .filter(distinctByKey(dataFile -> dataFile.path().toString())) + .filter(distinctByKey(ContentFile::location)) .collect(Collectors.toList()); List deleteFiles = @@ -217,7 +218,7 @@ private void commitToTable( .filter(payload -> payload.deleteFiles() != null) .flatMap(payload -> payload.deleteFiles().stream()) .filter(deleteFile -> deleteFile.recordCount() > 0) - .filter(distinctByKey(deleteFile -> deleteFile.path().toString())) + .filter(distinctByKey(ContentFile::location)) .collect(Collectors.toList()); if (terminated) { diff --git a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java index 8b9cd3d3d4db..5b1b8222534f 100644 --- a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java +++ b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java @@ -277,8 +277,10 @@ public void abortJob(JobContext originalContext, int status) throws IOException .onFailure( (file, exc) -> LOG.warn( - "Failed to remove data file {} on abort job", file.path(), exc)) - .run(file -> table.io().deleteFile(file.path().toString())); + "Failed to remove data file {} on abort job", + file.location(), + exc)) + .run(file -> table.io().deleteFile(file.location())); } }); } finally { diff --git a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergRecordWriter.java b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergRecordWriter.java index 0c698aa4b2fd..9fc578e88a62 100644 --- a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergRecordWriter.java +++ b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergRecordWriter.java @@ -113,7 +113,7 @@ public void close(boolean abort) throws IOException { .onFailure( (file, exception) -> LOG.debug("Failed on to remove file {} on abort", file, exception)) - .run(dataFile -> io.deleteFile(dataFile.path().toString())); + .run(dataFile -> io.deleteFile(dataFile.location())); } LOG.info( diff --git a/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java b/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java index 9b8d4e9247a2..492729d97338 100644 --- a/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java +++ b/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java @@ -320,8 +320,7 @@ private CloseableIterable openTask(FileScanTask currentTask, Schema readSchem DataFile file = currentTask.file(); InputFile inputFile = encryptionManager.decrypt( - EncryptedFiles.encryptedInput( - io.newInputFile(file.path().toString()), file.keyMetadata())); + EncryptedFiles.encryptedInput(io.newInputFile(file.location()), file.keyMetadata())); CloseableIterable iterable; switch (file.format()) { @@ -336,7 +335,7 @@ private CloseableIterable openTask(FileScanTask currentTask, Schema readSchem break; default: throw new UnsupportedOperationException( - String.format("Cannot read %s file: %s", file.format().name(), file.path())); + String.format("Cannot read %s file: %s", file.format().name(), file.location())); } return iterable; diff --git a/orc/src/test/java/org/apache/iceberg/orc/TestOrcDataWriter.java b/orc/src/test/java/org/apache/iceberg/orc/TestOrcDataWriter.java index e026c0e4c115..a8a8c3f570e8 100644 --- a/orc/src/test/java/org/apache/iceberg/orc/TestOrcDataWriter.java +++ b/orc/src/test/java/org/apache/iceberg/orc/TestOrcDataWriter.java @@ -85,7 +85,7 @@ private List stripeOffsetsFromReader(DataFile dataFile) throws IOException private List stripeOffsetsFromReader(DataFile dataFile, OrcFile.ReaderOptions options) throws IOException { - return OrcFile.createReader(new Path(dataFile.path().toString()), options).getStripes().stream() + return OrcFile.createReader(new Path(dataFile.location()), options).getStripes().stream() .map(StripeInformation::getOffset) .collect(Collectors.toList()); } diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java index 9c3c7e589c07..17820b14b011 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java @@ -300,7 +300,7 @@ public void testExpireDeleteFiles() throws Exception { Assert.assertEquals("Should have 1 delete file", 1, TestHelpers.deleteFiles(table).size()); Path deleteManifestPath = new Path(TestHelpers.deleteManifests(table).iterator().next().path()); DeleteFile deleteFile = TestHelpers.deleteFiles(table).iterator().next(); - Path deleteFilePath = new Path(String.valueOf(deleteFile.path())); + Path deleteFilePath = new Path(deleteFile.location()); sql( "CALL %s.system.rewrite_data_files(" diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java index e1966520893a..76ef5cfc9695 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java @@ -433,8 +433,7 @@ public void testRemoveOrphanFilesWithDeleteFiles() throws Exception { "Should have 1 delete manifest", 1, TestHelpers.deleteManifests(table).size()); Assert.assertEquals("Should have 1 delete file", 1, TestHelpers.deleteFiles(table).size()); Path deleteManifestPath = new Path(TestHelpers.deleteManifests(table).iterator().next().path()); - Path deleteFilePath = - new Path(String.valueOf(TestHelpers.deleteFiles(table).iterator().next().path())); + Path deleteFilePath = new Path(TestHelpers.deleteFiles(table).iterator().next().location()); // wait to ensure files are old enough waitUntilAfter(System.currentTimeMillis()); diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java index da9b61f4d74d..7175ad4ec499 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java @@ -305,7 +305,7 @@ private void writePosDeletesForFiles(Table table, List files) throws I List> deletes = Lists.newArrayList(); for (DataFile partitionFile : partitionFiles) { for (int deletePos = 0; deletePos < DELETE_FILE_SIZE; deletePos++) { - deletes.add(Pair.of(partitionFile.path(), (long) deletePos)); + deletes.add(Pair.of(partitionFile.location(), (long) deletePos)); counter++; if (counter == deleteFileSize) { // Dump to file and reset variables diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java index 63af42ff4c4f..4dc2e852aae9 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java @@ -1131,7 +1131,7 @@ public void testUpdateWithStaticPredicatePushdown() { // remove the data file from the 'hr' partition to ensure it is not scanned DataFile dataFile = Iterables.getOnlyElement(snapshot.addedDataFiles(table.io())); - table.io().deleteFile(dataFile.path().toString()); + table.io().deleteFile(dataFile.location()); // disable dynamic pruning and rely only on static predicate pushdown withSQLConf( diff --git a/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java b/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java index 2ac3de2ff947..60430339d392 100644 --- a/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java +++ b/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java @@ -45,7 +45,7 @@ protected void appendData() throws IOException { table().refresh(); for (DataFile file : table().currentSnapshot().addedDataFiles(table().io())) { - writePosDeletes(file.path(), NUM_ROWS, 0.25, numDeleteFile); + writePosDeletes(file.location(), NUM_ROWS, 0.25, numDeleteFile); } } } diff --git a/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java b/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java index 8cd6fb36fcf5..72fffaa50b8e 100644 --- a/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java +++ b/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java @@ -47,7 +47,7 @@ protected void appendData() throws IOException { // add pos-deletes table().refresh(); for (DataFile file : table().currentSnapshot().addedDataFiles(table().io())) { - writePosDeletes(file.path(), NUM_ROWS, percentDeleteRow); + writePosDeletes(file.location(), NUM_ROWS, percentDeleteRow); } } } diff --git a/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java b/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java index 1ae48e213cb7..892e2c5db19f 100644 --- a/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java +++ b/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java @@ -48,7 +48,7 @@ protected void appendData() throws IOException { table().refresh(); for (DataFile file : table().currentSnapshot().addedDataFiles(table().io())) { writePosDeletesWithNoise( - file.path(), + file.location(), NUM_ROWS, PERCENT_DELETE_ROW, (int) (percentUnrelatedDeletes / PERCENT_DELETE_ROW), diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java index dfd0b58ffbee..7ab17361b13e 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java @@ -33,6 +33,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.HasTableOperations; @@ -672,7 +673,7 @@ private static void importUnpartitionedSparkTable( if (checkDuplicateFiles) { Dataset importedFiles = spark - .createDataset(Lists.transform(files, f -> f.path().toString()), Encoders.STRING()) + .createDataset(Lists.transform(files, ContentFile::location), Encoders.STRING()) .toDF("file_path"); Dataset existingFiles = loadMetadataTable(spark, targetTable, MetadataTableType.ENTRIES).filter("status != 2"); @@ -800,7 +801,7 @@ public static void importSparkPartitions( if (checkDuplicateFiles) { Dataset importedFiles = filesToImport - .map((MapFunction) f -> f.path().toString(), Encoders.STRING()) + .map((MapFunction) ContentFile::location, Encoders.STRING()) .toDF("file_path"); Dataset existingFiles = loadMetadataTable(spark, targetTable, MetadataTableType.ENTRIES).filter("status != 2"); @@ -819,7 +820,7 @@ public static void importSparkPartitions( .repartition(numShufflePartitions) .map( (MapFunction>) - file -> Tuple2.apply(file.path().toString(), file), + file -> Tuple2.apply(file.location(), file), Encoders.tuple(Encoders.STRING(), Encoders.javaSerialization(DataFile.class))) .orderBy(col("_1")) .mapPartitions( diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java index c5b80835833e..e98494484b69 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java @@ -440,7 +440,7 @@ public CloseableIterator entries(ManifestFileBean manifest) { } static FileInfo toFileInfo(ContentFile file) { - return new FileInfo(file.path().toString(), file.content().toString()); + return new FileInfo(file.location(), file.content().toString()); } } } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java index 96ff430c179d..40d907e12c08 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java @@ -148,7 +148,7 @@ public boolean next() throws IOException { if (currentTask != null && !currentTask.isDataTask()) { String filePaths = referencedFiles(currentTask) - .map(file -> file.path().toString()) + .map(ContentFile::location) .collect(Collectors.joining(", ")); LOG.error("Error reading file(s): {}", filePaths, e); } @@ -194,7 +194,7 @@ private Map inputFiles() { } private EncryptedInputFile toEncryptedInputFile(ContentFile file) { - InputFile inputFile = table.io().newInputFile(file.path().toString()); + InputFile inputFile = table.io().newInputFile(file.location()); return EncryptedFiles.encryptedInput(inputFile, file.keyMetadata()); } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java index a2cb74c926c9..f45c152203ee 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java @@ -82,7 +82,7 @@ protected Stream> referencedFiles(FileScanTask task) { @Override protected CloseableIterator open(FileScanTask task) { - String filePath = task.file().path().toString(); + String filePath = task.file().location(); LOG.debug("Opening data file {}", filePath); // update the current file for Spark's filename() function diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java index 25cd9eda6bce..a6e2891ae696 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java @@ -110,13 +110,13 @@ private CloseableIterable openChangelogScanTask(ChangelogScanTask t } CloseableIterable openAddedRowsScanTask(AddedRowsScanTask task) { - String filePath = task.file().path().toString(); + String filePath = task.file().location(); SparkDeleteFilter deletes = new SparkDeleteFilter(filePath, task.deletes(), counter(), true); return deletes.filter(rows(task, deletes.requiredSchema())); } private CloseableIterable openDeletedDataFileScanTask(DeletedDataFileScanTask task) { - String filePath = task.file().path().toString(); + String filePath = task.file().location(); SparkDeleteFilter deletes = new SparkDeleteFilter(filePath, task.existingDeletes(), counter(), true); return deletes.filter(rows(task, deletes.requiredSchema())); @@ -125,7 +125,7 @@ private CloseableIterable openDeletedDataFileScanTask(DeletedDataFi private CloseableIterable rows(ContentScanTask task, Schema readSchema) { Map idToConstant = constantsMap(task, readSchema); - String filePath = task.file().path().toString(); + String filePath = task.file().location(); // update the current file for Spark's filename() function InputFileBlockHolder.set(filePath, task.start(), task.length()); diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java index 7aa5a971562a..ee9449ee13c8 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java @@ -41,7 +41,7 @@ public EqualityDeleteRowReader( @Override protected CloseableIterator open(FileScanTask task) { SparkDeleteFilter matches = - new SparkDeleteFilter(task.file().path().toString(), task.deletes(), counter(), true); + new SparkDeleteFilter(task.file().location(), task.deletes(), counter(), true); // schema or rows returned by readers Schema requiredSchema = matches.requiredSchema(); @@ -49,7 +49,7 @@ protected CloseableIterator open(FileScanTask task) { DataFile file = task.file(); // update the current file for Spark's filename() function - InputFileBlockHolder.set(file.path().toString(), task.start(), task.length()); + InputFileBlockHolder.set(file.location(), task.start(), task.length()); return matches.findEqualityDeleteRows(open(task, requiredSchema, idToConstant)).iterator(); } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java index 4b847474153c..1a894df29166 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java @@ -74,13 +74,13 @@ protected Stream> referencedFiles(PositionDeletesScanTask task) { @Override protected CloseableIterator open(PositionDeletesScanTask task) { - String filePath = task.file().path().toString(); + String filePath = task.file().location(); LOG.debug("Opening position delete file {}", filePath); // update the current file for Spark's filename() function InputFileBlockHolder.set(filePath, task.start(), task.length()); - InputFile inputFile = getInputFile(task.file().path().toString()); + InputFile inputFile = getInputFile(task.file().location()); Preconditions.checkNotNull(inputFile, "Could not find InputFile associated with %s", task); // select out constant fields when pushing down filter to row reader diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java index 33b1d6275d6e..f24602fd5583 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java @@ -81,7 +81,7 @@ protected Stream> referencedFiles(FileScanTask task) { @Override protected CloseableIterator open(FileScanTask task) { - String filePath = task.file().path().toString(); + String filePath = task.file().location(); LOG.debug("Opening data file {}", filePath); SparkDeleteFilter deleteFilter = new SparkDeleteFilter(filePath, task.deletes(), counter(), true); @@ -101,7 +101,7 @@ protected CloseableIterable open( if (task.isDataTask()) { return newDataIterable(task.asDataTask(), readSchema); } else { - InputFile inputFile = getInputFile(task.file().path().toString()); + InputFile inputFile = getInputFile(task.file().location()); Preconditions.checkNotNull( inputFile, "Could not find InputFile associated with FileScanTask"); return newIterable( diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkCleanupUtil.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkCleanupUtil.java index a103a5003222..5f343128161d 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkCleanupUtil.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkCleanupUtil.java @@ -81,7 +81,7 @@ private static String taskInfo() { * @param files a list of files to delete */ public static void deleteFiles(String context, FileIO io, List> files) { - List paths = Lists.transform(files, file -> file.path().toString()); + List paths = Lists.transform(files, ContentFile::location); deletePaths(context, io, paths); } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java index d978b81e67bd..345a3bb75797 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java @@ -126,7 +126,7 @@ public void filter(Filter[] filters) { this.filteredLocations = fileLocations; List filteredTasks = tasks().stream() - .filter(file -> fileLocations.contains(file.file().path().toString())) + .filter(file -> fileLocations.contains(file.file().location())) .collect(Collectors.toList()); LOG.info( diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/TaskCheckHelper.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/TaskCheckHelper.java index c44bacf149b5..b241b8ed6363 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/TaskCheckHelper.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/TaskCheckHelper.java @@ -60,7 +60,8 @@ public static void assertEquals(FileScanTask expected, FileScanTask actual) { } public static void assertEquals(DataFile expected, DataFile actual) { - Assert.assertEquals("Should match the serialized record path", expected.path(), actual.path()); + Assert.assertEquals( + "Should match the serialized record path", expected.location(), actual.location()); Assert.assertEquals( "Should match the serialized record format", expected.format(), actual.format()); Assert.assertEquals( @@ -105,7 +106,7 @@ private static List getFileScanTasksInFilePathOrder( ScanTaskGroup taskGroup) { return taskGroup.tasks().stream() // use file path + start position to differentiate the tasks - .sorted(Comparator.comparing(o -> o.file().path().toString() + "##" + o.start())) + .sorted(Comparator.comparing(o -> o.file().location() + "##" + o.start())) .collect(Collectors.toList()); } } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java index b0b3085dca70..dac09fea7562 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java @@ -42,7 +42,7 @@ public static List snapshotIds(Long... ids) { } public static List files(ContentFile... files) { - return Arrays.stream(files).map(file -> file.path().toString()).collect(Collectors.toList()); + return Arrays.stream(files).map(ContentFile::location).collect(Collectors.toList()); } public static void validateDataManifest( @@ -62,7 +62,7 @@ public static void validateDataManifest( actualDataSeqs.add(entry.dataSequenceNumber()); actualFileSeqs.add(entry.fileSequenceNumber()); actualSnapshotIds.add(entry.snapshotId()); - actualFiles.add(entry.file().path().toString()); + actualFiles.add(entry.file().location()); } assertSameElements("data seqs", actualDataSeqs, dataSeqs); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java index 82b36528996f..3e8953fb950c 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java @@ -142,7 +142,7 @@ protected static String dbPath(String dbName) { } protected void withUnavailableFiles(Iterable> files, Action action) { - Iterable fileLocations = Iterables.transform(files, file -> file.path().toString()); + Iterable fileLocations = Iterables.transform(files, ContentFile::location); withUnavailableLocations(fileLocations, action); } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java index 6d1bbc2c18f1..65a21e28a712 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java @@ -204,7 +204,7 @@ public void dataFilesCleanupWithParallelTasks() { .forEach( file -> Assert.assertTrue( - "FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString()))); + "FILE_A should be deleted", deletedFiles.contains(FILE_A.location()))); checkRemoveFilesResults(4L, 0, 0, 6L, 4L, 6, result); } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java index 71339b32421a..72294fdbdd76 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java @@ -239,8 +239,8 @@ public void dataFilesCleanupWithParallelTasks() throws IOException { Sets.newHashSet( "remove-snapshot-0", "remove-snapshot-1", "remove-snapshot-2", "remove-snapshot-3")); - Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString())); - Assert.assertTrue("FILE_B should be deleted", deletedFiles.contains(FILE_B.path().toString())); + Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.location())); + Assert.assertTrue("FILE_B should be deleted", deletedFiles.contains(FILE_B.location())); checkExpirationResults(2L, 0L, 0L, 3L, 3L, result); } @@ -555,7 +555,7 @@ public void testScanExpiredManifestInValidSnapshotAppend() { .deleteWith(deletedFiles::add) .execute(); - Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString())); + Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.location())); checkExpirationResults(1L, 0L, 0L, 1L, 2L, result); } @@ -584,7 +584,7 @@ public void testScanExpiredManifestInValidSnapshotFastAppend() { .deleteWith(deletedFiles::add) .execute(); - Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString())); + Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.location())); checkExpirationResults(1L, 0L, 0L, 1L, 2L, result); } @@ -627,7 +627,7 @@ public void testWithExpiringDanglingStageCommit() { .addedDataFiles(table.io()) .forEach( i -> { - expectedDeletes.add(i.path().toString()); + expectedDeletes.add(i.location()); }); // ManifestList should be deleted too @@ -696,7 +696,7 @@ public void testWithCherryPickTableSnapshot() { i.addedDataFiles(table.io()) .forEach( item -> { - Assert.assertFalse(deletedFiles.contains(item.path().toString())); + Assert.assertFalse(deletedFiles.contains(item.location())); }); }); @@ -745,7 +745,7 @@ public void testWithExpiringStagedThenCherrypick() { i.addedDataFiles(table.io()) .forEach( item -> { - Assert.assertFalse(deletedFiles.contains(item.path().toString())); + Assert.assertFalse(deletedFiles.contains(item.location())); }); }); checkExpirationResults(0L, 0L, 0L, 1L, 1L, firstResult); @@ -765,7 +765,7 @@ public void testWithExpiringStagedThenCherrypick() { i.addedDataFiles(table.io()) .forEach( item -> { - Assert.assertFalse(deletedFiles.contains(item.path().toString())); + Assert.assertFalse(deletedFiles.contains(item.location())); }); }); checkExpirationResults(0L, 0L, 0L, 0L, 2L, secondResult); @@ -866,7 +866,7 @@ public void testExpireOlderThanWithDelete() { .allManifests(table.io()) .get(0) .path(), // manifest contained only deletes, was dropped - FILE_A.path()), // deleted + FILE_A.location()), // deleted deletedFiles); checkExpirationResults(1, 0, 0, 2, 2, result); @@ -935,7 +935,7 @@ public void testExpireOlderThanWithDeleteInMergedManifests() { .get(0) .path(), // manifest was rewritten for delete secondSnapshot.manifestListLocation(), // snapshot expired - FILE_A.path()), // deleted + FILE_A.location()), // deleted deletedFiles); checkExpirationResults(1, 0, 0, 1, 2, result); @@ -1049,7 +1049,7 @@ public void testExpireOlderThanWithRollbackAndMergedManifests() { secondSnapshot.manifestListLocation(), // snapshot expired Iterables.getOnlyElement(secondSnapshotManifests) .path(), // manifest is no longer referenced - FILE_B.path()), // added, but rolled back + FILE_B.location()), // added, but rolled back deletedFiles); checkExpirationResults(1, 0, 0, 1, 1, result); @@ -1098,9 +1098,9 @@ public void testExpireOlderThanWithDeleteFile() { secondSnapshot.manifestListLocation(), thirdSnapshot.manifestListLocation(), fourthSnapshot.manifestListLocation(), - FILE_A.path().toString(), - FILE_A_POS_DELETES.path().toString(), - FILE_A_EQ_DELETES.path().toString()); + FILE_A.location(), + FILE_A_POS_DELETES.location(), + FILE_A_EQ_DELETES.location()); expectedDeletes.addAll( thirdSnapshot.allManifests(table.io()).stream() @@ -1273,7 +1273,7 @@ public void textExpireAllCheckFilesDeleted(int dataFilesExpired, int dataFilesRe .withPartitionPath("c1=1") .withRecordCount(1) .build(); - dataFiles.add(df.path().toString()); + dataFiles.add(df.location()); table.newFastAppend().appendFile(df).commit(); } @@ -1346,9 +1346,9 @@ public void testExpireSomeCheckFilesDeleted() { // C, D should be retained (live) // B should be retained (previous snapshot points to it) // A should be deleted - Assert.assertTrue(deletedFiles.contains(FILE_A.path().toString())); - Assert.assertFalse(deletedFiles.contains(FILE_B.path().toString())); - Assert.assertFalse(deletedFiles.contains(FILE_C.path().toString())); - Assert.assertFalse(deletedFiles.contains(FILE_D.path().toString())); + Assert.assertTrue(deletedFiles.contains(FILE_A.location())); + Assert.assertFalse(deletedFiles.contains(FILE_B.location())); + Assert.assertFalse(deletedFiles.contains(FILE_C.location())); + Assert.assertFalse(deletedFiles.contains(FILE_D.location())); } } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java index c978be9e3d91..6b7d5b9e1cfd 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java @@ -1814,8 +1814,7 @@ private void writeDF(Dataset df) { private List writePosDeletesToFile( Table table, DataFile dataFile, int outputDeleteFiles) { - return writePosDeletes( - table, dataFile.partition(), dataFile.path().toString(), outputDeleteFiles); + return writePosDeletes(table, dataFile.partition(), dataFile.location(), outputDeleteFiles); } private List writePosDeletes( diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java index 43b762813de2..15d4b45ab902 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java @@ -831,7 +831,7 @@ private void writePosDeletesForFiles( List> deletes = Lists.newArrayList(); for (DataFile partitionFile : partitionFiles) { for (int deletePos = 0; deletePos < deletesPerDataFile; deletePos++) { - deletes.add(Pair.of(partitionFile.path(), (long) deletePos)); + deletes.add(Pair.of(partitionFile.location(), (long) deletePos)); counter++; if (counter == deleteFileSize) { // Dump to file and reset variables @@ -868,17 +868,17 @@ private List deleteFiles(Table table) { private > List except(List first, List second) { Set secondPaths = - second.stream().map(f -> f.path().toString()).collect(Collectors.toSet()); + second.stream().map(ContentFile::location).collect(Collectors.toSet()); return first.stream() - .filter(f -> !secondPaths.contains(f.path().toString())) + .filter(f -> !secondPaths.contains(f.location())) .collect(Collectors.toList()); } private void assertNotContains(List original, List rewritten) { Set originalPaths = - original.stream().map(f -> f.path().toString()).collect(Collectors.toSet()); + original.stream().map(ContentFile::location).collect(Collectors.toSet()); Set rewrittenPaths = - rewritten.stream().map(f -> f.path().toString()).collect(Collectors.toSet()); + rewritten.stream().map(ContentFile::location).collect(Collectors.toSet()); rewrittenPaths.retainAll(originalPaths); Assert.assertEquals(0, rewrittenPaths.size()); } @@ -887,7 +887,7 @@ private void assertLocallySorted(List deleteFiles) { for (DeleteFile deleteFile : deleteFiles) { Dataset deletes = spark.read().format("iceberg").load("default." + TABLE_NAME + ".position_deletes"); - deletes.filter(deletes.col("delete_file_path").equalTo(deleteFile.path().toString())); + deletes.filter(deletes.col("delete_file_path").equalTo(deleteFile.location())); List rows = deletes.collectAsList(); Assert.assertFalse("Empty delete file found", rows.isEmpty()); int lastPos = 0; diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java index e5e0e350e50b..d417acdeb20a 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java @@ -114,7 +114,7 @@ public Boolean hasIterator(FileScanTask task) { } private String getKey(FileScanTask task) { - return task.file().path().toString(); + return task.file().location(); } } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java index 693bcec9eb05..b5c055925bf7 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java @@ -197,8 +197,8 @@ private void writeAndValidateWithLocations(Table table, File location, File expe Assert.assertTrue( String.format( "File should have the parent directory %s, but has: %s.", - expectedDataDir.getAbsolutePath(), dataFile.path()), - URI.create(dataFile.path().toString()) + expectedDataDir.getAbsolutePath(), dataFile.location()), + URI.create(dataFile.location()) .getPath() .startsWith(expectedDataDir.getAbsolutePath()))); } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java index 9cee25560366..01424a1abdfe 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java @@ -109,7 +109,7 @@ public void testWriteFormatOptionOverridesTableProperties() throws IOException { try (CloseableIterable tasks = table.newScan().planFiles()) { tasks.forEach( task -> { - FileFormat fileFormat = FileFormat.fromFileName(task.file().path()); + FileFormat fileFormat = FileFormat.fromFileName(task.file().location()); Assert.assertEquals(FileFormat.PARQUET, fileFormat); }); } @@ -134,7 +134,7 @@ public void testNoWriteFormatOption() throws IOException { try (CloseableIterable tasks = table.newScan().planFiles()) { tasks.forEach( task -> { - FileFormat fileFormat = FileFormat.fromFileName(task.file().path()); + FileFormat fileFormat = FileFormat.fromFileName(task.file().location()); Assert.assertEquals(FileFormat.AVRO, fileFormat); }); } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index d82bc0b4ba8e..99c621997701 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -268,7 +268,7 @@ public void testEntriesTableDataFilePrune() throws Exception { .select("data_file.file_path") .collectAsList()); - List singleExpected = ImmutableList.of(row(file.path())); + List singleExpected = ImmutableList.of(row(file.location())); assertEquals( "Should prune a single element from a nested struct", singleExpected, singleActual); @@ -307,7 +307,7 @@ public void testEntriesTableDataFilePruneMulti() throws Exception { List multiExpected = ImmutableList.of( - row(file.path(), file.valueCounts(), file.recordCount(), file.columnSizes())); + row(file.location(), file.valueCounts(), file.recordCount(), file.columnSizes())); assertEquals("Should prune a single element from a nested struct", multiExpected, multiActual); } @@ -341,7 +341,7 @@ public void testFilesSelectMap() throws Exception { List multiExpected = ImmutableList.of( - row(file.path(), file.valueCounts(), file.recordCount(), file.columnSizes())); + row(file.location(), file.valueCounts(), file.recordCount(), file.columnSizes())); assertEquals("Should prune a single element from a row", multiExpected, multiActual); } @@ -2316,7 +2316,7 @@ private DeleteFile writePosDeleteFile(Table table, long pos) { StructLike dataFilePartition = dataFile.partition(); PositionDelete delete = PositionDelete.create(); - delete.set(dataFile.path(), pos, null); + delete.set(dataFile.location(), pos, null); return writePositionDeletes(table, dataFileSpec, dataFilePartition, ImmutableList.of(delete)); } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java index 2ec4f2f4f907..af3ec224cb4f 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java @@ -26,6 +26,7 @@ import java.util.UUID; import java.util.function.BiFunction; import java.util.stream.Collectors; +import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; @@ -140,8 +141,8 @@ public void testNullRows() throws IOException { tab.newAppend().appendFile(dFile).commit(); List> deletes = Lists.newArrayList(); - deletes.add(Pair.of(dFile.path(), 0L)); - deletes.add(Pair.of(dFile.path(), 1L)); + deletes.add(Pair.of(dFile.location(), 0L)); + deletes.add(Pair.of(dFile.location(), 1L)); Pair posDeletes = FileHelpers.writeDeleteFile( tab, Files.localOutput(temp.newFile()), TestHelpers.Row.of(0), deletes); @@ -150,9 +151,9 @@ public void testNullRows() throws IOException { StructLikeSet actual = actual(tableName, tab); List> expectedDeletes = - Lists.newArrayList(positionDelete(dFile.path(), 0L), positionDelete(dFile.path(), 1L)); - StructLikeSet expected = - expected(tab, expectedDeletes, null, posDeletes.first().path().toString()); + Lists.newArrayList( + positionDelete(dFile.location(), 0L), positionDelete(dFile.location(), 1L)); + StructLikeSet expected = expected(tab, expectedDeletes, null, posDeletes.first().location()); Assert.assertEquals("Position Delete table should contain expected rows", expected, actual); dropTable(tableName); @@ -181,7 +182,7 @@ public void testPartitionedTable() throws IOException { GenericRecord partitionB = GenericRecord.create(tab.spec().partitionType()); partitionB.setField("data", "b"); StructLikeSet expected = - expected(tab, deletesB.first(), partitionB, deletesB.second().path().toString()); + expected(tab, deletesB.first(), partitionB, deletesB.second().location()); Assert.assertEquals("Position Delete table should contain expected rows", expected, actual); dropTable(tableName); @@ -221,7 +222,7 @@ public void testSelect() throws IOException { (delete, file) -> { int rowData = delete.get(2, GenericRecord.class).get(0, Integer.class); long pos = delete.get(1, Long.class); - return row(rowData, pos, file.path().toString(), file.path().toString()); + return row(rowData, pos, file.location(), file.location()); }; expected.addAll( deletesA.first().stream() @@ -270,7 +271,7 @@ public void testSplitTasks() throws IOException { List> deletes = Lists.newArrayList(); for (long i = 0; i < records; i++) { - deletes.add(positionDelete(tab.schema(), dFile.path(), i, (int) i, String.valueOf(i))); + deletes.add(positionDelete(tab.schema(), dFile.location(), i, (int) i, String.valueOf(i))); } DeleteFile posDeletes = FileHelpers.writePosDeleteFile( @@ -292,7 +293,7 @@ public void testSplitTasks() throws IOException { } StructLikeSet actual = actual(tableName, tab); - StructLikeSet expected = expected(tab, deletes, null, posDeletes.path().toString()); + StructLikeSet expected = expected(tab, deletes, null, posDeletes.location()); Assert.assertEquals("Position Delete table should contain expected rows", expected, actual); dropTable(tableName); @@ -322,9 +323,9 @@ public void testPartitionFilter() throws IOException { Record partitionA = partitionRecordTemplate.copy("data", "a"); Record partitionB = partitionRecordTemplate.copy("data", "b"); StructLikeSet expectedA = - expected(tab, deletesA.first(), partitionA, deletesA.second().path().toString()); + expected(tab, deletesA.first(), partitionA, deletesA.second().location()); StructLikeSet expectedB = - expected(tab, deletesB.first(), partitionB, deletesB.second().path().toString()); + expected(tab, deletesB.first(), partitionB, deletesB.second().location()); StructLikeSet allExpected = StructLikeSet.create(deletesTab.schema().asStruct()); allExpected.addAll(expectedA); allExpected.addAll(expectedB); @@ -365,9 +366,9 @@ public void testPartitionTransformFilter() throws IOException { Record partitionA = partitionRecordTemplate.copy("data_trunc", "a"); Record partitionB = partitionRecordTemplate.copy("data_trunc", "b"); StructLikeSet expectedA = - expected(tab, deletesA.first(), partitionA, deletesA.second().path().toString()); + expected(tab, deletesA.first(), partitionA, deletesA.second().location()); StructLikeSet expectedB = - expected(tab, deletesB.first(), partitionB, deletesB.second().path().toString()); + expected(tab, deletesB.first(), partitionB, deletesB.second().location()); StructLikeSet allExpected = StructLikeSet.create(deletesTable.schema().asStruct()); allExpected.addAll(expectedA); allExpected.addAll(expectedB); @@ -415,7 +416,7 @@ public void testPartitionEvolutionReplace() throws Exception { GenericRecord partitionRecordTemplate = GenericRecord.create(Partitioning.partitionType(tab)); Record partitionA = partitionRecordTemplate.copy("data", "a"); StructLikeSet expectedA = - expected(tab, deletesA.first(), partitionA, dataSpec, deletesA.second().path().toString()); + expected(tab, deletesA.first(), partitionA, dataSpec, deletesA.second().location()); StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); Assert.assertEquals("Position Delete table should contain expected rows", expectedA, actualA); @@ -427,7 +428,7 @@ public void testPartitionEvolutionReplace() throws Exception { deletes10.first(), partition10, tab.spec().specId(), - deletes10.second().path().toString()); + deletes10.second().location()); StructLikeSet actual10 = actual(tableName, tab, "partition.id = 10 AND pos >= 0"); Assert.assertEquals("Position Delete table should contain expected rows", expected10, actual10); @@ -465,7 +466,7 @@ public void testPartitionEvolutionAdd() throws Exception { GenericRecord partitionRecordTemplate = GenericRecord.create(Partitioning.partitionType(tab)); Record partitionA = partitionRecordTemplate.copy("data", "a"); StructLikeSet expectedA = - expected(tab, deletesA.first(), partitionA, specId1, deletesA.second().path().toString()); + expected(tab, deletesA.first(), partitionA, specId1, deletesA.second().location()); StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); Assert.assertEquals("Position Delete table should contain expected rows", expectedA, actualA); @@ -477,7 +478,7 @@ public void testPartitionEvolutionAdd() throws Exception { deletesUnpartitioned.first(), unpartitionedRecord, specId0, - deletesUnpartitioned.second().path().toString()); + deletesUnpartitioned.second().location()); StructLikeSet actualUnpartitioned = actual(tableName, tab, "partition.data IS NULL and pos >= 0"); @@ -520,7 +521,7 @@ public void testPartitionEvolutionRemove() throws Exception { GenericRecord partitionRecordTemplate = GenericRecord.create(Partitioning.partitionType(tab)); Record partitionA = partitionRecordTemplate.copy("data", "a"); StructLikeSet expectedA = - expected(tab, deletesA.first(), partitionA, specId0, deletesA.second().path().toString()); + expected(tab, deletesA.first(), partitionA, specId0, deletesA.second().location()); StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); Assert.assertEquals("Position Delete table should contain expected rows", expectedA, actualA); @@ -532,7 +533,7 @@ public void testPartitionEvolutionRemove() throws Exception { deletesUnpartitioned.first(), unpartitionedRecord, specId1, - deletesUnpartitioned.second().path().toString()); + deletesUnpartitioned.second().location()); StructLikeSet actualUnpartitioned = actual(tableName, tab, "partition.data IS NULL and pos >= 0"); @@ -577,7 +578,7 @@ public void testSpecIdFilter() throws Exception { deletesUnpartitioned.first(), partitionRecordTemplate, unpartitionedSpec, - deletesUnpartitioned.second().path().toString()); + deletesUnpartitioned.second().location()); StructLikeSet actualUnpartitioned = actual(tableName, tab, String.format("spec_id = %d", unpartitionedSpec)); Assert.assertEquals( @@ -589,9 +590,9 @@ public void testSpecIdFilter() throws Exception { StructLike partitionA = partitionRecordTemplate.copy("data", "a"); StructLike partitionB = partitionRecordTemplate.copy("data", "b"); StructLikeSet expected = - expected(tab, deletesA.first(), partitionA, dataSpec, deletesA.second().path().toString()); + expected(tab, deletesA.first(), partitionA, dataSpec, deletesA.second().location()); expected.addAll( - expected(tab, deletesB.first(), partitionB, dataSpec, deletesB.second().path().toString())); + expected(tab, deletesB.first(), partitionB, dataSpec, deletesB.second().location())); StructLikeSet actual = actual(tableName, tab, String.format("spec_id = %d", dataSpec)); Assert.assertEquals("Position Delete table should contain expected rows", expected, actual); @@ -645,14 +646,14 @@ public void testSchemaEvolutionAdd() throws Exception { d.set(2, padded); }); StructLikeSet expectedA = - expected(tab, expectedDeletesA, partitionA, deletesA.second().path().toString()); + expected(tab, expectedDeletesA, partitionA, deletesA.second().location()); StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); Assert.assertEquals("Position Delete table should contain expected rows", expectedA, actualA); // Select deletes from new schema Record partitionC = partitionRecordTemplate.copy("data", "c"); StructLikeSet expectedC = - expected(tab, deletesC.first(), partitionC, deletesC.second().path().toString()); + expected(tab, deletesC.first(), partitionC, deletesC.second().location()); StructLikeSet actualC = actual(tableName, tab, "partition.data = 'c' and pos >= 0"); Assert.assertEquals("Position Delete table should contain expected rows", expectedC, actualC); @@ -707,14 +708,14 @@ public void testSchemaEvolutionRemove() throws Exception { d.set(2, padded); }); StructLikeSet expectedA = - expected(tab, expectedDeletesA, partitionA, deletesA.second().path().toString()); + expected(tab, expectedDeletesA, partitionA, deletesA.second().location()); StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); Assert.assertEquals("Position Delete table should contain expected rows", expectedA, actualA); // Select deletes from new schema Record partitionC = partitionRecordTemplate.copy("data", "c"); StructLikeSet expectedC = - expected(tab, deletesC.first(), partitionC, deletesC.second().path().toString()); + expected(tab, deletesC.first(), partitionC, deletesC.second().location()); StructLikeSet actualC = actual(tableName, tab, "partition.data = 'c' and pos >= 0"); Assert.assertEquals("Position Delete table should contain expected rows", expectedC, actualC); @@ -792,8 +793,8 @@ public void testWriteUnpartitionedNullRows() throws Exception { tab.newAppend().appendFile(dFile).commit(); List> deletes = Lists.newArrayList(); - deletes.add(Pair.of(dFile.path(), 0L)); - deletes.add(Pair.of(dFile.path(), 1L)); + deletes.add(Pair.of(dFile.location(), 0L)); + deletes.add(Pair.of(dFile.location(), 1L)); Pair posDeletes = FileHelpers.writeDeleteFile( tab, Files.localOutput(temp.newFile()), TestHelpers.Row.of(0), deletes); @@ -827,7 +828,8 @@ public void testWriteUnpartitionedNullRows() throws Exception { actual(tableName, tab, null, ImmutableList.of("file_path", "pos", "row", "spec_id")); List> expectedDeletes = - Lists.newArrayList(positionDelete(dFile.path(), 0L), positionDelete(dFile.path(), 1L)); + Lists.newArrayList( + positionDelete(dFile.location(), 0L), positionDelete(dFile.location(), 1L)); StructLikeSet expected = expected(tab, expectedDeletes, null, null); Assert.assertEquals("Position Delete table should contain expected rows", expected, actual); @@ -846,8 +848,8 @@ public void testWriteMixedRows() throws Exception { // Add a delete file with row and without row List> deletes = Lists.newArrayList(); - deletes.add(Pair.of(dataFileA.path(), 0L)); - deletes.add(Pair.of(dataFileA.path(), 1L)); + deletes.add(Pair.of(dataFileA.location(), 0L)); + deletes.add(Pair.of(dataFileA.location(), 1L)); Pair deletesWithoutRow = FileHelpers.writeDeleteFile( tab, Files.localOutput(temp.newFile()), TestHelpers.Row.of("a"), deletes); @@ -905,7 +907,7 @@ public void testWriteMixedRows() throws Exception { expected( tab, Lists.newArrayList( - positionDelete(dataFileA.path(), 0L), positionDelete(dataFileA.path(), 1L)), + positionDelete(dataFileA.location(), 0L), positionDelete(dataFileA.location(), 1L)), partitionA, null)); allExpected.addAll(expected(tab, deletesWithRow.first(), partitionB, null)); @@ -1481,13 +1483,13 @@ private Pair>, DeleteFile> deleteFile( Lists.newArrayList( positionDelete( tab.schema(), - dataFile.path(), + dataFile.location(), 0L, idPartition != null ? idPartition : 29, dataPartition != null ? dataPartition : "c"), positionDelete( tab.schema(), - dataFile.path(), + dataFile.location(), 1L, idPartition != null ? idPartition : 61, dataPartition != null ? dataPartition : "r")); @@ -1533,9 +1535,9 @@ private void commit( Assert.assertEquals(expectedTargetFiles, addedFiles.size()); List sortedAddedFiles = - addedFiles.stream().map(f -> f.path().toString()).sorted().collect(Collectors.toList()); + addedFiles.stream().map(ContentFile::location).sorted().collect(Collectors.toList()); List sortedRewrittenFiles = - rewrittenFiles.stream().map(f -> f.path().toString()).sorted().collect(Collectors.toList()); + rewrittenFiles.stream().map(ContentFile::location).sorted().collect(Collectors.toList()); Assert.assertNotEquals("Lists should not be the same", sortedAddedFiles, sortedRewrittenFiles); baseTab diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java index beaf7b75c6c0..d59c280bc060 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java @@ -444,7 +444,7 @@ private void deleteNotMatchingFiles(Expression filter, int expectedDeletedFileCo Set matchingFileLocations = Sets.newHashSet(); try (CloseableIterable files = table.newScan().filter(filter).planFiles()) { for (FileScanTask file : files) { - String path = file.file().path().toString(); + String path = file.file().location(); matchingFileLocations.add(path); } } catch (IOException e) { @@ -454,7 +454,7 @@ private void deleteNotMatchingFiles(Expression filter, int expectedDeletedFileCo Set deletedFileLocations = Sets.newHashSet(); try (CloseableIterable files = table.newScan().planFiles()) { for (FileScanTask file : files) { - String path = file.file().path().toString(); + String path = file.file().location(); if (!matchingFileLocations.contains(path)) { io.deleteFile(path); deletedFileLocations.add(path); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java index b1f2082b5d9b..e03ba3794a18 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java @@ -193,7 +193,7 @@ private void checkSparkDataFile(Table table) throws IOException { } private void checkDataFile(DataFile expected, DataFile actual) { - Assert.assertEquals("Path must match", expected.path(), actual.path()); + Assert.assertEquals("Path must match", expected.location(), actual.location()); Assert.assertEquals("Format must match", expected.format(), actual.format()); Assert.assertEquals("Record count must match", expected.recordCount(), actual.recordCount()); Assert.assertEquals("Size must match", expected.fileSizeInBytes(), actual.fileSizeInBytes()); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java index bbdd49b63df8..dda49b49465c 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java @@ -343,10 +343,10 @@ public void testPosDeletesAllRowsInBatch() throws IOException { // deleted. List> deletes = Lists.newArrayList( - Pair.of(dataFile.path(), 0L), // id = 29 - Pair.of(dataFile.path(), 1L), // id = 43 - Pair.of(dataFile.path(), 2L), // id = 61 - Pair.of(dataFile.path(), 3L) // id = 89 + Pair.of(dataFile.location(), 0L), // id = 29 + Pair.of(dataFile.location(), 1L), // id = 43 + Pair.of(dataFile.location(), 2L), // id = 61 + Pair.of(dataFile.location(), 3L) // id = 89 ); Pair posDeletes = @@ -376,10 +376,10 @@ public void testPosDeletesWithDeletedColumn() throws IOException { // deleted. List> deletes = Lists.newArrayList( - Pair.of(dataFile.path(), 0L), // id = 29 - Pair.of(dataFile.path(), 1L), // id = 43 - Pair.of(dataFile.path(), 2L), // id = 61 - Pair.of(dataFile.path(), 3L) // id = 89 + Pair.of(dataFile.location(), 0L), // id = 29 + Pair.of(dataFile.location(), 1L), // id = 43 + Pair.of(dataFile.location(), 2L), // id = 61 + Pair.of(dataFile.location(), 3L) // id = 89 ); Pair posDeletes = @@ -455,8 +455,8 @@ public void testMixedPosAndEqDeletesWithDeletedColumn() throws IOException { List> deletes = Lists.newArrayList( - Pair.of(dataFile.path(), 3L), // id = 89 - Pair.of(dataFile.path(), 5L) // id = 121 + Pair.of(dataFile.location(), 3L), // id = 89 + Pair.of(dataFile.location(), 5L) // id = 121 ); Pair posDeletes = @@ -486,10 +486,10 @@ public void testMixedPosAndEqDeletesWithDeletedColumn() throws IOException { public void testFilterOnDeletedMetadataColumn() throws IOException { List> deletes = Lists.newArrayList( - Pair.of(dataFile.path(), 0L), // id = 29 - Pair.of(dataFile.path(), 1L), // id = 43 - Pair.of(dataFile.path(), 2L), // id = 61 - Pair.of(dataFile.path(), 3L) // id = 89 + Pair.of(dataFile.location(), 0L), // id = 29 + Pair.of(dataFile.location(), 1L), // id = 43 + Pair.of(dataFile.location(), 2L), // id = 61 + Pair.of(dataFile.location(), 3L) // id = 89 ); Pair posDeletes = @@ -611,13 +611,13 @@ public void testPosDeletesOnParquetFileWithMultipleRowGroups() throws IOExceptio // Add positional deletes to the table List> deletes = Lists.newArrayList( - Pair.of(dataFile.path(), 97L), - Pair.of(dataFile.path(), 98L), - Pair.of(dataFile.path(), 99L), - Pair.of(dataFile.path(), 101L), - Pair.of(dataFile.path(), 103L), - Pair.of(dataFile.path(), 107L), - Pair.of(dataFile.path(), 109L)); + Pair.of(dataFile.location(), 97L), + Pair.of(dataFile.location(), 98L), + Pair.of(dataFile.location(), 99L), + Pair.of(dataFile.location(), 101L), + Pair.of(dataFile.location(), 103L), + Pair.of(dataFile.location(), 107L), + Pair.of(dataFile.location(), 109L)); Pair posDeletes = FileHelpers.writeDeleteFile( table, diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java index 8c355612ceb5..d130ccbc2cd1 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java @@ -292,7 +292,7 @@ public void testExpireDeleteFiles() throws Exception { Assert.assertEquals("Should have 1 delete file", 1, TestHelpers.deleteFiles(table).size()); Path deleteManifestPath = new Path(TestHelpers.deleteManifests(table).iterator().next().path()); DeleteFile deleteFile = TestHelpers.deleteFiles(table).iterator().next(); - Path deleteFilePath = new Path(String.valueOf(deleteFile.path())); + Path deleteFilePath = new Path(deleteFile.location()); sql( "CALL %s.system.rewrite_data_files(" diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java index 3b1ca1b239b8..01e341eead89 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java @@ -424,8 +424,7 @@ public void testRemoveOrphanFilesWithDeleteFiles() throws Exception { "Should have 1 delete manifest", 1, TestHelpers.deleteManifests(table).size()); Assert.assertEquals("Should have 1 delete file", 1, TestHelpers.deleteFiles(table).size()); Path deleteManifestPath = new Path(TestHelpers.deleteManifests(table).iterator().next().path()); - Path deleteFilePath = - new Path(String.valueOf(TestHelpers.deleteFiles(table).iterator().next().path())); + Path deleteFilePath = new Path(TestHelpers.deleteFiles(table).iterator().next().location()); // wait to ensure files are old enough waitUntilAfter(System.currentTimeMillis()); diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java index 4c5082cabbb6..2d3d3b851a99 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java @@ -314,7 +314,7 @@ private void writePosDeletesForFiles(Table table, List files) throws I List> deletes = Lists.newArrayList(); for (DataFile partitionFile : partitionFiles) { for (int deletePos = 0; deletePos < DELETE_FILE_SIZE; deletePos++) { - deletes.add(Pair.of(partitionFile.path(), (long) deletePos)); + deletes.add(Pair.of(partitionFile.location(), (long) deletePos)); counter++; if (counter == deleteFileSize) { // Dump to file and reset variables diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSparkExecutorCache.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSparkExecutorCache.java index 3d995cc4f096..ed0685735941 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSparkExecutorCache.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSparkExecutorCache.java @@ -222,7 +222,7 @@ private void checkMerge(RowLevelOperationMode mode) throws Exception { } private int streamCount(DeleteFile deleteFile) { - CustomInputFile inputFile = INPUT_FILES.get(deleteFile.path().toString()); + CustomInputFile inputFile = INPUT_FILES.get(deleteFile.location()); return inputFile.streamCount(); } @@ -247,7 +247,7 @@ private List createAndInitTable(String operation, RowLevelOperationM List> posDeletes = dataFiles(table).stream() - .map(dataFile -> Pair.of(dataFile.path(), 0L)) + .map(dataFile -> Pair.of((CharSequence) dataFile.location(), 0L)) .collect(Collectors.toList()); Pair posDeleteResult = writePosDeletes(table, posDeletes); DeleteFile posDeleteFile = posDeleteResult.first(); diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java index e84789de7abc..9968f5eb0f2c 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java @@ -1299,7 +1299,7 @@ public void testUpdateWithStaticPredicatePushdown() { // remove the data file from the 'hr' partition to ensure it is not scanned DataFile dataFile = Iterables.getOnlyElement(snapshot.addedDataFiles(table.io())); - table.io().deleteFile(dataFile.path().toString()); + table.io().deleteFile(dataFile.location()); // disable dynamic pruning and rely only on static predicate pushdown withSQLConf( diff --git a/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java b/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java index 2ac3de2ff947..60430339d392 100644 --- a/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java +++ b/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java @@ -45,7 +45,7 @@ protected void appendData() throws IOException { table().refresh(); for (DataFile file : table().currentSnapshot().addedDataFiles(table().io())) { - writePosDeletes(file.path(), NUM_ROWS, 0.25, numDeleteFile); + writePosDeletes(file.location(), NUM_ROWS, 0.25, numDeleteFile); } } } diff --git a/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java b/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java index 8cd6fb36fcf5..72fffaa50b8e 100644 --- a/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java +++ b/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java @@ -47,7 +47,7 @@ protected void appendData() throws IOException { // add pos-deletes table().refresh(); for (DataFile file : table().currentSnapshot().addedDataFiles(table().io())) { - writePosDeletes(file.path(), NUM_ROWS, percentDeleteRow); + writePosDeletes(file.location(), NUM_ROWS, percentDeleteRow); } } } diff --git a/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java b/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java index 1ae48e213cb7..892e2c5db19f 100644 --- a/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java +++ b/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java @@ -48,7 +48,7 @@ protected void appendData() throws IOException { table().refresh(); for (DataFile file : table().currentSnapshot().addedDataFiles(table().io())) { writePosDeletesWithNoise( - file.path(), + file.location(), NUM_ROWS, PERCENT_DELETE_ROW, (int) (percentUnrelatedDeletes / PERCENT_DELETE_ROW), diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java index e103104171f6..aaeabe350f54 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java @@ -33,6 +33,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.HasTableOperations; @@ -635,7 +636,7 @@ private static void importUnpartitionedSparkTable( if (checkDuplicateFiles) { Dataset importedFiles = spark - .createDataset(Lists.transform(files, f -> f.path().toString()), Encoders.STRING()) + .createDataset(Lists.transform(files, ContentFile::location), Encoders.STRING()) .toDF("file_path"); Dataset existingFiles = loadMetadataTable(spark, targetTable, MetadataTableType.ENTRIES).filter("status != 2"); @@ -763,7 +764,7 @@ public static void importSparkPartitions( if (checkDuplicateFiles) { Dataset importedFiles = filesToImport - .map((MapFunction) f -> f.path().toString(), Encoders.STRING()) + .map((MapFunction) ContentFile::location, Encoders.STRING()) .toDF("file_path"); Dataset existingFiles = loadMetadataTable(spark, targetTable, MetadataTableType.ENTRIES).filter("status != 2"); @@ -782,7 +783,7 @@ public static void importSparkPartitions( .repartition(numShufflePartitions) .map( (MapFunction>) - file -> Tuple2.apply(file.path().toString(), file), + file -> Tuple2.apply(file.location(), file), Encoders.tuple(Encoders.STRING(), Encoders.javaSerialization(DataFile.class))) .orderBy(col("_1")) .mapPartitions( diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java index cff07c05d423..13ce67cda18e 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java @@ -433,7 +433,7 @@ public CloseableIterator entries(ManifestFileBean manifest) { } static FileInfo toFileInfo(ContentFile file) { - return new FileInfo(file.path().toString(), file.content().toString()); + return new FileInfo(file.location(), file.content().toString()); } } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java index b9dc46f5e1bc..c8c883e553b7 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java @@ -88,7 +88,7 @@ Result doExecute() { RewriteFiles rewriteFiles = table.newRewrite(); List danglingDeletes = findDanglingDeletes(); for (DeleteFile deleteFile : danglingDeletes) { - LOG.debug("Removing dangling delete file {}", deleteFile.path()); + LOG.debug("Removing dangling delete file {}", deleteFile.location()); rewriteFiles.deleteFile(deleteFile); } if (!danglingDeletes.isEmpty()) { diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java index 891640843668..ee8690124c43 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java @@ -153,7 +153,7 @@ public boolean next() throws IOException { if (currentTask != null && !currentTask.isDataTask()) { String filePaths = referencedFiles(currentTask) - .map(file -> file.path().toString()) + .map(ContentFile::location) .collect(Collectors.joining(", ")); LOG.error("Error reading file(s): {}", filePaths, e); } @@ -199,7 +199,7 @@ private Map inputFiles() { } private EncryptedInputFile toEncryptedInputFile(ContentFile file) { - InputFile inputFile = table.io().newInputFile(file.path().toString()); + InputFile inputFile = table.io().newInputFile(file.location()); return EncryptedFiles.encryptedInput(inputFile, file.keyMetadata()); } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java index a2cb74c926c9..f45c152203ee 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java @@ -82,7 +82,7 @@ protected Stream> referencedFiles(FileScanTask task) { @Override protected CloseableIterator open(FileScanTask task) { - String filePath = task.file().path().toString(); + String filePath = task.file().location(); LOG.debug("Opening data file {}", filePath); // update the current file for Spark's filename() function diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java index 25cd9eda6bce..a6e2891ae696 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java @@ -110,13 +110,13 @@ private CloseableIterable openChangelogScanTask(ChangelogScanTask t } CloseableIterable openAddedRowsScanTask(AddedRowsScanTask task) { - String filePath = task.file().path().toString(); + String filePath = task.file().location(); SparkDeleteFilter deletes = new SparkDeleteFilter(filePath, task.deletes(), counter(), true); return deletes.filter(rows(task, deletes.requiredSchema())); } private CloseableIterable openDeletedDataFileScanTask(DeletedDataFileScanTask task) { - String filePath = task.file().path().toString(); + String filePath = task.file().location(); SparkDeleteFilter deletes = new SparkDeleteFilter(filePath, task.existingDeletes(), counter(), true); return deletes.filter(rows(task, deletes.requiredSchema())); @@ -125,7 +125,7 @@ private CloseableIterable openDeletedDataFileScanTask(DeletedDataFi private CloseableIterable rows(ContentScanTask task, Schema readSchema) { Map idToConstant = constantsMap(task, readSchema); - String filePath = task.file().path().toString(); + String filePath = task.file().location(); // update the current file for Spark's filename() function InputFileBlockHolder.set(filePath, task.start(), task.length()); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java index 7aa5a971562a..ee9449ee13c8 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java @@ -41,7 +41,7 @@ public EqualityDeleteRowReader( @Override protected CloseableIterator open(FileScanTask task) { SparkDeleteFilter matches = - new SparkDeleteFilter(task.file().path().toString(), task.deletes(), counter(), true); + new SparkDeleteFilter(task.file().location(), task.deletes(), counter(), true); // schema or rows returned by readers Schema requiredSchema = matches.requiredSchema(); @@ -49,7 +49,7 @@ protected CloseableIterator open(FileScanTask task) { DataFile file = task.file(); // update the current file for Spark's filename() function - InputFileBlockHolder.set(file.path().toString(), task.start(), task.length()); + InputFileBlockHolder.set(file.location(), task.start(), task.length()); return matches.findEqualityDeleteRows(open(task, requiredSchema, idToConstant)).iterator(); } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java index 4b847474153c..1a894df29166 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java @@ -74,13 +74,13 @@ protected Stream> referencedFiles(PositionDeletesScanTask task) { @Override protected CloseableIterator open(PositionDeletesScanTask task) { - String filePath = task.file().path().toString(); + String filePath = task.file().location(); LOG.debug("Opening position delete file {}", filePath); // update the current file for Spark's filename() function InputFileBlockHolder.set(filePath, task.start(), task.length()); - InputFile inputFile = getInputFile(task.file().path().toString()); + InputFile inputFile = getInputFile(task.file().location()); Preconditions.checkNotNull(inputFile, "Could not find InputFile associated with %s", task); // select out constant fields when pushing down filter to row reader diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java index 33b1d6275d6e..f24602fd5583 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java @@ -81,7 +81,7 @@ protected Stream> referencedFiles(FileScanTask task) { @Override protected CloseableIterator open(FileScanTask task) { - String filePath = task.file().path().toString(); + String filePath = task.file().location(); LOG.debug("Opening data file {}", filePath); SparkDeleteFilter deleteFilter = new SparkDeleteFilter(filePath, task.deletes(), counter(), true); @@ -101,7 +101,7 @@ protected CloseableIterable open( if (task.isDataTask()) { return newDataIterable(task.asDataTask(), readSchema); } else { - InputFile inputFile = getInputFile(task.file().path().toString()); + InputFile inputFile = getInputFile(task.file().location()); Preconditions.checkNotNull( inputFile, "Could not find InputFile associated with FileScanTask"); return newIterable( diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkCleanupUtil.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkCleanupUtil.java index a103a5003222..5f343128161d 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkCleanupUtil.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkCleanupUtil.java @@ -81,7 +81,7 @@ private static String taskInfo() { * @param files a list of files to delete */ public static void deleteFiles(String context, FileIO io, List> files) { - List paths = Lists.transform(files, file -> file.path().toString()); + List paths = Lists.transform(files, ContentFile::location); deletePaths(context, io, paths); } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java index 5a3963bc614c..7a6025b0731a 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java @@ -130,7 +130,7 @@ public void filter(Filter[] filters) { this.filteredLocations = fileLocations; List filteredTasks = tasks().stream() - .filter(file -> fileLocations.contains(file.file().path().toString())) + .filter(file -> fileLocations.contains(file.file().location())) .collect(Collectors.toList()); LOG.info( diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/TaskCheckHelper.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/TaskCheckHelper.java index c44bacf149b5..b241b8ed6363 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/TaskCheckHelper.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/TaskCheckHelper.java @@ -60,7 +60,8 @@ public static void assertEquals(FileScanTask expected, FileScanTask actual) { } public static void assertEquals(DataFile expected, DataFile actual) { - Assert.assertEquals("Should match the serialized record path", expected.path(), actual.path()); + Assert.assertEquals( + "Should match the serialized record path", expected.location(), actual.location()); Assert.assertEquals( "Should match the serialized record format", expected.format(), actual.format()); Assert.assertEquals( @@ -105,7 +106,7 @@ private static List getFileScanTasksInFilePathOrder( ScanTaskGroup taskGroup) { return taskGroup.tasks().stream() // use file path + start position to differentiate the tasks - .sorted(Comparator.comparing(o -> o.file().path().toString() + "##" + o.start())) + .sorted(Comparator.comparing(o -> o.file().location() + "##" + o.start())) .collect(Collectors.toList()); } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java index b0b3085dca70..dac09fea7562 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java @@ -42,7 +42,7 @@ public static List snapshotIds(Long... ids) { } public static List files(ContentFile... files) { - return Arrays.stream(files).map(file -> file.path().toString()).collect(Collectors.toList()); + return Arrays.stream(files).map(ContentFile::location).collect(Collectors.toList()); } public static void validateDataManifest( @@ -62,7 +62,7 @@ public static void validateDataManifest( actualDataSeqs.add(entry.dataSequenceNumber()); actualFileSeqs.add(entry.fileSequenceNumber()); actualSnapshotIds.add(entry.snapshotId()); - actualFiles.add(entry.file().path().toString()); + actualFiles.add(entry.file().location()); } assertSameElements("data seqs", actualDataSeqs, dataSeqs); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java index 82b36528996f..3e8953fb950c 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java @@ -142,7 +142,7 @@ protected static String dbPath(String dbName) { } protected void withUnavailableFiles(Iterable> files, Action action) { - Iterable fileLocations = Iterables.transform(files, file -> file.path().toString()); + Iterable fileLocations = Iterables.transform(files, ContentFile::location); withUnavailableLocations(fileLocations, action); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestSparkExecutorCache.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestSparkExecutorCache.java index 35dfb55d5bb2..0d523b659cc1 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestSparkExecutorCache.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestSparkExecutorCache.java @@ -284,7 +284,7 @@ private void checkDelete(RowLevelOperationMode mode) throws Exception { } private int streamCount(DeleteFile deleteFile) { - CustomInputFile inputFile = INPUT_FILES.get(deleteFile.path().toString()); + CustomInputFile inputFile = INPUT_FILES.get(deleteFile.location()); return inputFile.streamCount(); } @@ -309,7 +309,7 @@ private List createAndInitTable(String operation, RowLevelOperationM List> posDeletes = dataFiles(table).stream() - .map(dataFile -> Pair.of(dataFile.path(), 0L)) + .map(dataFile -> Pair.of((CharSequence) dataFile.location(), 0L)) .collect(Collectors.toList()); Pair posDeleteResult = writePosDeletes(table, posDeletes); DeleteFile posDeleteFile = posDeleteResult.first(); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java index 97843c152811..f4b270528bb1 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java @@ -204,7 +204,7 @@ public void dataFilesCleanupWithParallelTasks() { .forEach( file -> Assert.assertTrue( - "FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString()))); + "FILE_A should be deleted", deletedFiles.contains(FILE_A.location()))); checkRemoveFilesResults(4L, 0, 0, 6L, 4L, 6, result); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java index 7a74329f3550..03ae730312fb 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java @@ -239,8 +239,8 @@ public void dataFilesCleanupWithParallelTasks() throws IOException { Sets.newHashSet( "remove-snapshot-0", "remove-snapshot-1", "remove-snapshot-2", "remove-snapshot-3")); - Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString())); - Assert.assertTrue("FILE_B should be deleted", deletedFiles.contains(FILE_B.path().toString())); + Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.location())); + Assert.assertTrue("FILE_B should be deleted", deletedFiles.contains(FILE_B.location())); checkExpirationResults(2L, 0L, 0L, 3L, 3L, result); } @@ -552,7 +552,7 @@ public void testScanExpiredManifestInValidSnapshotAppend() { .deleteWith(deletedFiles::add) .execute(); - Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString())); + Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.location())); checkExpirationResults(1L, 0L, 0L, 1L, 2L, result); } @@ -581,7 +581,7 @@ public void testScanExpiredManifestInValidSnapshotFastAppend() { .deleteWith(deletedFiles::add) .execute(); - Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString())); + Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.location())); checkExpirationResults(1L, 0L, 0L, 1L, 2L, result); } @@ -624,7 +624,7 @@ public void testWithExpiringDanglingStageCommit() { .addedDataFiles(table.io()) .forEach( i -> { - expectedDeletes.add(i.path().toString()); + expectedDeletes.add(i.location()); }); // ManifestList should be deleted too @@ -693,7 +693,7 @@ public void testWithCherryPickTableSnapshot() { i.addedDataFiles(table.io()) .forEach( item -> { - Assert.assertFalse(deletedFiles.contains(item.path().toString())); + Assert.assertFalse(deletedFiles.contains(item.location())); }); }); @@ -742,7 +742,7 @@ public void testWithExpiringStagedThenCherrypick() { i.addedDataFiles(table.io()) .forEach( item -> { - Assert.assertFalse(deletedFiles.contains(item.path().toString())); + Assert.assertFalse(deletedFiles.contains(item.location())); }); }); checkExpirationResults(0L, 0L, 0L, 1L, 1L, firstResult); @@ -762,7 +762,7 @@ public void testWithExpiringStagedThenCherrypick() { i.addedDataFiles(table.io()) .forEach( item -> { - Assert.assertFalse(deletedFiles.contains(item.path().toString())); + Assert.assertFalse(deletedFiles.contains(item.location())); }); }); checkExpirationResults(0L, 0L, 0L, 0L, 2L, secondResult); @@ -863,7 +863,7 @@ public void testExpireOlderThanWithDelete() { .allManifests(table.io()) .get(0) .path(), // manifest contained only deletes, was dropped - FILE_A.path()), // deleted + FILE_A.location()), // deleted deletedFiles); checkExpirationResults(1, 0, 0, 2, 2, result); @@ -932,7 +932,7 @@ public void testExpireOlderThanWithDeleteInMergedManifests() { .get(0) .path(), // manifest was rewritten for delete secondSnapshot.manifestListLocation(), // snapshot expired - FILE_A.path()), // deleted + FILE_A.location()), // deleted deletedFiles); checkExpirationResults(1, 0, 0, 1, 2, result); @@ -1046,7 +1046,7 @@ public void testExpireOlderThanWithRollbackAndMergedManifests() { secondSnapshot.manifestListLocation(), // snapshot expired Iterables.getOnlyElement(secondSnapshotManifests) .path(), // manifest is no longer referenced - FILE_B.path()), // added, but rolled back + FILE_B.location()), // added, but rolled back deletedFiles); checkExpirationResults(1, 0, 0, 1, 1, result); @@ -1095,9 +1095,9 @@ public void testExpireOlderThanWithDeleteFile() { secondSnapshot.manifestListLocation(), thirdSnapshot.manifestListLocation(), fourthSnapshot.manifestListLocation(), - FILE_A.path().toString(), - FILE_A_POS_DELETES.path().toString(), - FILE_A_EQ_DELETES.path().toString()); + FILE_A.location(), + FILE_A_POS_DELETES.location(), + FILE_A_EQ_DELETES.location()); expectedDeletes.addAll( thirdSnapshot.allManifests(table.io()).stream() @@ -1270,7 +1270,7 @@ public void textExpireAllCheckFilesDeleted(int dataFilesExpired, int dataFilesRe .withPartitionPath("c1=1") .withRecordCount(1) .build(); - dataFiles.add(df.path().toString()); + dataFiles.add(df.location()); table.newFastAppend().appendFile(df).commit(); } @@ -1343,9 +1343,9 @@ public void testExpireSomeCheckFilesDeleted() { // C, D should be retained (live) // B should be retained (previous snapshot points to it) // A should be deleted - Assert.assertTrue(deletedFiles.contains(FILE_A.path().toString())); - Assert.assertFalse(deletedFiles.contains(FILE_B.path().toString())); - Assert.assertFalse(deletedFiles.contains(FILE_C.path().toString())); - Assert.assertFalse(deletedFiles.contains(FILE_D.path().toString())); + Assert.assertTrue(deletedFiles.contains(FILE_A.location())); + Assert.assertFalse(deletedFiles.contains(FILE_B.location())); + Assert.assertFalse(deletedFiles.contains(FILE_C.location())); + Assert.assertFalse(deletedFiles.contains(FILE_D.location())); } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java index 8ac9be00ae90..dd330e461b1d 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java @@ -264,21 +264,21 @@ public void testPartitionedDeletesWithLesserSeqNo() { .collectAsList(); List> expected = ImmutableList.of( - Tuple2.apply(1L, FILE_B.path().toString()), - Tuple2.apply(1L, FILE_C.path().toString()), - Tuple2.apply(1L, FILE_D.path().toString()), - Tuple2.apply(2L, FILE_A_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_A_POS_DELETES.path().toString()), - Tuple2.apply(2L, FILE_A2_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_A2_POS_DELETES.path().toString()), - Tuple2.apply(2L, FILE_B_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_B_POS_DELETES.path().toString()), - Tuple2.apply(2L, FILE_B2_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_B2_POS_DELETES.path().toString()), - Tuple2.apply(3L, FILE_A2.path().toString()), - Tuple2.apply(3L, FILE_B2.path().toString()), - Tuple2.apply(3L, FILE_C2.path().toString()), - Tuple2.apply(3L, FILE_D2.path().toString())); + Tuple2.apply(1L, FILE_B.location()), + Tuple2.apply(1L, FILE_C.location()), + Tuple2.apply(1L, FILE_D.location()), + Tuple2.apply(2L, FILE_A_EQ_DELETES.location()), + Tuple2.apply(2L, FILE_A_POS_DELETES.location()), + Tuple2.apply(2L, FILE_A2_EQ_DELETES.location()), + Tuple2.apply(2L, FILE_A2_POS_DELETES.location()), + Tuple2.apply(2L, FILE_B_EQ_DELETES.location()), + Tuple2.apply(2L, FILE_B_POS_DELETES.location()), + Tuple2.apply(2L, FILE_B2_EQ_DELETES.location()), + Tuple2.apply(2L, FILE_B2_POS_DELETES.location()), + Tuple2.apply(3L, FILE_A2.location()), + Tuple2.apply(3L, FILE_B2.location()), + Tuple2.apply(3L, FILE_C2.location()), + Tuple2.apply(3L, FILE_D2.location())); assertThat(actual).isEqualTo(expected); RemoveDanglingDeleteFiles.Result result = SparkActions.get().removeDanglingDeleteFiles(table).execute(); @@ -286,16 +286,16 @@ public void testPartitionedDeletesWithLesserSeqNo() { // because there are no data files in partition with a lesser sequence number Set removedDeleteFiles = StreamSupport.stream(result.removedDeleteFiles().spliterator(), false) - .map(DeleteFile::path) + .map(DeleteFile::location) .collect(Collectors.toSet()); assertThat(removedDeleteFiles) .as("Expected 4 delete files removed") .hasSize(4) .containsExactlyInAnyOrder( - FILE_A_POS_DELETES.path(), - FILE_A2_POS_DELETES.path(), - FILE_A_EQ_DELETES.path(), - FILE_A2_EQ_DELETES.path()); + FILE_A_POS_DELETES.location(), + FILE_A2_POS_DELETES.location(), + FILE_A_EQ_DELETES.location(), + FILE_A2_EQ_DELETES.location()); List> actualAfter = spark .read() @@ -308,17 +308,17 @@ public void testPartitionedDeletesWithLesserSeqNo() { .collectAsList(); List> expectedAfter = ImmutableList.of( - Tuple2.apply(1L, FILE_B.path().toString()), - Tuple2.apply(1L, FILE_C.path().toString()), - Tuple2.apply(1L, FILE_D.path().toString()), - Tuple2.apply(2L, FILE_B_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_B_POS_DELETES.path().toString()), - Tuple2.apply(2L, FILE_B2_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_B2_POS_DELETES.path().toString()), - Tuple2.apply(3L, FILE_A2.path().toString()), - Tuple2.apply(3L, FILE_B2.path().toString()), - Tuple2.apply(3L, FILE_C2.path().toString()), - Tuple2.apply(3L, FILE_D2.path().toString())); + Tuple2.apply(1L, FILE_B.location()), + Tuple2.apply(1L, FILE_C.location()), + Tuple2.apply(1L, FILE_D.location()), + Tuple2.apply(2L, FILE_B_EQ_DELETES.location()), + Tuple2.apply(2L, FILE_B_POS_DELETES.location()), + Tuple2.apply(2L, FILE_B2_EQ_DELETES.location()), + Tuple2.apply(2L, FILE_B2_POS_DELETES.location()), + Tuple2.apply(3L, FILE_A2.location()), + Tuple2.apply(3L, FILE_B2.location()), + Tuple2.apply(3L, FILE_C2.location()), + Tuple2.apply(3L, FILE_D2.location())); assertThat(actualAfter).isEqualTo(expectedAfter); } @@ -354,21 +354,21 @@ public void testPartitionedDeletesWithEqSeqNo() { .collectAsList(); List> expected = ImmutableList.of( - Tuple2.apply(1L, FILE_A.path().toString()), - Tuple2.apply(1L, FILE_C.path().toString()), - Tuple2.apply(1L, FILE_D.path().toString()), - Tuple2.apply(2L, FILE_A_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_A_POS_DELETES.path().toString()), - Tuple2.apply(2L, FILE_A2.path().toString()), - Tuple2.apply(2L, FILE_A2_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_A2_POS_DELETES.path().toString()), - Tuple2.apply(2L, FILE_B_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_B_POS_DELETES.path().toString()), - Tuple2.apply(2L, FILE_B2.path().toString()), - Tuple2.apply(2L, FILE_B2_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_B2_POS_DELETES.path().toString()), - Tuple2.apply(2L, FILE_C2.path().toString()), - Tuple2.apply(2L, FILE_D2.path().toString())); + Tuple2.apply(1L, FILE_A.location()), + Tuple2.apply(1L, FILE_C.location()), + Tuple2.apply(1L, FILE_D.location()), + Tuple2.apply(2L, FILE_A_EQ_DELETES.location()), + Tuple2.apply(2L, FILE_A_POS_DELETES.location()), + Tuple2.apply(2L, FILE_A2.location()), + Tuple2.apply(2L, FILE_A2_EQ_DELETES.location()), + Tuple2.apply(2L, FILE_A2_POS_DELETES.location()), + Tuple2.apply(2L, FILE_B_EQ_DELETES.location()), + Tuple2.apply(2L, FILE_B_POS_DELETES.location()), + Tuple2.apply(2L, FILE_B2.location()), + Tuple2.apply(2L, FILE_B2_EQ_DELETES.location()), + Tuple2.apply(2L, FILE_B2_POS_DELETES.location()), + Tuple2.apply(2L, FILE_C2.location()), + Tuple2.apply(2L, FILE_D2.location())); assertThat(actual).isEqualTo(expected); RemoveDanglingDeleteFiles.Result result = SparkActions.get().removeDanglingDeleteFiles(table).execute(); @@ -376,12 +376,12 @@ public void testPartitionedDeletesWithEqSeqNo() { // because there are no data files in partition with a lesser sequence number Set removedDeleteFiles = StreamSupport.stream(result.removedDeleteFiles().spliterator(), false) - .map(DeleteFile::path) + .map(DeleteFile::location) .collect(Collectors.toSet()); assertThat(removedDeleteFiles) .as("Expected two delete files removed") .hasSize(2) - .containsExactlyInAnyOrder(FILE_B_EQ_DELETES.path(), FILE_B2_EQ_DELETES.path()); + .containsExactlyInAnyOrder(FILE_B_EQ_DELETES.location(), FILE_B2_EQ_DELETES.location()); List> actualAfter = spark .read() @@ -394,19 +394,19 @@ public void testPartitionedDeletesWithEqSeqNo() { .collectAsList(); List> expectedAfter = ImmutableList.of( - Tuple2.apply(1L, FILE_A.path().toString()), - Tuple2.apply(1L, FILE_C.path().toString()), - Tuple2.apply(1L, FILE_D.path().toString()), - Tuple2.apply(2L, FILE_A_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_A_POS_DELETES.path().toString()), - Tuple2.apply(2L, FILE_A2.path().toString()), - Tuple2.apply(2L, FILE_A2_EQ_DELETES.path().toString()), - Tuple2.apply(2L, FILE_A2_POS_DELETES.path().toString()), - Tuple2.apply(2L, FILE_B_POS_DELETES.path().toString()), - Tuple2.apply(2L, FILE_B2.path().toString()), - Tuple2.apply(2L, FILE_B2_POS_DELETES.path().toString()), - Tuple2.apply(2L, FILE_C2.path().toString()), - Tuple2.apply(2L, FILE_D2.path().toString())); + Tuple2.apply(1L, FILE_A.location()), + Tuple2.apply(1L, FILE_C.location()), + Tuple2.apply(1L, FILE_D.location()), + Tuple2.apply(2L, FILE_A_EQ_DELETES.location()), + Tuple2.apply(2L, FILE_A_POS_DELETES.location()), + Tuple2.apply(2L, FILE_A2.location()), + Tuple2.apply(2L, FILE_A2_EQ_DELETES.location()), + Tuple2.apply(2L, FILE_A2_POS_DELETES.location()), + Tuple2.apply(2L, FILE_B_POS_DELETES.location()), + Tuple2.apply(2L, FILE_B2.location()), + Tuple2.apply(2L, FILE_B2_POS_DELETES.location()), + Tuple2.apply(2L, FILE_C2.location()), + Tuple2.apply(2L, FILE_D2.location())); assertThat(actualAfter).isEqualTo(expectedAfter); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java index 656b7358d1a5..490c7119304c 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java @@ -1983,8 +1983,7 @@ private void writeDF(Dataset df) { private List writePosDeletesToFile( Table table, DataFile dataFile, int outputDeleteFiles) { - return writePosDeletes( - table, dataFile.partition(), dataFile.path().toString(), outputDeleteFiles); + return writePosDeletes(table, dataFile.partition(), dataFile.location(), outputDeleteFiles); } private List writePosDeletes( diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java index 5820d46703e2..9104cdadbb88 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java @@ -867,7 +867,7 @@ private void writePosDeletesForFiles( List> deletes = Lists.newArrayList(); for (DataFile partitionFile : partitionFiles) { for (int deletePos = 0; deletePos < deletesPerDataFile; deletePos++) { - deletes.add(Pair.of(partitionFile.path(), (long) deletePos)); + deletes.add(Pair.of(partitionFile.location(), (long) deletePos)); counter++; if (counter == deleteFileSize) { // Dump to file and reset variables @@ -904,17 +904,17 @@ private List deleteFiles(Table table) { private > List except(List first, List second) { Set secondPaths = - second.stream().map(f -> f.path().toString()).collect(Collectors.toSet()); + second.stream().map(ContentFile::location).collect(Collectors.toSet()); return first.stream() - .filter(f -> !secondPaths.contains(f.path().toString())) + .filter(f -> !secondPaths.contains(f.location())) .collect(Collectors.toList()); } private void assertNotContains(List original, List rewritten) { Set originalPaths = - original.stream().map(f -> f.path().toString()).collect(Collectors.toSet()); + original.stream().map(ContentFile::location).collect(Collectors.toSet()); Set rewrittenPaths = - rewritten.stream().map(f -> f.path().toString()).collect(Collectors.toSet()); + rewritten.stream().map(ContentFile::location).collect(Collectors.toSet()); rewrittenPaths.retainAll(originalPaths); Assert.assertEquals(0, rewrittenPaths.size()); } @@ -923,7 +923,7 @@ private void assertLocallySorted(List deleteFiles) { for (DeleteFile deleteFile : deleteFiles) { Dataset deletes = spark.read().format("iceberg").load("default." + TABLE_NAME + ".position_deletes"); - deletes.filter(deletes.col("delete_file_path").equalTo(deleteFile.path().toString())); + deletes.filter(deletes.col("delete_file_path").equalTo(deleteFile.location())); List rows = deletes.collectAsList(); Assert.assertFalse("Empty delete file found", rows.isEmpty()); int lastPos = 0; diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java index e5e0e350e50b..d417acdeb20a 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java @@ -114,7 +114,7 @@ public Boolean hasIterator(FileScanTask task) { } private String getKey(FileScanTask task) { - return task.file().path().toString(); + return task.file().location(); } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java index 15325fe03c4e..724c6edde26a 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java @@ -168,7 +168,7 @@ public void testWriteDataWithDifferentSetting() throws Exception { List manifestFiles = table.currentSnapshot().dataManifests(table.io()); try (ManifestReader reader = ManifestFiles.read(manifestFiles.get(0), table.io())) { DataFile file = reader.iterator().next(); - InputFile inputFile = table.io().newInputFile(file.path().toString()); + InputFile inputFile = table.io().newInputFile(file.location()); assertThat(getCompressionType(inputFile)) .isEqualToIgnoringCase(properties.get(COMPRESSION_CODEC)); } @@ -182,7 +182,7 @@ public void testWriteDataWithDifferentSetting() throws Exception { try (ManifestReader reader = ManifestFiles.readDeleteManifest(deleteManifestFiles.get(0), table.io(), specMap)) { DeleteFile file = reader.iterator().next(); - InputFile inputFile = table.io().newInputFile(file.path().toString()); + InputFile inputFile = table.io().newInputFile(file.location()); assertThat(getCompressionType(inputFile)) .isEqualToIgnoringCase(properties.get(COMPRESSION_CODEC)); } @@ -196,7 +196,7 @@ public void testWriteDataWithDifferentSetting() throws Exception { try (ManifestReader reader = ManifestFiles.readDeleteManifest(deleteManifestFiles.get(0), table.io(), specMap)) { DeleteFile file = reader.iterator().next(); - InputFile inputFile = table.io().newInputFile(file.path().toString()); + InputFile inputFile = table.io().newInputFile(file.location()); assertThat(getCompressionType(inputFile)) .isEqualToIgnoringCase(properties.get(COMPRESSION_CODEC)); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java index 693bcec9eb05..b5c055925bf7 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java @@ -197,8 +197,8 @@ private void writeAndValidateWithLocations(Table table, File location, File expe Assert.assertTrue( String.format( "File should have the parent directory %s, but has: %s.", - expectedDataDir.getAbsolutePath(), dataFile.path()), - URI.create(dataFile.path().toString()) + expectedDataDir.getAbsolutePath(), dataFile.location()), + URI.create(dataFile.location()) .getPath() .startsWith(expectedDataDir.getAbsolutePath()))); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java index 77e1c373e8d9..013b8d4386af 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java @@ -110,7 +110,7 @@ public void testWriteFormatOptionOverridesTableProperties() throws IOException { try (CloseableIterable tasks = table.newScan().planFiles()) { tasks.forEach( task -> { - FileFormat fileFormat = FileFormat.fromFileName(task.file().path()); + FileFormat fileFormat = FileFormat.fromFileName(task.file().location()); Assert.assertEquals(FileFormat.PARQUET, fileFormat); }); } @@ -135,7 +135,7 @@ public void testNoWriteFormatOption() throws IOException { try (CloseableIterable tasks = table.newScan().planFiles()) { tasks.forEach( task -> { - FileFormat fileFormat = FileFormat.fromFileName(task.file().path()); + FileFormat fileFormat = FileFormat.fromFileName(task.file().location()); Assert.assertEquals(FileFormat.AVRO, fileFormat); }); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index 84dff6666423..fd644cc49ca8 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -268,7 +268,7 @@ public void testEntriesTableDataFilePrune() { .select("data_file.file_path") .collectAsList()); - List singleExpected = ImmutableList.of(row(file.path())); + List singleExpected = ImmutableList.of(row(file.location())); assertEquals( "Should prune a single element from a nested struct", singleExpected, singleActual); @@ -307,7 +307,7 @@ public void testEntriesTableDataFilePruneMulti() { List multiExpected = ImmutableList.of( - row(file.path(), file.valueCounts(), file.recordCount(), file.columnSizes())); + row(file.location(), file.valueCounts(), file.recordCount(), file.columnSizes())); assertEquals("Should prune a single element from a nested struct", multiExpected, multiActual); } @@ -341,7 +341,7 @@ public void testFilesSelectMap() { List multiExpected = ImmutableList.of( - row(file.path(), file.valueCounts(), file.recordCount(), file.columnSizes())); + row(file.location(), file.valueCounts(), file.recordCount(), file.columnSizes())); assertEquals("Should prune a single element from a row", multiExpected, multiActual); } @@ -2315,7 +2315,7 @@ private DeleteFile writePosDeleteFile(Table table, long pos) { StructLike dataFilePartition = dataFile.partition(); PositionDelete delete = PositionDelete.create(); - delete.set(dataFile.path(), pos, null); + delete.set(dataFile.location(), pos, null); return writePositionDeletes(table, dataFileSpec, dataFilePartition, ImmutableList.of(delete)); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java index 2ec4f2f4f907..af3ec224cb4f 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java @@ -26,6 +26,7 @@ import java.util.UUID; import java.util.function.BiFunction; import java.util.stream.Collectors; +import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; @@ -140,8 +141,8 @@ public void testNullRows() throws IOException { tab.newAppend().appendFile(dFile).commit(); List> deletes = Lists.newArrayList(); - deletes.add(Pair.of(dFile.path(), 0L)); - deletes.add(Pair.of(dFile.path(), 1L)); + deletes.add(Pair.of(dFile.location(), 0L)); + deletes.add(Pair.of(dFile.location(), 1L)); Pair posDeletes = FileHelpers.writeDeleteFile( tab, Files.localOutput(temp.newFile()), TestHelpers.Row.of(0), deletes); @@ -150,9 +151,9 @@ public void testNullRows() throws IOException { StructLikeSet actual = actual(tableName, tab); List> expectedDeletes = - Lists.newArrayList(positionDelete(dFile.path(), 0L), positionDelete(dFile.path(), 1L)); - StructLikeSet expected = - expected(tab, expectedDeletes, null, posDeletes.first().path().toString()); + Lists.newArrayList( + positionDelete(dFile.location(), 0L), positionDelete(dFile.location(), 1L)); + StructLikeSet expected = expected(tab, expectedDeletes, null, posDeletes.first().location()); Assert.assertEquals("Position Delete table should contain expected rows", expected, actual); dropTable(tableName); @@ -181,7 +182,7 @@ public void testPartitionedTable() throws IOException { GenericRecord partitionB = GenericRecord.create(tab.spec().partitionType()); partitionB.setField("data", "b"); StructLikeSet expected = - expected(tab, deletesB.first(), partitionB, deletesB.second().path().toString()); + expected(tab, deletesB.first(), partitionB, deletesB.second().location()); Assert.assertEquals("Position Delete table should contain expected rows", expected, actual); dropTable(tableName); @@ -221,7 +222,7 @@ public void testSelect() throws IOException { (delete, file) -> { int rowData = delete.get(2, GenericRecord.class).get(0, Integer.class); long pos = delete.get(1, Long.class); - return row(rowData, pos, file.path().toString(), file.path().toString()); + return row(rowData, pos, file.location(), file.location()); }; expected.addAll( deletesA.first().stream() @@ -270,7 +271,7 @@ public void testSplitTasks() throws IOException { List> deletes = Lists.newArrayList(); for (long i = 0; i < records; i++) { - deletes.add(positionDelete(tab.schema(), dFile.path(), i, (int) i, String.valueOf(i))); + deletes.add(positionDelete(tab.schema(), dFile.location(), i, (int) i, String.valueOf(i))); } DeleteFile posDeletes = FileHelpers.writePosDeleteFile( @@ -292,7 +293,7 @@ public void testSplitTasks() throws IOException { } StructLikeSet actual = actual(tableName, tab); - StructLikeSet expected = expected(tab, deletes, null, posDeletes.path().toString()); + StructLikeSet expected = expected(tab, deletes, null, posDeletes.location()); Assert.assertEquals("Position Delete table should contain expected rows", expected, actual); dropTable(tableName); @@ -322,9 +323,9 @@ public void testPartitionFilter() throws IOException { Record partitionA = partitionRecordTemplate.copy("data", "a"); Record partitionB = partitionRecordTemplate.copy("data", "b"); StructLikeSet expectedA = - expected(tab, deletesA.first(), partitionA, deletesA.second().path().toString()); + expected(tab, deletesA.first(), partitionA, deletesA.second().location()); StructLikeSet expectedB = - expected(tab, deletesB.first(), partitionB, deletesB.second().path().toString()); + expected(tab, deletesB.first(), partitionB, deletesB.second().location()); StructLikeSet allExpected = StructLikeSet.create(deletesTab.schema().asStruct()); allExpected.addAll(expectedA); allExpected.addAll(expectedB); @@ -365,9 +366,9 @@ public void testPartitionTransformFilter() throws IOException { Record partitionA = partitionRecordTemplate.copy("data_trunc", "a"); Record partitionB = partitionRecordTemplate.copy("data_trunc", "b"); StructLikeSet expectedA = - expected(tab, deletesA.first(), partitionA, deletesA.second().path().toString()); + expected(tab, deletesA.first(), partitionA, deletesA.second().location()); StructLikeSet expectedB = - expected(tab, deletesB.first(), partitionB, deletesB.second().path().toString()); + expected(tab, deletesB.first(), partitionB, deletesB.second().location()); StructLikeSet allExpected = StructLikeSet.create(deletesTable.schema().asStruct()); allExpected.addAll(expectedA); allExpected.addAll(expectedB); @@ -415,7 +416,7 @@ public void testPartitionEvolutionReplace() throws Exception { GenericRecord partitionRecordTemplate = GenericRecord.create(Partitioning.partitionType(tab)); Record partitionA = partitionRecordTemplate.copy("data", "a"); StructLikeSet expectedA = - expected(tab, deletesA.first(), partitionA, dataSpec, deletesA.second().path().toString()); + expected(tab, deletesA.first(), partitionA, dataSpec, deletesA.second().location()); StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); Assert.assertEquals("Position Delete table should contain expected rows", expectedA, actualA); @@ -427,7 +428,7 @@ public void testPartitionEvolutionReplace() throws Exception { deletes10.first(), partition10, tab.spec().specId(), - deletes10.second().path().toString()); + deletes10.second().location()); StructLikeSet actual10 = actual(tableName, tab, "partition.id = 10 AND pos >= 0"); Assert.assertEquals("Position Delete table should contain expected rows", expected10, actual10); @@ -465,7 +466,7 @@ public void testPartitionEvolutionAdd() throws Exception { GenericRecord partitionRecordTemplate = GenericRecord.create(Partitioning.partitionType(tab)); Record partitionA = partitionRecordTemplate.copy("data", "a"); StructLikeSet expectedA = - expected(tab, deletesA.first(), partitionA, specId1, deletesA.second().path().toString()); + expected(tab, deletesA.first(), partitionA, specId1, deletesA.second().location()); StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); Assert.assertEquals("Position Delete table should contain expected rows", expectedA, actualA); @@ -477,7 +478,7 @@ public void testPartitionEvolutionAdd() throws Exception { deletesUnpartitioned.first(), unpartitionedRecord, specId0, - deletesUnpartitioned.second().path().toString()); + deletesUnpartitioned.second().location()); StructLikeSet actualUnpartitioned = actual(tableName, tab, "partition.data IS NULL and pos >= 0"); @@ -520,7 +521,7 @@ public void testPartitionEvolutionRemove() throws Exception { GenericRecord partitionRecordTemplate = GenericRecord.create(Partitioning.partitionType(tab)); Record partitionA = partitionRecordTemplate.copy("data", "a"); StructLikeSet expectedA = - expected(tab, deletesA.first(), partitionA, specId0, deletesA.second().path().toString()); + expected(tab, deletesA.first(), partitionA, specId0, deletesA.second().location()); StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); Assert.assertEquals("Position Delete table should contain expected rows", expectedA, actualA); @@ -532,7 +533,7 @@ public void testPartitionEvolutionRemove() throws Exception { deletesUnpartitioned.first(), unpartitionedRecord, specId1, - deletesUnpartitioned.second().path().toString()); + deletesUnpartitioned.second().location()); StructLikeSet actualUnpartitioned = actual(tableName, tab, "partition.data IS NULL and pos >= 0"); @@ -577,7 +578,7 @@ public void testSpecIdFilter() throws Exception { deletesUnpartitioned.first(), partitionRecordTemplate, unpartitionedSpec, - deletesUnpartitioned.second().path().toString()); + deletesUnpartitioned.second().location()); StructLikeSet actualUnpartitioned = actual(tableName, tab, String.format("spec_id = %d", unpartitionedSpec)); Assert.assertEquals( @@ -589,9 +590,9 @@ public void testSpecIdFilter() throws Exception { StructLike partitionA = partitionRecordTemplate.copy("data", "a"); StructLike partitionB = partitionRecordTemplate.copy("data", "b"); StructLikeSet expected = - expected(tab, deletesA.first(), partitionA, dataSpec, deletesA.second().path().toString()); + expected(tab, deletesA.first(), partitionA, dataSpec, deletesA.second().location()); expected.addAll( - expected(tab, deletesB.first(), partitionB, dataSpec, deletesB.second().path().toString())); + expected(tab, deletesB.first(), partitionB, dataSpec, deletesB.second().location())); StructLikeSet actual = actual(tableName, tab, String.format("spec_id = %d", dataSpec)); Assert.assertEquals("Position Delete table should contain expected rows", expected, actual); @@ -645,14 +646,14 @@ public void testSchemaEvolutionAdd() throws Exception { d.set(2, padded); }); StructLikeSet expectedA = - expected(tab, expectedDeletesA, partitionA, deletesA.second().path().toString()); + expected(tab, expectedDeletesA, partitionA, deletesA.second().location()); StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); Assert.assertEquals("Position Delete table should contain expected rows", expectedA, actualA); // Select deletes from new schema Record partitionC = partitionRecordTemplate.copy("data", "c"); StructLikeSet expectedC = - expected(tab, deletesC.first(), partitionC, deletesC.second().path().toString()); + expected(tab, deletesC.first(), partitionC, deletesC.second().location()); StructLikeSet actualC = actual(tableName, tab, "partition.data = 'c' and pos >= 0"); Assert.assertEquals("Position Delete table should contain expected rows", expectedC, actualC); @@ -707,14 +708,14 @@ public void testSchemaEvolutionRemove() throws Exception { d.set(2, padded); }); StructLikeSet expectedA = - expected(tab, expectedDeletesA, partitionA, deletesA.second().path().toString()); + expected(tab, expectedDeletesA, partitionA, deletesA.second().location()); StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); Assert.assertEquals("Position Delete table should contain expected rows", expectedA, actualA); // Select deletes from new schema Record partitionC = partitionRecordTemplate.copy("data", "c"); StructLikeSet expectedC = - expected(tab, deletesC.first(), partitionC, deletesC.second().path().toString()); + expected(tab, deletesC.first(), partitionC, deletesC.second().location()); StructLikeSet actualC = actual(tableName, tab, "partition.data = 'c' and pos >= 0"); Assert.assertEquals("Position Delete table should contain expected rows", expectedC, actualC); @@ -792,8 +793,8 @@ public void testWriteUnpartitionedNullRows() throws Exception { tab.newAppend().appendFile(dFile).commit(); List> deletes = Lists.newArrayList(); - deletes.add(Pair.of(dFile.path(), 0L)); - deletes.add(Pair.of(dFile.path(), 1L)); + deletes.add(Pair.of(dFile.location(), 0L)); + deletes.add(Pair.of(dFile.location(), 1L)); Pair posDeletes = FileHelpers.writeDeleteFile( tab, Files.localOutput(temp.newFile()), TestHelpers.Row.of(0), deletes); @@ -827,7 +828,8 @@ public void testWriteUnpartitionedNullRows() throws Exception { actual(tableName, tab, null, ImmutableList.of("file_path", "pos", "row", "spec_id")); List> expectedDeletes = - Lists.newArrayList(positionDelete(dFile.path(), 0L), positionDelete(dFile.path(), 1L)); + Lists.newArrayList( + positionDelete(dFile.location(), 0L), positionDelete(dFile.location(), 1L)); StructLikeSet expected = expected(tab, expectedDeletes, null, null); Assert.assertEquals("Position Delete table should contain expected rows", expected, actual); @@ -846,8 +848,8 @@ public void testWriteMixedRows() throws Exception { // Add a delete file with row and without row List> deletes = Lists.newArrayList(); - deletes.add(Pair.of(dataFileA.path(), 0L)); - deletes.add(Pair.of(dataFileA.path(), 1L)); + deletes.add(Pair.of(dataFileA.location(), 0L)); + deletes.add(Pair.of(dataFileA.location(), 1L)); Pair deletesWithoutRow = FileHelpers.writeDeleteFile( tab, Files.localOutput(temp.newFile()), TestHelpers.Row.of("a"), deletes); @@ -905,7 +907,7 @@ public void testWriteMixedRows() throws Exception { expected( tab, Lists.newArrayList( - positionDelete(dataFileA.path(), 0L), positionDelete(dataFileA.path(), 1L)), + positionDelete(dataFileA.location(), 0L), positionDelete(dataFileA.location(), 1L)), partitionA, null)); allExpected.addAll(expected(tab, deletesWithRow.first(), partitionB, null)); @@ -1481,13 +1483,13 @@ private Pair>, DeleteFile> deleteFile( Lists.newArrayList( positionDelete( tab.schema(), - dataFile.path(), + dataFile.location(), 0L, idPartition != null ? idPartition : 29, dataPartition != null ? dataPartition : "c"), positionDelete( tab.schema(), - dataFile.path(), + dataFile.location(), 1L, idPartition != null ? idPartition : 61, dataPartition != null ? dataPartition : "r")); @@ -1533,9 +1535,9 @@ private void commit( Assert.assertEquals(expectedTargetFiles, addedFiles.size()); List sortedAddedFiles = - addedFiles.stream().map(f -> f.path().toString()).sorted().collect(Collectors.toList()); + addedFiles.stream().map(ContentFile::location).sorted().collect(Collectors.toList()); List sortedRewrittenFiles = - rewrittenFiles.stream().map(f -> f.path().toString()).sorted().collect(Collectors.toList()); + rewrittenFiles.stream().map(ContentFile::location).sorted().collect(Collectors.toList()); Assert.assertNotEquals("Lists should not be the same", sortedAddedFiles, sortedRewrittenFiles); baseTab diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java index edd4cdf08360..b09c995b30fa 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java @@ -470,7 +470,7 @@ private void deleteNotMatchingFiles(Expression filter, int expectedDeletedFileCo Set matchingFileLocations = Sets.newHashSet(); try (CloseableIterable files = table.newScan().filter(filter).planFiles()) { for (FileScanTask file : files) { - String path = file.file().path().toString(); + String path = file.file().location(); matchingFileLocations.add(path); } } catch (IOException e) { @@ -480,7 +480,7 @@ private void deleteNotMatchingFiles(Expression filter, int expectedDeletedFileCo Set deletedFileLocations = Sets.newHashSet(); try (CloseableIterable files = table.newScan().planFiles()) { for (FileScanTask file : files) { - String path = file.file().path().toString(); + String path = file.file().location(); if (!matchingFileLocations.contains(path)) { io.deleteFile(path); deletedFileLocations.add(path); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java index b894d32326dc..16fde3c95444 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java @@ -281,7 +281,7 @@ private void checkDeleteFile(DeleteFile expected, DeleteFile actual) { private void checkContentFile(ContentFile expected, ContentFile actual) { assertThat(actual.content()).isEqualTo(expected.content()); - assertThat(actual.path()).isEqualTo(expected.path()); + assertThat(actual.location()).isEqualTo(expected.location()); assertThat(actual.format()).isEqualTo(expected.format()); assertThat(actual.recordCount()).isEqualTo(expected.recordCount()); assertThat(actual.fileSizeInBytes()).isEqualTo(expected.fileSizeInBytes()); @@ -319,10 +319,10 @@ private DeleteFile createPositionDeleteFile(Table table, DataFile dataFile) { null, // no NaN counts ImmutableMap.of( MetadataColumns.DELETE_FILE_PATH.fieldId(), - Conversions.toByteBuffer(Types.StringType.get(), dataFile.path())), + Conversions.toByteBuffer(Types.StringType.get(), dataFile.location())), ImmutableMap.of( MetadataColumns.DELETE_FILE_PATH.fieldId(), - Conversions.toByteBuffer(Types.StringType.get(), dataFile.path())))) + Conversions.toByteBuffer(Types.StringType.get(), dataFile.location())))) .withEncryptionKeyMetadata(ByteBuffer.allocate(4).putInt(35)) .build(); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java index bbdd49b63df8..dda49b49465c 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java @@ -343,10 +343,10 @@ public void testPosDeletesAllRowsInBatch() throws IOException { // deleted. List> deletes = Lists.newArrayList( - Pair.of(dataFile.path(), 0L), // id = 29 - Pair.of(dataFile.path(), 1L), // id = 43 - Pair.of(dataFile.path(), 2L), // id = 61 - Pair.of(dataFile.path(), 3L) // id = 89 + Pair.of(dataFile.location(), 0L), // id = 29 + Pair.of(dataFile.location(), 1L), // id = 43 + Pair.of(dataFile.location(), 2L), // id = 61 + Pair.of(dataFile.location(), 3L) // id = 89 ); Pair posDeletes = @@ -376,10 +376,10 @@ public void testPosDeletesWithDeletedColumn() throws IOException { // deleted. List> deletes = Lists.newArrayList( - Pair.of(dataFile.path(), 0L), // id = 29 - Pair.of(dataFile.path(), 1L), // id = 43 - Pair.of(dataFile.path(), 2L), // id = 61 - Pair.of(dataFile.path(), 3L) // id = 89 + Pair.of(dataFile.location(), 0L), // id = 29 + Pair.of(dataFile.location(), 1L), // id = 43 + Pair.of(dataFile.location(), 2L), // id = 61 + Pair.of(dataFile.location(), 3L) // id = 89 ); Pair posDeletes = @@ -455,8 +455,8 @@ public void testMixedPosAndEqDeletesWithDeletedColumn() throws IOException { List> deletes = Lists.newArrayList( - Pair.of(dataFile.path(), 3L), // id = 89 - Pair.of(dataFile.path(), 5L) // id = 121 + Pair.of(dataFile.location(), 3L), // id = 89 + Pair.of(dataFile.location(), 5L) // id = 121 ); Pair posDeletes = @@ -486,10 +486,10 @@ public void testMixedPosAndEqDeletesWithDeletedColumn() throws IOException { public void testFilterOnDeletedMetadataColumn() throws IOException { List> deletes = Lists.newArrayList( - Pair.of(dataFile.path(), 0L), // id = 29 - Pair.of(dataFile.path(), 1L), // id = 43 - Pair.of(dataFile.path(), 2L), // id = 61 - Pair.of(dataFile.path(), 3L) // id = 89 + Pair.of(dataFile.location(), 0L), // id = 29 + Pair.of(dataFile.location(), 1L), // id = 43 + Pair.of(dataFile.location(), 2L), // id = 61 + Pair.of(dataFile.location(), 3L) // id = 89 ); Pair posDeletes = @@ -611,13 +611,13 @@ public void testPosDeletesOnParquetFileWithMultipleRowGroups() throws IOExceptio // Add positional deletes to the table List> deletes = Lists.newArrayList( - Pair.of(dataFile.path(), 97L), - Pair.of(dataFile.path(), 98L), - Pair.of(dataFile.path(), 99L), - Pair.of(dataFile.path(), 101L), - Pair.of(dataFile.path(), 103L), - Pair.of(dataFile.path(), 107L), - Pair.of(dataFile.path(), 109L)); + Pair.of(dataFile.location(), 97L), + Pair.of(dataFile.location(), 98L), + Pair.of(dataFile.location(), 99L), + Pair.of(dataFile.location(), 101L), + Pair.of(dataFile.location(), 103L), + Pair.of(dataFile.location(), 107L), + Pair.of(dataFile.location(), 109L)); Pair posDeletes = FileHelpers.writeDeleteFile( table, diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java index c58277ec084d..d3bbff5712ce 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java @@ -284,7 +284,7 @@ public void testExpireDeleteFiles() throws Exception { assertThat(TestHelpers.deleteFiles(table)).as("Should have 1 delete file").hasSize(1); Path deleteManifestPath = new Path(TestHelpers.deleteManifests(table).iterator().next().path()); DeleteFile deleteFile = TestHelpers.deleteFiles(table).iterator().next(); - Path deleteFilePath = new Path(String.valueOf(deleteFile.location())); + Path deleteFilePath = new Path(deleteFile.location()); sql( "CALL %s.system.rewrite_data_files(" diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java index e7698e5d5d07..57d4550d52a3 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java @@ -411,8 +411,7 @@ public void testRemoveOrphanFilesWithDeleteFiles() throws Exception { assertThat(TestHelpers.deleteManifests(table)).as("Should have 1 delete manifest").hasSize(1); assertThat(TestHelpers.deleteFiles(table)).as("Should have 1 delete file").hasSize(1); Path deleteManifestPath = new Path(TestHelpers.deleteManifests(table).iterator().next().path()); - Path deleteFilePath = - new Path(String.valueOf(TestHelpers.deleteFiles(table).iterator().next().location())); + Path deleteFilePath = new Path(TestHelpers.deleteFiles(table).iterator().next().location()); // wait to ensure files are old enough waitUntilAfter(System.currentTimeMillis()); diff --git a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java index 3b39847cd11d..f410fcc905fe 100644 --- a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java +++ b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java @@ -45,7 +45,7 @@ protected void appendData() throws IOException { table().refresh(); for (DataFile file : table().currentSnapshot().addedDataFiles(table().io())) { - writePosDeletes(file.path(), NUM_ROWS, 0.25, numDeleteFile); + writePosDeletes(file.location(), NUM_ROWS, 0.25, numDeleteFile); } } } diff --git a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java index 3b41552052b0..b145a8178898 100644 --- a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java +++ b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java @@ -47,7 +47,7 @@ protected void appendData() throws IOException { // add pos-deletes table().refresh(); for (DataFile file : table().currentSnapshot().addedDataFiles(table().io())) { - writePosDeletes(file.path(), NUM_ROWS, percentDeleteRow); + writePosDeletes(file.location(), NUM_ROWS, percentDeleteRow); } } } diff --git a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java index c3823e88275e..b0439fe07ee8 100644 --- a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java +++ b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java @@ -48,7 +48,7 @@ protected void appendData() throws IOException { table().refresh(); for (DataFile file : table().currentSnapshot().addedDataFiles(table().io())) { writePosDeletesWithNoise( - file.path(), + file.location(), NUM_ROWS, PERCENT_DELETE_ROW, (int) (percentUnrelatedDeletes / PERCENT_DELETE_ROW), diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java index 01912c81ccdf..160dee9e7058 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java @@ -39,6 +39,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.HasTableOperations; @@ -647,7 +648,7 @@ private static void importUnpartitionedSparkTable( if (checkDuplicateFiles) { Dataset importedFiles = spark - .createDataset(Lists.transform(files, f -> f.path().toString()), Encoders.STRING()) + .createDataset(Lists.transform(files, ContentFile::location), Encoders.STRING()) .toDF("file_path"); Dataset existingFiles = loadMetadataTable(spark, targetTable, MetadataTableType.ENTRIES).filter("status != 2"); @@ -777,7 +778,7 @@ public static void importSparkPartitions( if (checkDuplicateFiles) { Dataset importedFiles = filesToImport - .map((MapFunction) f -> f.path().toString(), Encoders.STRING()) + .map((MapFunction) ContentFile::location, Encoders.STRING()) .toDF("file_path"); Dataset existingFiles = loadMetadataTable(spark, targetTable, MetadataTableType.ENTRIES).filter("status != 2"); @@ -796,7 +797,7 @@ public static void importSparkPartitions( .repartition(numShufflePartitions) .map( (MapFunction>) - file -> Tuple2.apply(file.path().toString(), file), + file -> Tuple2.apply(file.location(), file), Encoders.tuple(Encoders.STRING(), Encoders.javaSerialization(DataFile.class))) .orderBy(col("_1")) .mapPartitions( diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java index c8e6f5679cd8..a6e2891ae696 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java @@ -116,7 +116,7 @@ CloseableIterable openAddedRowsScanTask(AddedRowsScanTask task) { } private CloseableIterable openDeletedDataFileScanTask(DeletedDataFileScanTask task) { - String filePath = task.file().path().toString(); + String filePath = task.file().location(); SparkDeleteFilter deletes = new SparkDeleteFilter(filePath, task.existingDeletes(), counter(), true); return deletes.filter(rows(task, deletes.requiredSchema())); @@ -125,7 +125,7 @@ private CloseableIterable openDeletedDataFileScanTask(DeletedDataFi private CloseableIterable rows(ContentScanTask task, Schema readSchema) { Map idToConstant = constantsMap(task, readSchema); - String filePath = task.file().path().toString(); + String filePath = task.file().location(); // update the current file for Spark's filename() function InputFileBlockHolder.set(filePath, task.start(), task.length()); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java index 7314043b15e2..dac09fea7562 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java @@ -42,7 +42,7 @@ public static List snapshotIds(Long... ids) { } public static List files(ContentFile... files) { - return Arrays.stream(files).map(file -> file.location()).collect(Collectors.toList()); + return Arrays.stream(files).map(ContentFile::location).collect(Collectors.toList()); } public static void validateDataManifest( diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java index 86afd2f890ae..de68351f6e39 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java @@ -142,7 +142,7 @@ protected static String dbPath(String dbName) { } protected void withUnavailableFiles(Iterable> files, Action action) { - Iterable fileLocations = Iterables.transform(files, file -> file.location()); + Iterable fileLocations = Iterables.transform(files, ContentFile::location); withUnavailableLocations(fileLocations, action); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkExecutorCache.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkExecutorCache.java index d9d7b78799ba..8114df1b5314 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkExecutorCache.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkExecutorCache.java @@ -375,7 +375,7 @@ private List createAndInitTable(String operation, RowLevelOperationM List> posDeletes = dataFiles(table).stream() - .map(dataFile -> Pair.of(dataFile.path(), 0L)) + .map(dataFile -> Pair.of((CharSequence) dataFile.location(), 0L)) .collect(Collectors.toList()); Pair posDeleteResult = writePosDeletes(table, posDeletes); DeleteFile posDeleteFile = posDeleteResult.first(); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java index ffbe988e8d41..6a1eb0ed9054 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java @@ -874,7 +874,7 @@ public void testExpireOlderThanWithDelete() { .allManifests(table.io()) .get(0) .path(), // manifest contained only deletes, was dropped - FILE_A.path()) // deleted + FILE_A.location()) // deleted ); checkExpirationResults(1, 0, 0, 2, 2, result); @@ -941,7 +941,7 @@ public void testExpireOlderThanWithDeleteInMergedManifests() { .get(0) .path(), // manifest was rewritten for delete secondSnapshot.manifestListLocation(), // snapshot expired - FILE_A.path()) // deleted + FILE_A.location()) // deleted ); checkExpirationResults(1, 0, 0, 1, 2, result); } @@ -1053,7 +1053,7 @@ public void testExpireOlderThanWithRollbackAndMergedManifests() { secondSnapshot.manifestListLocation(), // snapshot expired Iterables.getOnlyElement(secondSnapshotManifests) .path(), // manifest is no longer referenced - FILE_B.path()) // added, but rolled back + FILE_B.location()) // added, but rolled back ); checkExpirationResults(1, 0, 0, 1, 1, result); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java index e58966cfea3f..ce86474c82f1 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java @@ -313,21 +313,21 @@ public void testPartitionedDeletesWithLesserSeqNo() { .collectAsList(); List> expected = ImmutableList.of( - Tuple2.apply(1L, FILE_B.path().toString()), - Tuple2.apply(1L, FILE_C.path().toString()), - Tuple2.apply(1L, FILE_D.path().toString()), - Tuple2.apply(2L, FILE_A_EQ_DELETES.path().toString()), - Tuple2.apply(2L, fileADeletes.path().toString()), - Tuple2.apply(2L, FILE_A2_EQ_DELETES.path().toString()), - Tuple2.apply(2L, fileA2Deletes.path().toString()), - Tuple2.apply(2L, FILE_B_EQ_DELETES.path().toString()), - Tuple2.apply(2L, fileBDeletes.path().toString()), - Tuple2.apply(2L, FILE_B2_EQ_DELETES.path().toString()), - Tuple2.apply(2L, fileB2Deletes.path().toString()), - Tuple2.apply(3L, FILE_A2.path().toString()), - Tuple2.apply(3L, FILE_B2.path().toString()), - Tuple2.apply(3L, FILE_C2.path().toString()), - Tuple2.apply(3L, FILE_D2.path().toString())); + Tuple2.apply(1L, FILE_B.location()), + Tuple2.apply(1L, FILE_C.location()), + Tuple2.apply(1L, FILE_D.location()), + Tuple2.apply(2L, FILE_A_EQ_DELETES.location()), + Tuple2.apply(2L, fileADeletes.location()), + Tuple2.apply(2L, FILE_A2_EQ_DELETES.location()), + Tuple2.apply(2L, fileA2Deletes.location()), + Tuple2.apply(2L, FILE_B_EQ_DELETES.location()), + Tuple2.apply(2L, fileBDeletes.location()), + Tuple2.apply(2L, FILE_B2_EQ_DELETES.location()), + Tuple2.apply(2L, fileB2Deletes.location()), + Tuple2.apply(3L, FILE_A2.location()), + Tuple2.apply(3L, FILE_B2.location()), + Tuple2.apply(3L, FILE_C2.location()), + Tuple2.apply(3L, FILE_D2.location())); assertThat(actual).containsExactlyInAnyOrderElementsOf(expected); RemoveDanglingDeleteFiles.Result result = @@ -338,16 +338,16 @@ public void testPartitionedDeletesWithLesserSeqNo() { Set removedDeleteFiles = StreamSupport.stream(result.removedDeleteFiles().spliterator(), false) - .map(DeleteFile::path) + .map(DeleteFile::location) .collect(Collectors.toSet()); assertThat(removedDeleteFiles) .as("Expected 4 delete files removed") .hasSize(4) .containsExactlyInAnyOrder( - fileADeletes.path(), - fileA2Deletes.path(), - FILE_A_EQ_DELETES.path(), - FILE_A2_EQ_DELETES.path()); + fileADeletes.location(), + fileA2Deletes.location(), + FILE_A_EQ_DELETES.location(), + FILE_A2_EQ_DELETES.location()); List> actualAfter = spark @@ -361,17 +361,17 @@ public void testPartitionedDeletesWithLesserSeqNo() { .collectAsList(); List> expectedAfter = ImmutableList.of( - Tuple2.apply(1L, FILE_B.path().toString()), - Tuple2.apply(1L, FILE_C.path().toString()), - Tuple2.apply(1L, FILE_D.path().toString()), - Tuple2.apply(2L, FILE_B_EQ_DELETES.path().toString()), - Tuple2.apply(2L, fileBDeletes.path().toString()), - Tuple2.apply(2L, FILE_B2_EQ_DELETES.path().toString()), - Tuple2.apply(2L, fileB2Deletes.path().toString()), - Tuple2.apply(3L, FILE_A2.path().toString()), - Tuple2.apply(3L, FILE_B2.path().toString()), - Tuple2.apply(3L, FILE_C2.path().toString()), - Tuple2.apply(3L, FILE_D2.path().toString())); + Tuple2.apply(1L, FILE_B.location()), + Tuple2.apply(1L, FILE_C.location()), + Tuple2.apply(1L, FILE_D.location()), + Tuple2.apply(2L, FILE_B_EQ_DELETES.location()), + Tuple2.apply(2L, fileBDeletes.location()), + Tuple2.apply(2L, FILE_B2_EQ_DELETES.location()), + Tuple2.apply(2L, fileB2Deletes.location()), + Tuple2.apply(3L, FILE_A2.location()), + Tuple2.apply(3L, FILE_B2.location()), + Tuple2.apply(3L, FILE_C2.location()), + Tuple2.apply(3L, FILE_D2.location())); assertThat(actualAfter).containsExactlyInAnyOrderElementsOf(expectedAfter); } @@ -414,21 +414,21 @@ public void testPartitionedDeletesWithEqSeqNo() { .collectAsList(); List> expected = ImmutableList.of( - Tuple2.apply(1L, FILE_A.path().toString()), - Tuple2.apply(1L, FILE_C.path().toString()), - Tuple2.apply(1L, FILE_D.path().toString()), - Tuple2.apply(2L, FILE_A_EQ_DELETES.path().toString()), - Tuple2.apply(2L, fileADeletes.path().toString()), - Tuple2.apply(2L, FILE_A2.path().toString()), - Tuple2.apply(2L, FILE_A2_EQ_DELETES.path().toString()), - Tuple2.apply(2L, fileA2Deletes.path().toString()), - Tuple2.apply(2L, FILE_B_EQ_DELETES.path().toString()), - Tuple2.apply(2L, fileBDeletes.path().toString()), - Tuple2.apply(2L, FILE_B2.path().toString()), - Tuple2.apply(2L, FILE_B2_EQ_DELETES.path().toString()), - Tuple2.apply(2L, fileB2Deletes.path().toString()), - Tuple2.apply(2L, FILE_C2.path().toString()), - Tuple2.apply(2L, FILE_D2.path().toString())); + Tuple2.apply(1L, FILE_A.location()), + Tuple2.apply(1L, FILE_C.location()), + Tuple2.apply(1L, FILE_D.location()), + Tuple2.apply(2L, FILE_A_EQ_DELETES.location()), + Tuple2.apply(2L, fileADeletes.location()), + Tuple2.apply(2L, FILE_A2.location()), + Tuple2.apply(2L, FILE_A2_EQ_DELETES.location()), + Tuple2.apply(2L, fileA2Deletes.location()), + Tuple2.apply(2L, FILE_B_EQ_DELETES.location()), + Tuple2.apply(2L, fileBDeletes.location()), + Tuple2.apply(2L, FILE_B2.location()), + Tuple2.apply(2L, FILE_B2_EQ_DELETES.location()), + Tuple2.apply(2L, fileB2Deletes.location()), + Tuple2.apply(2L, FILE_C2.location()), + Tuple2.apply(2L, FILE_D2.location())); assertThat(actual).containsExactlyInAnyOrderElementsOf(expected); RemoveDanglingDeleteFiles.Result result = @@ -438,12 +438,12 @@ public void testPartitionedDeletesWithEqSeqNo() { // because there are no data files in partition with a lesser sequence number Set removedDeleteFiles = StreamSupport.stream(result.removedDeleteFiles().spliterator(), false) - .map(DeleteFile::path) + .map(DeleteFile::location) .collect(Collectors.toSet()); assertThat(removedDeleteFiles) .as("Expected two delete files removed") .hasSize(2) - .containsExactlyInAnyOrder(FILE_B_EQ_DELETES.path(), FILE_B2_EQ_DELETES.path()); + .containsExactlyInAnyOrder(FILE_B_EQ_DELETES.location(), FILE_B2_EQ_DELETES.location()); List> actualAfter = spark @@ -457,19 +457,19 @@ public void testPartitionedDeletesWithEqSeqNo() { .collectAsList(); List> expectedAfter = ImmutableList.of( - Tuple2.apply(1L, FILE_A.path().toString()), - Tuple2.apply(1L, FILE_C.path().toString()), - Tuple2.apply(1L, FILE_D.path().toString()), - Tuple2.apply(2L, FILE_A_EQ_DELETES.path().toString()), - Tuple2.apply(2L, fileADeletes.path().toString()), - Tuple2.apply(2L, FILE_A2.path().toString()), - Tuple2.apply(2L, FILE_A2_EQ_DELETES.path().toString()), - Tuple2.apply(2L, fileA2Deletes.path().toString()), - Tuple2.apply(2L, fileBDeletes.path().toString()), - Tuple2.apply(2L, FILE_B2.path().toString()), - Tuple2.apply(2L, fileB2Deletes.path().toString()), - Tuple2.apply(2L, FILE_C2.path().toString()), - Tuple2.apply(2L, FILE_D2.path().toString())); + Tuple2.apply(1L, FILE_A.location()), + Tuple2.apply(1L, FILE_C.location()), + Tuple2.apply(1L, FILE_D.location()), + Tuple2.apply(2L, FILE_A_EQ_DELETES.location()), + Tuple2.apply(2L, fileADeletes.location()), + Tuple2.apply(2L, FILE_A2.location()), + Tuple2.apply(2L, FILE_A2_EQ_DELETES.location()), + Tuple2.apply(2L, fileA2Deletes.location()), + Tuple2.apply(2L, fileBDeletes.location()), + Tuple2.apply(2L, FILE_B2.location()), + Tuple2.apply(2L, fileB2Deletes.location()), + Tuple2.apply(2L, FILE_C2.location()), + Tuple2.apply(2L, FILE_D2.location())); assertThat(actualAfter).containsExactlyInAnyOrderElementsOf(expectedAfter); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java index 38c4d32a90d2..bd307a468279 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java @@ -2152,8 +2152,7 @@ private void writeDF(Dataset df) { private List writePosDeletesToFile( Table table, DataFile dataFile, int outputDeleteFiles) { - return writePosDeletes( - table, dataFile.partition(), dataFile.path().toString(), outputDeleteFiles); + return writePosDeletes(table, dataFile.partition(), dataFile.location(), outputDeleteFiles); } private List writePosDeletes( diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java index 12b104fca27c..3bce399b34a9 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java @@ -898,7 +898,7 @@ private void writePosDeletesForFiles( List> deletes = Lists.newArrayList(); for (DataFile partitionFile : partitionFiles) { for (int deletePos = 0; deletePos < deletesPerDataFile; deletePos++) { - deletes.add(Pair.of(partitionFile.path(), (long) deletePos)); + deletes.add(Pair.of(partitionFile.location(), (long) deletePos)); counter++; if (counter == deleteFileSize) { // Dump to file and reset variables @@ -936,17 +936,17 @@ private List deleteFiles(Table table) { private > List except(List first, List second) { Set secondPaths = - second.stream().map(f -> f.path().toString()).collect(Collectors.toSet()); + second.stream().map(ContentFile::location).collect(Collectors.toSet()); return first.stream() - .filter(f -> !secondPaths.contains(f.path().toString())) + .filter(f -> !secondPaths.contains(f.location())) .collect(Collectors.toList()); } private void assertNotContains(List original, List rewritten) { Set originalPaths = - original.stream().map(f -> f.path().toString()).collect(Collectors.toSet()); + original.stream().map(ContentFile::location).collect(Collectors.toSet()); Set rewrittenPaths = - rewritten.stream().map(f -> f.path().toString()).collect(Collectors.toSet()); + rewritten.stream().map(ContentFile::location).collect(Collectors.toSet()); rewrittenPaths.retainAll(originalPaths); assertThat(rewrittenPaths).hasSize(0); } @@ -955,7 +955,7 @@ private void assertLocallySorted(List deleteFiles) { for (DeleteFile deleteFile : deleteFiles) { Dataset deletes = spark.read().format("iceberg").load("default." + TABLE_NAME + ".position_deletes"); - deletes.filter(deletes.col("delete_file_path").equalTo(deleteFile.path().toString())); + deletes.filter(deletes.col("delete_file_path").equalTo(deleteFile.location())); List rows = deletes.collectAsList(); assertThat(rows).as("Empty delete file found").isNotEmpty(); int lastPos = 0; diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index dcd347a34f81..7d38cf55c98c 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -266,7 +266,7 @@ public void testEntriesTableDataFilePrune() { .select("data_file.file_path") .collectAsList()); - List singleExpected = ImmutableList.of(row(file.path())); + List singleExpected = ImmutableList.of(row(file.location())); assertEquals( "Should prune a single element from a nested struct", singleExpected, singleActual); @@ -305,7 +305,7 @@ public void testEntriesTableDataFilePruneMulti() { List multiExpected = ImmutableList.of( - row(file.path(), file.valueCounts(), file.recordCount(), file.columnSizes())); + row(file.location(), file.valueCounts(), file.recordCount(), file.columnSizes())); assertEquals("Should prune a single element from a nested struct", multiExpected, multiActual); } @@ -339,7 +339,7 @@ public void testFilesSelectMap() { List multiExpected = ImmutableList.of( - row(file.path(), file.valueCounts(), file.recordCount(), file.columnSizes())); + row(file.location(), file.valueCounts(), file.recordCount(), file.columnSizes())); assertEquals("Should prune a single element from a row", multiExpected, multiActual); } @@ -2341,7 +2341,7 @@ private DeleteFile writePosDeleteFile(Table table, long pos) { StructLike dataFilePartition = dataFile.partition(); PositionDelete delete = PositionDelete.create(); - delete.set(dataFile.path(), pos, null); + delete.set(dataFile.location(), pos, null); return writePositionDeletes(table, dataFileSpec, dataFilePartition, ImmutableList.of(delete)); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java index 983e52cdf897..357cd0cdaa06 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java @@ -30,6 +30,7 @@ import java.util.UUID; import java.util.function.BiFunction; import java.util.stream.Collectors; +import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; @@ -1586,9 +1587,9 @@ private void commit( assertThat(addedFiles).hasSize(expectedTargetFiles); List sortedAddedFiles = - addedFiles.stream().map(f -> f.location()).sorted().collect(Collectors.toList()); + addedFiles.stream().map(ContentFile::location).sorted().collect(Collectors.toList()); List sortedRewrittenFiles = - rewrittenFiles.stream().map(f -> f.location()).sorted().collect(Collectors.toList()); + rewrittenFiles.stream().map(ContentFile::location).sorted().collect(Collectors.toList()); assertThat(sortedRewrittenFiles) .as("Lists should not be the same") .isNotEqualTo(sortedAddedFiles);