amogh-jahagirdar commented on code in PR #5669: URL: https://github.com/apache/iceberg/pull/5669#discussion_r986049674
########## core/src/main/java/org/apache/iceberg/ReachableFileCleanup.java: ########## @@ -0,0 +1,178 @@ +/* + * 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.iceberg; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.function.Consumer; +import java.util.stream.Collectors; +import org.apache.iceberg.exceptions.NotFoundException; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.base.Joiner; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.util.Tasks; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * File cleanup strategy for snapshot expiration which determines, via an in-memory reference set, + * metadata and data files that are not reachable given the previous and current table states. + */ +class ReachableFileCleanup extends FileCleanupStrategy { + + private static final Logger LOG = LoggerFactory.getLogger(ReachableFileCleanup.class); + + private final TableMetadata afterExpiration; + private final TableMetadata beforeExpiration; + + ReachableFileCleanup( + TableOperations ops, + TableMetadata beforeExpiration, + ExecutorService deleteExecutorService, + ExecutorService planExecutorService, + Consumer<String> deleteFunc) { + super(ops, deleteExecutorService, planExecutorService, deleteFunc); + this.beforeExpiration = beforeExpiration; + this.afterExpiration = ops.refresh(); + } + + @Override + public void cleanFiles() { + // Identify all of the manifest lists to retain + Set<String> manifestListsBeforeExpiration = + ReachableFileUtil.manifestListLocations(beforeExpiration); + + // Identify all of the manifest lists that currently exist + Set<String> manifestListsAfterExpiration = + ReachableFileUtil.manifestListLocations(afterExpiration); + + Set<String> manifestListsToDelete = + Sets.difference(manifestListsBeforeExpiration, manifestListsAfterExpiration); + Map<String, ManifestFile> reachableManifestsAfterExpiration = + computeManifestsForManifestLists(manifestListsAfterExpiration); + + // Compute the manifests to delete based on the current manifest lists to delete and the current + // reachable manifests + List<ManifestFile> manifestsToDelete = + computeManifestsToDelete(manifestListsToDelete, reachableManifestsAfterExpiration); + + List<ManifestFile> currentManifests = + Lists.newArrayList(reachableManifestsAfterExpiration.values()); + + Set<String> filesToDelete = findFilesToDelete(manifestsToDelete, currentManifests); + Tasks.foreach(filesToDelete) + .executeWith(deleteExecutorService) + .retry(3) + .stopRetryOn(NotFoundException.class) + .suppressFailureWhenFinished() + .onFailure((file, exc) -> LOG.warn("Delete failed for data file: {}", file, exc)) + .run(deleteFunc::accept); + + Set<String> manifestPathsToDelete = + manifestsToDelete.stream().map(ManifestFile::path).collect(Collectors.toSet()); + + LOG.warn("Manifests to delete: {}", Joiner.on(", ").join(manifestsToDelete)); Review Comment: Yeah agreed, that it should be removed, that's what I was mentioning here https://github.com/apache/iceberg/pull/5669#discussion_r985147634 but the existing incremental approach logs it as warn https://github.com/apache/iceberg/blame/8b8a1038609610873c7d40023b4fc98ba341a502/core/src/main/java/org/apache/iceberg/RemoveSnapshots.java#L538 so before changing that, I wanted to make sure it wasn't done intentionally. ########## core/src/main/java/org/apache/iceberg/FileCleanupStrategy.java: ########## @@ -0,0 +1,86 @@ +/* + * 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.iceberg; + +import java.util.Set; +import java.util.concurrent.ExecutorService; +import java.util.function.Consumer; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.exceptions.NotFoundException; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.base.Joiner; +import org.apache.iceberg.util.Tasks; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings("checkstyle:VisibilityModifier") +abstract class FileCleanupStrategy { + + private final Logger log = LoggerFactory.getLogger(getClass()); + + protected final TableOperations ops; + protected final ExecutorService deleteExecutorService; + protected final ExecutorService planExecutorService; + protected final Consumer<String> deleteFunc; + + protected FileCleanupStrategy( + TableOperations ops, + ExecutorService deleteExecutorService, + ExecutorService planExecutorService, + Consumer<String> deleteFunc) { + this.ops = ops; + this.deleteExecutorService = deleteExecutorService; + this.planExecutorService = planExecutorService; + this.deleteFunc = deleteFunc; + } + + public abstract void cleanFiles(); + + protected static final Schema MANIFEST_PROJECTION = + ManifestFile.schema() + .select( + "manifest_path", "manifest_length", "added_snapshot_id", "deleted_data_files_count"); + + protected CloseableIterable<ManifestFile> readManifestFiles(String manifestListLocation) { + return Avro.read(ops.io().newInputFile(manifestListLocation)) + .rename("manifest_file", GenericManifestFile.class.getName()) + .classLoader(GenericManifestFile.class.getClassLoader()) + .project(MANIFEST_PROJECTION) + .reuseContainers(true) + .build(); + } + + protected void deleteMetadataFiles( + Set<String> manifestsToDelete, Set<String> manifestListsToDelete) { + log.warn("Manifests to delete: {}", Joiner.on(", ").join(manifestsToDelete)); Review Comment: https://github.com/apache/iceberg/blob/master/core/src/main/java/org/apache/iceberg/RemoveSnapshots.java#L538 This is still warn as it is currently, but I'm doubting if it really needs to be warn level or if we even need this logging? warn level implies that something is in a "not fatally wrong" which is not the case here. We are intentionally deleting these files after they've been correctly computed. It may mislead users. @rdblue @singhpk234 @jackye1995 ########## core/src/main/java/org/apache/iceberg/ReachableFileCleanup.java: ########## @@ -0,0 +1,178 @@ +/* + * 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.iceberg; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.function.Consumer; +import java.util.stream.Collectors; +import org.apache.iceberg.exceptions.NotFoundException; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.base.Joiner; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.util.Tasks; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * File cleanup strategy for snapshot expiration which determines, via an in-memory reference set, + * metadata and data files that are not reachable given the previous and current table states. + */ +class ReachableFileCleanup extends FileCleanupStrategy { + + private static final Logger LOG = LoggerFactory.getLogger(ReachableFileCleanup.class); + + private final TableMetadata afterExpiration; + private final TableMetadata beforeExpiration; + + ReachableFileCleanup( + TableOperations ops, + TableMetadata beforeExpiration, + ExecutorService deleteExecutorService, + ExecutorService planExecutorService, + Consumer<String> deleteFunc) { + super(ops, deleteExecutorService, planExecutorService, deleteFunc); + this.beforeExpiration = beforeExpiration; + this.afterExpiration = ops.refresh(); + } + + @Override + public void cleanFiles() { + // Identify all of the manifest lists to retain + Set<String> manifestListsBeforeExpiration = + ReachableFileUtil.manifestListLocations(beforeExpiration); + + // Identify all of the manifest lists that currently exist + Set<String> manifestListsAfterExpiration = + ReachableFileUtil.manifestListLocations(afterExpiration); + + Set<String> manifestListsToDelete = + Sets.difference(manifestListsBeforeExpiration, manifestListsAfterExpiration); + Map<String, ManifestFile> reachableManifestsAfterExpiration = + computeManifestsForManifestLists(manifestListsAfterExpiration); + + // Compute the manifests to delete based on the current manifest lists to delete and the current + // reachable manifests + List<ManifestFile> manifestsToDelete = + computeManifestsToDelete(manifestListsToDelete, reachableManifestsAfterExpiration); + + List<ManifestFile> currentManifests = + Lists.newArrayList(reachableManifestsAfterExpiration.values()); + + Set<String> filesToDelete = findFilesToDelete(manifestsToDelete, currentManifests); + Tasks.foreach(filesToDelete) + .executeWith(deleteExecutorService) + .retry(3) + .stopRetryOn(NotFoundException.class) + .suppressFailureWhenFinished() + .onFailure((file, exc) -> LOG.warn("Delete failed for data file: {}", file, exc)) + .run(deleteFunc::accept); + + Set<String> manifestPathsToDelete = + manifestsToDelete.stream().map(ManifestFile::path).collect(Collectors.toSet()); + + LOG.warn("Manifests to delete: {}", Joiner.on(", ").join(manifestsToDelete)); Review Comment: Yeah agreed, that it should be removed, that's what I was mentioning here https://github.com/apache/iceberg/pull/5669#discussion_r985147634 but the existing incremental approach in the current code logs it as warn https://github.com/apache/iceberg/blame/8b8a1038609610873c7d40023b4fc98ba341a502/core/src/main/java/org/apache/iceberg/RemoveSnapshots.java#L538 so before changing that, I wanted to make sure it wasn't done intentionally. ########## core/src/main/java/org/apache/iceberg/ReachableFileCleanup.java: ########## @@ -0,0 +1,178 @@ +/* + * 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.iceberg; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.function.Consumer; +import java.util.stream.Collectors; +import org.apache.iceberg.exceptions.NotFoundException; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.base.Joiner; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.util.Tasks; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * File cleanup strategy for snapshot expiration which determines, via an in-memory reference set, + * metadata and data files that are not reachable given the previous and current table states. + */ +class ReachableFileCleanup extends FileCleanupStrategy { + + private static final Logger LOG = LoggerFactory.getLogger(ReachableFileCleanup.class); + + private final TableMetadata afterExpiration; + private final TableMetadata beforeExpiration; + + ReachableFileCleanup( + TableOperations ops, + TableMetadata beforeExpiration, + ExecutorService deleteExecutorService, + ExecutorService planExecutorService, + Consumer<String> deleteFunc) { + super(ops, deleteExecutorService, planExecutorService, deleteFunc); + this.beforeExpiration = beforeExpiration; + this.afterExpiration = ops.refresh(); + } + + @Override + public void cleanFiles() { + // Identify all of the manifest lists to retain + Set<String> manifestListsBeforeExpiration = + ReachableFileUtil.manifestListLocations(beforeExpiration); + + // Identify all of the manifest lists that currently exist + Set<String> manifestListsAfterExpiration = + ReachableFileUtil.manifestListLocations(afterExpiration); + + Set<String> manifestListsToDelete = + Sets.difference(manifestListsBeforeExpiration, manifestListsAfterExpiration); + Map<String, ManifestFile> reachableManifestsAfterExpiration = + computeManifestsForManifestLists(manifestListsAfterExpiration); + + // Compute the manifests to delete based on the current manifest lists to delete and the current + // reachable manifests + List<ManifestFile> manifestsToDelete = + computeManifestsToDelete(manifestListsToDelete, reachableManifestsAfterExpiration); + + List<ManifestFile> currentManifests = + Lists.newArrayList(reachableManifestsAfterExpiration.values()); + + Set<String> filesToDelete = findFilesToDelete(manifestsToDelete, currentManifests); + Tasks.foreach(filesToDelete) + .executeWith(deleteExecutorService) + .retry(3) + .stopRetryOn(NotFoundException.class) + .suppressFailureWhenFinished() + .onFailure((file, exc) -> LOG.warn("Delete failed for data file: {}", file, exc)) + .run(deleteFunc::accept); + + Set<String> manifestPathsToDelete = + manifestsToDelete.stream().map(ManifestFile::path).collect(Collectors.toSet()); + + LOG.warn("Manifests to delete: {}", Joiner.on(", ").join(manifestsToDelete)); Review Comment: Yeah agreed, that it should be removed, that's what I was mentioning here https://github.com/apache/iceberg/pull/5669#discussion_r985147634 but the existing incremental approach in the current code logs it at warning level https://github.com/apache/iceberg/blame/8b8a1038609610873c7d40023b4fc98ba341a502/core/src/main/java/org/apache/iceberg/RemoveSnapshots.java#L538 so before changing that, I wanted to make sure it wasn't done intentionally. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
