From d0cd4564589d78d6c0c3801efec5fcd5179eb017 Mon Sep 17 00:00:00 2001 From: Daniel Tu Date: Sun, 22 Sep 2024 15:48:12 -0700 Subject: [PATCH 01/29] delete manifest, manifest list, prev files, stats when drop table with purge --- .../service/task/TableCleanupTaskHandler.java | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java b/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java index 7f323174b..5d7cc2d54 100644 --- a/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java +++ b/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java @@ -22,7 +22,10 @@ import java.util.UUID; import java.util.function.Function; import java.util.stream.Collectors; + import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StatisticsFile; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableMetadataParser; import org.apache.iceberg.io.FileIO; @@ -158,6 +161,22 @@ public boolean handleTask(TaskEntity cleanupTask) { for (PolarisBaseEntity createdTask : createdTasks) { taskExecutor.addTaskHandlerContext(createdTask.getId(), CallContext.getCurrentContext()); } + + tableMetadata.snapshots().stream() + .flatMap(sn -> sn.allManifests(fileIO).stream()) + // remove duplication + .collect(Collectors.toMap(ManifestFile::path, Function.identity(), (mf1, mf2) -> mf1)) + .keySet() + .forEach(file -> fileIO.deleteFile(file)); + tableMetadata.snapshots().stream() + .map(Snapshot::manifestListLocation) + .forEach(file -> fileIO.deleteFile(file)); + tableMetadata.previousFiles().stream() + .map(TableMetadata.MetadataLogEntry::file) + .forEach(file -> fileIO.deleteFile(file)); + tableMetadata.statisticsFiles().stream() + .map(StatisticsFile::path) + .forEach(file -> fileIO.deleteFile(file)); fileIO.deleteFile(tableEntity.getMetadataLocation()); return true; From 26e03acee10177ee9ba5c63fd6bae9d530ce7156 Mon Sep 17 00:00:00 2001 From: Daniel Tu Date: Sun, 22 Sep 2024 15:49:02 -0700 Subject: [PATCH 02/29] unit test for drop table --- .../task/TableCleanupTaskHandlerTest.java | 165 +++++++++++++++++- 1 file changed, 163 insertions(+), 2 deletions(-) diff --git a/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java b/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java index ab9f9324c..058d35748 100644 --- a/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java +++ b/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java @@ -22,10 +22,20 @@ import java.io.IOException; import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; + import org.apache.commons.codec.binary.Base64; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; import org.apache.iceberg.ManifestFile; import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.PartitionStatisticsFile; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StatisticsFile; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableMetadataParser; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.inmemory.InMemoryFileIO; @@ -43,6 +53,7 @@ import org.assertj.core.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; import org.mockito.Mockito; import org.slf4j.LoggerFactory; @@ -64,7 +75,7 @@ public void testTableCleanup() throws IOException { new PolarisDefaultDiagServiceImpl()); try (CallContext callCtx = CallContext.of(realmContext, polarisCallContext)) { CallContext.setCurrentContext(callCtx); - FileIO fileIO = new InMemoryFileIO(); + FileIO fileIO = createMockFileIO(new InMemoryFileIO()); TableIdentifier tableIdentifier = TableIdentifier.of(Namespace.of("db1", "schema1"), "table1"); TableCleanupTaskHandler handler = @@ -91,6 +102,21 @@ public void testTableCleanup() throws IOException { .build(); Assertions.assertThatPredicate(handler::canHandleTask).accepts(task); + PolarisBaseEntity baseEntity = task.readData(PolarisBaseEntity.class); + TableLikeEntity tableEntity = TableLikeEntity.of(baseEntity); + TableMetadata tableMetadata = + TableMetadataParser.read(fileIO, tableEntity.getMetadataLocation()); + Set manifestListLocations = manifestListLocations(tableMetadata.snapshots()); + Set manifestLocations = manifestLocations(tableMetadata.snapshots(), fileIO); + Set metadataLocations = metadataLocations(tableMetadata); + Set statsLocations = statsLocations(tableMetadata); + Set partitionStatsLocations = partitionStatsLocations(tableMetadata); + assertThat(manifestListLocations).hasSize(1); + assertThat(manifestLocations).hasSize(1); + assertThat(metadataLocations).hasSize(1); + assertThat(statsLocations).hasSize(0); + assertThat(partitionStatsLocations).hasSize(0); + CallContext.setCurrentContext(CallContext.of(realmContext, polarisCallContext)); handler.handleTask(task); @@ -113,6 +139,35 @@ public void testTableCleanup() throws IOException { entity -> entity.readData( ManifestFileCleanupTaskHandler.ManifestCleanupTask.class))); + + ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(String.class); + Mockito.verify( + fileIO, + Mockito.times( + manifestListLocations.size() + + manifestLocations.size() + + metadataLocations.size() + + statsLocations.size() + + partitionStatsLocations.size() + ) + ).deleteFile(argumentCaptor.capture()); + + List deletedPaths = argumentCaptor.getAllValues(); + assertThat(deletedPaths) + .as("should contain all created manifest lists") + .containsAll(manifestListLocations); + assertThat(deletedPaths) + .as("should contain all created manifests") + .containsAll(manifestLocations); + assertThat(deletedPaths) + .as("should contain all created metadata locations") + .containsAll(metadataLocations); + assertThat(deletedPaths) + .as("should contain all created statistics") + .containsAll(statsLocations); + assertThat(deletedPaths) + .as("should contain all created partition stats files") + .containsAll(partitionStatsLocations); } } @@ -275,7 +330,7 @@ public void testTableCleanupMultipleSnapshots() throws IOException { new PolarisDefaultDiagServiceImpl()); try (CallContext callCtx = CallContext.of(realmContext, polarisCallContext)) { CallContext.setCurrentContext(callCtx); - FileIO fileIO = new InMemoryFileIO(); + FileIO fileIO = createMockFileIO(new InMemoryFileIO()); TableIdentifier tableIdentifier = TableIdentifier.of(Namespace.of("db1", "schema1"), "table1"); TableCleanupTaskHandler handler = @@ -318,6 +373,22 @@ public void testTableCleanupMultipleSnapshots() throws IOException { Assertions.assertThatPredicate(handler::canHandleTask).accepts(task); CallContext.setCurrentContext(CallContext.of(realmContext, polarisCallContext)); + + PolarisBaseEntity baseEntity = task.readData(PolarisBaseEntity.class); + TableLikeEntity tableEntity = TableLikeEntity.of(baseEntity); + TableMetadata tableMetadata = + TableMetadataParser.read(fileIO, tableEntity.getMetadataLocation()); + Set manifestListLocations = manifestListLocations(tableMetadata.snapshots()); + Set manifestLocations = manifestLocations(tableMetadata.snapshots(), fileIO); + Set metadataLocations = metadataLocations(tableMetadata); + Set statsLocations = statsLocations(tableMetadata); + Set partitionStatsLocations = partitionStatsLocations(tableMetadata); + assertThat(manifestListLocations).hasSize(2); + assertThat(manifestLocations).hasSize(3); + assertThat(metadataLocations).hasSize(1); + assertThat(statsLocations).hasSize(0); + assertThat(partitionStatsLocations).hasSize(0); + handler.handleTask(task); assertThat( @@ -362,6 +433,96 @@ public void testTableCleanupMultipleSnapshots() throws IOException { entity -> entity.readData( ManifestFileCleanupTaskHandler.ManifestCleanupTask.class))); + + ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(String.class); + Mockito.verify( + fileIO, + Mockito.times( + manifestListLocations.size() + + manifestLocations.size() + + metadataLocations.size() + + statsLocations.size() + + partitionStatsLocations.size() + ) + ).deleteFile(argumentCaptor.capture()); + + List deletedPaths = argumentCaptor.getAllValues(); + assertThat(deletedPaths) + .as("should contain all created manifest lists") + .containsAll(manifestListLocations); + assertThat(deletedPaths) + .as("should contain all created manifests") + .containsAll(manifestLocations); + assertThat(deletedPaths) + .as("should contain all created metadata locations") + .containsAll(metadataLocations); + assertThat(deletedPaths) + .as("should contain all created statistics") + .containsAll(statsLocations); + assertThat(deletedPaths) + .as("should contain all created partition stats files") + .containsAll(partitionStatsLocations); } } + + private FileIO createMockFileIO(InMemoryFileIO wrapped) { + InMemoryFileIO mockIO = Mockito.mock(InMemoryFileIO.class); + + Mockito.when(mockIO.newInputFile(Mockito.anyString())) + .thenAnswer(invocation -> wrapped.newInputFile((String) invocation.getArgument(0))); + Mockito.when(mockIO.newInputFile(Mockito.anyString(), Mockito.anyLong())) + .thenAnswer( + invocation -> + wrapped.newInputFile(invocation.getArgument(0), invocation.getArgument(1))); + Mockito.when(mockIO.newInputFile(Mockito.any(ManifestFile.class))) + .thenAnswer(invocation -> wrapped.newInputFile((ManifestFile) invocation.getArgument(0))); + Mockito.when(mockIO.newInputFile(Mockito.any(DataFile.class))) + .thenAnswer(invocation -> wrapped.newInputFile((DataFile) invocation.getArgument(0))); + Mockito.when(mockIO.newInputFile(Mockito.any(DeleteFile.class))) + .thenAnswer(invocation -> wrapped.newInputFile((DeleteFile) invocation.getArgument(0))); + Mockito.when(mockIO.newOutputFile(Mockito.anyString())) + .thenAnswer(invocation -> wrapped.newOutputFile((String) invocation.getArgument(0))); + + return mockIO; + } + + private Set manifestListLocations(List snapshotList) { + return snapshotList.stream().map(Snapshot::manifestListLocation).collect(Collectors.toSet()); + } + + private Set manifestLocations(List snapshotSet, FileIO io) { + return snapshotSet.stream() + .flatMap(snapshot -> snapshot.allManifests(io).stream()) + .map(ManifestFile::path) + .collect(Collectors.toSet()); + } + + private Set dataLocations(List snapshotList, FileIO io) { + return snapshotList.stream() + .flatMap(snapshot -> StreamSupport.stream(snapshot.addedDataFiles(io).spliterator(), false)) + .map(dataFile -> dataFile.path().toString()) + .collect(Collectors.toSet()); + } + + + private Set metadataLocations(TableMetadata tableMetadata) { + Set metadataLocations = + tableMetadata.previousFiles().stream() + .map(TableMetadata.MetadataLogEntry::file) + .collect(Collectors.toSet()); + metadataLocations.add(tableMetadata.metadataFileLocation()); + return metadataLocations; + } + + private Set statsLocations(TableMetadata tableMetadata) { + return tableMetadata.statisticsFiles().stream() + .map(StatisticsFile::path) + .collect(Collectors.toSet()); + } + + private Set partitionStatsLocations(TableMetadata tableMetadata) { + return tableMetadata.partitionStatisticsFiles().stream() + .map(PartitionStatisticsFile::path) + .collect(Collectors.toSet()); + } } From 0f8e8f45114e8b219086b93d413bd823832b2f3c Mon Sep 17 00:00:00 2001 From: Daniel Tu Date: Sun, 22 Sep 2024 15:56:26 -0700 Subject: [PATCH 03/29] refine warning code --- .../polaris/service/task/TableCleanupTaskHandler.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java b/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java index 5d7cc2d54..30ef5a048 100644 --- a/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java +++ b/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java @@ -167,16 +167,16 @@ public boolean handleTask(TaskEntity cleanupTask) { // remove duplication .collect(Collectors.toMap(ManifestFile::path, Function.identity(), (mf1, mf2) -> mf1)) .keySet() - .forEach(file -> fileIO.deleteFile(file)); + .forEach(fileIO::deleteFile); tableMetadata.snapshots().stream() .map(Snapshot::manifestListLocation) - .forEach(file -> fileIO.deleteFile(file)); + .forEach(fileIO::deleteFile); tableMetadata.previousFiles().stream() .map(TableMetadata.MetadataLogEntry::file) - .forEach(file -> fileIO.deleteFile(file)); + .forEach(fileIO::deleteFile); tableMetadata.statisticsFiles().stream() .map(StatisticsFile::path) - .forEach(file -> fileIO.deleteFile(file)); + .forEach(fileIO::deleteFile); fileIO.deleteFile(tableEntity.getMetadataLocation()); return true; From 1b525de2f43d294ac2ca6b5216b9baac372384e2 Mon Sep 17 00:00:00 2001 From: Daniel Tu Date: Sun, 22 Sep 2024 16:03:07 -0700 Subject: [PATCH 04/29] code format --- .../apache/polaris/service/task/TableCleanupTaskHandlerTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java b/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java index 058d35748..744b9950a 100644 --- a/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java +++ b/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java @@ -504,7 +504,6 @@ private Set dataLocations(List snapshotList, FileIO io) { .collect(Collectors.toSet()); } - private Set metadataLocations(TableMetadata tableMetadata) { Set metadataLocations = tableMetadata.previousFiles().stream() From e8b26d286cdcb094defda3d9336231697698e62c Mon Sep 17 00:00:00 2001 From: Daniel Tu Date: Sun, 22 Sep 2024 16:04:19 -0700 Subject: [PATCH 05/29] refine warning code --- .../service/task/TableCleanupTaskHandlerTest.java | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java b/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java index 744b9950a..007b6442f 100644 --- a/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java +++ b/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java @@ -481,7 +481,7 @@ private FileIO createMockFileIO(InMemoryFileIO wrapped) { Mockito.when(mockIO.newInputFile(Mockito.any(DeleteFile.class))) .thenAnswer(invocation -> wrapped.newInputFile((DeleteFile) invocation.getArgument(0))); Mockito.when(mockIO.newOutputFile(Mockito.anyString())) - .thenAnswer(invocation -> wrapped.newOutputFile((String) invocation.getArgument(0))); + .thenAnswer(invocation -> wrapped.newOutputFile(invocation.getArgument(0))); return mockIO; } @@ -497,13 +497,6 @@ private Set manifestLocations(List snapshotSet, FileIO io) { .collect(Collectors.toSet()); } - private Set dataLocations(List snapshotList, FileIO io) { - return snapshotList.stream() - .flatMap(snapshot -> StreamSupport.stream(snapshot.addedDataFiles(io).spliterator(), false)) - .map(dataFile -> dataFile.path().toString()) - .collect(Collectors.toSet()); - } - private Set metadataLocations(TableMetadata tableMetadata) { Set metadataLocations = tableMetadata.previousFiles().stream() From 2ee6deef00fc54905f301a69e1136a6d7453839a Mon Sep 17 00:00:00 2001 From: Daniel Tu Date: Sun, 22 Sep 2024 16:44:55 -0700 Subject: [PATCH 06/29] remove unused code --- .../task/TableCleanupTaskHandlerTest.java | 36 ------------------- 1 file changed, 36 deletions(-) diff --git a/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java b/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java index 007b6442f..ca0980ee5 100644 --- a/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java +++ b/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java @@ -109,13 +109,9 @@ public void testTableCleanup() throws IOException { Set manifestListLocations = manifestListLocations(tableMetadata.snapshots()); Set manifestLocations = manifestLocations(tableMetadata.snapshots(), fileIO); Set metadataLocations = metadataLocations(tableMetadata); - Set statsLocations = statsLocations(tableMetadata); - Set partitionStatsLocations = partitionStatsLocations(tableMetadata); assertThat(manifestListLocations).hasSize(1); assertThat(manifestLocations).hasSize(1); assertThat(metadataLocations).hasSize(1); - assertThat(statsLocations).hasSize(0); - assertThat(partitionStatsLocations).hasSize(0); CallContext.setCurrentContext(CallContext.of(realmContext, polarisCallContext)); handler.handleTask(task); @@ -147,8 +143,6 @@ public void testTableCleanup() throws IOException { manifestListLocations.size() + manifestLocations.size() + metadataLocations.size() - + statsLocations.size() - + partitionStatsLocations.size() ) ).deleteFile(argumentCaptor.capture()); @@ -162,12 +156,6 @@ public void testTableCleanup() throws IOException { assertThat(deletedPaths) .as("should contain all created metadata locations") .containsAll(metadataLocations); - assertThat(deletedPaths) - .as("should contain all created statistics") - .containsAll(statsLocations); - assertThat(deletedPaths) - .as("should contain all created partition stats files") - .containsAll(partitionStatsLocations); } } @@ -381,13 +369,9 @@ public void testTableCleanupMultipleSnapshots() throws IOException { Set manifestListLocations = manifestListLocations(tableMetadata.snapshots()); Set manifestLocations = manifestLocations(tableMetadata.snapshots(), fileIO); Set metadataLocations = metadataLocations(tableMetadata); - Set statsLocations = statsLocations(tableMetadata); - Set partitionStatsLocations = partitionStatsLocations(tableMetadata); assertThat(manifestListLocations).hasSize(2); assertThat(manifestLocations).hasSize(3); assertThat(metadataLocations).hasSize(1); - assertThat(statsLocations).hasSize(0); - assertThat(partitionStatsLocations).hasSize(0); handler.handleTask(task); @@ -441,8 +425,6 @@ public void testTableCleanupMultipleSnapshots() throws IOException { manifestListLocations.size() + manifestLocations.size() + metadataLocations.size() - + statsLocations.size() - + partitionStatsLocations.size() ) ).deleteFile(argumentCaptor.capture()); @@ -456,12 +438,6 @@ public void testTableCleanupMultipleSnapshots() throws IOException { assertThat(deletedPaths) .as("should contain all created metadata locations") .containsAll(metadataLocations); - assertThat(deletedPaths) - .as("should contain all created statistics") - .containsAll(statsLocations); - assertThat(deletedPaths) - .as("should contain all created partition stats files") - .containsAll(partitionStatsLocations); } } @@ -505,16 +481,4 @@ private Set metadataLocations(TableMetadata tableMetadata) { metadataLocations.add(tableMetadata.metadataFileLocation()); return metadataLocations; } - - private Set statsLocations(TableMetadata tableMetadata) { - return tableMetadata.statisticsFiles().stream() - .map(StatisticsFile::path) - .collect(Collectors.toSet()); - } - - private Set partitionStatsLocations(TableMetadata tableMetadata) { - return tableMetadata.partitionStatisticsFiles().stream() - .map(PartitionStatisticsFile::path) - .collect(Collectors.toSet()); - } } From 806f46dff94693a0fdf5193d577d0de9937bddf1 Mon Sep 17 00:00:00 2001 From: Daniel Tu Date: Sun, 22 Sep 2024 16:45:21 -0700 Subject: [PATCH 07/29] remove unused import --- .../polaris/service/task/TableCleanupTaskHandlerTest.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java b/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java index ca0980ee5..30f6f0fa7 100644 --- a/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java +++ b/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java @@ -24,16 +24,13 @@ import java.util.List; import java.util.Set; import java.util.stream.Collectors; -import java.util.stream.StreamSupport; import org.apache.commons.codec.binary.Base64; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.ManifestFile; import org.apache.iceberg.ManifestFiles; -import org.apache.iceberg.PartitionStatisticsFile; import org.apache.iceberg.Snapshot; -import org.apache.iceberg.StatisticsFile; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableMetadataParser; import org.apache.iceberg.catalog.Namespace; From 4f1d3c9634b3a268b142c9109ca80479eece87b6 Mon Sep 17 00:00:00 2001 From: Daniel Tu Date: Mon, 23 Sep 2024 09:23:19 -0700 Subject: [PATCH 08/29] code format --- .../service/task/TableCleanupTaskHandler.java | 23 +++--- .../task/TableCleanupTaskHandlerTest.java | 75 +++++++++---------- 2 files changed, 47 insertions(+), 51 deletions(-) diff --git a/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java b/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java index 30ef5a048..ae4089306 100644 --- a/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java +++ b/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java @@ -22,7 +22,6 @@ import java.util.UUID; import java.util.function.Function; import java.util.stream.Collectors; - import org.apache.iceberg.ManifestFile; import org.apache.iceberg.Snapshot; import org.apache.iceberg.StatisticsFile; @@ -163,20 +162,20 @@ public boolean handleTask(TaskEntity cleanupTask) { } tableMetadata.snapshots().stream() - .flatMap(sn -> sn.allManifests(fileIO).stream()) - // remove duplication - .collect(Collectors.toMap(ManifestFile::path, Function.identity(), (mf1, mf2) -> mf1)) - .keySet() - .forEach(fileIO::deleteFile); + .flatMap(sn -> sn.allManifests(fileIO).stream()) + // remove duplication + .collect(Collectors.toMap(ManifestFile::path, Function.identity(), (mf1, mf2) -> mf1)) + .keySet() + .forEach(fileIO::deleteFile); tableMetadata.snapshots().stream() - .map(Snapshot::manifestListLocation) - .forEach(fileIO::deleteFile); + .map(Snapshot::manifestListLocation) + .forEach(fileIO::deleteFile); tableMetadata.previousFiles().stream() - .map(TableMetadata.MetadataLogEntry::file) - .forEach(fileIO::deleteFile); + .map(TableMetadata.MetadataLogEntry::file) + .forEach(fileIO::deleteFile); tableMetadata.statisticsFiles().stream() - .map(StatisticsFile::path) - .forEach(fileIO::deleteFile); + .map(StatisticsFile::path) + .forEach(fileIO::deleteFile); fileIO.deleteFile(tableEntity.getMetadataLocation()); return true; diff --git a/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java b/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java index 30f6f0fa7..663d92ff6 100644 --- a/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java +++ b/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java @@ -24,7 +24,6 @@ import java.util.List; import java.util.Set; import java.util.stream.Collectors; - import org.apache.commons.codec.binary.Base64; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; @@ -102,7 +101,7 @@ public void testTableCleanup() throws IOException { PolarisBaseEntity baseEntity = task.readData(PolarisBaseEntity.class); TableLikeEntity tableEntity = TableLikeEntity.of(baseEntity); TableMetadata tableMetadata = - TableMetadataParser.read(fileIO, tableEntity.getMetadataLocation()); + TableMetadataParser.read(fileIO, tableEntity.getMetadataLocation()); Set manifestListLocations = manifestListLocations(tableMetadata.snapshots()); Set manifestLocations = manifestLocations(tableMetadata.snapshots(), fileIO); Set metadataLocations = metadataLocations(tableMetadata); @@ -137,22 +136,21 @@ public void testTableCleanup() throws IOException { Mockito.verify( fileIO, Mockito.times( - manifestListLocations.size() - + manifestLocations.size() - + metadataLocations.size() - ) - ).deleteFile(argumentCaptor.capture()); + manifestListLocations.size() + + manifestLocations.size() + + metadataLocations.size())) + .deleteFile(argumentCaptor.capture()); List deletedPaths = argumentCaptor.getAllValues(); assertThat(deletedPaths) - .as("should contain all created manifest lists") - .containsAll(manifestListLocations); + .as("should contain all created manifest lists") + .containsAll(manifestListLocations); assertThat(deletedPaths) - .as("should contain all created manifests") - .containsAll(manifestLocations); + .as("should contain all created manifests") + .containsAll(manifestLocations); assertThat(deletedPaths) - .as("should contain all created metadata locations") - .containsAll(metadataLocations); + .as("should contain all created metadata locations") + .containsAll(metadataLocations); } } @@ -362,7 +360,7 @@ public void testTableCleanupMultipleSnapshots() throws IOException { PolarisBaseEntity baseEntity = task.readData(PolarisBaseEntity.class); TableLikeEntity tableEntity = TableLikeEntity.of(baseEntity); TableMetadata tableMetadata = - TableMetadataParser.read(fileIO, tableEntity.getMetadataLocation()); + TableMetadataParser.read(fileIO, tableEntity.getMetadataLocation()); Set manifestListLocations = manifestListLocations(tableMetadata.snapshots()); Set manifestLocations = manifestLocations(tableMetadata.snapshots(), fileIO); Set metadataLocations = metadataLocations(tableMetadata); @@ -419,22 +417,21 @@ public void testTableCleanupMultipleSnapshots() throws IOException { Mockito.verify( fileIO, Mockito.times( - manifestListLocations.size() - + manifestLocations.size() - + metadataLocations.size() - ) - ).deleteFile(argumentCaptor.capture()); + manifestListLocations.size() + + manifestLocations.size() + + metadataLocations.size())) + .deleteFile(argumentCaptor.capture()); List deletedPaths = argumentCaptor.getAllValues(); assertThat(deletedPaths) - .as("should contain all created manifest lists") - .containsAll(manifestListLocations); + .as("should contain all created manifest lists") + .containsAll(manifestListLocations); assertThat(deletedPaths) - .as("should contain all created manifests") - .containsAll(manifestLocations); + .as("should contain all created manifests") + .containsAll(manifestLocations); assertThat(deletedPaths) - .as("should contain all created metadata locations") - .containsAll(metadataLocations); + .as("should contain all created metadata locations") + .containsAll(metadataLocations); } } @@ -442,19 +439,19 @@ private FileIO createMockFileIO(InMemoryFileIO wrapped) { InMemoryFileIO mockIO = Mockito.mock(InMemoryFileIO.class); Mockito.when(mockIO.newInputFile(Mockito.anyString())) - .thenAnswer(invocation -> wrapped.newInputFile((String) invocation.getArgument(0))); + .thenAnswer(invocation -> wrapped.newInputFile((String) invocation.getArgument(0))); Mockito.when(mockIO.newInputFile(Mockito.anyString(), Mockito.anyLong())) - .thenAnswer( - invocation -> - wrapped.newInputFile(invocation.getArgument(0), invocation.getArgument(1))); + .thenAnswer( + invocation -> + wrapped.newInputFile(invocation.getArgument(0), invocation.getArgument(1))); Mockito.when(mockIO.newInputFile(Mockito.any(ManifestFile.class))) - .thenAnswer(invocation -> wrapped.newInputFile((ManifestFile) invocation.getArgument(0))); + .thenAnswer(invocation -> wrapped.newInputFile((ManifestFile) invocation.getArgument(0))); Mockito.when(mockIO.newInputFile(Mockito.any(DataFile.class))) - .thenAnswer(invocation -> wrapped.newInputFile((DataFile) invocation.getArgument(0))); + .thenAnswer(invocation -> wrapped.newInputFile((DataFile) invocation.getArgument(0))); Mockito.when(mockIO.newInputFile(Mockito.any(DeleteFile.class))) - .thenAnswer(invocation -> wrapped.newInputFile((DeleteFile) invocation.getArgument(0))); + .thenAnswer(invocation -> wrapped.newInputFile((DeleteFile) invocation.getArgument(0))); Mockito.when(mockIO.newOutputFile(Mockito.anyString())) - .thenAnswer(invocation -> wrapped.newOutputFile(invocation.getArgument(0))); + .thenAnswer(invocation -> wrapped.newOutputFile(invocation.getArgument(0))); return mockIO; } @@ -465,16 +462,16 @@ private Set manifestListLocations(List snapshotList) { private Set manifestLocations(List snapshotSet, FileIO io) { return snapshotSet.stream() - .flatMap(snapshot -> snapshot.allManifests(io).stream()) - .map(ManifestFile::path) - .collect(Collectors.toSet()); + .flatMap(snapshot -> snapshot.allManifests(io).stream()) + .map(ManifestFile::path) + .collect(Collectors.toSet()); } private Set metadataLocations(TableMetadata tableMetadata) { Set metadataLocations = - tableMetadata.previousFiles().stream() - .map(TableMetadata.MetadataLogEntry::file) - .collect(Collectors.toSet()); + tableMetadata.previousFiles().stream() + .map(TableMetadata.MetadataLogEntry::file) + .collect(Collectors.toSet()); metadataLocations.add(tableMetadata.metadataFileLocation()); return metadataLocations; } From 0a77bfa3064f845599309cd5d401af1d7232cd43 Mon Sep 17 00:00:00 2001 From: Daniel Tu Date: Mon, 23 Sep 2024 22:49:36 -0700 Subject: [PATCH 09/29] remove additional manifest and manifest list deletion --- .../polaris/service/task/TableCleanupTaskHandler.java | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java b/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java index ae4089306..94182dcdf 100644 --- a/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java +++ b/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java @@ -23,7 +23,6 @@ import java.util.function.Function; import java.util.stream.Collectors; import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.Snapshot; import org.apache.iceberg.StatisticsFile; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableMetadataParser; @@ -161,15 +160,6 @@ public boolean handleTask(TaskEntity cleanupTask) { taskExecutor.addTaskHandlerContext(createdTask.getId(), CallContext.getCurrentContext()); } - tableMetadata.snapshots().stream() - .flatMap(sn -> sn.allManifests(fileIO).stream()) - // remove duplication - .collect(Collectors.toMap(ManifestFile::path, Function.identity(), (mf1, mf2) -> mf1)) - .keySet() - .forEach(fileIO::deleteFile); - tableMetadata.snapshots().stream() - .map(Snapshot::manifestListLocation) - .forEach(fileIO::deleteFile); tableMetadata.previousFiles().stream() .map(TableMetadata.MetadataLogEntry::file) .forEach(fileIO::deleteFile); From 47dc60a6cee137f024e5d674adbfaa0b558bce14 Mon Sep 17 00:00:00 2001 From: Daniel Tu Date: Mon, 23 Sep 2024 22:49:52 -0700 Subject: [PATCH 10/29] add stat deletion test --- .../task/TableCleanupTaskHandlerTest.java | 99 ++++++++++--------- .../polaris/service/task/TaskTestUtils.java | 21 +++- 2 files changed, 72 insertions(+), 48 deletions(-) diff --git a/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java b/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java index 663d92ff6..c7d956283 100644 --- a/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java +++ b/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java @@ -21,21 +21,30 @@ import static org.assertj.core.api.Assertions.assertThat; import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Set; +import java.util.UUID; import java.util.stream.Collectors; import org.apache.commons.codec.binary.Base64; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.GenericBlobMetadata; +import org.apache.iceberg.GenericStatisticsFile; import org.apache.iceberg.ManifestFile; import org.apache.iceberg.ManifestFiles; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StatisticsFile; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableMetadataParser; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.inmemory.InMemoryFileIO; import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.puffin.Blob; +import org.apache.iceberg.puffin.Puffin; +import org.apache.iceberg.puffin.PuffinWriter; import org.apache.polaris.core.PolarisCallContext; import org.apache.polaris.core.PolarisDefaultDiagServiceImpl; import org.apache.polaris.core.context.CallContext; @@ -83,7 +92,8 @@ public void testTableCleanup() throws IOException { TestSnapshot snapshot = TaskTestUtils.newSnapshot(fileIO, "manifestList.avro", 1, snapshotId, 99L, manifestFile); String metadataFile = "v1-49494949.metadata.json"; - TaskTestUtils.writeTableMetadata(fileIO, metadataFile, snapshot); + StatisticsFile statisticsFile = writeStatsFile(snapshot.snapshotId(), snapshot.sequenceNumber(), "/metadata/" + UUID.randomUUID() + ".stats", fileIO); + TaskTestUtils.writeTableMetadata(fileIO, metadataFile, List.of(statisticsFile), snapshot); TaskEntity task = new TaskEntity.Builder() @@ -102,12 +112,10 @@ public void testTableCleanup() throws IOException { TableLikeEntity tableEntity = TableLikeEntity.of(baseEntity); TableMetadata tableMetadata = TableMetadataParser.read(fileIO, tableEntity.getMetadataLocation()); - Set manifestListLocations = manifestListLocations(tableMetadata.snapshots()); - Set manifestLocations = manifestLocations(tableMetadata.snapshots(), fileIO); Set metadataLocations = metadataLocations(tableMetadata); - assertThat(manifestListLocations).hasSize(1); - assertThat(manifestLocations).hasSize(1); + Set statsLocation = statsLocations(tableMetadata); assertThat(metadataLocations).hasSize(1); + assertThat(statsLocation).hasSize(1); CallContext.setCurrentContext(CallContext.of(realmContext, polarisCallContext)); handler.handleTask(task); @@ -133,24 +141,16 @@ public void testTableCleanup() throws IOException { ManifestFileCleanupTaskHandler.ManifestCleanupTask.class))); ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(String.class); - Mockito.verify( - fileIO, - Mockito.times( - manifestListLocations.size() - + manifestLocations.size() - + metadataLocations.size())) + Mockito.verify(fileIO, Mockito.times(metadataLocations.size() + statsLocation.size())) .deleteFile(argumentCaptor.capture()); List deletedPaths = argumentCaptor.getAllValues(); - assertThat(deletedPaths) - .as("should contain all created manifest lists") - .containsAll(manifestListLocations); - assertThat(deletedPaths) - .as("should contain all created manifests") - .containsAll(manifestLocations); assertThat(deletedPaths) .as("should contain all created metadata locations") .containsAll(metadataLocations); + assertThat(deletedPaths) + .as("should contain all created stats locations") + .containsAll(statsLocation); } } @@ -341,7 +341,9 @@ public void testTableCleanupMultipleSnapshots() throws IOException { manifestFile1, manifestFile3); // exclude manifest2 from the new snapshot String metadataFile = "v1-295495059.metadata.json"; - TaskTestUtils.writeTableMetadata(fileIO, metadataFile, snapshot, snapshot2); + StatisticsFile statisticsFile1 = writeStatsFile(snapshot.snapshotId(), snapshot.sequenceNumber(), "/metadata/" + UUID.randomUUID() + ".stats", fileIO); + StatisticsFile statisticsFile2 = writeStatsFile(snapshot2.snapshotId(), snapshot2.sequenceNumber(), "/metadata/" + UUID.randomUUID() + ".stats", fileIO); + TaskTestUtils.writeTableMetadata(fileIO, metadataFile, List.of(statisticsFile1, statisticsFile2), snapshot, snapshot2); TaskEntity task = new TaskEntity.Builder() @@ -361,12 +363,10 @@ public void testTableCleanupMultipleSnapshots() throws IOException { TableLikeEntity tableEntity = TableLikeEntity.of(baseEntity); TableMetadata tableMetadata = TableMetadataParser.read(fileIO, tableEntity.getMetadataLocation()); - Set manifestListLocations = manifestListLocations(tableMetadata.snapshots()); - Set manifestLocations = manifestLocations(tableMetadata.snapshots(), fileIO); Set metadataLocations = metadataLocations(tableMetadata); - assertThat(manifestListLocations).hasSize(2); - assertThat(manifestLocations).hasSize(3); + Set statsLocations = statsLocations(tableMetadata); assertThat(metadataLocations).hasSize(1); + assertThat(statsLocations).hasSize(2); handler.handleTask(task); @@ -414,24 +414,16 @@ public void testTableCleanupMultipleSnapshots() throws IOException { ManifestFileCleanupTaskHandler.ManifestCleanupTask.class))); ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(String.class); - Mockito.verify( - fileIO, - Mockito.times( - manifestListLocations.size() - + manifestLocations.size() - + metadataLocations.size())) + Mockito.verify(fileIO, Mockito.times(metadataLocations.size() + statsLocations.size())) .deleteFile(argumentCaptor.capture()); List deletedPaths = argumentCaptor.getAllValues(); - assertThat(deletedPaths) - .as("should contain all created manifest lists") - .containsAll(manifestListLocations); - assertThat(deletedPaths) - .as("should contain all created manifests") - .containsAll(manifestLocations); assertThat(deletedPaths) .as("should contain all created metadata locations") .containsAll(metadataLocations); + assertThat(deletedPaths) + .as("should contain all created stats locations") + .containsAll(statsLocations); } } @@ -456,17 +448,6 @@ private FileIO createMockFileIO(InMemoryFileIO wrapped) { return mockIO; } - private Set manifestListLocations(List snapshotList) { - return snapshotList.stream().map(Snapshot::manifestListLocation).collect(Collectors.toSet()); - } - - private Set manifestLocations(List snapshotSet, FileIO io) { - return snapshotSet.stream() - .flatMap(snapshot -> snapshot.allManifests(io).stream()) - .map(ManifestFile::path) - .collect(Collectors.toSet()); - } - private Set metadataLocations(TableMetadata tableMetadata) { Set metadataLocations = tableMetadata.previousFiles().stream() @@ -475,4 +456,32 @@ private Set metadataLocations(TableMetadata tableMetadata) { metadataLocations.add(tableMetadata.metadataFileLocation()); return metadataLocations; } + + private Set statsLocations(TableMetadata tableMetadata) { + return tableMetadata.statisticsFiles().stream() + .map(StatisticsFile::path) + .collect(Collectors.toSet()); + } + + private StatisticsFile writeStatsFile( + long snapshotId, long snapshotSequenceNumber, String statsLocation, FileIO fileIO) + throws IOException { + try (PuffinWriter puffinWriter = Puffin.write(fileIO.newOutputFile(statsLocation)).build()) { + puffinWriter.add( + new Blob( + "some-blob-type", + List.of(1), + snapshotId, + snapshotSequenceNumber, + ByteBuffer.wrap("blob content".getBytes(StandardCharsets.UTF_8)))); + puffinWriter.finish(); + + return new GenericStatisticsFile( + snapshotId, + statsLocation, + puffinWriter.fileSize(), + puffinWriter.footerSize(), + puffinWriter.writtenBlobsMetadata().stream().map(GenericBlobMetadata::from).toList()); + } + } } diff --git a/polaris-service/src/test/java/org/apache/polaris/service/task/TaskTestUtils.java b/polaris-service/src/test/java/org/apache/polaris/service/task/TaskTestUtils.java index 1e5612e25..b9e81d717 100644 --- a/polaris-service/src/test/java/org/apache/polaris/service/task/TaskTestUtils.java +++ b/polaris-service/src/test/java/org/apache/polaris/service/task/TaskTestUtils.java @@ -33,6 +33,7 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StatisticsFile; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableMetadataParser; import org.apache.iceberg.avro.Avro; @@ -64,7 +65,16 @@ static ManifestFile manifestFile( static void writeTableMetadata(FileIO fileIO, String metadataFile, Snapshot... snapshots) throws IOException { - TableMetadata.Builder tmBuidler = + writeTableMetadata(fileIO, metadataFile, null, snapshots); + } + + static void writeTableMetadata( + FileIO fileIO, + String metadataFile, + List statisticsFiles, + Snapshot... snapshots) + throws IOException { + TableMetadata.Builder tmBuilder = TableMetadata.buildFromEmpty() .setLocation("path/to/table") .addSchema( @@ -74,10 +84,15 @@ static void writeTableMetadata(FileIO fileIO, String metadataFile, Snapshot... s .addSortOrder(SortOrder.unsorted()) .assignUUID(UUID.randomUUID().toString()) .addPartitionSpec(PartitionSpec.unpartitioned()); + + int statisticsFileIndex = 0; for (Snapshot snapshot : snapshots) { - tmBuidler.addSnapshot(snapshot); + tmBuilder.addSnapshot(snapshot); + if (statisticsFiles != null) { + tmBuilder.setStatistics(snapshot.snapshotId(), statisticsFiles.get(statisticsFileIndex++)); + } } - TableMetadata tableMetadata = tmBuidler.build(); + TableMetadata tableMetadata = tmBuilder.build(); PositionOutputStream out = fileIO.newOutputFile(metadataFile).createOrOverwrite(); out.write(TableMetadataParser.toJson(tableMetadata).getBytes(StandardCharsets.UTF_8)); out.close(); From 9d835b3820fe1b0882bb0bbfb65ae2679fca6feb Mon Sep 17 00:00:00 2001 From: Daniel Tu Date: Mon, 23 Sep 2024 22:58:44 -0700 Subject: [PATCH 11/29] code format --- .../task/TableCleanupTaskHandlerTest.java | 32 ++++++++++++++----- 1 file changed, 24 insertions(+), 8 deletions(-) diff --git a/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java b/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java index c7d956283..9b674c92d 100644 --- a/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java +++ b/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java @@ -92,7 +92,12 @@ public void testTableCleanup() throws IOException { TestSnapshot snapshot = TaskTestUtils.newSnapshot(fileIO, "manifestList.avro", 1, snapshotId, 99L, manifestFile); String metadataFile = "v1-49494949.metadata.json"; - StatisticsFile statisticsFile = writeStatsFile(snapshot.snapshotId(), snapshot.sequenceNumber(), "/metadata/" + UUID.randomUUID() + ".stats", fileIO); + StatisticsFile statisticsFile = + writeStatsFile( + snapshot.snapshotId(), + snapshot.sequenceNumber(), + "/metadata/" + UUID.randomUUID() + ".stats", + fileIO); TaskTestUtils.writeTableMetadata(fileIO, metadataFile, List.of(statisticsFile), snapshot); TaskEntity task = @@ -149,8 +154,8 @@ public void testTableCleanup() throws IOException { .as("should contain all created metadata locations") .containsAll(metadataLocations); assertThat(deletedPaths) - .as("should contain all created stats locations") - .containsAll(statsLocation); + .as("should contain all created stats locations") + .containsAll(statsLocation); } } @@ -341,9 +346,20 @@ public void testTableCleanupMultipleSnapshots() throws IOException { manifestFile1, manifestFile3); // exclude manifest2 from the new snapshot String metadataFile = "v1-295495059.metadata.json"; - StatisticsFile statisticsFile1 = writeStatsFile(snapshot.snapshotId(), snapshot.sequenceNumber(), "/metadata/" + UUID.randomUUID() + ".stats", fileIO); - StatisticsFile statisticsFile2 = writeStatsFile(snapshot2.snapshotId(), snapshot2.sequenceNumber(), "/metadata/" + UUID.randomUUID() + ".stats", fileIO); - TaskTestUtils.writeTableMetadata(fileIO, metadataFile, List.of(statisticsFile1, statisticsFile2), snapshot, snapshot2); + StatisticsFile statisticsFile1 = + writeStatsFile( + snapshot.snapshotId(), + snapshot.sequenceNumber(), + "/metadata/" + UUID.randomUUID() + ".stats", + fileIO); + StatisticsFile statisticsFile2 = + writeStatsFile( + snapshot2.snapshotId(), + snapshot2.sequenceNumber(), + "/metadata/" + UUID.randomUUID() + ".stats", + fileIO); + TaskTestUtils.writeTableMetadata( + fileIO, metadataFile, List.of(statisticsFile1, statisticsFile2), snapshot, snapshot2); TaskEntity task = new TaskEntity.Builder() @@ -422,8 +438,8 @@ public void testTableCleanupMultipleSnapshots() throws IOException { .as("should contain all created metadata locations") .containsAll(metadataLocations); assertThat(deletedPaths) - .as("should contain all created stats locations") - .containsAll(statsLocations); + .as("should contain all created stats locations") + .containsAll(statsLocations); } } From 40c6147993d94422cb14ba811d86e12bd0ea1000 Mon Sep 17 00:00:00 2001 From: Daniel Tu Date: Sun, 6 Oct 2024 00:20:56 -0700 Subject: [PATCH 12/29] add new AsyncTaskType --- .../java/org/apache/polaris/core/entity/AsyncTaskType.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/polaris-core/src/main/java/org/apache/polaris/core/entity/AsyncTaskType.java b/polaris-core/src/main/java/org/apache/polaris/core/entity/AsyncTaskType.java index 32c478e86..11997b1d6 100644 --- a/polaris-core/src/main/java/org/apache/polaris/core/entity/AsyncTaskType.java +++ b/polaris-core/src/main/java/org/apache/polaris/core/entity/AsyncTaskType.java @@ -23,7 +23,8 @@ public enum AsyncTaskType { ENTITY_CLEANUP_SCHEDULER(1), - FILE_CLEANUP(2); + FILE_CLEANUP(2), + TABLE_CONTENT_CLEANUP(3); private final int typeCode; From f354d1c769cdc71d9424fcae316a4f5daa1b1ec5 Mon Sep 17 00:00:00 2001 From: Daniel Tu Date: Sun, 6 Oct 2024 00:21:45 -0700 Subject: [PATCH 13/29] Schedule prev metadata and stat files deletion in seperated tasks --- .../service/task/TableCleanupTaskHandler.java | 88 +++++++++++++++++-- 1 file changed, 82 insertions(+), 6 deletions(-) diff --git a/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java b/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java index 94182dcdf..474baddef 100644 --- a/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java +++ b/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java @@ -22,6 +22,8 @@ import java.util.UUID; import java.util.function.Function; import java.util.stream.Collectors; +import java.util.stream.Stream; + import org.apache.iceberg.ManifestFile; import org.apache.iceberg.StatisticsFile; import org.apache.iceberg.TableMetadata; @@ -160,12 +162,24 @@ public boolean handleTask(TaskEntity cleanupTask) { taskExecutor.addTaskHandlerContext(createdTask.getId(), CallContext.getCurrentContext()); } - tableMetadata.previousFiles().stream() - .map(TableMetadata.MetadataLogEntry::file) - .forEach(fileIO::deleteFile); - tableMetadata.statisticsFiles().stream() - .map(StatisticsFile::path) - .forEach(fileIO::deleteFile); + // Schedule and dispatch prev metadata and stat files in seperated tasks + scheduleTableContentCleanupTask( + tableMetadata.previousFiles().stream().map(TableMetadata.MetadataLogEntry::file), + CleanupTableContentFileType.PREV_METADATA, + fileIO, + cleanupTask, + metaStoreManager, + polarisCallContext + ); + scheduleTableContentCleanupTask( + tableMetadata.statisticsFiles().stream().map(StatisticsFile::path), + CleanupTableContentFileType.STATISTICS, + fileIO, + cleanupTask, + metaStoreManager, + polarisCallContext + ); + fileIO.deleteFile(tableEntity.getMetadataLocation()); return true; @@ -173,4 +187,66 @@ public boolean handleTask(TaskEntity cleanupTask) { } return false; } + + private void scheduleTableContentCleanupTask(Stream fileStream, + CleanupTableContentFileType fileType, + FileIO fileIO, + TaskEntity cleanupTask, + PolarisMetaStoreManager metaStoreManager, + PolarisCallContext polarisCallContext) { + PolarisBaseEntity entity = cleanupTask.readData(PolarisBaseEntity.class); + TableLikeEntity tableEntity = TableLikeEntity.of(entity); + + List cleanupTaskEntities = fileStream + .filter(file -> TaskUtils.exists(file, fileIO)) + .map(file -> { + String taskName = cleanupTask.getName() + "_" + file + "_" + UUID.randomUUID(); + LOGGER.atDebug() + .addKeyValue("taskName", taskName) + .addKeyValue("tableIdentifier", tableEntity.getTableIdentifier()) + .addKeyValue("filePath", file) + .log("Queueing task to delete " + fileType.getTypeName()); + + return new TaskEntity.Builder() + .setName(taskName) + .setId(metaStoreManager.generateNewEntityId(polarisCallContext).getId()) + .setCreateTimestamp(polarisCallContext.getClock().millis()) + .withTaskType(AsyncTaskType.TABLE_CONTENT_CLEANUP) + .withData(new TableContentCleanupTaskHandler.TableContentCleanupTask( + tableEntity.getTableIdentifier(), file)) + .setInternalProperties(cleanupTask.getInternalPropertiesAsMap()) + .build(); + }) + .toList(); + + List createdTasks = metaStoreManager.createEntitiesIfNotExist( + polarisCallContext, null, cleanupTaskEntities).getEntities(); + + if (createdTasks != null) { + LOGGER.atInfo() + .addKeyValue("tableIdentifier", tableEntity.getTableIdentifier()) + .addKeyValue("taskCount", cleanupTaskEntities.size()) + .log("Successfully queued tasks to delete " + fileType.getTypeName() + "s"); + + for (PolarisBaseEntity createdTask : createdTasks) { + taskExecutor.addTaskHandlerContext(createdTask.getId(), CallContext.getCurrentContext()); + } + } + } + + private enum CleanupTableContentFileType { + PREV_METADATA("previous metadata file"), + STATISTICS("statistics file"), + ; + + private final String typeName; + + CleanupTableContentFileType(String typeName) { + this.typeName = typeName; + } + + public String getTypeName() { + return typeName; + } + } } From 88c66514d912cea4234614e8c05a2a410ee75258 Mon Sep 17 00:00:00 2001 From: Daniel Tu Date: Sun, 6 Oct 2024 00:22:19 -0700 Subject: [PATCH 14/29] Table content cleanup task handler --- .../task/TableContentCleanupTaskHandler.java | 158 ++++++++++++++++++ 1 file changed, 158 insertions(+) create mode 100644 polaris-service/src/main/java/org/apache/polaris/service/task/TableContentCleanupTaskHandler.java diff --git a/polaris-service/src/main/java/org/apache/polaris/service/task/TableContentCleanupTaskHandler.java b/polaris-service/src/main/java/org/apache/polaris/service/task/TableContentCleanupTaskHandler.java new file mode 100644 index 000000000..e3c71f2a3 --- /dev/null +++ b/polaris-service/src/main/java/org/apache/polaris/service/task/TableContentCleanupTaskHandler.java @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.polaris.service.task; + +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.io.FileIO; +import org.apache.polaris.core.entity.AsyncTaskType; +import org.apache.polaris.core.entity.TaskEntity; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Objects; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; + +/** + * {@link TaskHandler} responsible for deleting previous metadata and statistics files of a table. + */ +public class TableContentCleanupTaskHandler implements TaskHandler { + public static final int MAX_ATTEMPTS = 3; + public static final int FILE_DELETION_RETRY_MILLIS = 100; + private static final Logger LOGGER = + LoggerFactory.getLogger(ManifestFileCleanupTaskHandler.class); + private final Function fileIOSupplier; + private final ExecutorService executorService; + + public TableContentCleanupTaskHandler(Function fileIOSupplier, ExecutorService executorService) { + this.fileIOSupplier = fileIOSupplier; + this.executorService = executorService; + } + + @Override + public boolean canHandleTask(TaskEntity task) { + return task.getTaskType() == AsyncTaskType.TABLE_CONTENT_CLEANUP; + } + + @Override + public boolean handleTask(TaskEntity task) { + TableContentCleanupTask cleanupTask = task.readData(TableContentCleanupTask.class); + String filePath = cleanupTask.getFilePath(); + TableIdentifier tableId = cleanupTask.getTableId(); + try (FileIO authorizedFileIO = fileIOSupplier.apply(task)) { + if (!TaskUtils.exists(filePath, authorizedFileIO)) { + LOGGER.atWarn() + .addKeyValue("filePath", filePath) + .addKeyValue("tableId", tableId) + .log("Table content cleanup task scheduled, but the file doesn't exist"); + return true; + } + + tryDelete(tableId, authorizedFileIO, filePath, null, 1); + + return true; + } catch (Exception e) { + LOGGER.error("Error during table content cleanup for file {}", filePath, e); + return false; + } + } + + private CompletableFuture tryDelete(TableIdentifier tableId, + FileIO fileIO, + String filePath, + Throwable e, + int attempt) { + if (e != null && attempt < MAX_ATTEMPTS) { + LOGGER.atWarn() + .addKeyValue("filePath", filePath) + .addKeyValue("attempt", attempt) + .addKeyValue("error", e.getMessage()) + .log("Error encountered attempting to delete file"); + } + + if (attempt > MAX_ATTEMPTS && e != null) { + return CompletableFuture.failedFuture(e); + } + + return CompletableFuture.runAsync(() -> { + if (TaskUtils.exists(filePath, fileIO)) { + fileIO.deleteFile(filePath); + LOGGER.atInfo() + .addKeyValue("filePath", filePath) + .addKeyValue("tableId", tableId) + .log("Successfully deleted file"); + } else { + LOGGER.atInfo() + .addKeyValue("filePath", filePath) + .addKeyValue("tableId", tableId) + .log("File doesn't exist, likely already deleted"); + } + }, executorService).exceptionallyComposeAsync(newEx -> { + return tryDelete(tableId, fileIO, filePath, newEx, attempt + 1); + }, CompletableFuture.delayedExecutor(FILE_DELETION_RETRY_MILLIS, TimeUnit.MILLISECONDS, executorService)); + } + + + public static final class TableContentCleanupTask { + private TableIdentifier tableId; + private String filePath; + + public TableContentCleanupTask() {} + + public TableContentCleanupTask(TableIdentifier tableId, String filePath) { + this.tableId = tableId; + this.filePath = filePath; + } + + public TableIdentifier getTableId() { + return tableId; + } + + public void setTableId(TableIdentifier tableId) { + this.tableId = tableId; + } + + public String getFilePath() { + return filePath; + } + + public void setFilePath(String filePath) { + this.filePath = filePath; + } + + @Override + public boolean equals(Object object) { + if (this == object) { + return true; + } + if (!(object instanceof TableContentCleanupTask that)) { + return false; + } + return Objects.equals(tableId, that.tableId) && Objects.equals(filePath, that.filePath); + } + + @Override + public int hashCode() { + return Objects.hash(tableId, filePath); + } + } +} From af3efab364eb0a53fa379dfe1c774c459eb54e78 Mon Sep 17 00:00:00 2001 From: Daniel Tu Date: Sun, 6 Oct 2024 00:31:49 -0700 Subject: [PATCH 15/29] Unit test for table clean up --- .../task/TableCleanupTaskHandlerTest.java | 360 +++++++++++------- .../polaris/service/task/TaskTestUtils.java | 54 ++- 2 files changed, 277 insertions(+), 137 deletions(-) diff --git a/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java b/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java index 9b674c92d..d07c37f5d 100644 --- a/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java +++ b/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java @@ -21,30 +21,18 @@ import static org.assertj.core.api.Assertions.assertThat; import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; import java.util.List; -import java.util.Set; import java.util.UUID; -import java.util.stream.Collectors; import org.apache.commons.codec.binary.Base64; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.GenericBlobMetadata; -import org.apache.iceberg.GenericStatisticsFile; import org.apache.iceberg.ManifestFile; import org.apache.iceberg.ManifestFiles; import org.apache.iceberg.Snapshot; import org.apache.iceberg.StatisticsFile; import org.apache.iceberg.TableMetadata; -import org.apache.iceberg.TableMetadataParser; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.inmemory.InMemoryFileIO; import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.puffin.Blob; -import org.apache.iceberg.puffin.Puffin; -import org.apache.iceberg.puffin.PuffinWriter; import org.apache.polaris.core.PolarisCallContext; import org.apache.polaris.core.PolarisDefaultDiagServiceImpl; import org.apache.polaris.core.context.CallContext; @@ -58,7 +46,6 @@ import org.assertj.core.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; import org.mockito.Mockito; import org.slf4j.LoggerFactory; @@ -80,7 +67,7 @@ public void testTableCleanup() throws IOException { new PolarisDefaultDiagServiceImpl()); try (CallContext callCtx = CallContext.of(realmContext, polarisCallContext)) { CallContext.setCurrentContext(callCtx); - FileIO fileIO = createMockFileIO(new InMemoryFileIO()); + FileIO fileIO = new InMemoryFileIO(); TableIdentifier tableIdentifier = TableIdentifier.of(Namespace.of("db1", "schema1"), "table1"); TableCleanupTaskHandler handler = @@ -92,8 +79,7 @@ public void testTableCleanup() throws IOException { TestSnapshot snapshot = TaskTestUtils.newSnapshot(fileIO, "manifestList.avro", 1, snapshotId, 99L, manifestFile); String metadataFile = "v1-49494949.metadata.json"; - StatisticsFile statisticsFile = - writeStatsFile( + StatisticsFile statisticsFile = TaskTestUtils.writeStatsFile( snapshot.snapshotId(), snapshot.sequenceNumber(), "/metadata/" + UUID.randomUUID() + ".stats", @@ -113,29 +99,20 @@ public void testTableCleanup() throws IOException { .build(); Assertions.assertThatPredicate(handler::canHandleTask).accepts(task); - PolarisBaseEntity baseEntity = task.readData(PolarisBaseEntity.class); - TableLikeEntity tableEntity = TableLikeEntity.of(baseEntity); - TableMetadata tableMetadata = - TableMetadataParser.read(fileIO, tableEntity.getMetadataLocation()); - Set metadataLocations = metadataLocations(tableMetadata); - Set statsLocation = statsLocations(tableMetadata); - assertThat(metadataLocations).hasSize(1); - assertThat(statsLocation).hasSize(1); - CallContext.setCurrentContext(CallContext.of(realmContext, polarisCallContext)); handler.handleTask(task); assertThat( metaStoreManagerFactory .getOrCreateMetaStoreManager(realmContext) - .loadTasks(polarisCallContext, "test", 1) + .loadTasks(polarisCallContext, "test", 2) .getEntities()) - .hasSize(1) - .satisfiesExactly( + .hasSize(2) + .satisfiesExactlyInAnyOrder( taskEntity -> assertThat(taskEntity) .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) - .extracting(entity -> TaskEntity.of(entity)) + .extracting(TaskEntity::of) .returns(AsyncTaskType.FILE_CLEANUP, TaskEntity::getTaskType) .returns( new ManifestFileCleanupTaskHandler.ManifestCleanupTask( @@ -143,19 +120,15 @@ public void testTableCleanup() throws IOException { Base64.encodeBase64String(ManifestFiles.encode(manifestFile))), entity -> entity.readData( - ManifestFileCleanupTaskHandler.ManifestCleanupTask.class))); - - ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(String.class); - Mockito.verify(fileIO, Mockito.times(metadataLocations.size() + statsLocation.size())) - .deleteFile(argumentCaptor.capture()); - - List deletedPaths = argumentCaptor.getAllValues(); - assertThat(deletedPaths) - .as("should contain all created metadata locations") - .containsAll(metadataLocations); - assertThat(deletedPaths) - .as("should contain all created stats locations") - .containsAll(statsLocation); + ManifestFileCleanupTaskHandler.ManifestCleanupTask.class)), + taskEntity -> assertThat(taskEntity) + .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) + .extracting(TaskEntity::of) + .returns(AsyncTaskType.TABLE_CONTENT_CLEANUP, TaskEntity::getTaskType) + .returns( + new TableContentCleanupTaskHandler.TableContentCleanupTask(tableIdentifier, statisticsFile.path()), + entity -> entity.readData(TableContentCleanupTaskHandler.TableContentCleanupTask.class)) + ); } } @@ -286,7 +259,7 @@ public void close() { taskEntity -> assertThat(taskEntity) .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) - .extracting(entity -> TaskEntity.of(entity)) + .extracting(TaskEntity::of) .returns(AsyncTaskType.FILE_CLEANUP, TaskEntity::getTaskType) .returns( new ManifestFileCleanupTaskHandler.ManifestCleanupTask( @@ -298,7 +271,7 @@ public void close() { taskEntity -> assertThat(taskEntity) .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) - .extracting(entity -> TaskEntity.of(entity)) + .extracting(TaskEntity::of) .returns(AsyncTaskType.FILE_CLEANUP, TaskEntity::getTaskType) .returns( new ManifestFileCleanupTaskHandler.ManifestCleanupTask( @@ -318,7 +291,7 @@ public void testTableCleanupMultipleSnapshots() throws IOException { new PolarisDefaultDiagServiceImpl()); try (CallContext callCtx = CallContext.of(realmContext, polarisCallContext)) { CallContext.setCurrentContext(callCtx); - FileIO fileIO = createMockFileIO(new InMemoryFileIO()); + FileIO fileIO = new InMemoryFileIO(); TableIdentifier tableIdentifier = TableIdentifier.of(Namespace.of("db1", "schema1"), "table1"); TableCleanupTaskHandler handler = @@ -346,14 +319,12 @@ public void testTableCleanupMultipleSnapshots() throws IOException { manifestFile1, manifestFile3); // exclude manifest2 from the new snapshot String metadataFile = "v1-295495059.metadata.json"; - StatisticsFile statisticsFile1 = - writeStatsFile( + StatisticsFile statisticsFile1 = TaskTestUtils.writeStatsFile( snapshot.snapshotId(), snapshot.sequenceNumber(), "/metadata/" + UUID.randomUUID() + ".stats", fileIO); - StatisticsFile statisticsFile2 = - writeStatsFile( + StatisticsFile statisticsFile2 = TaskTestUtils.writeStatsFile( snapshot2.snapshotId(), snapshot2.sequenceNumber(), "/metadata/" + UUID.randomUUID() + ".stats", @@ -375,22 +346,26 @@ public void testTableCleanupMultipleSnapshots() throws IOException { CallContext.setCurrentContext(CallContext.of(realmContext, polarisCallContext)); - PolarisBaseEntity baseEntity = task.readData(PolarisBaseEntity.class); - TableLikeEntity tableEntity = TableLikeEntity.of(baseEntity); - TableMetadata tableMetadata = - TableMetadataParser.read(fileIO, tableEntity.getMetadataLocation()); - Set metadataLocations = metadataLocations(tableMetadata); - Set statsLocations = statsLocations(tableMetadata); - assertThat(metadataLocations).hasSize(1); - assertThat(statsLocations).hasSize(2); - handler.handleTask(task); - assertThat( - metaStoreManagerFactory - .getOrCreateMetaStoreManager(realmContext) - .loadTasks(polarisCallContext, "test", 5) - .getEntities()) + List entities = metaStoreManagerFactory + .getOrCreateMetaStoreManager(realmContext) + .loadTasks(polarisCallContext, "test", 5) + .getEntities(); + + List manifestCleanupTasks = entities.stream() + .filter(entity -> { + AsyncTaskType taskType = TaskEntity.of(entity).getTaskType(); + return taskType == AsyncTaskType.FILE_CLEANUP; + }).toList(); + + List StatsCleanupTasks = entities.stream() + .filter(entity -> { + AsyncTaskType taskType = TaskEntity.of(entity).getTaskType(); + return taskType == AsyncTaskType.TABLE_CONTENT_CLEANUP; + }).toList(); + + assertThat(manifestCleanupTasks) // all three manifests should be present, even though one is excluded from the latest // snapshot .hasSize(3) @@ -398,7 +373,7 @@ public void testTableCleanupMultipleSnapshots() throws IOException { taskEntity -> assertThat(taskEntity) .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) - .extracting(entity -> TaskEntity.of(entity)) + .extracting(TaskEntity::of) .returns( new ManifestFileCleanupTaskHandler.ManifestCleanupTask( tableIdentifier, @@ -409,7 +384,7 @@ public void testTableCleanupMultipleSnapshots() throws IOException { taskEntity -> assertThat(taskEntity) .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) - .extracting(entity -> TaskEntity.of(entity)) + .extracting(TaskEntity::of) .returns( new ManifestFileCleanupTaskHandler.ManifestCleanupTask( tableIdentifier, @@ -420,84 +395,207 @@ public void testTableCleanupMultipleSnapshots() throws IOException { taskEntity -> assertThat(taskEntity) .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) - .extracting(entity -> TaskEntity.of(entity)) + .extracting(TaskEntity::of) .returns( new ManifestFileCleanupTaskHandler.ManifestCleanupTask( tableIdentifier, Base64.encodeBase64String(ManifestFiles.encode(manifestFile3))), entity -> entity.readData( - ManifestFileCleanupTaskHandler.ManifestCleanupTask.class))); - - ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(String.class); - Mockito.verify(fileIO, Mockito.times(metadataLocations.size() + statsLocations.size())) - .deleteFile(argumentCaptor.capture()); - - List deletedPaths = argumentCaptor.getAllValues(); - assertThat(deletedPaths) - .as("should contain all created metadata locations") - .containsAll(metadataLocations); - assertThat(deletedPaths) - .as("should contain all created stats locations") - .containsAll(statsLocations); + ManifestFileCleanupTaskHandler.ManifestCleanupTask.class)) + ); + + assertThat(StatsCleanupTasks) + .hasSize(2) + .satisfiesExactlyInAnyOrder( + taskEntity -> + assertThat(taskEntity) + .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) + .extracting(TaskEntity::of) + .returns(AsyncTaskType.TABLE_CONTENT_CLEANUP, TaskEntity::getTaskType) + .returns( + new TableContentCleanupTaskHandler.TableContentCleanupTask( + tableIdentifier, statisticsFile1.path()), + entity -> entity.readData( + TableContentCleanupTaskHandler.TableContentCleanupTask.class)), + taskEntity -> + assertThat(taskEntity) + .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) + .extracting(TaskEntity::of) + .returns(AsyncTaskType.TABLE_CONTENT_CLEANUP, TaskEntity::getTaskType) + .returns( + new TableContentCleanupTaskHandler.TableContentCleanupTask( + tableIdentifier, statisticsFile2.path()), + entity -> entity.readData( + TableContentCleanupTaskHandler.TableContentCleanupTask.class)) + ); } } - private FileIO createMockFileIO(InMemoryFileIO wrapped) { - InMemoryFileIO mockIO = Mockito.mock(InMemoryFileIO.class); - - Mockito.when(mockIO.newInputFile(Mockito.anyString())) - .thenAnswer(invocation -> wrapped.newInputFile((String) invocation.getArgument(0))); - Mockito.when(mockIO.newInputFile(Mockito.anyString(), Mockito.anyLong())) - .thenAnswer( - invocation -> - wrapped.newInputFile(invocation.getArgument(0), invocation.getArgument(1))); - Mockito.when(mockIO.newInputFile(Mockito.any(ManifestFile.class))) - .thenAnswer(invocation -> wrapped.newInputFile((ManifestFile) invocation.getArgument(0))); - Mockito.when(mockIO.newInputFile(Mockito.any(DataFile.class))) - .thenAnswer(invocation -> wrapped.newInputFile((DataFile) invocation.getArgument(0))); - Mockito.when(mockIO.newInputFile(Mockito.any(DeleteFile.class))) - .thenAnswer(invocation -> wrapped.newInputFile((DeleteFile) invocation.getArgument(0))); - Mockito.when(mockIO.newOutputFile(Mockito.anyString())) - .thenAnswer(invocation -> wrapped.newOutputFile(invocation.getArgument(0))); - - return mockIO; - } + @Test + public void testTableCleanupMultipleMetadata() throws IOException { + PolarisCallContext polarisCallContext = + new PolarisCallContext( + metaStoreManagerFactory.getOrCreateSessionSupplier(realmContext).get(), + new PolarisDefaultDiagServiceImpl()); + try (CallContext callCtx = CallContext.of(realmContext, polarisCallContext)) { + CallContext.setCurrentContext(callCtx); + FileIO fileIO = new InMemoryFileIO(); + TableIdentifier tableIdentifier = + TableIdentifier.of(Namespace.of("db1", "schema1"), "table1"); + TableCleanupTaskHandler handler = + new TableCleanupTaskHandler(Mockito.mock(), metaStoreManagerFactory, (task) -> fileIO); + long snapshotId1 = 100L; + ManifestFile manifestFile1 = + TaskTestUtils.manifestFile( + fileIO, "manifest1.avro", snapshotId1, "dataFile1.parquet", "dataFile2.parquet"); + ManifestFile manifestFile2 = + TaskTestUtils.manifestFile( + fileIO, "manifest2.avro", snapshotId1, "dataFile3.parquet", "dataFile4.parquet"); + Snapshot snapshot = + TaskTestUtils.newSnapshot( + fileIO, "manifestList.avro", 1, snapshotId1, 99L, manifestFile1, manifestFile2); + StatisticsFile statisticsFile1 = TaskTestUtils.writeStatsFile( + snapshot.snapshotId(), + snapshot.sequenceNumber(), + "/metadata/" + UUID.randomUUID() + ".stats", + fileIO); + String firstMetadataFile = "v1-295495059.metadata.json"; + TableMetadata firstMetadata = TaskTestUtils.writeTableMetadata( + fileIO, firstMetadataFile, List.of(statisticsFile1), snapshot); + assertThat(TaskUtils.exists(firstMetadataFile, fileIO)).isTrue(); - private Set metadataLocations(TableMetadata tableMetadata) { - Set metadataLocations = - tableMetadata.previousFiles().stream() - .map(TableMetadata.MetadataLogEntry::file) - .collect(Collectors.toSet()); - metadataLocations.add(tableMetadata.metadataFileLocation()); - return metadataLocations; - } + ManifestFile manifestFile3 = + TaskTestUtils.manifestFile( + fileIO, "manifest3.avro", snapshot.snapshotId() + 1, "dataFile5.parquet"); + Snapshot snapshot2 = + TaskTestUtils.newSnapshot( + fileIO, + "manifestList2.avro", + snapshot.sequenceNumber() + 1, + snapshot.snapshotId() + 1, + snapshot.snapshotId(), + manifestFile1, + manifestFile3); // exclude manifest2 from the new snapshot + StatisticsFile statisticsFile2 = TaskTestUtils.writeStatsFile( + snapshot2.snapshotId(), + snapshot2.sequenceNumber(), + "/metadata/" + UUID.randomUUID() + ".stats", + fileIO); + String secondMetadataFile = "v1-295495060.metadata.json"; + TaskTestUtils.writeTableMetadata( + fileIO, secondMetadataFile, firstMetadata, firstMetadataFile, List.of(statisticsFile2), snapshot2); + assertThat(TaskUtils.exists(firstMetadataFile, fileIO)).isTrue(); + assertThat(TaskUtils.exists(secondMetadataFile, fileIO)).isTrue(); - private Set statsLocations(TableMetadata tableMetadata) { - return tableMetadata.statisticsFiles().stream() - .map(StatisticsFile::path) - .collect(Collectors.toSet()); - } + TaskEntity task = + new TaskEntity.Builder() + .withTaskType(AsyncTaskType.ENTITY_CLEANUP_SCHEDULER) + .withData( + new TableLikeEntity.Builder(tableIdentifier, secondMetadataFile) + .setName("table1") + .setCatalogId(1) + .setCreateTimestamp(100) + .build()) + .build(); + + Assertions.assertThatPredicate(handler::canHandleTask).accepts(task); + + CallContext.setCurrentContext(CallContext.of(realmContext, polarisCallContext)); + + handler.handleTask(task); - private StatisticsFile writeStatsFile( - long snapshotId, long snapshotSequenceNumber, String statsLocation, FileIO fileIO) - throws IOException { - try (PuffinWriter puffinWriter = Puffin.write(fileIO.newOutputFile(statsLocation)).build()) { - puffinWriter.add( - new Blob( - "some-blob-type", - List.of(1), - snapshotId, - snapshotSequenceNumber, - ByteBuffer.wrap("blob content".getBytes(StandardCharsets.UTF_8)))); - puffinWriter.finish(); - - return new GenericStatisticsFile( - snapshotId, - statsLocation, - puffinWriter.fileSize(), - puffinWriter.footerSize(), - puffinWriter.writtenBlobsMetadata().stream().map(GenericBlobMetadata::from).toList()); + List entities = metaStoreManagerFactory + .getOrCreateMetaStoreManager(realmContext) + .loadTasks(polarisCallContext, "test", 6) + .getEntities(); + + List manifestCleanupTasks = entities.stream() + .filter(entity -> { + AsyncTaskType taskType = TaskEntity.of(entity).getTaskType(); + return taskType == AsyncTaskType.FILE_CLEANUP; + }).toList(); + + List PrevMetadataNStatCleanupTasks = entities.stream() + .filter(entity -> { + AsyncTaskType taskType = TaskEntity.of(entity).getTaskType(); + return taskType == AsyncTaskType.TABLE_CONTENT_CLEANUP; + }).toList(); + + assertThat(manifestCleanupTasks) + // all three manifests should be present, even though one is excluded from the latest + // snapshot + .hasSize(3) + .satisfiesExactlyInAnyOrder( + taskEntity -> + assertThat(taskEntity) + .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) + .extracting(TaskEntity::of) + .returns( + new ManifestFileCleanupTaskHandler.ManifestCleanupTask( + tableIdentifier, + Base64.encodeBase64String(ManifestFiles.encode(manifestFile1))), + entity -> + entity.readData( + ManifestFileCleanupTaskHandler.ManifestCleanupTask.class)), + taskEntity -> + assertThat(taskEntity) + .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) + .extracting(TaskEntity::of) + .returns( + new ManifestFileCleanupTaskHandler.ManifestCleanupTask( + tableIdentifier, + Base64.encodeBase64String(ManifestFiles.encode(manifestFile2))), + entity -> + entity.readData( + ManifestFileCleanupTaskHandler.ManifestCleanupTask.class)), + taskEntity -> + assertThat(taskEntity) + .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) + .extracting(TaskEntity::of) + .returns( + new ManifestFileCleanupTaskHandler.ManifestCleanupTask( + tableIdentifier, + Base64.encodeBase64String(ManifestFiles.encode(manifestFile3))), + entity -> + entity.readData( + ManifestFileCleanupTaskHandler.ManifestCleanupTask.class)) + ); + + assertThat(PrevMetadataNStatCleanupTasks) + .hasSize(3) + .satisfiesExactlyInAnyOrder( + taskEntity -> + assertThat(taskEntity) + .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) + .extracting(TaskEntity::of) + .returns(AsyncTaskType.TABLE_CONTENT_CLEANUP, TaskEntity::getTaskType) + .returns( + new TableContentCleanupTaskHandler.TableContentCleanupTask( + tableIdentifier, statisticsFile1.path()), + entity -> entity.readData( + TableContentCleanupTaskHandler.TableContentCleanupTask.class)), + taskEntity -> + assertThat(taskEntity) + .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) + .extracting(TaskEntity::of) + .returns(AsyncTaskType.TABLE_CONTENT_CLEANUP, TaskEntity::getTaskType) + .returns( + new TableContentCleanupTaskHandler.TableContentCleanupTask( + tableIdentifier, statisticsFile2.path()), + entity -> entity.readData( + TableContentCleanupTaskHandler.TableContentCleanupTask.class)), + taskEntity -> + assertThat(taskEntity) + .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) + .extracting(TaskEntity::of) + .returns(AsyncTaskType.TABLE_CONTENT_CLEANUP, TaskEntity::getTaskType) + .returns( + new TableContentCleanupTaskHandler.TableContentCleanupTask( + tableIdentifier, firstMetadataFile), + entity -> entity.readData( + TableContentCleanupTaskHandler.TableContentCleanupTask.class)) + ); } } } diff --git a/polaris-service/src/test/java/org/apache/polaris/service/task/TaskTestUtils.java b/polaris-service/src/test/java/org/apache/polaris/service/task/TaskTestUtils.java index b9e81d717..49ab81aa9 100644 --- a/polaris-service/src/test/java/org/apache/polaris/service/task/TaskTestUtils.java +++ b/polaris-service/src/test/java/org/apache/polaris/service/task/TaskTestUtils.java @@ -19,6 +19,7 @@ package org.apache.polaris.service.task; import java.io.IOException; +import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.List; @@ -26,6 +27,8 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.GenericBlobMetadata; +import org.apache.iceberg.GenericStatisticsFile; import org.apache.iceberg.ManifestFile; import org.apache.iceberg.ManifestFiles; import org.apache.iceberg.ManifestWriter; @@ -40,6 +43,9 @@ import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.PositionOutputStream; +import org.apache.iceberg.puffin.Blob; +import org.apache.iceberg.puffin.Puffin; +import org.apache.iceberg.puffin.PuffinWriter; import org.apache.iceberg.types.Types; import org.jetbrains.annotations.NotNull; @@ -63,19 +69,32 @@ static ManifestFile manifestFile( return writer.toManifestFile(); } - static void writeTableMetadata(FileIO fileIO, String metadataFile, Snapshot... snapshots) + static TableMetadata writeTableMetadata(FileIO fileIO, String metadataFile, Snapshot... snapshots) throws IOException { - writeTableMetadata(fileIO, metadataFile, null, snapshots); + return writeTableMetadata(fileIO, metadataFile, null, null, null, snapshots); } - static void writeTableMetadata( + static TableMetadata writeTableMetadata(FileIO fileIO, String metadataFile, List statisticsFiles, Snapshot... snapshots) + throws IOException { + return writeTableMetadata(fileIO, metadataFile, null, null, statisticsFiles, snapshots); + } + + static TableMetadata writeTableMetadata( FileIO fileIO, String metadataFile, + TableMetadata prevMetadata, + String prevMetadataFile, List statisticsFiles, Snapshot... snapshots) - throws IOException { - TableMetadata.Builder tmBuilder = - TableMetadata.buildFromEmpty() + throws IOException { + TableMetadata.Builder tmBuilder; + if (prevMetadata == null) { + tmBuilder = TableMetadata.buildFromEmpty(); + } else { + tmBuilder = TableMetadata.buildFrom(prevMetadata) + .setPreviousFileLocation(prevMetadataFile); + } + tmBuilder .setLocation("path/to/table") .addSchema( new Schema( @@ -96,6 +115,7 @@ static void writeTableMetadata( PositionOutputStream out = fileIO.newOutputFile(metadataFile).createOrOverwrite(); out.write(TableMetadataParser.toJson(tableMetadata).getBytes(StandardCharsets.UTF_8)); out.close(); + return tableMetadata; } static @NotNull TestSnapshot newSnapshot( @@ -118,4 +138,26 @@ static void writeTableMetadata( new TestSnapshot(sequenceNumber, snapshotId, parentSnapshot, 1L, manifestListLocation); return snapshot; } + + static public StatisticsFile writeStatsFile( + long snapshotId, long snapshotSequenceNumber, String statsLocation, FileIO fileIO) + throws IOException { + try (PuffinWriter puffinWriter = Puffin.write(fileIO.newOutputFile(statsLocation)).build()) { + puffinWriter.add( + new Blob( + "some-blob-type", + List.of(1), + snapshotId, + snapshotSequenceNumber, + ByteBuffer.wrap("blob content".getBytes(StandardCharsets.UTF_8)))); + puffinWriter.finish(); + + return new GenericStatisticsFile( + snapshotId, + statsLocation, + puffinWriter.fileSize(), + puffinWriter.footerSize(), + puffinWriter.writtenBlobsMetadata().stream().map(GenericBlobMetadata::from).toList()); + } + } } From 278ab7e8ca0890ff7588be877eb4c08a963eab2e Mon Sep 17 00:00:00 2001 From: Daniel Tu Date: Sun, 6 Oct 2024 00:33:20 -0700 Subject: [PATCH 16/29] code format --- .../task/TableContentCleanupTaskHandler.java | 36 +++++++++---------- 1 file changed, 18 insertions(+), 18 deletions(-) diff --git a/polaris-service/src/main/java/org/apache/polaris/service/task/TableContentCleanupTaskHandler.java b/polaris-service/src/main/java/org/apache/polaris/service/task/TableContentCleanupTaskHandler.java index e3c71f2a3..a395b9fed 100644 --- a/polaris-service/src/main/java/org/apache/polaris/service/task/TableContentCleanupTaskHandler.java +++ b/polaris-service/src/main/java/org/apache/polaris/service/task/TableContentCleanupTaskHandler.java @@ -43,7 +43,8 @@ public class TableContentCleanupTaskHandler implements TaskHandler { private final Function fileIOSupplier; private final ExecutorService executorService; - public TableContentCleanupTaskHandler(Function fileIOSupplier, ExecutorService executorService) { + public TableContentCleanupTaskHandler(Function fileIOSupplier, + ExecutorService executorService) { this.fileIOSupplier = fileIOSupplier; this.executorService = executorService; } @@ -94,29 +95,28 @@ private CompletableFuture tryDelete(TableIdentifier tableId, } return CompletableFuture.runAsync(() -> { - if (TaskUtils.exists(filePath, fileIO)) { - fileIO.deleteFile(filePath); - LOGGER.atInfo() - .addKeyValue("filePath", filePath) - .addKeyValue("tableId", tableId) - .log("Successfully deleted file"); - } else { - LOGGER.atInfo() - .addKeyValue("filePath", filePath) - .addKeyValue("tableId", tableId) - .log("File doesn't exist, likely already deleted"); - } - }, executorService).exceptionallyComposeAsync(newEx -> { - return tryDelete(tableId, fileIO, filePath, newEx, attempt + 1); - }, CompletableFuture.delayedExecutor(FILE_DELETION_RETRY_MILLIS, TimeUnit.MILLISECONDS, executorService)); + if (TaskUtils.exists(filePath, fileIO)) { + fileIO.deleteFile(filePath); + LOGGER.atInfo() + .addKeyValue("filePath", filePath) + .addKeyValue("tableId", tableId) + .log("Successfully deleted file"); + } else { + LOGGER.atInfo() + .addKeyValue("filePath", filePath) + .addKeyValue("tableId", tableId) + .log("File doesn't exist, likely already deleted"); + } + }, executorService).exceptionallyComposeAsync(newEx -> tryDelete(tableId, fileIO, filePath, newEx, attempt + 1), + CompletableFuture.delayedExecutor(FILE_DELETION_RETRY_MILLIS, TimeUnit.MILLISECONDS, executorService)); } - public static final class TableContentCleanupTask { private TableIdentifier tableId; private String filePath; - public TableContentCleanupTask() {} + public TableContentCleanupTask() { + } public TableContentCleanupTask(TableIdentifier tableId, String filePath) { this.tableId = tableId; From ed30fb0ef2fcfe30eeba076e49988d4f0deed01a Mon Sep 17 00:00:00 2001 From: Daniel Tu Date: Sun, 6 Oct 2024 19:53:28 -0700 Subject: [PATCH 17/29] register task handler --- .../java/org/apache/polaris/service/PolarisApplication.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/polaris-service/src/main/java/org/apache/polaris/service/PolarisApplication.java b/polaris-service/src/main/java/org/apache/polaris/service/PolarisApplication.java index 5706fb272..016ee5b24 100644 --- a/polaris-service/src/main/java/org/apache/polaris/service/PolarisApplication.java +++ b/polaris-service/src/main/java/org/apache/polaris/service/PolarisApplication.java @@ -104,6 +104,7 @@ import org.apache.polaris.service.storage.PolarisStorageIntegrationProviderImpl; import org.apache.polaris.service.task.ManifestFileCleanupTaskHandler; import org.apache.polaris.service.task.TableCleanupTaskHandler; +import org.apache.polaris.service.task.TableContentCleanupTaskHandler; import org.apache.polaris.service.task.TaskExecutorImpl; import org.apache.polaris.service.task.TaskFileIOSupplier; import org.apache.polaris.service.throttling.StreamReadConstraintsExceptionMapper; @@ -211,6 +212,9 @@ public void run(PolarisApplicationConfig configuration, Environment environment) taskExecutor.addTaskHandler( new ManifestFileCleanupTaskHandler( fileIOSupplier, Executors.newVirtualThreadPerTaskExecutor())); + taskExecutor.addTaskHandler( + new TableContentCleanupTaskHandler( + fileIOSupplier, Executors.newVirtualThreadPerTaskExecutor())); LOGGER.info( "Initializing PolarisCallContextCatalogFactory for metaStoreManagerType {}", From 05c3dd9464da3a005ea0c2978b413eea34bb4eec Mon Sep 17 00:00:00 2001 From: Daniel Tu Date: Sun, 6 Oct 2024 20:03:45 -0700 Subject: [PATCH 18/29] handler table content files in batch --- .../service/task/TableCleanupTaskHandler.java | 60 ++++++++-------- .../task/TableContentCleanupTaskHandler.java | 69 +++++++++++++------ 2 files changed, 80 insertions(+), 49 deletions(-) diff --git a/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java b/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java index 474baddef..54b504327 100644 --- a/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java +++ b/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java @@ -52,6 +52,7 @@ public class TableCleanupTaskHandler implements TaskHandler { private final TaskExecutor taskExecutor; private final MetaStoreManagerFactory metaStoreManagerFactory; private final Function fileIOSupplier; + private static final int BATCH_SIZE = 10; public TableCleanupTaskHandler( TaskExecutor taskExecutor, @@ -197,39 +198,42 @@ private void scheduleTableContentCleanupTask(Stream fileStream, PolarisBaseEntity entity = cleanupTask.readData(PolarisBaseEntity.class); TableLikeEntity tableEntity = TableLikeEntity.of(entity); - List cleanupTaskEntities = fileStream + List validFiles = fileStream .filter(file -> TaskUtils.exists(file, fileIO)) - .map(file -> { - String taskName = cleanupTask.getName() + "_" + file + "_" + UUID.randomUUID(); - LOGGER.atDebug() - .addKeyValue("taskName", taskName) - .addKeyValue("tableIdentifier", tableEntity.getTableIdentifier()) - .addKeyValue("filePath", file) - .log("Queueing task to delete " + fileType.getTypeName()); - - return new TaskEntity.Builder() - .setName(taskName) - .setId(metaStoreManager.generateNewEntityId(polarisCallContext).getId()) - .setCreateTimestamp(polarisCallContext.getClock().millis()) - .withTaskType(AsyncTaskType.TABLE_CONTENT_CLEANUP) - .withData(new TableContentCleanupTaskHandler.TableContentCleanupTask( - tableEntity.getTableIdentifier(), file)) - .setInternalProperties(cleanupTask.getInternalPropertiesAsMap()) - .build(); - }) .toList(); - List createdTasks = metaStoreManager.createEntitiesIfNotExist( - polarisCallContext, null, cleanupTaskEntities).getEntities(); - - if (createdTasks != null) { - LOGGER.atInfo() + for (int i = 0; i < validFiles.size(); i += BATCH_SIZE) { + List fileBatch = validFiles.subList(i, Math.min(i + BATCH_SIZE, validFiles.size())); + String taskName = cleanupTask.getName() + "_batch" + i + "_" + UUID.randomUUID(); + LOGGER.atDebug() + .addKeyValue("taskName", taskName) .addKeyValue("tableIdentifier", tableEntity.getTableIdentifier()) - .addKeyValue("taskCount", cleanupTaskEntities.size()) - .log("Successfully queued tasks to delete " + fileType.getTypeName() + "s"); + .addKeyValue("fileBatch", fileBatch.toString()) + .log("Queueing task to delete a batch of " + fileType.getTypeName()); + + TaskEntity batchTask = new TaskEntity.Builder() + .setName(taskName) + .setId(metaStoreManager.generateNewEntityId(polarisCallContext).getId()) + .setCreateTimestamp(polarisCallContext.getClock().millis()) + .withTaskType(AsyncTaskType.TABLE_CONTENT_CLEANUP) + .withData(new TableContentCleanupTaskHandler.TableContentCleanupTask( + tableEntity.getTableIdentifier(), fileBatch)) + .setInternalProperties(cleanupTask.getInternalPropertiesAsMap()) + .build(); + + List createdTasks = metaStoreManager.createEntitiesIfNotExist( + polarisCallContext, null, List.of(batchTask)).getEntities(); + + if (createdTasks != null) { + LOGGER.atInfo() + .addKeyValue("tableIdentifier", tableEntity.getTableIdentifier()) + .addKeyValue("taskCount", createdTasks.size()) + .addKeyValue("fileBatch", fileBatch.toString()) + .log("Successfully queued task to delete a batch of " + fileType.getTypeName() + "s"); - for (PolarisBaseEntity createdTask : createdTasks) { - taskExecutor.addTaskHandlerContext(createdTask.getId(), CallContext.getCurrentContext()); + for (PolarisBaseEntity createdTask : createdTasks) { + taskExecutor.addTaskHandlerContext(createdTask.getId(), CallContext.getCurrentContext()); + } } } } diff --git a/polaris-service/src/main/java/org/apache/polaris/service/task/TableContentCleanupTaskHandler.java b/polaris-service/src/main/java/org/apache/polaris/service/task/TableContentCleanupTaskHandler.java index a395b9fed..10d7200d0 100644 --- a/polaris-service/src/main/java/org/apache/polaris/service/task/TableContentCleanupTaskHandler.java +++ b/polaris-service/src/main/java/org/apache/polaris/service/task/TableContentCleanupTaskHandler.java @@ -26,6 +26,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.List; import java.util.Objects; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; @@ -39,7 +40,7 @@ public class TableContentCleanupTaskHandler implements TaskHandler { public static final int MAX_ATTEMPTS = 3; public static final int FILE_DELETION_RETRY_MILLIS = 100; private static final Logger LOGGER = - LoggerFactory.getLogger(ManifestFileCleanupTaskHandler.class); + LoggerFactory.getLogger(TableContentCleanupTaskHandler.class); private final Function fileIOSupplier; private final ExecutorService executorService; @@ -57,22 +58,39 @@ public boolean canHandleTask(TaskEntity task) { @Override public boolean handleTask(TaskEntity task) { TableContentCleanupTask cleanupTask = task.readData(TableContentCleanupTask.class); - String filePath = cleanupTask.getFilePath(); + List fileBatch = cleanupTask.getFileBatch(); TableIdentifier tableId = cleanupTask.getTableId(); try (FileIO authorizedFileIO = fileIOSupplier.apply(task)) { - if (!TaskUtils.exists(filePath, authorizedFileIO)) { + List validFiles = fileBatch.stream() + .filter(file -> TaskUtils.exists(file, authorizedFileIO)) + .toList(); + if (validFiles.isEmpty()) { LOGGER.atWarn() - .addKeyValue("filePath", filePath) + .addKeyValue("taskName", task.getName()) + .addKeyValue("fileBatch", fileBatch.toString()) .addKeyValue("tableId", tableId) - .log("Table content cleanup task scheduled, but the file doesn't exist"); + .log("Table content cleanup task scheduled, but the none of the file in batch exists"); return true; } - tryDelete(tableId, authorizedFileIO, filePath, null, 1); + // Schedule the deletion for each file asynchronously + List> deleteFutures = validFiles.stream() + .map(file -> tryDelete(tableId, authorizedFileIO, file, null, 1)) + .toList(); + + // Wait for all delete operations to finish + CompletableFuture allDeletes = CompletableFuture.allOf(deleteFutures.toArray(new CompletableFuture[0])); + allDeletes.join(); + + LOGGER.atInfo() + .addKeyValue("taskName", task.getName()) + .addKeyValue("fileBatch", fileBatch.toString()) + .addKeyValue("tableId", tableId) + .log("All the files in task have been deleted"); return true; } catch (Exception e) { - LOGGER.error("Error during table content cleanup for file {}", filePath, e); + LOGGER.error("Error during table content cleanup for file batch {}", fileBatch.toString(), e); return false; } } @@ -82,7 +100,7 @@ private CompletableFuture tryDelete(TableIdentifier tableId, String filePath, Throwable e, int attempt) { - if (e != null && attempt < MAX_ATTEMPTS) { + if (e != null && attempt <= MAX_ATTEMPTS) { LOGGER.atWarn() .addKeyValue("filePath", filePath) .addKeyValue("attempt", attempt) @@ -100,27 +118,36 @@ private CompletableFuture tryDelete(TableIdentifier tableId, LOGGER.atInfo() .addKeyValue("filePath", filePath) .addKeyValue("tableId", tableId) - .log("Successfully deleted file"); + .addKeyValue("attempt", attempt) + .log("Successfully deleted file {}", filePath); } else { LOGGER.atInfo() .addKeyValue("filePath", filePath) .addKeyValue("tableId", tableId) .log("File doesn't exist, likely already deleted"); } - }, executorService).exceptionallyComposeAsync(newEx -> tryDelete(tableId, fileIO, filePath, newEx, attempt + 1), - CompletableFuture.delayedExecutor(FILE_DELETION_RETRY_MILLIS, TimeUnit.MILLISECONDS, executorService)); + }, executorService).exceptionallyComposeAsync( + newEx -> { + LOGGER.atWarn() + .addKeyValue("filePath", filePath) + .addKeyValue("tableId", tableId) + .log("Exception caught deleting table content file", newEx); + return tryDelete(tableId, fileIO, filePath, newEx, attempt + 1); + }, + CompletableFuture.delayedExecutor(FILE_DELETION_RETRY_MILLIS, TimeUnit.MILLISECONDS, executorService) + ); } public static final class TableContentCleanupTask { private TableIdentifier tableId; - private String filePath; + private List fileBatch; public TableContentCleanupTask() { } - public TableContentCleanupTask(TableIdentifier tableId, String filePath) { + public TableContentCleanupTask(TableIdentifier tableId, List fileBatch) { this.tableId = tableId; - this.filePath = filePath; + this.fileBatch = fileBatch; } public TableIdentifier getTableId() { @@ -131,12 +158,12 @@ public void setTableId(TableIdentifier tableId) { this.tableId = tableId; } - public String getFilePath() { - return filePath; + public List getFileBatch() { + return fileBatch; } - public void setFilePath(String filePath) { - this.filePath = filePath; + public void setFileBatch(List fileBatch) { + this.fileBatch = fileBatch; } @Override @@ -144,15 +171,15 @@ public boolean equals(Object object) { if (this == object) { return true; } - if (!(object instanceof TableContentCleanupTask that)) { + if (!(object instanceof TableContentCleanupTask other)) { return false; } - return Objects.equals(tableId, that.tableId) && Objects.equals(filePath, that.filePath); + return Objects.equals(tableId, other.tableId) && Objects.equals(fileBatch, other.fileBatch); } @Override public int hashCode() { - return Objects.hash(tableId, filePath); + return Objects.hash(tableId, fileBatch.toString()); } } } From 49dbe682c9e32c22e5e60f735ca0d98b1bd9d9da Mon Sep 17 00:00:00 2001 From: Daniel Tu Date: Sun, 6 Oct 2024 20:04:04 -0700 Subject: [PATCH 19/29] adjust unit test after batch processing --- .../task/TableCleanupTaskHandlerTest.java | 32 ++++--------------- 1 file changed, 6 insertions(+), 26 deletions(-) diff --git a/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java b/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java index d07c37f5d..32b6df464 100644 --- a/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java +++ b/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java @@ -126,7 +126,7 @@ public void testTableCleanup() throws IOException { .extracting(TaskEntity::of) .returns(AsyncTaskType.TABLE_CONTENT_CLEANUP, TaskEntity::getTaskType) .returns( - new TableContentCleanupTaskHandler.TableContentCleanupTask(tableIdentifier, statisticsFile.path()), + new TableContentCleanupTaskHandler.TableContentCleanupTask(tableIdentifier, List.of(statisticsFile.path())), entity -> entity.readData(TableContentCleanupTaskHandler.TableContentCleanupTask.class)) ); } @@ -406,7 +406,7 @@ public void testTableCleanupMultipleSnapshots() throws IOException { ); assertThat(StatsCleanupTasks) - .hasSize(2) + .hasSize(1) .satisfiesExactlyInAnyOrder( taskEntity -> assertThat(taskEntity) @@ -415,17 +415,7 @@ public void testTableCleanupMultipleSnapshots() throws IOException { .returns(AsyncTaskType.TABLE_CONTENT_CLEANUP, TaskEntity::getTaskType) .returns( new TableContentCleanupTaskHandler.TableContentCleanupTask( - tableIdentifier, statisticsFile1.path()), - entity -> entity.readData( - TableContentCleanupTaskHandler.TableContentCleanupTask.class)), - taskEntity -> - assertThat(taskEntity) - .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) - .extracting(TaskEntity::of) - .returns(AsyncTaskType.TABLE_CONTENT_CLEANUP, TaskEntity::getTaskType) - .returns( - new TableContentCleanupTaskHandler.TableContentCleanupTask( - tableIdentifier, statisticsFile2.path()), + tableIdentifier, List.of(statisticsFile1.path(), statisticsFile2.path())), entity -> entity.readData( TableContentCleanupTaskHandler.TableContentCleanupTask.class)) ); @@ -563,7 +553,7 @@ public void testTableCleanupMultipleMetadata() throws IOException { ); assertThat(PrevMetadataNStatCleanupTasks) - .hasSize(3) + .hasSize(2) .satisfiesExactlyInAnyOrder( taskEntity -> assertThat(taskEntity) @@ -572,17 +562,7 @@ public void testTableCleanupMultipleMetadata() throws IOException { .returns(AsyncTaskType.TABLE_CONTENT_CLEANUP, TaskEntity::getTaskType) .returns( new TableContentCleanupTaskHandler.TableContentCleanupTask( - tableIdentifier, statisticsFile1.path()), - entity -> entity.readData( - TableContentCleanupTaskHandler.TableContentCleanupTask.class)), - taskEntity -> - assertThat(taskEntity) - .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) - .extracting(TaskEntity::of) - .returns(AsyncTaskType.TABLE_CONTENT_CLEANUP, TaskEntity::getTaskType) - .returns( - new TableContentCleanupTaskHandler.TableContentCleanupTask( - tableIdentifier, statisticsFile2.path()), + tableIdentifier, List.of(statisticsFile1.path(), statisticsFile2.path())), entity -> entity.readData( TableContentCleanupTaskHandler.TableContentCleanupTask.class)), taskEntity -> @@ -592,7 +572,7 @@ public void testTableCleanupMultipleMetadata() throws IOException { .returns(AsyncTaskType.TABLE_CONTENT_CLEANUP, TaskEntity::getTaskType) .returns( new TableContentCleanupTaskHandler.TableContentCleanupTask( - tableIdentifier, firstMetadataFile), + tableIdentifier, List.of(firstMetadataFile)), entity -> entity.readData( TableContentCleanupTaskHandler.TableContentCleanupTask.class)) ); From 8eea50d8f5cf3c42e05f78c7faff70a96465457e Mon Sep 17 00:00:00 2001 From: Daniel Tu Date: Sun, 6 Oct 2024 20:04:18 -0700 Subject: [PATCH 20/29] add unit test for TableContentCleanupTaskHandler --- .../TableContentCleanupTaskHandlerTest.java | 269 ++++++++++++++++++ 1 file changed, 269 insertions(+) create mode 100644 polaris-service/src/test/java/org/apache/polaris/service/task/TableContentCleanupTaskHandlerTest.java diff --git a/polaris-service/src/test/java/org/apache/polaris/service/task/TableContentCleanupTaskHandlerTest.java b/polaris-service/src/test/java/org/apache/polaris/service/task/TableContentCleanupTaskHandlerTest.java new file mode 100644 index 000000000..bf879a3ae --- /dev/null +++ b/polaris-service/src/test/java/org/apache/polaris/service/task/TableContentCleanupTaskHandlerTest.java @@ -0,0 +1,269 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.polaris.service.task; + +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StatisticsFile; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.inmemory.InMemoryFileIO; +import org.apache.iceberg.io.FileIO; +import org.apache.polaris.core.PolarisCallContext; +import org.apache.polaris.core.PolarisDefaultDiagServiceImpl; +import org.apache.polaris.core.context.CallContext; +import org.apache.polaris.core.context.RealmContext; +import org.apache.polaris.core.entity.AsyncTaskType; +import org.apache.polaris.core.entity.TaskEntity; +import org.apache.polaris.service.persistence.InMemoryPolarisMetaStoreManagerFactory; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Stream; + +import static org.assertj.core.api.Assertions.assertThatPredicate; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +public class TableContentCleanupTaskHandlerTest { + private InMemoryPolarisMetaStoreManagerFactory metaStoreManagerFactory; + private RealmContext realmContext; + private static final Logger LOGGER = + LoggerFactory.getLogger(TableContentCleanupTaskHandlerTest.class); + + @BeforeEach + void setUp() { + metaStoreManagerFactory = new InMemoryPolarisMetaStoreManagerFactory(); + realmContext = () -> "realmName"; + } + + @Test + public void testCleanup() throws IOException { + PolarisCallContext polarisCallContext = + new PolarisCallContext( + metaStoreManagerFactory.getOrCreateSessionSupplier(realmContext).get(), + new PolarisDefaultDiagServiceImpl()); + try (CallContext callCtx = CallContext.of(realmContext, polarisCallContext)) { + CallContext.setCurrentContext(callCtx); + FileIO fileIO = new InMemoryFileIO() { + @Override + public void close() { + // no-op + } + }; + TableIdentifier tableIdentifier = + TableIdentifier.of(Namespace.of("db1", "schema1"), "table1"); + ExecutorService executorService = Executors.newCachedThreadPool(); + TableContentCleanupTaskHandler handler = + new TableContentCleanupTaskHandler((task) -> fileIO, executorService); + + long snapshotId1 = 100L; + ManifestFile manifestFile1 = + TaskTestUtils.manifestFile( + fileIO, "manifest1.avro", snapshotId1, "dataFile1.parquet", "dataFile2.parquet"); + ManifestFile manifestFile2 = + TaskTestUtils.manifestFile( + fileIO, "manifest2.avro", snapshotId1, "dataFile3.parquet", "dataFile4.parquet"); + Snapshot snapshot = + TaskTestUtils.newSnapshot( + fileIO, "manifestList.avro", 1, snapshotId1, 99L, manifestFile1, manifestFile2); + StatisticsFile statisticsFile1 = TaskTestUtils.writeStatsFile( + snapshot.snapshotId(), + snapshot.sequenceNumber(), + "/metadata/" + UUID.randomUUID() + ".stats", + fileIO); + String firstMetadataFile = "v1-295495059.metadata.json"; + TableMetadata firstMetadata = TaskTestUtils.writeTableMetadata( + fileIO, firstMetadataFile, List.of(statisticsFile1), snapshot); + assertThat(TaskUtils.exists(firstMetadataFile, fileIO)).isTrue(); + + ManifestFile manifestFile3 = + TaskTestUtils.manifestFile( + fileIO, "manifest3.avro", snapshot.snapshotId() + 1, "dataFile5.parquet"); + Snapshot snapshot2 = + TaskTestUtils.newSnapshot( + fileIO, + "manifestList2.avro", + snapshot.sequenceNumber() + 1, + snapshot.snapshotId() + 1, + snapshot.snapshotId(), + manifestFile1, + manifestFile3); // exclude manifest2 from the new snapshot + StatisticsFile statisticsFile2 = TaskTestUtils.writeStatsFile( + snapshot2.snapshotId(), + snapshot2.sequenceNumber(), + "/metadata/" + UUID.randomUUID() + ".stats", + fileIO); + String secondMetadataFile = "v1-295495060.metadata.json"; + TableMetadata secondMetadata = TaskTestUtils.writeTableMetadata( + fileIO, secondMetadataFile, firstMetadata, firstMetadataFile, List.of(statisticsFile2), snapshot2); + assertThat(TaskUtils.exists(firstMetadataFile, fileIO)).isTrue(); + assertThat(TaskUtils.exists(secondMetadataFile, fileIO)).isTrue(); + + List cleanupFiles = Stream.concat( + secondMetadata.previousFiles() + .stream() + .map(TableMetadata.MetadataLogEntry::file) + .filter(file -> TaskUtils.exists(file, fileIO)), + secondMetadata.statisticsFiles() + .stream() + .map(StatisticsFile::path) + .filter(file -> TaskUtils.exists(file, fileIO)) + ).toList(); + + TaskEntity task = new TaskEntity.Builder() + .withTaskType(AsyncTaskType.TABLE_CONTENT_CLEANUP) + .withData(new TableContentCleanupTaskHandler.TableContentCleanupTask(tableIdentifier, cleanupFiles)) + .setName(UUID.randomUUID().toString()) + .build(); + + assertThatPredicate(handler::canHandleTask).accepts(task); + assertThatPredicate(handler::handleTask).accepts(task); + + assertThatPredicate((String file) -> TaskUtils.exists(file, fileIO)).rejects(firstMetadataFile); + assertThatPredicate((String file) -> TaskUtils.exists(file, fileIO)).rejects(statisticsFile1.path()); + assertThatPredicate((String file) -> TaskUtils.exists(file, fileIO)).rejects(statisticsFile2.path()); + } + } + + @Test + public void testCleanupIfFileNotExist() throws IOException { + PolarisCallContext polarisCallContext = + new PolarisCallContext( + metaStoreManagerFactory.getOrCreateSessionSupplier(realmContext).get(), + new PolarisDefaultDiagServiceImpl()); + try (CallContext callCtx = CallContext.of(realmContext, polarisCallContext)) { + CallContext.setCurrentContext(callCtx); + FileIO fileIO = new InMemoryFileIO(); + TableIdentifier tableIdentifier = + TableIdentifier.of(Namespace.of("db1", "schema1"), "table1"); + TableContentCleanupTaskHandler handler = + new TableContentCleanupTaskHandler((task) -> fileIO, Executors.newCachedThreadPool()); + long snapshotId = 100L; + ManifestFile manifestFile = + TaskTestUtils.manifestFile( + fileIO, "manifest1.avro", snapshotId, "dataFile1.parquet", "dataFile2.parquet"); + TestSnapshot snapshot = + TaskTestUtils.newSnapshot(fileIO, "manifestList.avro", 1, snapshotId, 99L, manifestFile); + String metadataFile = "v1-49494949.metadata.json"; + StatisticsFile statisticsFile = TaskTestUtils.writeStatsFile( + snapshot.snapshotId(), + snapshot.sequenceNumber(), + "/metadata/" + UUID.randomUUID() + ".stats", + fileIO); + TaskTestUtils.writeTableMetadata(fileIO, metadataFile, List.of(statisticsFile), snapshot); + + fileIO.deleteFile(statisticsFile.path()); + assertThat(TaskUtils.exists(statisticsFile.path(), fileIO)).isFalse(); + + TaskEntity task = new TaskEntity.Builder() + .withTaskType(AsyncTaskType.TABLE_CONTENT_CLEANUP) + .withData(new TableContentCleanupTaskHandler.TableContentCleanupTask(tableIdentifier, List.of(statisticsFile.path()))) + .setName(UUID.randomUUID().toString()) + .build(); + assertThatPredicate(handler::canHandleTask).accepts(task); + assertThatPredicate(handler::handleTask).accepts(task); + } + } + + @Test + public void testCleanupWithRetries() throws IOException { + PolarisCallContext polarisCallContext = + new PolarisCallContext( + metaStoreManagerFactory.getOrCreateSessionSupplier(realmContext).get(), + new PolarisDefaultDiagServiceImpl()); + try (CallContext callCtx = CallContext.of(realmContext, polarisCallContext)) { + CallContext.setCurrentContext(callCtx); + Map retryCounter = new HashMap<>(); + FileIO fileIO = new InMemoryFileIO() { + @Override + public void close() { + // no-op + } + + @Override + public void deleteFile(String location) { + int attempts = retryCounter + .computeIfAbsent(location, k -> new AtomicInteger(0)) + .incrementAndGet(); + LOGGER.info("Attempt {} to delete file: {}", attempts, location); + if (attempts < 3) { + LOGGER.warn("Simulating failure for retry {}", attempts); + throw new RuntimeException("Simulating failure to test retries"); + } else { + // Succeed on the third attempt + LOGGER.info("Actually delete file: {}", location); + super.deleteFile(location); + } + } + }; + TableIdentifier tableIdentifier = + TableIdentifier.of(Namespace.of("db1", "schema1"), "table1"); + TableContentCleanupTaskHandler handler = + new TableContentCleanupTaskHandler((task) -> fileIO, Executors.newCachedThreadPool()); + long snapshotId = 100L; + ManifestFile manifestFile = + TaskTestUtils.manifestFile( + fileIO, "manifest1.avro", snapshotId, "dataFile1.parquet", "dataFile2.parquet"); + TestSnapshot snapshot = + TaskTestUtils.newSnapshot(fileIO, "manifestList.avro", 1, snapshotId, 99L, manifestFile); + String metadataFile = "v1-49494949.metadata.json"; + StatisticsFile statisticsFile = TaskTestUtils.writeStatsFile( + snapshot.snapshotId(), + snapshot.sequenceNumber(), + "/metadata/" + UUID.randomUUID() + ".stats", + fileIO); + TaskTestUtils.writeTableMetadata(fileIO, metadataFile, List.of(statisticsFile), snapshot); + assertThat(TaskUtils.exists(statisticsFile.path(), fileIO)).isTrue(); + + TaskEntity task = new TaskEntity.Builder() + .withTaskType(AsyncTaskType.TABLE_CONTENT_CLEANUP) + .withData(new TableContentCleanupTaskHandler.TableContentCleanupTask(tableIdentifier, List.of(statisticsFile.path()))) + .setName(UUID.randomUUID().toString()) + .build(); + + CompletableFuture future = CompletableFuture.runAsync(() -> { + assertThatPredicate(handler::canHandleTask).accepts(task); + handler.handleTask(task); // this will schedule the batch deletion + }); + + // Wait for all async tasks to finish + future.join(); + + // Check if the file was successfully deleted after retries + assertThat(TaskUtils.exists(statisticsFile.path(), fileIO)).isFalse(); + + // Ensure that retries happened as expected + assertThat(retryCounter.containsKey(statisticsFile.path())).isTrue(); + assertThat(retryCounter.get(statisticsFile.path()).get()).isEqualTo(3); + } + } +} From d9804e6de17dac0e50b965d83283f4f93ed79df0 Mon Sep 17 00:00:00 2001 From: Daniel Tu Date: Sun, 6 Oct 2024 20:11:28 -0700 Subject: [PATCH 21/29] code format --- .../polaris/service/PolarisApplication.java | 4 +- .../service/task/TableCleanupTaskHandler.java | 78 ++-- .../task/TableContentCleanupTaskHandler.java | 284 ++++++------ .../task/TableCleanupTaskHandlerTest.java | 319 +++++++------ .../TableContentCleanupTaskHandlerTest.java | 433 +++++++++--------- .../polaris/service/task/TaskTestUtils.java | 56 +-- 6 files changed, 618 insertions(+), 556 deletions(-) diff --git a/polaris-service/src/main/java/org/apache/polaris/service/PolarisApplication.java b/polaris-service/src/main/java/org/apache/polaris/service/PolarisApplication.java index 016ee5b24..2558414c5 100644 --- a/polaris-service/src/main/java/org/apache/polaris/service/PolarisApplication.java +++ b/polaris-service/src/main/java/org/apache/polaris/service/PolarisApplication.java @@ -213,8 +213,8 @@ public void run(PolarisApplicationConfig configuration, Environment environment) new ManifestFileCleanupTaskHandler( fileIOSupplier, Executors.newVirtualThreadPerTaskExecutor())); taskExecutor.addTaskHandler( - new TableContentCleanupTaskHandler( - fileIOSupplier, Executors.newVirtualThreadPerTaskExecutor())); + new TableContentCleanupTaskHandler( + fileIOSupplier, Executors.newVirtualThreadPerTaskExecutor())); LOGGER.info( "Initializing PolarisCallContextCatalogFactory for metaStoreManagerType {}", diff --git a/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java b/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java index 54b504327..aa720fe26 100644 --- a/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java +++ b/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java @@ -23,7 +23,6 @@ import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.Stream; - import org.apache.iceberg.ManifestFile; import org.apache.iceberg.StatisticsFile; import org.apache.iceberg.TableMetadata; @@ -165,21 +164,19 @@ public boolean handleTask(TaskEntity cleanupTask) { // Schedule and dispatch prev metadata and stat files in seperated tasks scheduleTableContentCleanupTask( - tableMetadata.previousFiles().stream().map(TableMetadata.MetadataLogEntry::file), - CleanupTableContentFileType.PREV_METADATA, - fileIO, - cleanupTask, - metaStoreManager, - polarisCallContext - ); + tableMetadata.previousFiles().stream().map(TableMetadata.MetadataLogEntry::file), + CleanupTableContentFileType.PREV_METADATA, + fileIO, + cleanupTask, + metaStoreManager, + polarisCallContext); scheduleTableContentCleanupTask( - tableMetadata.statisticsFiles().stream().map(StatisticsFile::path), - CleanupTableContentFileType.STATISTICS, - fileIO, - cleanupTask, - metaStoreManager, - polarisCallContext - ); + tableMetadata.statisticsFiles().stream().map(StatisticsFile::path), + CleanupTableContentFileType.STATISTICS, + fileIO, + cleanupTask, + metaStoreManager, + polarisCallContext); fileIO.deleteFile(tableEntity.getMetadataLocation()); @@ -189,47 +186,52 @@ public boolean handleTask(TaskEntity cleanupTask) { return false; } - private void scheduleTableContentCleanupTask(Stream fileStream, - CleanupTableContentFileType fileType, - FileIO fileIO, - TaskEntity cleanupTask, - PolarisMetaStoreManager metaStoreManager, - PolarisCallContext polarisCallContext) { + private void scheduleTableContentCleanupTask( + Stream fileStream, + CleanupTableContentFileType fileType, + FileIO fileIO, + TaskEntity cleanupTask, + PolarisMetaStoreManager metaStoreManager, + PolarisCallContext polarisCallContext) { PolarisBaseEntity entity = cleanupTask.readData(PolarisBaseEntity.class); TableLikeEntity tableEntity = TableLikeEntity.of(entity); - List validFiles = fileStream - .filter(file -> TaskUtils.exists(file, fileIO)) - .toList(); + List validFiles = fileStream.filter(file -> TaskUtils.exists(file, fileIO)).toList(); for (int i = 0; i < validFiles.size(); i += BATCH_SIZE) { List fileBatch = validFiles.subList(i, Math.min(i + BATCH_SIZE, validFiles.size())); String taskName = cleanupTask.getName() + "_batch" + i + "_" + UUID.randomUUID(); - LOGGER.atDebug() - .addKeyValue("taskName", taskName) - .addKeyValue("tableIdentifier", tableEntity.getTableIdentifier()) - .addKeyValue("fileBatch", fileBatch.toString()) - .log("Queueing task to delete a batch of " + fileType.getTypeName()); + LOGGER + .atDebug() + .addKeyValue("taskName", taskName) + .addKeyValue("tableIdentifier", tableEntity.getTableIdentifier()) + .addKeyValue("fileBatch", fileBatch.toString()) + .log("Queueing task to delete a batch of " + fileType.getTypeName()); - TaskEntity batchTask = new TaskEntity.Builder() + TaskEntity batchTask = + new TaskEntity.Builder() .setName(taskName) .setId(metaStoreManager.generateNewEntityId(polarisCallContext).getId()) .setCreateTimestamp(polarisCallContext.getClock().millis()) .withTaskType(AsyncTaskType.TABLE_CONTENT_CLEANUP) - .withData(new TableContentCleanupTaskHandler.TableContentCleanupTask( + .withData( + new TableContentCleanupTaskHandler.TableContentCleanupTask( tableEntity.getTableIdentifier(), fileBatch)) .setInternalProperties(cleanupTask.getInternalPropertiesAsMap()) .build(); - List createdTasks = metaStoreManager.createEntitiesIfNotExist( - polarisCallContext, null, List.of(batchTask)).getEntities(); + List createdTasks = + metaStoreManager + .createEntitiesIfNotExist(polarisCallContext, null, List.of(batchTask)) + .getEntities(); if (createdTasks != null) { - LOGGER.atInfo() - .addKeyValue("tableIdentifier", tableEntity.getTableIdentifier()) - .addKeyValue("taskCount", createdTasks.size()) - .addKeyValue("fileBatch", fileBatch.toString()) - .log("Successfully queued task to delete a batch of " + fileType.getTypeName() + "s"); + LOGGER + .atInfo() + .addKeyValue("tableIdentifier", tableEntity.getTableIdentifier()) + .addKeyValue("taskCount", createdTasks.size()) + .addKeyValue("fileBatch", fileBatch.toString()) + .log("Successfully queued task to delete a batch of " + fileType.getTypeName() + "s"); for (PolarisBaseEntity createdTask : createdTasks) { taskExecutor.addTaskHandlerContext(createdTask.getId(), CallContext.getCurrentContext()); diff --git a/polaris-service/src/main/java/org/apache/polaris/service/task/TableContentCleanupTaskHandler.java b/polaris-service/src/main/java/org/apache/polaris/service/task/TableContentCleanupTaskHandler.java index 10d7200d0..300f3d202 100644 --- a/polaris-service/src/main/java/org/apache/polaris/service/task/TableContentCleanupTaskHandler.java +++ b/polaris-service/src/main/java/org/apache/polaris/service/task/TableContentCleanupTaskHandler.java @@ -16,170 +16,174 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.polaris.service.task; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.io.FileIO; -import org.apache.polaris.core.entity.AsyncTaskType; -import org.apache.polaris.core.entity.TaskEntity; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.util.List; import java.util.Objects; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; import java.util.function.Function; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.io.FileIO; +import org.apache.polaris.core.entity.AsyncTaskType; +import org.apache.polaris.core.entity.TaskEntity; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * {@link TaskHandler} responsible for deleting previous metadata and statistics files of a table. */ public class TableContentCleanupTaskHandler implements TaskHandler { - public static final int MAX_ATTEMPTS = 3; - public static final int FILE_DELETION_RETRY_MILLIS = 100; - private static final Logger LOGGER = - LoggerFactory.getLogger(TableContentCleanupTaskHandler.class); - private final Function fileIOSupplier; - private final ExecutorService executorService; - - public TableContentCleanupTaskHandler(Function fileIOSupplier, - ExecutorService executorService) { - this.fileIOSupplier = fileIOSupplier; - this.executorService = executorService; + public static final int MAX_ATTEMPTS = 3; + public static final int FILE_DELETION_RETRY_MILLIS = 100; + private static final Logger LOGGER = + LoggerFactory.getLogger(TableContentCleanupTaskHandler.class); + private final Function fileIOSupplier; + private final ExecutorService executorService; + + public TableContentCleanupTaskHandler( + Function fileIOSupplier, ExecutorService executorService) { + this.fileIOSupplier = fileIOSupplier; + this.executorService = executorService; + } + + @Override + public boolean canHandleTask(TaskEntity task) { + return task.getTaskType() == AsyncTaskType.TABLE_CONTENT_CLEANUP; + } + + @Override + public boolean handleTask(TaskEntity task) { + TableContentCleanupTask cleanupTask = task.readData(TableContentCleanupTask.class); + List fileBatch = cleanupTask.getFileBatch(); + TableIdentifier tableId = cleanupTask.getTableId(); + try (FileIO authorizedFileIO = fileIOSupplier.apply(task)) { + List validFiles = + fileBatch.stream().filter(file -> TaskUtils.exists(file, authorizedFileIO)).toList(); + if (validFiles.isEmpty()) { + LOGGER + .atWarn() + .addKeyValue("taskName", task.getName()) + .addKeyValue("fileBatch", fileBatch.toString()) + .addKeyValue("tableId", tableId) + .log("Table content cleanup task scheduled, but the none of the file in batch exists"); + return true; + } + + // Schedule the deletion for each file asynchronously + List> deleteFutures = + validFiles.stream() + .map(file -> tryDelete(tableId, authorizedFileIO, file, null, 1)) + .toList(); + + // Wait for all delete operations to finish + CompletableFuture allDeletes = + CompletableFuture.allOf(deleteFutures.toArray(new CompletableFuture[0])); + allDeletes.join(); + + LOGGER + .atInfo() + .addKeyValue("taskName", task.getName()) + .addKeyValue("fileBatch", fileBatch.toString()) + .addKeyValue("tableId", tableId) + .log("All the files in task have been deleted"); + + return true; + } catch (Exception e) { + LOGGER.error("Error during table content cleanup for file batch {}", fileBatch.toString(), e); + return false; + } + } + + private CompletableFuture tryDelete( + TableIdentifier tableId, FileIO fileIO, String filePath, Throwable e, int attempt) { + if (e != null && attempt <= MAX_ATTEMPTS) { + LOGGER + .atWarn() + .addKeyValue("filePath", filePath) + .addKeyValue("attempt", attempt) + .addKeyValue("error", e.getMessage()) + .log("Error encountered attempting to delete file"); } - @Override - public boolean canHandleTask(TaskEntity task) { - return task.getTaskType() == AsyncTaskType.TABLE_CONTENT_CLEANUP; + if (attempt > MAX_ATTEMPTS && e != null) { + return CompletableFuture.failedFuture(e); } - @Override - public boolean handleTask(TaskEntity task) { - TableContentCleanupTask cleanupTask = task.readData(TableContentCleanupTask.class); - List fileBatch = cleanupTask.getFileBatch(); - TableIdentifier tableId = cleanupTask.getTableId(); - try (FileIO authorizedFileIO = fileIOSupplier.apply(task)) { - List validFiles = fileBatch.stream() - .filter(file -> TaskUtils.exists(file, authorizedFileIO)) - .toList(); - if (validFiles.isEmpty()) { - LOGGER.atWarn() - .addKeyValue("taskName", task.getName()) - .addKeyValue("fileBatch", fileBatch.toString()) - .addKeyValue("tableId", tableId) - .log("Table content cleanup task scheduled, but the none of the file in batch exists"); - return true; - } - - // Schedule the deletion for each file asynchronously - List> deleteFutures = validFiles.stream() - .map(file -> tryDelete(tableId, authorizedFileIO, file, null, 1)) - .toList(); - - // Wait for all delete operations to finish - CompletableFuture allDeletes = CompletableFuture.allOf(deleteFutures.toArray(new CompletableFuture[0])); - allDeletes.join(); - - LOGGER.atInfo() - .addKeyValue("taskName", task.getName()) - .addKeyValue("fileBatch", fileBatch.toString()) + return CompletableFuture.runAsync( + () -> { + if (TaskUtils.exists(filePath, fileIO)) { + fileIO.deleteFile(filePath); + LOGGER + .atInfo() + .addKeyValue("filePath", filePath) + .addKeyValue("tableId", tableId) + .addKeyValue("attempt", attempt) + .log("Successfully deleted file {}", filePath); + } else { + LOGGER + .atInfo() + .addKeyValue("filePath", filePath) .addKeyValue("tableId", tableId) - .log("All the files in task have been deleted"); + .log("File doesn't exist, likely already deleted"); + } + }, + executorService) + .exceptionallyComposeAsync( + newEx -> { + LOGGER + .atWarn() + .addKeyValue("filePath", filePath) + .addKeyValue("tableId", tableId) + .log("Exception caught deleting table content file", newEx); + return tryDelete(tableId, fileIO, filePath, newEx, attempt + 1); + }, + CompletableFuture.delayedExecutor( + FILE_DELETION_RETRY_MILLIS, TimeUnit.MILLISECONDS, executorService)); + } + + public static final class TableContentCleanupTask { + private TableIdentifier tableId; + private List fileBatch; + + public TableContentCleanupTask() {} + + public TableContentCleanupTask(TableIdentifier tableId, List fileBatch) { + this.tableId = tableId; + this.fileBatch = fileBatch; + } - return true; - } catch (Exception e) { - LOGGER.error("Error during table content cleanup for file batch {}", fileBatch.toString(), e); - return false; - } + public TableIdentifier getTableId() { + return tableId; } - private CompletableFuture tryDelete(TableIdentifier tableId, - FileIO fileIO, - String filePath, - Throwable e, - int attempt) { - if (e != null && attempt <= MAX_ATTEMPTS) { - LOGGER.atWarn() - .addKeyValue("filePath", filePath) - .addKeyValue("attempt", attempt) - .addKeyValue("error", e.getMessage()) - .log("Error encountered attempting to delete file"); - } + public void setTableId(TableIdentifier tableId) { + this.tableId = tableId; + } - if (attempt > MAX_ATTEMPTS && e != null) { - return CompletableFuture.failedFuture(e); - } + public List getFileBatch() { + return fileBatch; + } - return CompletableFuture.runAsync(() -> { - if (TaskUtils.exists(filePath, fileIO)) { - fileIO.deleteFile(filePath); - LOGGER.atInfo() - .addKeyValue("filePath", filePath) - .addKeyValue("tableId", tableId) - .addKeyValue("attempt", attempt) - .log("Successfully deleted file {}", filePath); - } else { - LOGGER.atInfo() - .addKeyValue("filePath", filePath) - .addKeyValue("tableId", tableId) - .log("File doesn't exist, likely already deleted"); - } - }, executorService).exceptionallyComposeAsync( - newEx -> { - LOGGER.atWarn() - .addKeyValue("filePath", filePath) - .addKeyValue("tableId", tableId) - .log("Exception caught deleting table content file", newEx); - return tryDelete(tableId, fileIO, filePath, newEx, attempt + 1); - }, - CompletableFuture.delayedExecutor(FILE_DELETION_RETRY_MILLIS, TimeUnit.MILLISECONDS, executorService) - ); + public void setFileBatch(List fileBatch) { + this.fileBatch = fileBatch; } - public static final class TableContentCleanupTask { - private TableIdentifier tableId; - private List fileBatch; - - public TableContentCleanupTask() { - } - - public TableContentCleanupTask(TableIdentifier tableId, List fileBatch) { - this.tableId = tableId; - this.fileBatch = fileBatch; - } - - public TableIdentifier getTableId() { - return tableId; - } - - public void setTableId(TableIdentifier tableId) { - this.tableId = tableId; - } - - public List getFileBatch() { - return fileBatch; - } - - public void setFileBatch(List fileBatch) { - this.fileBatch = fileBatch; - } - - @Override - public boolean equals(Object object) { - if (this == object) { - return true; - } - if (!(object instanceof TableContentCleanupTask other)) { - return false; - } - return Objects.equals(tableId, other.tableId) && Objects.equals(fileBatch, other.fileBatch); - } - - @Override - public int hashCode() { - return Objects.hash(tableId, fileBatch.toString()); - } + @Override + public boolean equals(Object object) { + if (this == object) { + return true; + } + if (!(object instanceof TableContentCleanupTask other)) { + return false; + } + return Objects.equals(tableId, other.tableId) && Objects.equals(fileBatch, other.fileBatch); + } + + @Override + public int hashCode() { + return Objects.hash(tableId, fileBatch.toString()); } + } } diff --git a/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java b/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java index 32b6df464..f5df6ebb2 100644 --- a/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java +++ b/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java @@ -79,7 +79,8 @@ public void testTableCleanup() throws IOException { TestSnapshot snapshot = TaskTestUtils.newSnapshot(fileIO, "manifestList.avro", 1, snapshotId, 99L, manifestFile); String metadataFile = "v1-49494949.metadata.json"; - StatisticsFile statisticsFile = TaskTestUtils.writeStatsFile( + StatisticsFile statisticsFile = + TaskTestUtils.writeStatsFile( snapshot.snapshotId(), snapshot.sequenceNumber(), "/metadata/" + UUID.randomUUID() + ".stats", @@ -121,14 +122,17 @@ public void testTableCleanup() throws IOException { entity -> entity.readData( ManifestFileCleanupTaskHandler.ManifestCleanupTask.class)), - taskEntity -> assertThat(taskEntity) - .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) - .extracting(TaskEntity::of) - .returns(AsyncTaskType.TABLE_CONTENT_CLEANUP, TaskEntity::getTaskType) - .returns( - new TableContentCleanupTaskHandler.TableContentCleanupTask(tableIdentifier, List.of(statisticsFile.path())), - entity -> entity.readData(TableContentCleanupTaskHandler.TableContentCleanupTask.class)) - ); + taskEntity -> + assertThat(taskEntity) + .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) + .extracting(TaskEntity::of) + .returns(AsyncTaskType.TABLE_CONTENT_CLEANUP, TaskEntity::getTaskType) + .returns( + new TableContentCleanupTaskHandler.TableContentCleanupTask( + tableIdentifier, List.of(statisticsFile.path())), + entity -> + entity.readData( + TableContentCleanupTaskHandler.TableContentCleanupTask.class))); } } @@ -319,12 +323,14 @@ public void testTableCleanupMultipleSnapshots() throws IOException { manifestFile1, manifestFile3); // exclude manifest2 from the new snapshot String metadataFile = "v1-295495059.metadata.json"; - StatisticsFile statisticsFile1 = TaskTestUtils.writeStatsFile( + StatisticsFile statisticsFile1 = + TaskTestUtils.writeStatsFile( snapshot.snapshotId(), snapshot.sequenceNumber(), "/metadata/" + UUID.randomUUID() + ".stats", fileIO); - StatisticsFile statisticsFile2 = TaskTestUtils.writeStatsFile( + StatisticsFile statisticsFile2 = + TaskTestUtils.writeStatsFile( snapshot2.snapshotId(), snapshot2.sequenceNumber(), "/metadata/" + UUID.randomUUID() + ".stats", @@ -348,22 +354,29 @@ public void testTableCleanupMultipleSnapshots() throws IOException { handler.handleTask(task); - List entities = metaStoreManagerFactory + List entities = + metaStoreManagerFactory .getOrCreateMetaStoreManager(realmContext) .loadTasks(polarisCallContext, "test", 5) .getEntities(); - List manifestCleanupTasks = entities.stream() - .filter(entity -> { - AsyncTaskType taskType = TaskEntity.of(entity).getTaskType(); - return taskType == AsyncTaskType.FILE_CLEANUP; - }).toList(); - - List StatsCleanupTasks = entities.stream() - .filter(entity -> { - AsyncTaskType taskType = TaskEntity.of(entity).getTaskType(); - return taskType == AsyncTaskType.TABLE_CONTENT_CLEANUP; - }).toList(); + List manifestCleanupTasks = + entities.stream() + .filter( + entity -> { + AsyncTaskType taskType = TaskEntity.of(entity).getTaskType(); + return taskType == AsyncTaskType.FILE_CLEANUP; + }) + .toList(); + + List StatsCleanupTasks = + entities.stream() + .filter( + entity -> { + AsyncTaskType taskType = TaskEntity.of(entity).getTaskType(); + return taskType == AsyncTaskType.TABLE_CONTENT_CLEANUP; + }) + .toList(); assertThat(manifestCleanupTasks) // all three manifests should be present, even though one is excluded from the latest @@ -402,92 +415,100 @@ public void testTableCleanupMultipleSnapshots() throws IOException { Base64.encodeBase64String(ManifestFiles.encode(manifestFile3))), entity -> entity.readData( - ManifestFileCleanupTaskHandler.ManifestCleanupTask.class)) - ); + ManifestFileCleanupTaskHandler.ManifestCleanupTask.class))); assertThat(StatsCleanupTasks) - .hasSize(1) - .satisfiesExactlyInAnyOrder( - taskEntity -> - assertThat(taskEntity) - .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) - .extracting(TaskEntity::of) - .returns(AsyncTaskType.TABLE_CONTENT_CLEANUP, TaskEntity::getTaskType) - .returns( - new TableContentCleanupTaskHandler.TableContentCleanupTask( - tableIdentifier, List.of(statisticsFile1.path(), statisticsFile2.path())), - entity -> entity.readData( - TableContentCleanupTaskHandler.TableContentCleanupTask.class)) - ); + .hasSize(1) + .satisfiesExactlyInAnyOrder( + taskEntity -> + assertThat(taskEntity) + .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) + .extracting(TaskEntity::of) + .returns(AsyncTaskType.TABLE_CONTENT_CLEANUP, TaskEntity::getTaskType) + .returns( + new TableContentCleanupTaskHandler.TableContentCleanupTask( + tableIdentifier, + List.of(statisticsFile1.path(), statisticsFile2.path())), + entity -> + entity.readData( + TableContentCleanupTaskHandler.TableContentCleanupTask.class))); } } @Test public void testTableCleanupMultipleMetadata() throws IOException { PolarisCallContext polarisCallContext = - new PolarisCallContext( - metaStoreManagerFactory.getOrCreateSessionSupplier(realmContext).get(), - new PolarisDefaultDiagServiceImpl()); + new PolarisCallContext( + metaStoreManagerFactory.getOrCreateSessionSupplier(realmContext).get(), + new PolarisDefaultDiagServiceImpl()); try (CallContext callCtx = CallContext.of(realmContext, polarisCallContext)) { CallContext.setCurrentContext(callCtx); FileIO fileIO = new InMemoryFileIO(); TableIdentifier tableIdentifier = - TableIdentifier.of(Namespace.of("db1", "schema1"), "table1"); + TableIdentifier.of(Namespace.of("db1", "schema1"), "table1"); TableCleanupTaskHandler handler = - new TableCleanupTaskHandler(Mockito.mock(), metaStoreManagerFactory, (task) -> fileIO); + new TableCleanupTaskHandler(Mockito.mock(), metaStoreManagerFactory, (task) -> fileIO); long snapshotId1 = 100L; ManifestFile manifestFile1 = - TaskTestUtils.manifestFile( - fileIO, "manifest1.avro", snapshotId1, "dataFile1.parquet", "dataFile2.parquet"); + TaskTestUtils.manifestFile( + fileIO, "manifest1.avro", snapshotId1, "dataFile1.parquet", "dataFile2.parquet"); ManifestFile manifestFile2 = - TaskTestUtils.manifestFile( - fileIO, "manifest2.avro", snapshotId1, "dataFile3.parquet", "dataFile4.parquet"); + TaskTestUtils.manifestFile( + fileIO, "manifest2.avro", snapshotId1, "dataFile3.parquet", "dataFile4.parquet"); Snapshot snapshot = - TaskTestUtils.newSnapshot( - fileIO, "manifestList.avro", 1, snapshotId1, 99L, manifestFile1, manifestFile2); - StatisticsFile statisticsFile1 = TaskTestUtils.writeStatsFile( + TaskTestUtils.newSnapshot( + fileIO, "manifestList.avro", 1, snapshotId1, 99L, manifestFile1, manifestFile2); + StatisticsFile statisticsFile1 = + TaskTestUtils.writeStatsFile( snapshot.snapshotId(), snapshot.sequenceNumber(), "/metadata/" + UUID.randomUUID() + ".stats", fileIO); String firstMetadataFile = "v1-295495059.metadata.json"; - TableMetadata firstMetadata = TaskTestUtils.writeTableMetadata( + TableMetadata firstMetadata = + TaskTestUtils.writeTableMetadata( fileIO, firstMetadataFile, List.of(statisticsFile1), snapshot); assertThat(TaskUtils.exists(firstMetadataFile, fileIO)).isTrue(); ManifestFile manifestFile3 = - TaskTestUtils.manifestFile( - fileIO, "manifest3.avro", snapshot.snapshotId() + 1, "dataFile5.parquet"); + TaskTestUtils.manifestFile( + fileIO, "manifest3.avro", snapshot.snapshotId() + 1, "dataFile5.parquet"); Snapshot snapshot2 = - TaskTestUtils.newSnapshot( - fileIO, - "manifestList2.avro", - snapshot.sequenceNumber() + 1, - snapshot.snapshotId() + 1, - snapshot.snapshotId(), - manifestFile1, - manifestFile3); // exclude manifest2 from the new snapshot - StatisticsFile statisticsFile2 = TaskTestUtils.writeStatsFile( + TaskTestUtils.newSnapshot( + fileIO, + "manifestList2.avro", + snapshot.sequenceNumber() + 1, + snapshot.snapshotId() + 1, + snapshot.snapshotId(), + manifestFile1, + manifestFile3); // exclude manifest2 from the new snapshot + StatisticsFile statisticsFile2 = + TaskTestUtils.writeStatsFile( snapshot2.snapshotId(), snapshot2.sequenceNumber(), "/metadata/" + UUID.randomUUID() + ".stats", fileIO); String secondMetadataFile = "v1-295495060.metadata.json"; TaskTestUtils.writeTableMetadata( - fileIO, secondMetadataFile, firstMetadata, firstMetadataFile, List.of(statisticsFile2), snapshot2); + fileIO, + secondMetadataFile, + firstMetadata, + firstMetadataFile, + List.of(statisticsFile2), + snapshot2); assertThat(TaskUtils.exists(firstMetadataFile, fileIO)).isTrue(); assertThat(TaskUtils.exists(secondMetadataFile, fileIO)).isTrue(); TaskEntity task = - new TaskEntity.Builder() - .withTaskType(AsyncTaskType.ENTITY_CLEANUP_SCHEDULER) - .withData( - new TableLikeEntity.Builder(tableIdentifier, secondMetadataFile) - .setName("table1") - .setCatalogId(1) - .setCreateTimestamp(100) - .build()) - .build(); + new TaskEntity.Builder() + .withTaskType(AsyncTaskType.ENTITY_CLEANUP_SCHEDULER) + .withData( + new TableLikeEntity.Builder(tableIdentifier, secondMetadataFile) + .setName("table1") + .setCatalogId(1) + .setCreateTimestamp(100) + .build()) + .build(); Assertions.assertThatPredicate(handler::canHandleTask).accepts(task); @@ -495,87 +516,95 @@ public void testTableCleanupMultipleMetadata() throws IOException { handler.handleTask(task); - List entities = metaStoreManagerFactory + List entities = + metaStoreManagerFactory .getOrCreateMetaStoreManager(realmContext) .loadTasks(polarisCallContext, "test", 6) .getEntities(); - List manifestCleanupTasks = entities.stream() - .filter(entity -> { - AsyncTaskType taskType = TaskEntity.of(entity).getTaskType(); - return taskType == AsyncTaskType.FILE_CLEANUP; - }).toList(); - - List PrevMetadataNStatCleanupTasks = entities.stream() - .filter(entity -> { - AsyncTaskType taskType = TaskEntity.of(entity).getTaskType(); - return taskType == AsyncTaskType.TABLE_CONTENT_CLEANUP; - }).toList(); + List manifestCleanupTasks = + entities.stream() + .filter( + entity -> { + AsyncTaskType taskType = TaskEntity.of(entity).getTaskType(); + return taskType == AsyncTaskType.FILE_CLEANUP; + }) + .toList(); + + List PrevMetadataNStatCleanupTasks = + entities.stream() + .filter( + entity -> { + AsyncTaskType taskType = TaskEntity.of(entity).getTaskType(); + return taskType == AsyncTaskType.TABLE_CONTENT_CLEANUP; + }) + .toList(); assertThat(manifestCleanupTasks) - // all three manifests should be present, even though one is excluded from the latest - // snapshot - .hasSize(3) - .satisfiesExactlyInAnyOrder( - taskEntity -> - assertThat(taskEntity) - .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) - .extracting(TaskEntity::of) - .returns( - new ManifestFileCleanupTaskHandler.ManifestCleanupTask( - tableIdentifier, - Base64.encodeBase64String(ManifestFiles.encode(manifestFile1))), - entity -> - entity.readData( - ManifestFileCleanupTaskHandler.ManifestCleanupTask.class)), - taskEntity -> - assertThat(taskEntity) - .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) - .extracting(TaskEntity::of) - .returns( - new ManifestFileCleanupTaskHandler.ManifestCleanupTask( - tableIdentifier, - Base64.encodeBase64String(ManifestFiles.encode(manifestFile2))), - entity -> - entity.readData( - ManifestFileCleanupTaskHandler.ManifestCleanupTask.class)), - taskEntity -> - assertThat(taskEntity) - .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) - .extracting(TaskEntity::of) - .returns( - new ManifestFileCleanupTaskHandler.ManifestCleanupTask( - tableIdentifier, - Base64.encodeBase64String(ManifestFiles.encode(manifestFile3))), - entity -> - entity.readData( - ManifestFileCleanupTaskHandler.ManifestCleanupTask.class)) - ); + // all three manifests should be present, even though one is excluded from the latest + // snapshot + .hasSize(3) + .satisfiesExactlyInAnyOrder( + taskEntity -> + assertThat(taskEntity) + .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) + .extracting(TaskEntity::of) + .returns( + new ManifestFileCleanupTaskHandler.ManifestCleanupTask( + tableIdentifier, + Base64.encodeBase64String(ManifestFiles.encode(manifestFile1))), + entity -> + entity.readData( + ManifestFileCleanupTaskHandler.ManifestCleanupTask.class)), + taskEntity -> + assertThat(taskEntity) + .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) + .extracting(TaskEntity::of) + .returns( + new ManifestFileCleanupTaskHandler.ManifestCleanupTask( + tableIdentifier, + Base64.encodeBase64String(ManifestFiles.encode(manifestFile2))), + entity -> + entity.readData( + ManifestFileCleanupTaskHandler.ManifestCleanupTask.class)), + taskEntity -> + assertThat(taskEntity) + .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) + .extracting(TaskEntity::of) + .returns( + new ManifestFileCleanupTaskHandler.ManifestCleanupTask( + tableIdentifier, + Base64.encodeBase64String(ManifestFiles.encode(manifestFile3))), + entity -> + entity.readData( + ManifestFileCleanupTaskHandler.ManifestCleanupTask.class))); assertThat(PrevMetadataNStatCleanupTasks) - .hasSize(2) - .satisfiesExactlyInAnyOrder( - taskEntity -> - assertThat(taskEntity) - .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) - .extracting(TaskEntity::of) - .returns(AsyncTaskType.TABLE_CONTENT_CLEANUP, TaskEntity::getTaskType) - .returns( - new TableContentCleanupTaskHandler.TableContentCleanupTask( - tableIdentifier, List.of(statisticsFile1.path(), statisticsFile2.path())), - entity -> entity.readData( - TableContentCleanupTaskHandler.TableContentCleanupTask.class)), - taskEntity -> - assertThat(taskEntity) - .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) - .extracting(TaskEntity::of) - .returns(AsyncTaskType.TABLE_CONTENT_CLEANUP, TaskEntity::getTaskType) - .returns( - new TableContentCleanupTaskHandler.TableContentCleanupTask( - tableIdentifier, List.of(firstMetadataFile)), - entity -> entity.readData( - TableContentCleanupTaskHandler.TableContentCleanupTask.class)) - ); + .hasSize(2) + .satisfiesExactlyInAnyOrder( + taskEntity -> + assertThat(taskEntity) + .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) + .extracting(TaskEntity::of) + .returns(AsyncTaskType.TABLE_CONTENT_CLEANUP, TaskEntity::getTaskType) + .returns( + new TableContentCleanupTaskHandler.TableContentCleanupTask( + tableIdentifier, + List.of(statisticsFile1.path(), statisticsFile2.path())), + entity -> + entity.readData( + TableContentCleanupTaskHandler.TableContentCleanupTask.class)), + taskEntity -> + assertThat(taskEntity) + .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) + .extracting(TaskEntity::of) + .returns(AsyncTaskType.TABLE_CONTENT_CLEANUP, TaskEntity::getTaskType) + .returns( + new TableContentCleanupTaskHandler.TableContentCleanupTask( + tableIdentifier, List.of(firstMetadataFile)), + entity -> + entity.readData( + TableContentCleanupTaskHandler.TableContentCleanupTask.class))); } } } diff --git a/polaris-service/src/test/java/org/apache/polaris/service/task/TableContentCleanupTaskHandlerTest.java b/polaris-service/src/test/java/org/apache/polaris/service/task/TableContentCleanupTaskHandlerTest.java index bf879a3ae..4f2e8ae5d 100644 --- a/polaris-service/src/test/java/org/apache/polaris/service/task/TableContentCleanupTaskHandlerTest.java +++ b/polaris-service/src/test/java/org/apache/polaris/service/task/TableContentCleanupTaskHandlerTest.java @@ -16,9 +16,21 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.polaris.service.task; +import static org.assertj.core.api.Assertions.assertThatPredicate; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Stream; import org.apache.iceberg.ManifestFile; import org.apache.iceberg.Snapshot; import org.apache.iceberg.StatisticsFile; @@ -39,231 +51,244 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.stream.Stream; - -import static org.assertj.core.api.Assertions.assertThatPredicate; -import static org.assertj.core.api.AssertionsForClassTypes.assertThat; - public class TableContentCleanupTaskHandlerTest { - private InMemoryPolarisMetaStoreManagerFactory metaStoreManagerFactory; - private RealmContext realmContext; - private static final Logger LOGGER = - LoggerFactory.getLogger(TableContentCleanupTaskHandlerTest.class); + private InMemoryPolarisMetaStoreManagerFactory metaStoreManagerFactory; + private RealmContext realmContext; + private static final Logger LOGGER = + LoggerFactory.getLogger(TableContentCleanupTaskHandlerTest.class); - @BeforeEach - void setUp() { - metaStoreManagerFactory = new InMemoryPolarisMetaStoreManagerFactory(); - realmContext = () -> "realmName"; - } + @BeforeEach + void setUp() { + metaStoreManagerFactory = new InMemoryPolarisMetaStoreManagerFactory(); + realmContext = () -> "realmName"; + } - @Test - public void testCleanup() throws IOException { - PolarisCallContext polarisCallContext = - new PolarisCallContext( - metaStoreManagerFactory.getOrCreateSessionSupplier(realmContext).get(), - new PolarisDefaultDiagServiceImpl()); - try (CallContext callCtx = CallContext.of(realmContext, polarisCallContext)) { - CallContext.setCurrentContext(callCtx); - FileIO fileIO = new InMemoryFileIO() { - @Override - public void close() { - // no-op - } - }; - TableIdentifier tableIdentifier = - TableIdentifier.of(Namespace.of("db1", "schema1"), "table1"); - ExecutorService executorService = Executors.newCachedThreadPool(); - TableContentCleanupTaskHandler handler = - new TableContentCleanupTaskHandler((task) -> fileIO, executorService); + @Test + public void testCleanup() throws IOException { + PolarisCallContext polarisCallContext = + new PolarisCallContext( + metaStoreManagerFactory.getOrCreateSessionSupplier(realmContext).get(), + new PolarisDefaultDiagServiceImpl()); + try (CallContext callCtx = CallContext.of(realmContext, polarisCallContext)) { + CallContext.setCurrentContext(callCtx); + FileIO fileIO = + new InMemoryFileIO() { + @Override + public void close() { + // no-op + } + }; + TableIdentifier tableIdentifier = + TableIdentifier.of(Namespace.of("db1", "schema1"), "table1"); + ExecutorService executorService = Executors.newCachedThreadPool(); + TableContentCleanupTaskHandler handler = + new TableContentCleanupTaskHandler((task) -> fileIO, executorService); - long snapshotId1 = 100L; - ManifestFile manifestFile1 = - TaskTestUtils.manifestFile( - fileIO, "manifest1.avro", snapshotId1, "dataFile1.parquet", "dataFile2.parquet"); - ManifestFile manifestFile2 = - TaskTestUtils.manifestFile( - fileIO, "manifest2.avro", snapshotId1, "dataFile3.parquet", "dataFile4.parquet"); - Snapshot snapshot = - TaskTestUtils.newSnapshot( - fileIO, "manifestList.avro", 1, snapshotId1, 99L, manifestFile1, manifestFile2); - StatisticsFile statisticsFile1 = TaskTestUtils.writeStatsFile( - snapshot.snapshotId(), - snapshot.sequenceNumber(), - "/metadata/" + UUID.randomUUID() + ".stats", - fileIO); - String firstMetadataFile = "v1-295495059.metadata.json"; - TableMetadata firstMetadata = TaskTestUtils.writeTableMetadata( - fileIO, firstMetadataFile, List.of(statisticsFile1), snapshot); - assertThat(TaskUtils.exists(firstMetadataFile, fileIO)).isTrue(); + long snapshotId1 = 100L; + ManifestFile manifestFile1 = + TaskTestUtils.manifestFile( + fileIO, "manifest1.avro", snapshotId1, "dataFile1.parquet", "dataFile2.parquet"); + ManifestFile manifestFile2 = + TaskTestUtils.manifestFile( + fileIO, "manifest2.avro", snapshotId1, "dataFile3.parquet", "dataFile4.parquet"); + Snapshot snapshot = + TaskTestUtils.newSnapshot( + fileIO, "manifestList.avro", 1, snapshotId1, 99L, manifestFile1, manifestFile2); + StatisticsFile statisticsFile1 = + TaskTestUtils.writeStatsFile( + snapshot.snapshotId(), + snapshot.sequenceNumber(), + "/metadata/" + UUID.randomUUID() + ".stats", + fileIO); + String firstMetadataFile = "v1-295495059.metadata.json"; + TableMetadata firstMetadata = + TaskTestUtils.writeTableMetadata( + fileIO, firstMetadataFile, List.of(statisticsFile1), snapshot); + assertThat(TaskUtils.exists(firstMetadataFile, fileIO)).isTrue(); - ManifestFile manifestFile3 = - TaskTestUtils.manifestFile( - fileIO, "manifest3.avro", snapshot.snapshotId() + 1, "dataFile5.parquet"); - Snapshot snapshot2 = - TaskTestUtils.newSnapshot( - fileIO, - "manifestList2.avro", - snapshot.sequenceNumber() + 1, - snapshot.snapshotId() + 1, - snapshot.snapshotId(), - manifestFile1, - manifestFile3); // exclude manifest2 from the new snapshot - StatisticsFile statisticsFile2 = TaskTestUtils.writeStatsFile( - snapshot2.snapshotId(), - snapshot2.sequenceNumber(), - "/metadata/" + UUID.randomUUID() + ".stats", - fileIO); - String secondMetadataFile = "v1-295495060.metadata.json"; - TableMetadata secondMetadata = TaskTestUtils.writeTableMetadata( - fileIO, secondMetadataFile, firstMetadata, firstMetadataFile, List.of(statisticsFile2), snapshot2); - assertThat(TaskUtils.exists(firstMetadataFile, fileIO)).isTrue(); - assertThat(TaskUtils.exists(secondMetadataFile, fileIO)).isTrue(); + ManifestFile manifestFile3 = + TaskTestUtils.manifestFile( + fileIO, "manifest3.avro", snapshot.snapshotId() + 1, "dataFile5.parquet"); + Snapshot snapshot2 = + TaskTestUtils.newSnapshot( + fileIO, + "manifestList2.avro", + snapshot.sequenceNumber() + 1, + snapshot.snapshotId() + 1, + snapshot.snapshotId(), + manifestFile1, + manifestFile3); // exclude manifest2 from the new snapshot + StatisticsFile statisticsFile2 = + TaskTestUtils.writeStatsFile( + snapshot2.snapshotId(), + snapshot2.sequenceNumber(), + "/metadata/" + UUID.randomUUID() + ".stats", + fileIO); + String secondMetadataFile = "v1-295495060.metadata.json"; + TableMetadata secondMetadata = + TaskTestUtils.writeTableMetadata( + fileIO, + secondMetadataFile, + firstMetadata, + firstMetadataFile, + List.of(statisticsFile2), + snapshot2); + assertThat(TaskUtils.exists(firstMetadataFile, fileIO)).isTrue(); + assertThat(TaskUtils.exists(secondMetadataFile, fileIO)).isTrue(); - List cleanupFiles = Stream.concat( - secondMetadata.previousFiles() - .stream() - .map(TableMetadata.MetadataLogEntry::file) - .filter(file -> TaskUtils.exists(file, fileIO)), - secondMetadata.statisticsFiles() - .stream() - .map(StatisticsFile::path) - .filter(file -> TaskUtils.exists(file, fileIO)) - ).toList(); + List cleanupFiles = + Stream.concat( + secondMetadata.previousFiles().stream() + .map(TableMetadata.MetadataLogEntry::file) + .filter(file -> TaskUtils.exists(file, fileIO)), + secondMetadata.statisticsFiles().stream() + .map(StatisticsFile::path) + .filter(file -> TaskUtils.exists(file, fileIO))) + .toList(); - TaskEntity task = new TaskEntity.Builder() - .withTaskType(AsyncTaskType.TABLE_CONTENT_CLEANUP) - .withData(new TableContentCleanupTaskHandler.TableContentCleanupTask(tableIdentifier, cleanupFiles)) - .setName(UUID.randomUUID().toString()) - .build(); + TaskEntity task = + new TaskEntity.Builder() + .withTaskType(AsyncTaskType.TABLE_CONTENT_CLEANUP) + .withData( + new TableContentCleanupTaskHandler.TableContentCleanupTask( + tableIdentifier, cleanupFiles)) + .setName(UUID.randomUUID().toString()) + .build(); - assertThatPredicate(handler::canHandleTask).accepts(task); - assertThatPredicate(handler::handleTask).accepts(task); + assertThatPredicate(handler::canHandleTask).accepts(task); + assertThatPredicate(handler::handleTask).accepts(task); - assertThatPredicate((String file) -> TaskUtils.exists(file, fileIO)).rejects(firstMetadataFile); - assertThatPredicate((String file) -> TaskUtils.exists(file, fileIO)).rejects(statisticsFile1.path()); - assertThatPredicate((String file) -> TaskUtils.exists(file, fileIO)).rejects(statisticsFile2.path()); - } + assertThatPredicate((String file) -> TaskUtils.exists(file, fileIO)) + .rejects(firstMetadataFile); + assertThatPredicate((String file) -> TaskUtils.exists(file, fileIO)) + .rejects(statisticsFile1.path()); + assertThatPredicate((String file) -> TaskUtils.exists(file, fileIO)) + .rejects(statisticsFile2.path()); } + } - @Test - public void testCleanupIfFileNotExist() throws IOException { - PolarisCallContext polarisCallContext = - new PolarisCallContext( - metaStoreManagerFactory.getOrCreateSessionSupplier(realmContext).get(), - new PolarisDefaultDiagServiceImpl()); - try (CallContext callCtx = CallContext.of(realmContext, polarisCallContext)) { - CallContext.setCurrentContext(callCtx); - FileIO fileIO = new InMemoryFileIO(); - TableIdentifier tableIdentifier = - TableIdentifier.of(Namespace.of("db1", "schema1"), "table1"); - TableContentCleanupTaskHandler handler = - new TableContentCleanupTaskHandler((task) -> fileIO, Executors.newCachedThreadPool()); - long snapshotId = 100L; - ManifestFile manifestFile = - TaskTestUtils.manifestFile( - fileIO, "manifest1.avro", snapshotId, "dataFile1.parquet", "dataFile2.parquet"); - TestSnapshot snapshot = - TaskTestUtils.newSnapshot(fileIO, "manifestList.avro", 1, snapshotId, 99L, manifestFile); - String metadataFile = "v1-49494949.metadata.json"; - StatisticsFile statisticsFile = TaskTestUtils.writeStatsFile( - snapshot.snapshotId(), - snapshot.sequenceNumber(), - "/metadata/" + UUID.randomUUID() + ".stats", - fileIO); - TaskTestUtils.writeTableMetadata(fileIO, metadataFile, List.of(statisticsFile), snapshot); + @Test + public void testCleanupIfFileNotExist() throws IOException { + PolarisCallContext polarisCallContext = + new PolarisCallContext( + metaStoreManagerFactory.getOrCreateSessionSupplier(realmContext).get(), + new PolarisDefaultDiagServiceImpl()); + try (CallContext callCtx = CallContext.of(realmContext, polarisCallContext)) { + CallContext.setCurrentContext(callCtx); + FileIO fileIO = new InMemoryFileIO(); + TableIdentifier tableIdentifier = + TableIdentifier.of(Namespace.of("db1", "schema1"), "table1"); + TableContentCleanupTaskHandler handler = + new TableContentCleanupTaskHandler((task) -> fileIO, Executors.newCachedThreadPool()); + long snapshotId = 100L; + ManifestFile manifestFile = + TaskTestUtils.manifestFile( + fileIO, "manifest1.avro", snapshotId, "dataFile1.parquet", "dataFile2.parquet"); + TestSnapshot snapshot = + TaskTestUtils.newSnapshot(fileIO, "manifestList.avro", 1, snapshotId, 99L, manifestFile); + String metadataFile = "v1-49494949.metadata.json"; + StatisticsFile statisticsFile = + TaskTestUtils.writeStatsFile( + snapshot.snapshotId(), + snapshot.sequenceNumber(), + "/metadata/" + UUID.randomUUID() + ".stats", + fileIO); + TaskTestUtils.writeTableMetadata(fileIO, metadataFile, List.of(statisticsFile), snapshot); - fileIO.deleteFile(statisticsFile.path()); - assertThat(TaskUtils.exists(statisticsFile.path(), fileIO)).isFalse(); + fileIO.deleteFile(statisticsFile.path()); + assertThat(TaskUtils.exists(statisticsFile.path(), fileIO)).isFalse(); - TaskEntity task = new TaskEntity.Builder() - .withTaskType(AsyncTaskType.TABLE_CONTENT_CLEANUP) - .withData(new TableContentCleanupTaskHandler.TableContentCleanupTask(tableIdentifier, List.of(statisticsFile.path()))) - .setName(UUID.randomUUID().toString()) - .build(); - assertThatPredicate(handler::canHandleTask).accepts(task); - assertThatPredicate(handler::handleTask).accepts(task); - } + TaskEntity task = + new TaskEntity.Builder() + .withTaskType(AsyncTaskType.TABLE_CONTENT_CLEANUP) + .withData( + new TableContentCleanupTaskHandler.TableContentCleanupTask( + tableIdentifier, List.of(statisticsFile.path()))) + .setName(UUID.randomUUID().toString()) + .build(); + assertThatPredicate(handler::canHandleTask).accepts(task); + assertThatPredicate(handler::handleTask).accepts(task); } + } - @Test - public void testCleanupWithRetries() throws IOException { - PolarisCallContext polarisCallContext = - new PolarisCallContext( - metaStoreManagerFactory.getOrCreateSessionSupplier(realmContext).get(), - new PolarisDefaultDiagServiceImpl()); - try (CallContext callCtx = CallContext.of(realmContext, polarisCallContext)) { - CallContext.setCurrentContext(callCtx); - Map retryCounter = new HashMap<>(); - FileIO fileIO = new InMemoryFileIO() { - @Override - public void close() { - // no-op - } + @Test + public void testCleanupWithRetries() throws IOException { + PolarisCallContext polarisCallContext = + new PolarisCallContext( + metaStoreManagerFactory.getOrCreateSessionSupplier(realmContext).get(), + new PolarisDefaultDiagServiceImpl()); + try (CallContext callCtx = CallContext.of(realmContext, polarisCallContext)) { + CallContext.setCurrentContext(callCtx); + Map retryCounter = new HashMap<>(); + FileIO fileIO = + new InMemoryFileIO() { + @Override + public void close() { + // no-op + } - @Override - public void deleteFile(String location) { - int attempts = retryCounter - .computeIfAbsent(location, k -> new AtomicInteger(0)) - .incrementAndGet(); - LOGGER.info("Attempt {} to delete file: {}", attempts, location); - if (attempts < 3) { - LOGGER.warn("Simulating failure for retry {}", attempts); - throw new RuntimeException("Simulating failure to test retries"); - } else { - // Succeed on the third attempt - LOGGER.info("Actually delete file: {}", location); - super.deleteFile(location); - } - } - }; - TableIdentifier tableIdentifier = - TableIdentifier.of(Namespace.of("db1", "schema1"), "table1"); - TableContentCleanupTaskHandler handler = - new TableContentCleanupTaskHandler((task) -> fileIO, Executors.newCachedThreadPool()); - long snapshotId = 100L; - ManifestFile manifestFile = - TaskTestUtils.manifestFile( - fileIO, "manifest1.avro", snapshotId, "dataFile1.parquet", "dataFile2.parquet"); - TestSnapshot snapshot = - TaskTestUtils.newSnapshot(fileIO, "manifestList.avro", 1, snapshotId, 99L, manifestFile); - String metadataFile = "v1-49494949.metadata.json"; - StatisticsFile statisticsFile = TaskTestUtils.writeStatsFile( - snapshot.snapshotId(), - snapshot.sequenceNumber(), - "/metadata/" + UUID.randomUUID() + ".stats", - fileIO); - TaskTestUtils.writeTableMetadata(fileIO, metadataFile, List.of(statisticsFile), snapshot); - assertThat(TaskUtils.exists(statisticsFile.path(), fileIO)).isTrue(); + @Override + public void deleteFile(String location) { + int attempts = + retryCounter + .computeIfAbsent(location, k -> new AtomicInteger(0)) + .incrementAndGet(); + LOGGER.info("Attempt {} to delete file: {}", attempts, location); + if (attempts < 3) { + LOGGER.warn("Simulating failure for retry {}", attempts); + throw new RuntimeException("Simulating failure to test retries"); + } else { + // Succeed on the third attempt + LOGGER.info("Actually delete file: {}", location); + super.deleteFile(location); + } + } + }; + TableIdentifier tableIdentifier = + TableIdentifier.of(Namespace.of("db1", "schema1"), "table1"); + TableContentCleanupTaskHandler handler = + new TableContentCleanupTaskHandler((task) -> fileIO, Executors.newCachedThreadPool()); + long snapshotId = 100L; + ManifestFile manifestFile = + TaskTestUtils.manifestFile( + fileIO, "manifest1.avro", snapshotId, "dataFile1.parquet", "dataFile2.parquet"); + TestSnapshot snapshot = + TaskTestUtils.newSnapshot(fileIO, "manifestList.avro", 1, snapshotId, 99L, manifestFile); + String metadataFile = "v1-49494949.metadata.json"; + StatisticsFile statisticsFile = + TaskTestUtils.writeStatsFile( + snapshot.snapshotId(), + snapshot.sequenceNumber(), + "/metadata/" + UUID.randomUUID() + ".stats", + fileIO); + TaskTestUtils.writeTableMetadata(fileIO, metadataFile, List.of(statisticsFile), snapshot); + assertThat(TaskUtils.exists(statisticsFile.path(), fileIO)).isTrue(); - TaskEntity task = new TaskEntity.Builder() - .withTaskType(AsyncTaskType.TABLE_CONTENT_CLEANUP) - .withData(new TableContentCleanupTaskHandler.TableContentCleanupTask(tableIdentifier, List.of(statisticsFile.path()))) - .setName(UUID.randomUUID().toString()) - .build(); + TaskEntity task = + new TaskEntity.Builder() + .withTaskType(AsyncTaskType.TABLE_CONTENT_CLEANUP) + .withData( + new TableContentCleanupTaskHandler.TableContentCleanupTask( + tableIdentifier, List.of(statisticsFile.path()))) + .setName(UUID.randomUUID().toString()) + .build(); - CompletableFuture future = CompletableFuture.runAsync(() -> { + CompletableFuture future = + CompletableFuture.runAsync( + () -> { assertThatPredicate(handler::canHandleTask).accepts(task); handler.handleTask(task); // this will schedule the batch deletion - }); + }); - // Wait for all async tasks to finish - future.join(); + // Wait for all async tasks to finish + future.join(); - // Check if the file was successfully deleted after retries - assertThat(TaskUtils.exists(statisticsFile.path(), fileIO)).isFalse(); + // Check if the file was successfully deleted after retries + assertThat(TaskUtils.exists(statisticsFile.path(), fileIO)).isFalse(); - // Ensure that retries happened as expected - assertThat(retryCounter.containsKey(statisticsFile.path())).isTrue(); - assertThat(retryCounter.get(statisticsFile.path()).get()).isEqualTo(3); - } + // Ensure that retries happened as expected + assertThat(retryCounter.containsKey(statisticsFile.path())).isTrue(); + assertThat(retryCounter.get(statisticsFile.path()).get()).isEqualTo(3); } + } } diff --git a/polaris-service/src/test/java/org/apache/polaris/service/task/TaskTestUtils.java b/polaris-service/src/test/java/org/apache/polaris/service/task/TaskTestUtils.java index 49ab81aa9..5864ff160 100644 --- a/polaris-service/src/test/java/org/apache/polaris/service/task/TaskTestUtils.java +++ b/polaris-service/src/test/java/org/apache/polaris/service/task/TaskTestUtils.java @@ -74,8 +74,12 @@ static TableMetadata writeTableMetadata(FileIO fileIO, String metadataFile, Snap return writeTableMetadata(fileIO, metadataFile, null, null, null, snapshots); } - static TableMetadata writeTableMetadata(FileIO fileIO, String metadataFile, List statisticsFiles, Snapshot... snapshots) - throws IOException { + static TableMetadata writeTableMetadata( + FileIO fileIO, + String metadataFile, + List statisticsFiles, + Snapshot... snapshots) + throws IOException { return writeTableMetadata(fileIO, metadataFile, null, null, statisticsFiles, snapshots); } @@ -86,23 +90,21 @@ static TableMetadata writeTableMetadata( String prevMetadataFile, List statisticsFiles, Snapshot... snapshots) - throws IOException { + throws IOException { TableMetadata.Builder tmBuilder; if (prevMetadata == null) { tmBuilder = TableMetadata.buildFromEmpty(); } else { - tmBuilder = TableMetadata.buildFrom(prevMetadata) - .setPreviousFileLocation(prevMetadataFile); + tmBuilder = TableMetadata.buildFrom(prevMetadata).setPreviousFileLocation(prevMetadataFile); } tmBuilder - .setLocation("path/to/table") - .addSchema( - new Schema( - List.of(Types.NestedField.of(1, false, "field1", Types.StringType.get()))), - 1) - .addSortOrder(SortOrder.unsorted()) - .assignUUID(UUID.randomUUID().toString()) - .addPartitionSpec(PartitionSpec.unpartitioned()); + .setLocation("path/to/table") + .addSchema( + new Schema(List.of(Types.NestedField.of(1, false, "field1", Types.StringType.get()))), + 1) + .addSortOrder(SortOrder.unsorted()) + .assignUUID(UUID.randomUUID().toString()) + .addPartitionSpec(PartitionSpec.unpartitioned()); int statisticsFileIndex = 0; for (Snapshot snapshot : snapshots) { @@ -139,25 +141,25 @@ static TableMetadata writeTableMetadata( return snapshot; } - static public StatisticsFile writeStatsFile( - long snapshotId, long snapshotSequenceNumber, String statsLocation, FileIO fileIO) - throws IOException { + public static StatisticsFile writeStatsFile( + long snapshotId, long snapshotSequenceNumber, String statsLocation, FileIO fileIO) + throws IOException { try (PuffinWriter puffinWriter = Puffin.write(fileIO.newOutputFile(statsLocation)).build()) { puffinWriter.add( - new Blob( - "some-blob-type", - List.of(1), - snapshotId, - snapshotSequenceNumber, - ByteBuffer.wrap("blob content".getBytes(StandardCharsets.UTF_8)))); + new Blob( + "some-blob-type", + List.of(1), + snapshotId, + snapshotSequenceNumber, + ByteBuffer.wrap("blob content".getBytes(StandardCharsets.UTF_8)))); puffinWriter.finish(); return new GenericStatisticsFile( - snapshotId, - statsLocation, - puffinWriter.fileSize(), - puffinWriter.footerSize(), - puffinWriter.writtenBlobsMetadata().stream().map(GenericBlobMetadata::from).toList()); + snapshotId, + statsLocation, + puffinWriter.fileSize(), + puffinWriter.footerSize(), + puffinWriter.writtenBlobsMetadata().stream().map(GenericBlobMetadata::from).toList()); } } } From 27ea1b33d086ef45d7f4ac30199c3b01b51898c0 Mon Sep 17 00:00:00 2001 From: Daniel Tu Date: Sun, 3 Nov 2024 21:46:48 -0800 Subject: [PATCH 22/29] merge cleanup tasks into one --- .../polaris/core/entity/AsyncTaskType.java | 3 +- .../polaris/service/PolarisApplication.java | 4 - .../task/ManifestFileCleanupTaskHandler.java | 186 +++++++---- .../service/task/TableCleanupTaskHandler.java | 137 +++----- .../task/TableContentCleanupTaskHandler.java | 189 ----------- .../ManifestFileCleanupTaskHandlerTest.java | 231 ++++++++++++++ .../task/TableCleanupTaskHandlerTest.java | 66 +--- .../TableContentCleanupTaskHandlerTest.java | 294 ------------------ 8 files changed, 423 insertions(+), 687 deletions(-) delete mode 100644 polaris-service/src/main/java/org/apache/polaris/service/task/TableContentCleanupTaskHandler.java delete mode 100644 polaris-service/src/test/java/org/apache/polaris/service/task/TableContentCleanupTaskHandlerTest.java diff --git a/polaris-core/src/main/java/org/apache/polaris/core/entity/AsyncTaskType.java b/polaris-core/src/main/java/org/apache/polaris/core/entity/AsyncTaskType.java index 11997b1d6..32c478e86 100644 --- a/polaris-core/src/main/java/org/apache/polaris/core/entity/AsyncTaskType.java +++ b/polaris-core/src/main/java/org/apache/polaris/core/entity/AsyncTaskType.java @@ -23,8 +23,7 @@ public enum AsyncTaskType { ENTITY_CLEANUP_SCHEDULER(1), - FILE_CLEANUP(2), - TABLE_CONTENT_CLEANUP(3); + FILE_CLEANUP(2); private final int typeCode; diff --git a/polaris-service/src/main/java/org/apache/polaris/service/PolarisApplication.java b/polaris-service/src/main/java/org/apache/polaris/service/PolarisApplication.java index 2558414c5..5706fb272 100644 --- a/polaris-service/src/main/java/org/apache/polaris/service/PolarisApplication.java +++ b/polaris-service/src/main/java/org/apache/polaris/service/PolarisApplication.java @@ -104,7 +104,6 @@ import org.apache.polaris.service.storage.PolarisStorageIntegrationProviderImpl; import org.apache.polaris.service.task.ManifestFileCleanupTaskHandler; import org.apache.polaris.service.task.TableCleanupTaskHandler; -import org.apache.polaris.service.task.TableContentCleanupTaskHandler; import org.apache.polaris.service.task.TaskExecutorImpl; import org.apache.polaris.service.task.TaskFileIOSupplier; import org.apache.polaris.service.throttling.StreamReadConstraintsExceptionMapper; @@ -212,9 +211,6 @@ public void run(PolarisApplicationConfig configuration, Environment environment) taskExecutor.addTaskHandler( new ManifestFileCleanupTaskHandler( fileIOSupplier, Executors.newVirtualThreadPerTaskExecutor())); - taskExecutor.addTaskHandler( - new TableContentCleanupTaskHandler( - fileIOSupplier, Executors.newVirtualThreadPerTaskExecutor())); LOGGER.info( "Initializing PolarisCallContextCatalogFactory for metaStoreManagerType {}", diff --git a/polaris-service/src/main/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandler.java b/polaris-service/src/main/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandler.java index e17e24fd7..8ee8fe8eb 100644 --- a/polaris-service/src/main/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandler.java +++ b/polaris-service/src/main/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandler.java @@ -42,9 +42,11 @@ import org.slf4j.LoggerFactory; /** - * {@link TaskHandler} responsible for deleting all of the files in a manifest and the manifest - * itself. Since data files may be present in multiple manifests across different snapshots, we - * assume a data file that doesn't exist is missing because it was already deleted by another task. + * {@link TaskHandler} responsible for deleting table files: 1. Manifest files: It contains all the + * files in a manifest and the manifest itself. Since data files may be present in multiple + * manifests across different snapshots, we assume a data file that doesn't exist is missing because + * it was already deleted by another task. 2. Table content files: It contains previous metadata and + * statistics files, which are grouped and deleted in batch */ public class ManifestFileCleanupTaskHandler implements TaskHandler { public static final int MAX_ATTEMPTS = 3; @@ -68,58 +70,107 @@ public boolean canHandleTask(TaskEntity task) { @Override public boolean handleTask(TaskEntity task) { ManifestCleanupTask cleanupTask = task.readData(ManifestCleanupTask.class); - ManifestFile manifestFile = decodeManifestData(cleanupTask.getManifestFileData()); TableIdentifier tableId = cleanupTask.getTableId(); try (FileIO authorizedFileIO = fileIOSupplier.apply(task)) { - - // if the file doesn't exist, we assume that another task execution was successful, but failed - // to drop the task entity. Log a warning and return success - if (!TaskUtils.exists(manifestFile.path(), authorizedFileIO)) { + if (cleanupTask.getManifestFileData() != null) { + ManifestFile manifestFile = decodeManifestData(cleanupTask.getManifestFileData()); + return manifestFileCleanUp(manifestFile, authorizedFileIO, tableId); + } else if (cleanupTask.getContentFileBatch() != null) { + return contentFileCleanup(cleanupTask.getContentFileBatch(), authorizedFileIO, tableId); + } else { LOGGER .atWarn() - .addKeyValue("manifestFile", manifestFile.path()) .addKeyValue("tableId", tableId) - .log("Manifest cleanup task scheduled, but manifest file doesn't exist"); + .log("Cleanup task scheduled, but no file inputs were found"); return true; } + } + } - ManifestReader dataFiles = ManifestFiles.read(manifestFile, authorizedFileIO); - List> dataFileDeletes = - StreamSupport.stream( - Spliterators.spliteratorUnknownSize(dataFiles.iterator(), Spliterator.IMMUTABLE), - false) - .map( - file -> - tryDelete( - tableId, authorizedFileIO, manifestFile, file.path().toString(), null, 1)) - .toList(); - LOGGER.debug( - "Scheduled {} data files to be deleted from manifest {}", - dataFileDeletes.size(), - manifestFile.path()); - try { - // wait for all data files to be deleted, then wait for the manifest itself to be deleted - CompletableFuture.allOf(dataFileDeletes.toArray(CompletableFuture[]::new)) - .thenCompose( - (v) -> { - LOGGER - .atInfo() - .addKeyValue("manifestFile", manifestFile.path()) - .log("All data files in manifest deleted - deleting manifest"); - return tryDelete( - tableId, authorizedFileIO, manifestFile, manifestFile.path(), null, 1); - }) - .get(); - return true; - } catch (InterruptedException e) { - LOGGER.error( - "Interrupted exception deleting data files from manifest {}", manifestFile.path(), e); - throw new RuntimeException(e); - } catch (ExecutionException e) { - LOGGER.error("Unable to delete data files from manifest {}", manifestFile.path(), e); - return false; - } + private boolean manifestFileCleanUp( + ManifestFile manifestFile, FileIO fileIO, TableIdentifier tableId) { + // if the file doesn't exist, we assume that another task execution was successful, but + // failed to drop the task entity. Log a warning and return success + if (!TaskUtils.exists(manifestFile.path(), fileIO)) { + LOGGER + .atWarn() + .addKeyValue("manifestFile", manifestFile.path()) + .addKeyValue("tableId", tableId) + .log("Manifest cleanup task scheduled, but manifest file doesn't exist"); + return true; + } + + ManifestReader dataFiles = ManifestFiles.read(manifestFile, fileIO); + List> dataFileDeletes = + StreamSupport.stream( + Spliterators.spliteratorUnknownSize(dataFiles.iterator(), Spliterator.IMMUTABLE), + false) + .map(file -> tryDelete(tableId, fileIO, manifestFile, file.path().toString(), null, 1)) + .toList(); + LOGGER.debug( + "Scheduled {} data files to be deleted from manifest {}", + dataFileDeletes.size(), + manifestFile.path()); + try { + // wait for all data files to be deleted, then wait for the manifest itself to be deleted + CompletableFuture.allOf(dataFileDeletes.toArray(CompletableFuture[]::new)) + .thenCompose( + (v) -> { + LOGGER + .atInfo() + .addKeyValue("manifestFile", manifestFile.path()) + .log("All data files in manifest deleted - deleting manifest"); + return tryDelete(tableId, fileIO, manifestFile, manifestFile.path(), null, 1); + }) + .get(); + return true; + } catch (InterruptedException e) { + LOGGER.error( + "Interrupted exception deleting data files from manifest {}", manifestFile.path(), e); + throw new RuntimeException(e); + } catch (ExecutionException e) { + LOGGER.error("Unable to delete data files from manifest {}", manifestFile.path(), e); + return false; + } + } + + private boolean contentFileCleanup( + List contentFileBatch, FileIO fileIO, TableIdentifier tableId) { + List validFiles = + contentFileBatch.stream().filter(file -> TaskUtils.exists(file, fileIO)).toList(); + if (validFiles.isEmpty()) { + LOGGER + .atWarn() + .addKeyValue("contentFileBatch", contentFileBatch.toString()) + .addKeyValue("tableId", tableId) + .log("Table content cleanup task scheduled, but the none of the file in batch exists"); + return true; } + + // Schedule the deletion for each file asynchronously + List> deleteFutures = + validFiles.stream().map(file -> tryDelete(tableId, fileIO, null, file, null, 1)).toList(); + + // Wait for all delete operations to finish + try { + CompletableFuture allDeletes = + CompletableFuture.allOf(deleteFutures.toArray(new CompletableFuture[0])); + allDeletes.join(); + } catch (Exception e) { + LOGGER + .atWarn() + .addKeyValue("contentFileBatch", contentFileBatch.toString()) + .addKeyValue("tableId", tableId) + .log("Exception detected during content file batch deletion", e); + } + + LOGGER + .atInfo() + .addKeyValue("contentFileBatch", contentFileBatch.toString()) + .addKeyValue("tableId", tableId) + .log("Content file batch deletion has completed"); + + return true; } private static ManifestFile decodeManifestData(String manifestFileData) { @@ -134,16 +185,16 @@ private CompletableFuture tryDelete( TableIdentifier tableId, FileIO fileIO, ManifestFile manifestFile, - String dataFile, + String file, Throwable e, int attempt) { if (e != null && attempt <= MAX_ATTEMPTS) { LOGGER .atWarn() - .addKeyValue("dataFile", dataFile) + .addKeyValue("file", file) .addKeyValue("attempt", attempt) .addKeyValue("error", e.getMessage()) - .log("Error encountered attempting to delete data file"); + .log("Error encountered attempting to delete file"); } if (attempt > MAX_ATTEMPTS && e != null) { return CompletableFuture.failedFuture(e); @@ -155,15 +206,15 @@ private CompletableFuture tryDelete( // file's existence, but then it is deleted before we have a chance to // send the delete request. In such a case, we should retry // and find - if (TaskUtils.exists(dataFile, fileIO)) { - fileIO.deleteFile(dataFile); + if (TaskUtils.exists(file, fileIO)) { + fileIO.deleteFile(file); } else { LOGGER .atInfo() - .addKeyValue("dataFile", dataFile) - .addKeyValue("manifestFile", manifestFile.path()) + .addKeyValue("file", file) + .addKeyValue("manifestFile", manifestFile != null ? manifestFile.path() : "") .addKeyValue("tableId", tableId) - .log("Manifest cleanup task scheduled, but data file doesn't exist"); + .log("table file cleanup task scheduled, but data file doesn't exist"); } }, executorService) @@ -171,11 +222,11 @@ private CompletableFuture tryDelete( newEx -> { LOGGER .atWarn() - .addKeyValue("dataFile", dataFile) - .addKeyValue("tableIdentifer", tableId) - .addKeyValue("manifestFile", manifestFile.path()) + .addKeyValue("dataFile", file) + .addKeyValue("tableIdentifier", tableId) + .addKeyValue("manifestFile", manifestFile != null ? manifestFile.path() : "") .log("Exception caught deleting data file from manifest", newEx); - return tryDelete(tableId, fileIO, manifestFile, dataFile, newEx, attempt + 1); + return tryDelete(tableId, fileIO, manifestFile, file, newEx, attempt + 1); }, CompletableFuture.delayedExecutor( FILE_DELETION_RETRY_MILLIS, TimeUnit.MILLISECONDS, executorService)); @@ -185,12 +236,18 @@ private CompletableFuture tryDelete( public static final class ManifestCleanupTask { private TableIdentifier tableId; private String manifestFileData; + private List contentFileBatch; public ManifestCleanupTask(TableIdentifier tableId, String manifestFileData) { this.tableId = tableId; this.manifestFileData = manifestFileData; } + public ManifestCleanupTask(TableIdentifier tableId, List contentFileBatch) { + this.tableId = tableId; + this.contentFileBatch = contentFileBatch; + } + public ManifestCleanupTask() {} public TableIdentifier getTableId() { @@ -209,17 +266,26 @@ public void setManifestFileData(String manifestFileData) { this.manifestFileData = manifestFileData; } + public List getContentFileBatch() { + return contentFileBatch; + } + + public void setContentFileBatch(List contentFileBatch) { + this.contentFileBatch = contentFileBatch; + } + @Override public boolean equals(Object object) { if (this == object) return true; if (!(object instanceof ManifestCleanupTask that)) return false; return Objects.equals(tableId, that.tableId) - && Objects.equals(manifestFileData, that.manifestFileData); + && Objects.equals(manifestFileData, that.manifestFileData) + && Objects.equals(contentFileBatch, that.contentFileBatch); } @Override public int hashCode() { - return Objects.hash(tableId, manifestFileData); + return Objects.hash(tableId, manifestFileData, contentFileBatch); } } } diff --git a/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java b/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java index aa720fe26..10b1919fb 100644 --- a/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java +++ b/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java @@ -18,6 +18,7 @@ */ package org.apache.polaris.service.task; +import java.util.ArrayList; import java.util.List; import java.util.UUID; import java.util.function.Function; @@ -105,7 +106,7 @@ public boolean handleTask(TaskEntity cleanupTask) { // read the manifest list for each snapshot. dedupe the manifest files and schedule a // cleanupTask // for each manifest file and its data files to be deleted - List taskEntities = + Stream manifestCleanupTasks = tableMetadata.snapshots().stream() .flatMap(sn -> sn.allManifests(fileIO).stream()) // distinct by manifest path, since multiple snapshots will contain the same @@ -145,8 +146,40 @@ public boolean handleTask(TaskEntity cleanupTask) { // copy the internal properties, which will have storage info .setInternalProperties(cleanupTask.getInternalPropertiesAsMap()) .build(); - }) - .toList(); + }); + + Stream contentFileCleanupTasks = + getContentFileBatch(tableMetadata, fileIO).stream() + .map( + fileBatch -> { + String taskName = + cleanupTask.getName() + + "_" + + fileBatch.toString() + + "_" + + UUID.randomUUID(); + LOGGER + .atDebug() + .addKeyValue("taskName", taskName) + .addKeyValue("tableIdentifier", tableEntity.getTableIdentifier()) + .addKeyValue("fileBatch", fileBatch) + .log( + "Queueing task to delete content file (prev metadata and statistics files)"); + return new TaskEntity.Builder() + .setName(taskName) + .setId(metaStoreManager.generateNewEntityId(polarisCallContext).getId()) + .setCreateTimestamp(polarisCallContext.getClock().millis()) + .withTaskType(AsyncTaskType.FILE_CLEANUP) + .withData( + new ManifestFileCleanupTaskHandler.ManifestCleanupTask( + tableEntity.getTableIdentifier(), fileBatch)) + .setInternalProperties(cleanupTask.getInternalPropertiesAsMap()) + .build(); + }); + + List taskEntities = + Stream.concat(manifestCleanupTasks, contentFileCleanupTasks).toList(); + List createdTasks = metaStoreManager .createEntitiesIfNotExist(polarisCallContext, null, taskEntities) @@ -157,27 +190,12 @@ public boolean handleTask(TaskEntity cleanupTask) { .addKeyValue("tableIdentifier", tableEntity.getTableIdentifier()) .addKeyValue("metadataLocation", tableEntity.getMetadataLocation()) .addKeyValue("taskCount", taskEntities.size()) - .log("Successfully queued tasks to delete manifests - deleting table metadata file"); + .log( + "Successfully queued tasks to delete manifests, previous metadata, and statistics files - deleting table metadata file"); for (PolarisBaseEntity createdTask : createdTasks) { taskExecutor.addTaskHandlerContext(createdTask.getId(), CallContext.getCurrentContext()); } - // Schedule and dispatch prev metadata and stat files in seperated tasks - scheduleTableContentCleanupTask( - tableMetadata.previousFiles().stream().map(TableMetadata.MetadataLogEntry::file), - CleanupTableContentFileType.PREV_METADATA, - fileIO, - cleanupTask, - metaStoreManager, - polarisCallContext); - scheduleTableContentCleanupTask( - tableMetadata.statisticsFiles().stream().map(StatisticsFile::path), - CleanupTableContentFileType.STATISTICS, - fileIO, - cleanupTask, - metaStoreManager, - polarisCallContext); - fileIO.deleteFile(tableEntity.getMetadataLocation()); return true; @@ -186,73 +204,18 @@ public boolean handleTask(TaskEntity cleanupTask) { return false; } - private void scheduleTableContentCleanupTask( - Stream fileStream, - CleanupTableContentFileType fileType, - FileIO fileIO, - TaskEntity cleanupTask, - PolarisMetaStoreManager metaStoreManager, - PolarisCallContext polarisCallContext) { - PolarisBaseEntity entity = cleanupTask.readData(PolarisBaseEntity.class); - TableLikeEntity tableEntity = TableLikeEntity.of(entity); - - List validFiles = fileStream.filter(file -> TaskUtils.exists(file, fileIO)).toList(); - - for (int i = 0; i < validFiles.size(); i += BATCH_SIZE) { - List fileBatch = validFiles.subList(i, Math.min(i + BATCH_SIZE, validFiles.size())); - String taskName = cleanupTask.getName() + "_batch" + i + "_" + UUID.randomUUID(); - LOGGER - .atDebug() - .addKeyValue("taskName", taskName) - .addKeyValue("tableIdentifier", tableEntity.getTableIdentifier()) - .addKeyValue("fileBatch", fileBatch.toString()) - .log("Queueing task to delete a batch of " + fileType.getTypeName()); - - TaskEntity batchTask = - new TaskEntity.Builder() - .setName(taskName) - .setId(metaStoreManager.generateNewEntityId(polarisCallContext).getId()) - .setCreateTimestamp(polarisCallContext.getClock().millis()) - .withTaskType(AsyncTaskType.TABLE_CONTENT_CLEANUP) - .withData( - new TableContentCleanupTaskHandler.TableContentCleanupTask( - tableEntity.getTableIdentifier(), fileBatch)) - .setInternalProperties(cleanupTask.getInternalPropertiesAsMap()) - .build(); - - List createdTasks = - metaStoreManager - .createEntitiesIfNotExist(polarisCallContext, null, List.of(batchTask)) - .getEntities(); - - if (createdTasks != null) { - LOGGER - .atInfo() - .addKeyValue("tableIdentifier", tableEntity.getTableIdentifier()) - .addKeyValue("taskCount", createdTasks.size()) - .addKeyValue("fileBatch", fileBatch.toString()) - .log("Successfully queued task to delete a batch of " + fileType.getTypeName() + "s"); - - for (PolarisBaseEntity createdTask : createdTasks) { - taskExecutor.addTaskHandlerContext(createdTask.getId(), CallContext.getCurrentContext()); - } - } - } - } - - private enum CleanupTableContentFileType { - PREV_METADATA("previous metadata file"), - STATISTICS("statistics file"), - ; - - private final String typeName; - - CleanupTableContentFileType(String typeName) { - this.typeName = typeName; - } - - public String getTypeName() { - return typeName; + private List> getContentFileBatch(TableMetadata tableMetadata, FileIO fileIO) { + List> result = new ArrayList<>(); + List contentFiles = + Stream.concat( + tableMetadata.previousFiles().stream().map(TableMetadata.MetadataLogEntry::file), + tableMetadata.statisticsFiles().stream().map(StatisticsFile::path)) + .filter(file -> TaskUtils.exists(file, fileIO)) + .toList(); + + for (int i = 0; i < contentFiles.size(); i += BATCH_SIZE) { + result.add(contentFiles.subList(i, Math.min(i + BATCH_SIZE, contentFiles.size()))); } + return result; } } diff --git a/polaris-service/src/main/java/org/apache/polaris/service/task/TableContentCleanupTaskHandler.java b/polaris-service/src/main/java/org/apache/polaris/service/task/TableContentCleanupTaskHandler.java deleted file mode 100644 index 300f3d202..000000000 --- a/polaris-service/src/main/java/org/apache/polaris/service/task/TableContentCleanupTaskHandler.java +++ /dev/null @@ -1,189 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.polaris.service.task; - -import java.util.List; -import java.util.Objects; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.TimeUnit; -import java.util.function.Function; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.io.FileIO; -import org.apache.polaris.core.entity.AsyncTaskType; -import org.apache.polaris.core.entity.TaskEntity; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * {@link TaskHandler} responsible for deleting previous metadata and statistics files of a table. - */ -public class TableContentCleanupTaskHandler implements TaskHandler { - public static final int MAX_ATTEMPTS = 3; - public static final int FILE_DELETION_RETRY_MILLIS = 100; - private static final Logger LOGGER = - LoggerFactory.getLogger(TableContentCleanupTaskHandler.class); - private final Function fileIOSupplier; - private final ExecutorService executorService; - - public TableContentCleanupTaskHandler( - Function fileIOSupplier, ExecutorService executorService) { - this.fileIOSupplier = fileIOSupplier; - this.executorService = executorService; - } - - @Override - public boolean canHandleTask(TaskEntity task) { - return task.getTaskType() == AsyncTaskType.TABLE_CONTENT_CLEANUP; - } - - @Override - public boolean handleTask(TaskEntity task) { - TableContentCleanupTask cleanupTask = task.readData(TableContentCleanupTask.class); - List fileBatch = cleanupTask.getFileBatch(); - TableIdentifier tableId = cleanupTask.getTableId(); - try (FileIO authorizedFileIO = fileIOSupplier.apply(task)) { - List validFiles = - fileBatch.stream().filter(file -> TaskUtils.exists(file, authorizedFileIO)).toList(); - if (validFiles.isEmpty()) { - LOGGER - .atWarn() - .addKeyValue("taskName", task.getName()) - .addKeyValue("fileBatch", fileBatch.toString()) - .addKeyValue("tableId", tableId) - .log("Table content cleanup task scheduled, but the none of the file in batch exists"); - return true; - } - - // Schedule the deletion for each file asynchronously - List> deleteFutures = - validFiles.stream() - .map(file -> tryDelete(tableId, authorizedFileIO, file, null, 1)) - .toList(); - - // Wait for all delete operations to finish - CompletableFuture allDeletes = - CompletableFuture.allOf(deleteFutures.toArray(new CompletableFuture[0])); - allDeletes.join(); - - LOGGER - .atInfo() - .addKeyValue("taskName", task.getName()) - .addKeyValue("fileBatch", fileBatch.toString()) - .addKeyValue("tableId", tableId) - .log("All the files in task have been deleted"); - - return true; - } catch (Exception e) { - LOGGER.error("Error during table content cleanup for file batch {}", fileBatch.toString(), e); - return false; - } - } - - private CompletableFuture tryDelete( - TableIdentifier tableId, FileIO fileIO, String filePath, Throwable e, int attempt) { - if (e != null && attempt <= MAX_ATTEMPTS) { - LOGGER - .atWarn() - .addKeyValue("filePath", filePath) - .addKeyValue("attempt", attempt) - .addKeyValue("error", e.getMessage()) - .log("Error encountered attempting to delete file"); - } - - if (attempt > MAX_ATTEMPTS && e != null) { - return CompletableFuture.failedFuture(e); - } - - return CompletableFuture.runAsync( - () -> { - if (TaskUtils.exists(filePath, fileIO)) { - fileIO.deleteFile(filePath); - LOGGER - .atInfo() - .addKeyValue("filePath", filePath) - .addKeyValue("tableId", tableId) - .addKeyValue("attempt", attempt) - .log("Successfully deleted file {}", filePath); - } else { - LOGGER - .atInfo() - .addKeyValue("filePath", filePath) - .addKeyValue("tableId", tableId) - .log("File doesn't exist, likely already deleted"); - } - }, - executorService) - .exceptionallyComposeAsync( - newEx -> { - LOGGER - .atWarn() - .addKeyValue("filePath", filePath) - .addKeyValue("tableId", tableId) - .log("Exception caught deleting table content file", newEx); - return tryDelete(tableId, fileIO, filePath, newEx, attempt + 1); - }, - CompletableFuture.delayedExecutor( - FILE_DELETION_RETRY_MILLIS, TimeUnit.MILLISECONDS, executorService)); - } - - public static final class TableContentCleanupTask { - private TableIdentifier tableId; - private List fileBatch; - - public TableContentCleanupTask() {} - - public TableContentCleanupTask(TableIdentifier tableId, List fileBatch) { - this.tableId = tableId; - this.fileBatch = fileBatch; - } - - public TableIdentifier getTableId() { - return tableId; - } - - public void setTableId(TableIdentifier tableId) { - this.tableId = tableId; - } - - public List getFileBatch() { - return fileBatch; - } - - public void setFileBatch(List fileBatch) { - this.fileBatch = fileBatch; - } - - @Override - public boolean equals(Object object) { - if (this == object) { - return true; - } - if (!(object instanceof TableContentCleanupTask other)) { - return false; - } - return Objects.equals(tableId, other.tableId) && Objects.equals(fileBatch, other.fileBatch); - } - - @Override - public int hashCode() { - return Objects.hash(tableId, fileBatch.toString()); - } - } -} diff --git a/polaris-service/src/test/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandlerTest.java b/polaris-service/src/test/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandlerTest.java index 711e661f2..862e90237 100644 --- a/polaris-service/src/test/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandlerTest.java +++ b/polaris-service/src/test/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandlerTest.java @@ -19,17 +19,24 @@ package org.apache.polaris.service.task; import static java.nio.charset.StandardCharsets.UTF_8; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatPredicate; import java.io.IOException; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.UUID; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Stream; import org.apache.commons.codec.binary.Base64; import org.apache.iceberg.ManifestFile; import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StatisticsFile; +import org.apache.iceberg.TableMetadata; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.inmemory.InMemoryFileIO; @@ -226,4 +233,228 @@ public void deleteFile(String location) { assertThatPredicate((String f) -> TaskUtils.exists(f, fileIO)).rejects(dataFile2Path); } } + + @Test + public void testContentFileCleanup() throws IOException { + PolarisCallContext polarisCallContext = + new PolarisCallContext( + metaStoreManagerFactory.getOrCreateSessionSupplier(realmContext).get(), + new PolarisDefaultDiagServiceImpl()); + try (CallContext callCtx = CallContext.of(realmContext, polarisCallContext)) { + CallContext.setCurrentContext(callCtx); + FileIO fileIO = + new InMemoryFileIO() { + @Override + public void close() { + // no-op + } + }; + TableIdentifier tableIdentifier = + TableIdentifier.of(Namespace.of("db1", "schema1"), "table1"); + ManifestFileCleanupTaskHandler handler = + new ManifestFileCleanupTaskHandler((task) -> fileIO, Executors.newSingleThreadExecutor()); + + long snapshotId1 = 100L; + ManifestFile manifestFile1 = + TaskTestUtils.manifestFile( + fileIO, "manifest1.avro", snapshotId1, "dataFile1.parquet", "dataFile2.parquet"); + ManifestFile manifestFile2 = + TaskTestUtils.manifestFile( + fileIO, "manifest2.avro", snapshotId1, "dataFile3.parquet", "dataFile4.parquet"); + Snapshot snapshot = + TaskTestUtils.newSnapshot( + fileIO, "manifestList.avro", 1, snapshotId1, 99L, manifestFile1, manifestFile2); + StatisticsFile statisticsFile1 = + TaskTestUtils.writeStatsFile( + snapshot.snapshotId(), + snapshot.sequenceNumber(), + "/metadata/" + UUID.randomUUID() + ".stats", + fileIO); + String firstMetadataFile = "v1-295495059.metadata.json"; + TableMetadata firstMetadata = + TaskTestUtils.writeTableMetadata( + fileIO, firstMetadataFile, List.of(statisticsFile1), snapshot); + assertThat(TaskUtils.exists(firstMetadataFile, fileIO)).isTrue(); + + ManifestFile manifestFile3 = + TaskTestUtils.manifestFile( + fileIO, "manifest3.avro", snapshot.snapshotId() + 1, "dataFile5.parquet"); + Snapshot snapshot2 = + TaskTestUtils.newSnapshot( + fileIO, + "manifestList2.avro", + snapshot.sequenceNumber() + 1, + snapshot.snapshotId() + 1, + snapshot.snapshotId(), + manifestFile1, + manifestFile3); // exclude manifest2 from the new snapshot + StatisticsFile statisticsFile2 = + TaskTestUtils.writeStatsFile( + snapshot2.snapshotId(), + snapshot2.sequenceNumber(), + "/metadata/" + UUID.randomUUID() + ".stats", + fileIO); + String secondMetadataFile = "v1-295495060.metadata.json"; + TableMetadata secondMetadata = + TaskTestUtils.writeTableMetadata( + fileIO, + secondMetadataFile, + firstMetadata, + firstMetadataFile, + List.of(statisticsFile2), + snapshot2); + assertThat(TaskUtils.exists(firstMetadataFile, fileIO)).isTrue(); + assertThat(TaskUtils.exists(secondMetadataFile, fileIO)).isTrue(); + + List cleanupFiles = + Stream.concat( + secondMetadata.previousFiles().stream() + .map(TableMetadata.MetadataLogEntry::file) + .filter(file -> TaskUtils.exists(file, fileIO)), + secondMetadata.statisticsFiles().stream() + .map(StatisticsFile::path) + .filter(file -> TaskUtils.exists(file, fileIO))) + .toList(); + + TaskEntity task = + new TaskEntity.Builder() + .withTaskType(AsyncTaskType.FILE_CLEANUP) + .withData( + new ManifestFileCleanupTaskHandler.ManifestCleanupTask( + tableIdentifier, cleanupFiles)) + .setName(UUID.randomUUID().toString()) + .build(); + + assertThatPredicate(handler::canHandleTask).accepts(task); + assertThatPredicate(handler::handleTask).accepts(task); + + assertThatPredicate((String file) -> TaskUtils.exists(file, fileIO)) + .rejects(firstMetadataFile); + assertThatPredicate((String file) -> TaskUtils.exists(file, fileIO)) + .rejects(statisticsFile1.path()); + assertThatPredicate((String file) -> TaskUtils.exists(file, fileIO)) + .rejects(statisticsFile2.path()); + } + } + + @Test + public void testContentFileCleanupIfFileNotExist() throws IOException { + PolarisCallContext polarisCallContext = + new PolarisCallContext( + metaStoreManagerFactory.getOrCreateSessionSupplier(realmContext).get(), + new PolarisDefaultDiagServiceImpl()); + try (CallContext callCtx = CallContext.of(realmContext, polarisCallContext)) { + CallContext.setCurrentContext(callCtx); + FileIO fileIO = new InMemoryFileIO(); + TableIdentifier tableIdentifier = + TableIdentifier.of(Namespace.of("db1", "schema1"), "table1"); + ManifestFileCleanupTaskHandler handler = + new ManifestFileCleanupTaskHandler((task) -> fileIO, Executors.newSingleThreadExecutor()); + long snapshotId = 100L; + ManifestFile manifestFile = + TaskTestUtils.manifestFile( + fileIO, "manifest1.avro", snapshotId, "dataFile1.parquet", "dataFile2.parquet"); + TestSnapshot snapshot = + TaskTestUtils.newSnapshot(fileIO, "manifestList.avro", 1, snapshotId, 99L, manifestFile); + String metadataFile = "v1-49494949.metadata.json"; + StatisticsFile statisticsFile = + TaskTestUtils.writeStatsFile( + snapshot.snapshotId(), + snapshot.sequenceNumber(), + "/metadata/" + UUID.randomUUID() + ".stats", + fileIO); + TaskTestUtils.writeTableMetadata(fileIO, metadataFile, List.of(statisticsFile), snapshot); + + fileIO.deleteFile(statisticsFile.path()); + assertThat(TaskUtils.exists(statisticsFile.path(), fileIO)).isFalse(); + + TaskEntity task = + new TaskEntity.Builder() + .withTaskType(AsyncTaskType.FILE_CLEANUP) + .withData( + new ManifestFileCleanupTaskHandler.ManifestCleanupTask( + tableIdentifier, List.of(statisticsFile.path()))) + .setName(UUID.randomUUID().toString()) + .build(); + assertThatPredicate(handler::canHandleTask).accepts(task); + assertThatPredicate(handler::handleTask).accepts(task); + } + } + + @Test + public void testCleanupWithRetries() throws IOException { + PolarisCallContext polarisCallContext = + new PolarisCallContext( + metaStoreManagerFactory.getOrCreateSessionSupplier(realmContext).get(), + new PolarisDefaultDiagServiceImpl()); + try (CallContext callCtx = CallContext.of(realmContext, polarisCallContext)) { + CallContext.setCurrentContext(callCtx); + Map retryCounter = new HashMap<>(); + FileIO fileIO = + new InMemoryFileIO() { + @Override + public void close() { + // no-op + } + + @Override + public void deleteFile(String location) { + int attempts = + retryCounter + .computeIfAbsent(location, k -> new AtomicInteger(0)) + .incrementAndGet(); + if (attempts < 3) { + throw new RuntimeException("Simulating failure to test retries"); + } else { + super.deleteFile(location); + } + } + }; + TableIdentifier tableIdentifier = + TableIdentifier.of(Namespace.of("db1", "schema1"), "table1"); + ManifestFileCleanupTaskHandler handler = + new ManifestFileCleanupTaskHandler((task) -> fileIO, Executors.newSingleThreadExecutor()); + long snapshotId = 100L; + ManifestFile manifestFile = + TaskTestUtils.manifestFile( + fileIO, "manifest1.avro", snapshotId, "dataFile1.parquet", "dataFile2.parquet"); + TestSnapshot snapshot = + TaskTestUtils.newSnapshot(fileIO, "manifestList.avro", 1, snapshotId, 99L, manifestFile); + String metadataFile = "v1-49494949.metadata.json"; + StatisticsFile statisticsFile = + TaskTestUtils.writeStatsFile( + snapshot.snapshotId(), + snapshot.sequenceNumber(), + "/metadata/" + UUID.randomUUID() + ".stats", + fileIO); + TaskTestUtils.writeTableMetadata(fileIO, metadataFile, List.of(statisticsFile), snapshot); + assertThat(TaskUtils.exists(statisticsFile.path(), fileIO)).isTrue(); + + TaskEntity task = + new TaskEntity.Builder() + .withTaskType(AsyncTaskType.FILE_CLEANUP) + .withData( + new ManifestFileCleanupTaskHandler.ManifestCleanupTask( + tableIdentifier, List.of(statisticsFile.path()))) + .setName(UUID.randomUUID().toString()) + .build(); + + CompletableFuture future = + CompletableFuture.runAsync( + () -> { + assertThatPredicate(handler::canHandleTask).accepts(task); + handler.handleTask(task); // this will schedule the batch deletion + }); + + // Wait for all async tasks to finish + future.join(); + + // Check if the file was successfully deleted after retries + assertThat(TaskUtils.exists(statisticsFile.path(), fileIO)).isFalse(); + + // Ensure that retries happened as expected + assertThat(retryCounter.containsKey(statisticsFile.path())).isTrue(); + assertThat(retryCounter.get(statisticsFile.path()).get()).isEqualTo(3); + } + } } diff --git a/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java b/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java index f5df6ebb2..17ae80618 100644 --- a/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java +++ b/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java @@ -126,13 +126,13 @@ public void testTableCleanup() throws IOException { assertThat(taskEntity) .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) .extracting(TaskEntity::of) - .returns(AsyncTaskType.TABLE_CONTENT_CLEANUP, TaskEntity::getTaskType) + .returns(AsyncTaskType.FILE_CLEANUP, TaskEntity::getTaskType) .returns( - new TableContentCleanupTaskHandler.TableContentCleanupTask( + new ManifestFileCleanupTaskHandler.ManifestCleanupTask( tableIdentifier, List.of(statisticsFile.path())), entity -> entity.readData( - TableContentCleanupTaskHandler.TableContentCleanupTask.class))); + ManifestFileCleanupTaskHandler.ManifestCleanupTask.class))); } } @@ -369,19 +369,10 @@ public void testTableCleanupMultipleSnapshots() throws IOException { }) .toList(); - List StatsCleanupTasks = - entities.stream() - .filter( - entity -> { - AsyncTaskType taskType = TaskEntity.of(entity).getTaskType(); - return taskType == AsyncTaskType.TABLE_CONTENT_CLEANUP; - }) - .toList(); - assertThat(manifestCleanupTasks) // all three manifests should be present, even though one is excluded from the latest // snapshot - .hasSize(3) + .hasSize(4) .satisfiesExactlyInAnyOrder( taskEntity -> assertThat(taskEntity) @@ -415,23 +406,18 @@ public void testTableCleanupMultipleSnapshots() throws IOException { Base64.encodeBase64String(ManifestFiles.encode(manifestFile3))), entity -> entity.readData( - ManifestFileCleanupTaskHandler.ManifestCleanupTask.class))); - - assertThat(StatsCleanupTasks) - .hasSize(1) - .satisfiesExactlyInAnyOrder( + ManifestFileCleanupTaskHandler.ManifestCleanupTask.class)), taskEntity -> assertThat(taskEntity) .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) .extracting(TaskEntity::of) - .returns(AsyncTaskType.TABLE_CONTENT_CLEANUP, TaskEntity::getTaskType) .returns( - new TableContentCleanupTaskHandler.TableContentCleanupTask( + new ManifestFileCleanupTaskHandler.ManifestCleanupTask( tableIdentifier, List.of(statisticsFile1.path(), statisticsFile2.path())), entity -> entity.readData( - TableContentCleanupTaskHandler.TableContentCleanupTask.class))); + ManifestFileCleanupTaskHandler.ManifestCleanupTask.class))); } } @@ -531,19 +517,10 @@ public void testTableCleanupMultipleMetadata() throws IOException { }) .toList(); - List PrevMetadataNStatCleanupTasks = - entities.stream() - .filter( - entity -> { - AsyncTaskType taskType = TaskEntity.of(entity).getTaskType(); - return taskType == AsyncTaskType.TABLE_CONTENT_CLEANUP; - }) - .toList(); - assertThat(manifestCleanupTasks) // all three manifests should be present, even though one is excluded from the latest // snapshot - .hasSize(3) + .hasSize(4) .satisfiesExactlyInAnyOrder( taskEntity -> assertThat(taskEntity) @@ -577,34 +554,21 @@ public void testTableCleanupMultipleMetadata() throws IOException { Base64.encodeBase64String(ManifestFiles.encode(manifestFile3))), entity -> entity.readData( - ManifestFileCleanupTaskHandler.ManifestCleanupTask.class))); - - assertThat(PrevMetadataNStatCleanupTasks) - .hasSize(2) - .satisfiesExactlyInAnyOrder( + ManifestFileCleanupTaskHandler.ManifestCleanupTask.class)), taskEntity -> assertThat(taskEntity) .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) .extracting(TaskEntity::of) - .returns(AsyncTaskType.TABLE_CONTENT_CLEANUP, TaskEntity::getTaskType) .returns( - new TableContentCleanupTaskHandler.TableContentCleanupTask( + new ManifestFileCleanupTaskHandler.ManifestCleanupTask( tableIdentifier, - List.of(statisticsFile1.path(), statisticsFile2.path())), + List.of( + firstMetadataFile, + statisticsFile1.path(), + statisticsFile2.path())), entity -> entity.readData( - TableContentCleanupTaskHandler.TableContentCleanupTask.class)), - taskEntity -> - assertThat(taskEntity) - .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) - .extracting(TaskEntity::of) - .returns(AsyncTaskType.TABLE_CONTENT_CLEANUP, TaskEntity::getTaskType) - .returns( - new TableContentCleanupTaskHandler.TableContentCleanupTask( - tableIdentifier, List.of(firstMetadataFile)), - entity -> - entity.readData( - TableContentCleanupTaskHandler.TableContentCleanupTask.class))); + ManifestFileCleanupTaskHandler.ManifestCleanupTask.class))); } } } diff --git a/polaris-service/src/test/java/org/apache/polaris/service/task/TableContentCleanupTaskHandlerTest.java b/polaris-service/src/test/java/org/apache/polaris/service/task/TableContentCleanupTaskHandlerTest.java deleted file mode 100644 index 4f2e8ae5d..000000000 --- a/polaris-service/src/test/java/org/apache/polaris/service/task/TableContentCleanupTaskHandlerTest.java +++ /dev/null @@ -1,294 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.polaris.service.task; - -import static org.assertj.core.api.Assertions.assertThatPredicate; -import static org.assertj.core.api.AssertionsForClassTypes.assertThat; - -import java.io.IOException; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.stream.Stream; -import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.StatisticsFile; -import org.apache.iceberg.TableMetadata; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.inmemory.InMemoryFileIO; -import org.apache.iceberg.io.FileIO; -import org.apache.polaris.core.PolarisCallContext; -import org.apache.polaris.core.PolarisDefaultDiagServiceImpl; -import org.apache.polaris.core.context.CallContext; -import org.apache.polaris.core.context.RealmContext; -import org.apache.polaris.core.entity.AsyncTaskType; -import org.apache.polaris.core.entity.TaskEntity; -import org.apache.polaris.service.persistence.InMemoryPolarisMetaStoreManagerFactory; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TableContentCleanupTaskHandlerTest { - private InMemoryPolarisMetaStoreManagerFactory metaStoreManagerFactory; - private RealmContext realmContext; - private static final Logger LOGGER = - LoggerFactory.getLogger(TableContentCleanupTaskHandlerTest.class); - - @BeforeEach - void setUp() { - metaStoreManagerFactory = new InMemoryPolarisMetaStoreManagerFactory(); - realmContext = () -> "realmName"; - } - - @Test - public void testCleanup() throws IOException { - PolarisCallContext polarisCallContext = - new PolarisCallContext( - metaStoreManagerFactory.getOrCreateSessionSupplier(realmContext).get(), - new PolarisDefaultDiagServiceImpl()); - try (CallContext callCtx = CallContext.of(realmContext, polarisCallContext)) { - CallContext.setCurrentContext(callCtx); - FileIO fileIO = - new InMemoryFileIO() { - @Override - public void close() { - // no-op - } - }; - TableIdentifier tableIdentifier = - TableIdentifier.of(Namespace.of("db1", "schema1"), "table1"); - ExecutorService executorService = Executors.newCachedThreadPool(); - TableContentCleanupTaskHandler handler = - new TableContentCleanupTaskHandler((task) -> fileIO, executorService); - - long snapshotId1 = 100L; - ManifestFile manifestFile1 = - TaskTestUtils.manifestFile( - fileIO, "manifest1.avro", snapshotId1, "dataFile1.parquet", "dataFile2.parquet"); - ManifestFile manifestFile2 = - TaskTestUtils.manifestFile( - fileIO, "manifest2.avro", snapshotId1, "dataFile3.parquet", "dataFile4.parquet"); - Snapshot snapshot = - TaskTestUtils.newSnapshot( - fileIO, "manifestList.avro", 1, snapshotId1, 99L, manifestFile1, manifestFile2); - StatisticsFile statisticsFile1 = - TaskTestUtils.writeStatsFile( - snapshot.snapshotId(), - snapshot.sequenceNumber(), - "/metadata/" + UUID.randomUUID() + ".stats", - fileIO); - String firstMetadataFile = "v1-295495059.metadata.json"; - TableMetadata firstMetadata = - TaskTestUtils.writeTableMetadata( - fileIO, firstMetadataFile, List.of(statisticsFile1), snapshot); - assertThat(TaskUtils.exists(firstMetadataFile, fileIO)).isTrue(); - - ManifestFile manifestFile3 = - TaskTestUtils.manifestFile( - fileIO, "manifest3.avro", snapshot.snapshotId() + 1, "dataFile5.parquet"); - Snapshot snapshot2 = - TaskTestUtils.newSnapshot( - fileIO, - "manifestList2.avro", - snapshot.sequenceNumber() + 1, - snapshot.snapshotId() + 1, - snapshot.snapshotId(), - manifestFile1, - manifestFile3); // exclude manifest2 from the new snapshot - StatisticsFile statisticsFile2 = - TaskTestUtils.writeStatsFile( - snapshot2.snapshotId(), - snapshot2.sequenceNumber(), - "/metadata/" + UUID.randomUUID() + ".stats", - fileIO); - String secondMetadataFile = "v1-295495060.metadata.json"; - TableMetadata secondMetadata = - TaskTestUtils.writeTableMetadata( - fileIO, - secondMetadataFile, - firstMetadata, - firstMetadataFile, - List.of(statisticsFile2), - snapshot2); - assertThat(TaskUtils.exists(firstMetadataFile, fileIO)).isTrue(); - assertThat(TaskUtils.exists(secondMetadataFile, fileIO)).isTrue(); - - List cleanupFiles = - Stream.concat( - secondMetadata.previousFiles().stream() - .map(TableMetadata.MetadataLogEntry::file) - .filter(file -> TaskUtils.exists(file, fileIO)), - secondMetadata.statisticsFiles().stream() - .map(StatisticsFile::path) - .filter(file -> TaskUtils.exists(file, fileIO))) - .toList(); - - TaskEntity task = - new TaskEntity.Builder() - .withTaskType(AsyncTaskType.TABLE_CONTENT_CLEANUP) - .withData( - new TableContentCleanupTaskHandler.TableContentCleanupTask( - tableIdentifier, cleanupFiles)) - .setName(UUID.randomUUID().toString()) - .build(); - - assertThatPredicate(handler::canHandleTask).accepts(task); - assertThatPredicate(handler::handleTask).accepts(task); - - assertThatPredicate((String file) -> TaskUtils.exists(file, fileIO)) - .rejects(firstMetadataFile); - assertThatPredicate((String file) -> TaskUtils.exists(file, fileIO)) - .rejects(statisticsFile1.path()); - assertThatPredicate((String file) -> TaskUtils.exists(file, fileIO)) - .rejects(statisticsFile2.path()); - } - } - - @Test - public void testCleanupIfFileNotExist() throws IOException { - PolarisCallContext polarisCallContext = - new PolarisCallContext( - metaStoreManagerFactory.getOrCreateSessionSupplier(realmContext).get(), - new PolarisDefaultDiagServiceImpl()); - try (CallContext callCtx = CallContext.of(realmContext, polarisCallContext)) { - CallContext.setCurrentContext(callCtx); - FileIO fileIO = new InMemoryFileIO(); - TableIdentifier tableIdentifier = - TableIdentifier.of(Namespace.of("db1", "schema1"), "table1"); - TableContentCleanupTaskHandler handler = - new TableContentCleanupTaskHandler((task) -> fileIO, Executors.newCachedThreadPool()); - long snapshotId = 100L; - ManifestFile manifestFile = - TaskTestUtils.manifestFile( - fileIO, "manifest1.avro", snapshotId, "dataFile1.parquet", "dataFile2.parquet"); - TestSnapshot snapshot = - TaskTestUtils.newSnapshot(fileIO, "manifestList.avro", 1, snapshotId, 99L, manifestFile); - String metadataFile = "v1-49494949.metadata.json"; - StatisticsFile statisticsFile = - TaskTestUtils.writeStatsFile( - snapshot.snapshotId(), - snapshot.sequenceNumber(), - "/metadata/" + UUID.randomUUID() + ".stats", - fileIO); - TaskTestUtils.writeTableMetadata(fileIO, metadataFile, List.of(statisticsFile), snapshot); - - fileIO.deleteFile(statisticsFile.path()); - assertThat(TaskUtils.exists(statisticsFile.path(), fileIO)).isFalse(); - - TaskEntity task = - new TaskEntity.Builder() - .withTaskType(AsyncTaskType.TABLE_CONTENT_CLEANUP) - .withData( - new TableContentCleanupTaskHandler.TableContentCleanupTask( - tableIdentifier, List.of(statisticsFile.path()))) - .setName(UUID.randomUUID().toString()) - .build(); - assertThatPredicate(handler::canHandleTask).accepts(task); - assertThatPredicate(handler::handleTask).accepts(task); - } - } - - @Test - public void testCleanupWithRetries() throws IOException { - PolarisCallContext polarisCallContext = - new PolarisCallContext( - metaStoreManagerFactory.getOrCreateSessionSupplier(realmContext).get(), - new PolarisDefaultDiagServiceImpl()); - try (CallContext callCtx = CallContext.of(realmContext, polarisCallContext)) { - CallContext.setCurrentContext(callCtx); - Map retryCounter = new HashMap<>(); - FileIO fileIO = - new InMemoryFileIO() { - @Override - public void close() { - // no-op - } - - @Override - public void deleteFile(String location) { - int attempts = - retryCounter - .computeIfAbsent(location, k -> new AtomicInteger(0)) - .incrementAndGet(); - LOGGER.info("Attempt {} to delete file: {}", attempts, location); - if (attempts < 3) { - LOGGER.warn("Simulating failure for retry {}", attempts); - throw new RuntimeException("Simulating failure to test retries"); - } else { - // Succeed on the third attempt - LOGGER.info("Actually delete file: {}", location); - super.deleteFile(location); - } - } - }; - TableIdentifier tableIdentifier = - TableIdentifier.of(Namespace.of("db1", "schema1"), "table1"); - TableContentCleanupTaskHandler handler = - new TableContentCleanupTaskHandler((task) -> fileIO, Executors.newCachedThreadPool()); - long snapshotId = 100L; - ManifestFile manifestFile = - TaskTestUtils.manifestFile( - fileIO, "manifest1.avro", snapshotId, "dataFile1.parquet", "dataFile2.parquet"); - TestSnapshot snapshot = - TaskTestUtils.newSnapshot(fileIO, "manifestList.avro", 1, snapshotId, 99L, manifestFile); - String metadataFile = "v1-49494949.metadata.json"; - StatisticsFile statisticsFile = - TaskTestUtils.writeStatsFile( - snapshot.snapshotId(), - snapshot.sequenceNumber(), - "/metadata/" + UUID.randomUUID() + ".stats", - fileIO); - TaskTestUtils.writeTableMetadata(fileIO, metadataFile, List.of(statisticsFile), snapshot); - assertThat(TaskUtils.exists(statisticsFile.path(), fileIO)).isTrue(); - - TaskEntity task = - new TaskEntity.Builder() - .withTaskType(AsyncTaskType.TABLE_CONTENT_CLEANUP) - .withData( - new TableContentCleanupTaskHandler.TableContentCleanupTask( - tableIdentifier, List.of(statisticsFile.path()))) - .setName(UUID.randomUUID().toString()) - .build(); - - CompletableFuture future = - CompletableFuture.runAsync( - () -> { - assertThatPredicate(handler::canHandleTask).accepts(task); - handler.handleTask(task); // this will schedule the batch deletion - }); - - // Wait for all async tasks to finish - future.join(); - - // Check if the file was successfully deleted after retries - assertThat(TaskUtils.exists(statisticsFile.path(), fileIO)).isFalse(); - - // Ensure that retries happened as expected - assertThat(retryCounter.containsKey(statisticsFile.path())).isTrue(); - assertThat(retryCounter.get(statisticsFile.path()).get()).isEqualTo(3); - } - } -} From eb533d747771d19d248f4a445ccbd5d103d78488 Mon Sep 17 00:00:00 2001 From: Daniel Tu Date: Sun, 3 Nov 2024 22:05:25 -0800 Subject: [PATCH 23/29] code format --- .../service/task/ManifestFileCleanupTaskHandler.java | 10 +++++----- .../polaris/service/task/TableCleanupTaskHandler.java | 10 +++++----- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/polaris-service/src/main/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandler.java b/polaris-service/src/main/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandler.java index 8ee8fe8eb..f906daba3 100644 --- a/polaris-service/src/main/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandler.java +++ b/polaris-service/src/main/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandler.java @@ -74,20 +74,20 @@ public boolean handleTask(TaskEntity task) { try (FileIO authorizedFileIO = fileIOSupplier.apply(task)) { if (cleanupTask.getManifestFileData() != null) { ManifestFile manifestFile = decodeManifestData(cleanupTask.getManifestFileData()); - return manifestFileCleanUp(manifestFile, authorizedFileIO, tableId); + return cleanUpManifestFile(manifestFile, authorizedFileIO, tableId); } else if (cleanupTask.getContentFileBatch() != null) { - return contentFileCleanup(cleanupTask.getContentFileBatch(), authorizedFileIO, tableId); + return cleanUpContentFiles(cleanupTask.getContentFileBatch(), authorizedFileIO, tableId); } else { LOGGER .atWarn() .addKeyValue("tableId", tableId) - .log("Cleanup task scheduled, but no file inputs were found"); + .log("Cleanup task scheduled, but input file doesn't exist"); return true; } } } - private boolean manifestFileCleanUp( + private boolean cleanUpManifestFile( ManifestFile manifestFile, FileIO fileIO, TableIdentifier tableId) { // if the file doesn't exist, we assume that another task execution was successful, but // failed to drop the task entity. Log a warning and return success @@ -134,7 +134,7 @@ private boolean manifestFileCleanUp( } } - private boolean contentFileCleanup( + private boolean cleanUpContentFiles( List contentFileBatch, FileIO fileIO, TableIdentifier tableId) { List validFiles = contentFileBatch.stream().filter(file -> TaskUtils.exists(file, fileIO)).toList(); diff --git a/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java b/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java index 10b1919fb..3f5486a1d 100644 --- a/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java +++ b/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java @@ -153,11 +153,11 @@ public boolean handleTask(TaskEntity cleanupTask) { .map( fileBatch -> { String taskName = - cleanupTask.getName() - + "_" - + fileBatch.toString() - + "_" - + UUID.randomUUID(); + String.join( + "_", + cleanupTask.getName(), + fileBatch.toString(), + UUID.randomUUID().toString()); LOGGER .atDebug() .addKeyValue("taskName", taskName) From 988e5309d1c3cf9aede473f52f17595a7e124001 Mon Sep 17 00:00:00 2001 From: Daniel Tu Date: Wed, 13 Nov 2024 22:00:42 -0800 Subject: [PATCH 24/29] refactor manifest cleanup handler based on comments - 1. renaming - 2. add log and exception handling - 3. remove unnecessary log --- .../task/ManifestFileCleanupTaskHandler.java | 63 ++++++++++--------- 1 file changed, 33 insertions(+), 30 deletions(-) diff --git a/polaris-service/src/main/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandler.java b/polaris-service/src/main/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandler.java index f906daba3..2c0814a68 100644 --- a/polaris-service/src/main/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandler.java +++ b/polaris-service/src/main/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandler.java @@ -45,8 +45,8 @@ * {@link TaskHandler} responsible for deleting table files: 1. Manifest files: It contains all the * files in a manifest and the manifest itself. Since data files may be present in multiple * manifests across different snapshots, we assume a data file that doesn't exist is missing because - * it was already deleted by another task. 2. Table content files: It contains previous metadata and - * statistics files, which are grouped and deleted in batch + * it was already deleted by another task. 2. Table metadata files: It contains previous metadata + * and statistics files, which are grouped and deleted in batch */ public class ManifestFileCleanupTaskHandler implements TaskHandler { public static final int MAX_ATTEMPTS = 3; @@ -75,8 +75,8 @@ public boolean handleTask(TaskEntity task) { if (cleanupTask.getManifestFileData() != null) { ManifestFile manifestFile = decodeManifestData(cleanupTask.getManifestFileData()); return cleanUpManifestFile(manifestFile, authorizedFileIO, tableId); - } else if (cleanupTask.getContentFileBatch() != null) { - return cleanUpContentFiles(cleanupTask.getContentFileBatch(), authorizedFileIO, tableId); + } else if (cleanupTask.getMetadataFiles() != null) { + return cleanUpMetadataFiles(cleanupTask.getMetadataFiles(), authorizedFileIO, tableId); } else { LOGGER .atWarn() @@ -134,42 +134,45 @@ private boolean cleanUpManifestFile( } } - private boolean cleanUpContentFiles( - List contentFileBatch, FileIO fileIO, TableIdentifier tableId) { + private boolean cleanUpMetadataFiles( + List metadataFiles, FileIO fileIO, TableIdentifier tableId) { List validFiles = - contentFileBatch.stream().filter(file -> TaskUtils.exists(file, fileIO)).toList(); + metadataFiles.stream().filter(file -> TaskUtils.exists(file, fileIO)).toList(); if (validFiles.isEmpty()) { LOGGER .atWarn() - .addKeyValue("contentFileBatch", contentFileBatch.toString()) + .addKeyValue("metadataFiles", metadataFiles.toString()) .addKeyValue("tableId", tableId) - .log("Table content cleanup task scheduled, but the none of the file in batch exists"); + .log("Table metadata cleanup task scheduled, but the none of the file in batch exists"); return true; } + if (validFiles.size() < metadataFiles.size()) { + List missingFiles = + metadataFiles.stream().filter(file -> !TaskUtils.exists(file, fileIO)).toList(); + LOGGER + .atWarn() + .addKeyValue("metadataFiles", metadataFiles.toString()) + .addKeyValue("missingFiles", missingFiles) + .addKeyValue("tableId", tableId) + .log( + "Table metadata cleanup task scheduled, but {} files in the batch are missing", + missingFiles.size()); + } // Schedule the deletion for each file asynchronously List> deleteFutures = validFiles.stream().map(file -> tryDelete(tableId, fileIO, null, file, null, 1)).toList(); - // Wait for all delete operations to finish try { + // Wait for all delete operations to finish CompletableFuture allDeletes = CompletableFuture.allOf(deleteFutures.toArray(new CompletableFuture[0])); allDeletes.join(); } catch (Exception e) { - LOGGER - .atWarn() - .addKeyValue("contentFileBatch", contentFileBatch.toString()) - .addKeyValue("tableId", tableId) - .log("Exception detected during content file batch deletion", e); + LOGGER.error("Exception detected during metadata file deletion", e); + return false; } - LOGGER - .atInfo() - .addKeyValue("contentFileBatch", contentFileBatch.toString()) - .addKeyValue("tableId", tableId) - .log("Content file batch deletion has completed"); - return true; } @@ -236,16 +239,16 @@ private CompletableFuture tryDelete( public static final class ManifestCleanupTask { private TableIdentifier tableId; private String manifestFileData; - private List contentFileBatch; + private List metadataFiles; public ManifestCleanupTask(TableIdentifier tableId, String manifestFileData) { this.tableId = tableId; this.manifestFileData = manifestFileData; } - public ManifestCleanupTask(TableIdentifier tableId, List contentFileBatch) { + public ManifestCleanupTask(TableIdentifier tableId, List metadataFiles) { this.tableId = tableId; - this.contentFileBatch = contentFileBatch; + this.metadataFiles = metadataFiles; } public ManifestCleanupTask() {} @@ -266,12 +269,12 @@ public void setManifestFileData(String manifestFileData) { this.manifestFileData = manifestFileData; } - public List getContentFileBatch() { - return contentFileBatch; + public List getMetadataFiles() { + return metadataFiles; } - public void setContentFileBatch(List contentFileBatch) { - this.contentFileBatch = contentFileBatch; + public void setMetadataFiles(List metadataFiles) { + this.metadataFiles = metadataFiles; } @Override @@ -280,12 +283,12 @@ public boolean equals(Object object) { if (!(object instanceof ManifestCleanupTask that)) return false; return Objects.equals(tableId, that.tableId) && Objects.equals(manifestFileData, that.manifestFileData) - && Objects.equals(contentFileBatch, that.contentFileBatch); + && Objects.equals(metadataFiles, that.metadataFiles); } @Override public int hashCode() { - return Objects.hash(tableId, manifestFileData, contentFileBatch); + return Objects.hash(tableId, manifestFileData, metadataFiles); } } } From 4965d5c0eb3e9e75d3661a14be74cecf677637d6 Mon Sep 17 00:00:00 2001 From: Daniel Tu Date: Wed, 13 Nov 2024 22:31:57 -0800 Subject: [PATCH 25/29] refactor table cleanup handler based on comments - 1. renaming - 2. extract task entities creation into methods - 3. remove unnecessary filtering --- .../service/task/TableCleanupTaskHandler.java | 187 ++++++++++-------- 1 file changed, 109 insertions(+), 78 deletions(-) diff --git a/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java b/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java index 3f5486a1d..b0fb351d9 100644 --- a/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java +++ b/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java @@ -103,82 +103,27 @@ public boolean handleTask(TaskEntity cleanupTask) { TableMetadata tableMetadata = TableMetadataParser.read(fileIO, tableEntity.getMetadataLocation()); - // read the manifest list for each snapshot. dedupe the manifest files and schedule a - // cleanupTask - // for each manifest file and its data files to be deleted Stream manifestCleanupTasks = - tableMetadata.snapshots().stream() - .flatMap(sn -> sn.allManifests(fileIO).stream()) - // distinct by manifest path, since multiple snapshots will contain the same - // manifest - .collect(Collectors.toMap(ManifestFile::path, Function.identity(), (mf1, mf2) -> mf1)) - .values() - .stream() - .filter(mf -> TaskUtils.exists(mf.path(), fileIO)) - .map( - mf -> { - // append a random uuid to the task name to avoid any potential conflict - // when - // storing the task entity. It's better to have duplicate tasks than to risk - // not storing the rest of the task entities. If a duplicate deletion task - // is - // queued, it will check for the manifest file's existence and simply exit - // if - // the task has already been handled. - String taskName = - cleanupTask.getName() + "_" + mf.path() + "_" + UUID.randomUUID(); - LOGGER - .atDebug() - .addKeyValue("taskName", taskName) - .addKeyValue("tableIdentifier", tableEntity.getTableIdentifier()) - .addKeyValue("metadataLocation", tableEntity.getMetadataLocation()) - .addKeyValue("manifestFile", mf.path()) - .log("Queueing task to delete manifest file"); - return new TaskEntity.Builder() - .setName(taskName) - .setId(metaStoreManager.generateNewEntityId(polarisCallContext).getId()) - .setCreateTimestamp(polarisCallContext.getClock().millis()) - .withTaskType(AsyncTaskType.FILE_CLEANUP) - .withData( - new ManifestFileCleanupTaskHandler.ManifestCleanupTask( - tableEntity.getTableIdentifier(), TaskUtils.encodeManifestFile(mf))) - .setId(metaStoreManager.generateNewEntityId(polarisCallContext).getId()) - // copy the internal properties, which will have storage info - .setInternalProperties(cleanupTask.getInternalPropertiesAsMap()) - .build(); - }); - - Stream contentFileCleanupTasks = - getContentFileBatch(tableMetadata, fileIO).stream() - .map( - fileBatch -> { - String taskName = - String.join( - "_", - cleanupTask.getName(), - fileBatch.toString(), - UUID.randomUUID().toString()); - LOGGER - .atDebug() - .addKeyValue("taskName", taskName) - .addKeyValue("tableIdentifier", tableEntity.getTableIdentifier()) - .addKeyValue("fileBatch", fileBatch) - .log( - "Queueing task to delete content file (prev metadata and statistics files)"); - return new TaskEntity.Builder() - .setName(taskName) - .setId(metaStoreManager.generateNewEntityId(polarisCallContext).getId()) - .setCreateTimestamp(polarisCallContext.getClock().millis()) - .withTaskType(AsyncTaskType.FILE_CLEANUP) - .withData( - new ManifestFileCleanupTaskHandler.ManifestCleanupTask( - tableEntity.getTableIdentifier(), fileBatch)) - .setInternalProperties(cleanupTask.getInternalPropertiesAsMap()) - .build(); - }); + getManifestTaskStream( + cleanupTask, + tableMetadata, + fileIO, + tableEntity, + metaStoreManager, + polarisCallContext); + + // TODO: handle partition statistics files + Stream metadataFileCleanupTasks = + getMetadataTaskStream( + cleanupTask, + tableMetadata, + fileIO, + tableEntity, + metaStoreManager, + polarisCallContext); List taskEntities = - Stream.concat(manifestCleanupTasks, contentFileCleanupTasks).toList(); + Stream.concat(manifestCleanupTasks, metadataFileCleanupTasks).toList(); List createdTasks = metaStoreManager @@ -204,17 +149,103 @@ public boolean handleTask(TaskEntity cleanupTask) { return false; } - private List> getContentFileBatch(TableMetadata tableMetadata, FileIO fileIO) { + private Stream getManifestTaskStream( + TaskEntity cleanupTask, + TableMetadata tableMetadata, + FileIO fileIO, + TableLikeEntity tableEntity, + PolarisMetaStoreManager metaStoreManager, + PolarisCallContext polarisCallContext) { + // read the manifest list for each snapshot. dedupe the manifest files and schedule a + // cleanupTask + // for each manifest file and its data files to be deleted + return tableMetadata.snapshots().stream() + .flatMap(sn -> sn.allManifests(fileIO).stream()) + // distinct by manifest path, since multiple snapshots will contain the same + // manifest + .collect(Collectors.toMap(ManifestFile::path, Function.identity(), (mf1, mf2) -> mf1)) + .values() + .stream() + .filter(mf -> TaskUtils.exists(mf.path(), fileIO)) + .map( + mf -> { + // append a random uuid to the task name to avoid any potential conflict + // when + // storing the task entity. It's better to have duplicate tasks than to risk + // not storing the rest of the task entities. If a duplicate deletion task + // is + // queued, it will check for the manifest file's existence and simply exit + // if + // the task has already been handled. + String taskName = cleanupTask.getName() + "_" + mf.path() + "_" + UUID.randomUUID(); + LOGGER + .atDebug() + .addKeyValue("taskName", taskName) + .addKeyValue("tableIdentifier", tableEntity.getTableIdentifier()) + .addKeyValue("metadataLocation", tableEntity.getMetadataLocation()) + .addKeyValue("manifestFile", mf.path()) + .log("Queueing task to delete manifest file"); + return new TaskEntity.Builder() + .setName(taskName) + .setId(metaStoreManager.generateNewEntityId(polarisCallContext).getId()) + .setCreateTimestamp(polarisCallContext.getClock().millis()) + .withTaskType(AsyncTaskType.FILE_CLEANUP) + .withData( + new ManifestFileCleanupTaskHandler.ManifestCleanupTask( + tableEntity.getTableIdentifier(), TaskUtils.encodeManifestFile(mf))) + .setId(metaStoreManager.generateNewEntityId(polarisCallContext).getId()) + // copy the internal properties, which will have storage info + .setInternalProperties(cleanupTask.getInternalPropertiesAsMap()) + .build(); + }); + } + + private Stream getMetadataTaskStream( + TaskEntity cleanupTask, + TableMetadata tableMetadata, + FileIO fileIO, + TableLikeEntity tableEntity, + PolarisMetaStoreManager metaStoreManager, + PolarisCallContext polarisCallContext) { + return getMetadataFiles(tableMetadata, fileIO).stream() + .map( + fileBatch -> { + String taskName = + String.join( + "_", + cleanupTask.getName(), + fileBatch.toString(), + UUID.randomUUID().toString()); + LOGGER + .atDebug() + .addKeyValue("taskName", taskName) + .addKeyValue("tableIdentifier", tableEntity.getTableIdentifier()) + .addKeyValue("fileBatch", fileBatch) + .log( + "Queueing task to delete metadata files (prev metadata and statistics files)"); + return new TaskEntity.Builder() + .setName(taskName) + .setId(metaStoreManager.generateNewEntityId(polarisCallContext).getId()) + .setCreateTimestamp(polarisCallContext.getClock().millis()) + .withTaskType(AsyncTaskType.FILE_CLEANUP) + .withData( + new ManifestFileCleanupTaskHandler.ManifestCleanupTask( + tableEntity.getTableIdentifier(), fileBatch)) + .setInternalProperties(cleanupTask.getInternalPropertiesAsMap()) + .build(); + }); + } + + private List> getMetadataFiles(TableMetadata tableMetadata, FileIO fileIO) { List> result = new ArrayList<>(); - List contentFiles = + List metadataFiles = Stream.concat( tableMetadata.previousFiles().stream().map(TableMetadata.MetadataLogEntry::file), tableMetadata.statisticsFiles().stream().map(StatisticsFile::path)) - .filter(file -> TaskUtils.exists(file, fileIO)) .toList(); - for (int i = 0; i < contentFiles.size(); i += BATCH_SIZE) { - result.add(contentFiles.subList(i, Math.min(i + BATCH_SIZE, contentFiles.size()))); + for (int i = 0; i < metadataFiles.size(); i += BATCH_SIZE) { + result.add(metadataFiles.subList(i, Math.min(i + BATCH_SIZE, metadataFiles.size()))); } return result; } From 5f8148354d4e3eba56ccd3f8a04ea7ea816acd30 Mon Sep 17 00:00:00 2001 From: Daniel Tu Date: Wed, 13 Nov 2024 22:32:06 -0800 Subject: [PATCH 26/29] add TODO --- .../polaris/service/task/ManifestFileCleanupTaskHandler.java | 1 + 1 file changed, 1 insertion(+) diff --git a/polaris-service/src/main/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandler.java b/polaris-service/src/main/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandler.java index 2c0814a68..b4057b1b5 100644 --- a/polaris-service/src/main/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandler.java +++ b/polaris-service/src/main/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandler.java @@ -48,6 +48,7 @@ * it was already deleted by another task. 2. Table metadata files: It contains previous metadata * and statistics files, which are grouped and deleted in batch */ +// TODO: Rename this class since we introducing metadata cleanup here public class ManifestFileCleanupTaskHandler implements TaskHandler { public static final int MAX_ATTEMPTS = 3; public static final int FILE_DELETION_RETRY_MILLIS = 100; From 16bb5fe610fda0e28ce46cb0986202fb36a37999 Mon Sep 17 00:00:00 2001 From: Daniel Tu Date: Wed, 13 Nov 2024 23:47:06 -0800 Subject: [PATCH 27/29] renaming --- .../service/task/TableCleanupTaskHandler.java | 12 ++++++------ .../task/ManifestFileCleanupTaskHandlerTest.java | 4 ++-- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java b/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java index b0fb351d9..372061aff 100644 --- a/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java +++ b/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java @@ -207,20 +207,20 @@ private Stream getMetadataTaskStream( TableLikeEntity tableEntity, PolarisMetaStoreManager metaStoreManager, PolarisCallContext polarisCallContext) { - return getMetadataFiles(tableMetadata, fileIO).stream() + return getMetadataFileBatches(tableMetadata).stream() .map( - fileBatch -> { + metadataBatch -> { String taskName = String.join( "_", cleanupTask.getName(), - fileBatch.toString(), + metadataBatch.toString(), UUID.randomUUID().toString()); LOGGER .atDebug() .addKeyValue("taskName", taskName) .addKeyValue("tableIdentifier", tableEntity.getTableIdentifier()) - .addKeyValue("fileBatch", fileBatch) + .addKeyValue("metadataFiles", metadataBatch.toString()) .log( "Queueing task to delete metadata files (prev metadata and statistics files)"); return new TaskEntity.Builder() @@ -230,13 +230,13 @@ private Stream getMetadataTaskStream( .withTaskType(AsyncTaskType.FILE_CLEANUP) .withData( new ManifestFileCleanupTaskHandler.ManifestCleanupTask( - tableEntity.getTableIdentifier(), fileBatch)) + tableEntity.getTableIdentifier(), metadataBatch)) .setInternalProperties(cleanupTask.getInternalPropertiesAsMap()) .build(); }); } - private List> getMetadataFiles(TableMetadata tableMetadata, FileIO fileIO) { + private List> getMetadataFileBatches(TableMetadata tableMetadata) { List> result = new ArrayList<>(); List metadataFiles = Stream.concat( diff --git a/polaris-service/src/test/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandlerTest.java b/polaris-service/src/test/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandlerTest.java index 862e90237..7e2fece9d 100644 --- a/polaris-service/src/test/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandlerTest.java +++ b/polaris-service/src/test/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandlerTest.java @@ -235,7 +235,7 @@ public void deleteFile(String location) { } @Test - public void testContentFileCleanup() throws IOException { + public void testMetadataFileCleanup() throws IOException { PolarisCallContext polarisCallContext = new PolarisCallContext( metaStoreManagerFactory.getOrCreateSessionSupplier(realmContext).get(), @@ -338,7 +338,7 @@ public void close() { } @Test - public void testContentFileCleanupIfFileNotExist() throws IOException { + public void testMetadataFileCleanupIfFileNotExist() throws IOException { PolarisCallContext polarisCallContext = new PolarisCallContext( metaStoreManagerFactory.getOrCreateSessionSupplier(realmContext).get(), From d276ae618d57817cf836e9b8481c4f0381d23055 Mon Sep 17 00:00:00 2001 From: Daniel Tu Date: Tue, 19 Nov 2024 21:12:04 -0800 Subject: [PATCH 28/29] split the task type in cleanup task handler --- .../polaris/core/entity/AsyncTaskType.java | 3 +- .../BasePolarisMetaStoreManagerTest.java | 2 +- .../task/ManifestFileCleanupTaskHandler.java | 7 +- .../service/task/TableCleanupTaskHandler.java | 18 +++-- .../ManifestFileCleanupTaskHandlerTest.java | 14 ++-- .../task/TableCleanupTaskHandlerTest.java | 76 ++++++++++++------- 6 files changed, 74 insertions(+), 46 deletions(-) diff --git a/polaris-core/src/main/java/org/apache/polaris/core/entity/AsyncTaskType.java b/polaris-core/src/main/java/org/apache/polaris/core/entity/AsyncTaskType.java index 32c478e86..6e3ce9ae5 100644 --- a/polaris-core/src/main/java/org/apache/polaris/core/entity/AsyncTaskType.java +++ b/polaris-core/src/main/java/org/apache/polaris/core/entity/AsyncTaskType.java @@ -23,7 +23,8 @@ public enum AsyncTaskType { ENTITY_CLEANUP_SCHEDULER(1), - FILE_CLEANUP(2); + MANIFEST_FILE_CLEANUP(2), + METADATA_FILE_BATCH_CLEANUP(3); private final int typeCode; diff --git a/polaris-core/src/testFixtures/java/org/apache/polaris/core/persistence/BasePolarisMetaStoreManagerTest.java b/polaris-core/src/testFixtures/java/org/apache/polaris/core/persistence/BasePolarisMetaStoreManagerTest.java index bf97e4128..1b5483afe 100644 --- a/polaris-core/src/testFixtures/java/org/apache/polaris/core/persistence/BasePolarisMetaStoreManagerTest.java +++ b/polaris-core/src/testFixtures/java/org/apache/polaris/core/persistence/BasePolarisMetaStoreManagerTest.java @@ -240,7 +240,7 @@ private static TaskEntity createTask(String taskName, long id) { .setName(taskName) .withData("data") .setId(id) - .withTaskType(AsyncTaskType.FILE_CLEANUP) + .withTaskType(AsyncTaskType.MANIFEST_FILE_CLEANUP) .setCreateTimestamp(Instant.now().toEpochMilli()) .build(); } diff --git a/polaris-service/src/main/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandler.java b/polaris-service/src/main/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandler.java index b4057b1b5..a38124a71 100644 --- a/polaris-service/src/main/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandler.java +++ b/polaris-service/src/main/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandler.java @@ -65,7 +65,8 @@ public ManifestFileCleanupTaskHandler( @Override public boolean canHandleTask(TaskEntity task) { - return task.getTaskType() == AsyncTaskType.FILE_CLEANUP; + return task.getTaskType() == AsyncTaskType.MANIFEST_FILE_CLEANUP + || task.getTaskType() == AsyncTaskType.METADATA_FILE_BATCH_CLEANUP; } @Override @@ -73,10 +74,10 @@ public boolean handleTask(TaskEntity task) { ManifestCleanupTask cleanupTask = task.readData(ManifestCleanupTask.class); TableIdentifier tableId = cleanupTask.getTableId(); try (FileIO authorizedFileIO = fileIOSupplier.apply(task)) { - if (cleanupTask.getManifestFileData() != null) { + if (task.getTaskType() == AsyncTaskType.MANIFEST_FILE_CLEANUP) { ManifestFile manifestFile = decodeManifestData(cleanupTask.getManifestFileData()); return cleanUpManifestFile(manifestFile, authorizedFileIO, tableId); - } else if (cleanupTask.getMetadataFiles() != null) { + } else if (task.getTaskType() == AsyncTaskType.METADATA_FILE_BATCH_CLEANUP) { return cleanUpMetadataFiles(cleanupTask.getMetadataFiles(), authorizedFileIO, tableId); } else { LOGGER diff --git a/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java b/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java index 372061aff..cc722bbb5 100644 --- a/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java +++ b/polaris-service/src/main/java/org/apache/polaris/service/task/TableCleanupTaskHandler.java @@ -52,7 +52,7 @@ public class TableCleanupTaskHandler implements TaskHandler { private final TaskExecutor taskExecutor; private final MetaStoreManagerFactory metaStoreManagerFactory; private final Function fileIOSupplier; - private static final int BATCH_SIZE = 10; + private static final String BATCH_SIZE_CONFIG_KEY = "TABLE_METADATA_CLEANUP_BATCH_SIZE"; public TableCleanupTaskHandler( TaskExecutor taskExecutor, @@ -189,7 +189,7 @@ private Stream getManifestTaskStream( .setName(taskName) .setId(metaStoreManager.generateNewEntityId(polarisCallContext).getId()) .setCreateTimestamp(polarisCallContext.getClock().millis()) - .withTaskType(AsyncTaskType.FILE_CLEANUP) + .withTaskType(AsyncTaskType.MANIFEST_FILE_CLEANUP) .withData( new ManifestFileCleanupTaskHandler.ManifestCleanupTask( tableEntity.getTableIdentifier(), TaskUtils.encodeManifestFile(mf))) @@ -207,7 +207,11 @@ private Stream getMetadataTaskStream( TableLikeEntity tableEntity, PolarisMetaStoreManager metaStoreManager, PolarisCallContext polarisCallContext) { - return getMetadataFileBatches(tableMetadata).stream() + int batchSize = + polarisCallContext + .getConfigurationStore() + .getConfiguration(polarisCallContext, BATCH_SIZE_CONFIG_KEY, 10); + return getMetadataFileBatches(tableMetadata, batchSize).stream() .map( metadataBatch -> { String taskName = @@ -227,7 +231,7 @@ private Stream getMetadataTaskStream( .setName(taskName) .setId(metaStoreManager.generateNewEntityId(polarisCallContext).getId()) .setCreateTimestamp(polarisCallContext.getClock().millis()) - .withTaskType(AsyncTaskType.FILE_CLEANUP) + .withTaskType(AsyncTaskType.METADATA_FILE_BATCH_CLEANUP) .withData( new ManifestFileCleanupTaskHandler.ManifestCleanupTask( tableEntity.getTableIdentifier(), metadataBatch)) @@ -236,7 +240,7 @@ private Stream getMetadataTaskStream( }); } - private List> getMetadataFileBatches(TableMetadata tableMetadata) { + private List> getMetadataFileBatches(TableMetadata tableMetadata, int batchSize) { List> result = new ArrayList<>(); List metadataFiles = Stream.concat( @@ -244,8 +248,8 @@ private List> getMetadataFileBatches(TableMetadata tableMetadata) { tableMetadata.statisticsFiles().stream().map(StatisticsFile::path)) .toList(); - for (int i = 0; i < metadataFiles.size(); i += BATCH_SIZE) { - result.add(metadataFiles.subList(i, Math.min(i + BATCH_SIZE, metadataFiles.size()))); + for (int i = 0; i < metadataFiles.size(); i += batchSize) { + result.add(metadataFiles.subList(i, Math.min(i + batchSize, metadataFiles.size()))); } return result; } diff --git a/polaris-service/src/test/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandlerTest.java b/polaris-service/src/test/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandlerTest.java index 7e2fece9d..709c0ad87 100644 --- a/polaris-service/src/test/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandlerTest.java +++ b/polaris-service/src/test/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandlerTest.java @@ -82,7 +82,7 @@ public void testCleanupFileNotExists() throws IOException { fileIO.deleteFile(manifestFile.path()); TaskEntity task = new TaskEntity.Builder() - .withTaskType(AsyncTaskType.FILE_CLEANUP) + .withTaskType(AsyncTaskType.MANIFEST_FILE_CLEANUP) .withData( new ManifestFileCleanupTaskHandler.ManifestCleanupTask( tableIdentifier, @@ -112,7 +112,7 @@ public void testCleanupFileManifestExistsDataFilesDontExist() throws IOException fileIO, "manifest1.avro", 100L, "dataFile1.parquet", "dataFile2.parquet"); TaskEntity task = new TaskEntity.Builder() - .withTaskType(AsyncTaskType.FILE_CLEANUP) + .withTaskType(AsyncTaskType.MANIFEST_FILE_CLEANUP) .withData( new ManifestFileCleanupTaskHandler.ManifestCleanupTask( tableIdentifier, @@ -157,7 +157,7 @@ public void close() { TaskTestUtils.manifestFile(fileIO, "manifest1.avro", 100L, dataFile1Path, dataFile2Path); TaskEntity task = new TaskEntity.Builder() - .withTaskType(AsyncTaskType.FILE_CLEANUP) + .withTaskType(AsyncTaskType.MANIFEST_FILE_CLEANUP) .withData( new ManifestFileCleanupTaskHandler.ManifestCleanupTask( tableIdentifier, @@ -220,7 +220,7 @@ public void deleteFile(String location) { TaskTestUtils.manifestFile(fileIO, "manifest1.avro", 100L, dataFile1Path, dataFile2Path); TaskEntity task = new TaskEntity.Builder() - .withTaskType(AsyncTaskType.FILE_CLEANUP) + .withTaskType(AsyncTaskType.MANIFEST_FILE_CLEANUP) .withData( new ManifestFileCleanupTaskHandler.ManifestCleanupTask( tableIdentifier, @@ -318,7 +318,7 @@ public void close() { TaskEntity task = new TaskEntity.Builder() - .withTaskType(AsyncTaskType.FILE_CLEANUP) + .withTaskType(AsyncTaskType.METADATA_FILE_BATCH_CLEANUP) .withData( new ManifestFileCleanupTaskHandler.ManifestCleanupTask( tableIdentifier, cleanupFiles)) @@ -370,7 +370,7 @@ public void testMetadataFileCleanupIfFileNotExist() throws IOException { TaskEntity task = new TaskEntity.Builder() - .withTaskType(AsyncTaskType.FILE_CLEANUP) + .withTaskType(AsyncTaskType.METADATA_FILE_BATCH_CLEANUP) .withData( new ManifestFileCleanupTaskHandler.ManifestCleanupTask( tableIdentifier, List.of(statisticsFile.path()))) @@ -432,7 +432,7 @@ public void deleteFile(String location) { TaskEntity task = new TaskEntity.Builder() - .withTaskType(AsyncTaskType.FILE_CLEANUP) + .withTaskType(AsyncTaskType.METADATA_FILE_BATCH_CLEANUP) .withData( new ManifestFileCleanupTaskHandler.ManifestCleanupTask( tableIdentifier, List.of(statisticsFile.path()))) diff --git a/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java b/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java index 17ae80618..383ac029d 100644 --- a/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java +++ b/polaris-service/src/test/java/org/apache/polaris/service/task/TableCleanupTaskHandlerTest.java @@ -114,7 +114,7 @@ public void testTableCleanup() throws IOException { assertThat(taskEntity) .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) .extracting(TaskEntity::of) - .returns(AsyncTaskType.FILE_CLEANUP, TaskEntity::getTaskType) + .returns(AsyncTaskType.MANIFEST_FILE_CLEANUP, TaskEntity::getTaskType) .returns( new ManifestFileCleanupTaskHandler.ManifestCleanupTask( tableIdentifier, @@ -126,7 +126,7 @@ public void testTableCleanup() throws IOException { assertThat(taskEntity) .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) .extracting(TaskEntity::of) - .returns(AsyncTaskType.FILE_CLEANUP, TaskEntity::getTaskType) + .returns(AsyncTaskType.METADATA_FILE_BATCH_CLEANUP, TaskEntity::getTaskType) .returns( new ManifestFileCleanupTaskHandler.ManifestCleanupTask( tableIdentifier, List.of(statisticsFile.path())), @@ -264,7 +264,7 @@ public void close() { assertThat(taskEntity) .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) .extracting(TaskEntity::of) - .returns(AsyncTaskType.FILE_CLEANUP, TaskEntity::getTaskType) + .returns(AsyncTaskType.MANIFEST_FILE_CLEANUP, TaskEntity::getTaskType) .returns( new ManifestFileCleanupTaskHandler.ManifestCleanupTask( tableIdentifier, @@ -276,7 +276,7 @@ public void close() { assertThat(taskEntity) .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) .extracting(TaskEntity::of) - .returns(AsyncTaskType.FILE_CLEANUP, TaskEntity::getTaskType) + .returns(AsyncTaskType.MANIFEST_FILE_CLEANUP, TaskEntity::getTaskType) .returns( new ManifestFileCleanupTaskHandler.ManifestCleanupTask( tableIdentifier, @@ -365,14 +365,20 @@ public void testTableCleanupMultipleSnapshots() throws IOException { .filter( entity -> { AsyncTaskType taskType = TaskEntity.of(entity).getTaskType(); - return taskType == AsyncTaskType.FILE_CLEANUP; + return taskType == AsyncTaskType.MANIFEST_FILE_CLEANUP; + }) + .toList(); + List metadataCleanupTasks = + entities.stream() + .filter( + entity -> { + AsyncTaskType taskType = TaskEntity.of(entity).getTaskType(); + return taskType == AsyncTaskType.METADATA_FILE_BATCH_CLEANUP; }) .toList(); - assertThat(manifestCleanupTasks) - // all three manifests should be present, even though one is excluded from the latest - // snapshot - .hasSize(4) + assertThat(metadataCleanupTasks) + .hasSize(1) .satisfiesExactlyInAnyOrder( taskEntity -> assertThat(taskEntity) @@ -381,10 +387,16 @@ public void testTableCleanupMultipleSnapshots() throws IOException { .returns( new ManifestFileCleanupTaskHandler.ManifestCleanupTask( tableIdentifier, - Base64.encodeBase64String(ManifestFiles.encode(manifestFile1))), + List.of(statisticsFile1.path(), statisticsFile2.path())), entity -> entity.readData( - ManifestFileCleanupTaskHandler.ManifestCleanupTask.class)), + ManifestFileCleanupTaskHandler.ManifestCleanupTask.class))); + + assertThat(manifestCleanupTasks) + // all three manifests should be present, even though one is excluded from the latest + // snapshot + .hasSize(3) + .satisfiesExactlyInAnyOrder( taskEntity -> assertThat(taskEntity) .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) @@ -392,7 +404,7 @@ public void testTableCleanupMultipleSnapshots() throws IOException { .returns( new ManifestFileCleanupTaskHandler.ManifestCleanupTask( tableIdentifier, - Base64.encodeBase64String(ManifestFiles.encode(manifestFile2))), + Base64.encodeBase64String(ManifestFiles.encode(manifestFile1))), entity -> entity.readData( ManifestFileCleanupTaskHandler.ManifestCleanupTask.class)), @@ -403,7 +415,7 @@ public void testTableCleanupMultipleSnapshots() throws IOException { .returns( new ManifestFileCleanupTaskHandler.ManifestCleanupTask( tableIdentifier, - Base64.encodeBase64String(ManifestFiles.encode(manifestFile3))), + Base64.encodeBase64String(ManifestFiles.encode(manifestFile2))), entity -> entity.readData( ManifestFileCleanupTaskHandler.ManifestCleanupTask.class)), @@ -414,7 +426,7 @@ public void testTableCleanupMultipleSnapshots() throws IOException { .returns( new ManifestFileCleanupTaskHandler.ManifestCleanupTask( tableIdentifier, - List.of(statisticsFile1.path(), statisticsFile2.path())), + Base64.encodeBase64String(ManifestFiles.encode(manifestFile3))), entity -> entity.readData( ManifestFileCleanupTaskHandler.ManifestCleanupTask.class))); @@ -513,14 +525,20 @@ public void testTableCleanupMultipleMetadata() throws IOException { .filter( entity -> { AsyncTaskType taskType = TaskEntity.of(entity).getTaskType(); - return taskType == AsyncTaskType.FILE_CLEANUP; + return taskType == AsyncTaskType.MANIFEST_FILE_CLEANUP; + }) + .toList(); + List metadataCleanupTasks = + entities.stream() + .filter( + entity -> { + AsyncTaskType taskType = TaskEntity.of(entity).getTaskType(); + return taskType == AsyncTaskType.METADATA_FILE_BATCH_CLEANUP; }) .toList(); - assertThat(manifestCleanupTasks) - // all three manifests should be present, even though one is excluded from the latest - // snapshot - .hasSize(4) + assertThat(metadataCleanupTasks) + .hasSize(1) .satisfiesExactlyInAnyOrder( taskEntity -> assertThat(taskEntity) @@ -529,10 +547,17 @@ public void testTableCleanupMultipleMetadata() throws IOException { .returns( new ManifestFileCleanupTaskHandler.ManifestCleanupTask( tableIdentifier, - Base64.encodeBase64String(ManifestFiles.encode(manifestFile1))), + List.of( + firstMetadataFile, + statisticsFile1.path(), + statisticsFile2.path())), entity -> entity.readData( - ManifestFileCleanupTaskHandler.ManifestCleanupTask.class)), + ManifestFileCleanupTaskHandler.ManifestCleanupTask.class))); + + assertThat(manifestCleanupTasks) + .hasSize(3) + .satisfiesExactlyInAnyOrder( taskEntity -> assertThat(taskEntity) .returns(PolarisEntityType.TASK.getCode(), PolarisBaseEntity::getTypeCode) @@ -540,7 +565,7 @@ public void testTableCleanupMultipleMetadata() throws IOException { .returns( new ManifestFileCleanupTaskHandler.ManifestCleanupTask( tableIdentifier, - Base64.encodeBase64String(ManifestFiles.encode(manifestFile2))), + Base64.encodeBase64String(ManifestFiles.encode(manifestFile1))), entity -> entity.readData( ManifestFileCleanupTaskHandler.ManifestCleanupTask.class)), @@ -551,7 +576,7 @@ public void testTableCleanupMultipleMetadata() throws IOException { .returns( new ManifestFileCleanupTaskHandler.ManifestCleanupTask( tableIdentifier, - Base64.encodeBase64String(ManifestFiles.encode(manifestFile3))), + Base64.encodeBase64String(ManifestFiles.encode(manifestFile2))), entity -> entity.readData( ManifestFileCleanupTaskHandler.ManifestCleanupTask.class)), @@ -562,10 +587,7 @@ public void testTableCleanupMultipleMetadata() throws IOException { .returns( new ManifestFileCleanupTaskHandler.ManifestCleanupTask( tableIdentifier, - List.of( - firstMetadataFile, - statisticsFile1.path(), - statisticsFile2.path())), + Base64.encodeBase64String(ManifestFiles.encode(manifestFile3))), entity -> entity.readData( ManifestFileCleanupTaskHandler.ManifestCleanupTask.class))); From 187b47e010f442b49626ccb48e2275e3b2d77cdc Mon Sep 17 00:00:00 2001 From: Daniel Tu Date: Tue, 19 Nov 2024 21:20:13 -0800 Subject: [PATCH 29/29] error handling --- .../polaris/service/task/ManifestFileCleanupTaskHandler.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/polaris-service/src/main/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandler.java b/polaris-service/src/main/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandler.java index a38124a71..441a56b27 100644 --- a/polaris-service/src/main/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandler.java +++ b/polaris-service/src/main/java/org/apache/polaris/service/task/ManifestFileCleanupTaskHandler.java @@ -83,8 +83,8 @@ public boolean handleTask(TaskEntity task) { LOGGER .atWarn() .addKeyValue("tableId", tableId) - .log("Cleanup task scheduled, but input file doesn't exist"); - return true; + .log("Unknown task type {}", task.getTaskType()); + return false; } } }