-
Notifications
You must be signed in to change notification settings - Fork 3.6k
Use s3 batch deletes for iceberg expire_snapshots #14434
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,30 @@ | ||
| /* | ||
| * Licensed 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 io.trino.filesystem; | ||
|
|
||
| import org.apache.hadoop.fs.FileSystem; | ||
| import org.apache.hadoop.fs.FilterFileSystem; | ||
|
|
||
| public final class FileSystemUtils | ||
| { | ||
| private FileSystemUtils() {} | ||
|
|
||
| public static FileSystem getRawFileSystem(FileSystem fileSystem) | ||
| { | ||
| if (fileSystem instanceof FilterFileSystem) { | ||
| return getRawFileSystem(((FilterFileSystem) fileSystem).getRawFileSystem()); | ||
| } | ||
| return fileSystem; | ||
| } | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,25 @@ | ||
| /* | ||
| * Licensed 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 io.trino.hdfs; | ||
|
|
||
| import org.apache.hadoop.fs.Path; | ||
|
|
||
| import java.io.IOException; | ||
| import java.util.Collection; | ||
|
|
||
| public interface FileSystemWithBatchDelete | ||
| { | ||
| void deleteFiles(Collection<Path> paths) | ||
| throws IOException; | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -49,6 +49,8 @@ | |
| import com.amazonaws.services.s3.model.CopyObjectRequest; | ||
| import com.amazonaws.services.s3.model.CryptoConfiguration; | ||
| import com.amazonaws.services.s3.model.DeleteObjectRequest; | ||
| import com.amazonaws.services.s3.model.DeleteObjectsRequest; | ||
| import com.amazonaws.services.s3.model.DeleteObjectsRequest.KeyVersion; | ||
| import com.amazonaws.services.s3.model.EncryptionMaterialsProvider; | ||
| import com.amazonaws.services.s3.model.GetObjectMetadataRequest; | ||
| import com.amazonaws.services.s3.model.GetObjectRequest; | ||
|
|
@@ -74,13 +76,15 @@ | |
| import com.google.common.base.Splitter; | ||
| import com.google.common.collect.AbstractSequentialIterator; | ||
| import com.google.common.collect.ImmutableSet; | ||
| import com.google.common.collect.Iterables; | ||
| import com.google.common.collect.Iterators; | ||
| import com.google.common.io.Closer; | ||
| import com.google.common.net.MediaType; | ||
| import io.airlift.log.Logger; | ||
| import io.airlift.units.DataSize; | ||
| import io.airlift.units.Duration; | ||
| import io.trino.hdfs.FSDataInputStreamTail; | ||
| import io.trino.hdfs.FileSystemWithBatchDelete; | ||
| import org.apache.hadoop.conf.Configurable; | ||
| import org.apache.hadoop.conf.Configuration; | ||
| import org.apache.hadoop.fs.BlockLocation; | ||
|
|
@@ -115,6 +119,7 @@ | |
| import java.util.ArrayList; | ||
| import java.util.Arrays; | ||
| import java.util.Base64; | ||
| import java.util.Collection; | ||
| import java.util.Date; | ||
| import java.util.Iterator; | ||
| import java.util.List; | ||
|
|
@@ -142,6 +147,7 @@ | |
| import static com.google.common.base.Throwables.throwIfInstanceOf; | ||
| import static com.google.common.base.Throwables.throwIfUnchecked; | ||
| import static com.google.common.base.Verify.verify; | ||
| import static com.google.common.collect.ImmutableList.toImmutableList; | ||
| import static com.google.common.collect.Iterables.toArray; | ||
| import static com.google.common.hash.Hashing.md5; | ||
| import static io.airlift.concurrent.Threads.threadsNamed; | ||
|
|
@@ -169,6 +175,7 @@ | |
|
|
||
| public class TrinoS3FileSystem | ||
| extends FileSystem | ||
| implements FileSystemWithBatchDelete | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Aren't we moving away from use of hadoop's
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Maybe it should, though currently only HdfsFileSystem implements it and it doesn't support batch delete. |
||
| { | ||
| public static final String S3_USER_AGENT_PREFIX = "trino.s3.user-agent-prefix"; | ||
| public static final String S3_CREDENTIALS_PROVIDER = "trino.s3.credentials-provider"; | ||
|
|
@@ -230,6 +237,7 @@ public class TrinoS3FileSystem | |
| private static final Set<String> GLACIER_STORAGE_CLASSES = ImmutableSet.of(Glacier.toString(), DeepArchive.toString()); | ||
| private static final MediaType DIRECTORY_MEDIA_TYPE = MediaType.create("application", "x-directory"); | ||
| private static final String S3_DEFAULT_ROLE_SESSION_NAME = "trino-session"; | ||
| public static final int DELETE_BATCH_SIZE = 1000; | ||
|
|
||
| private URI uri; | ||
| private Path workingDirectory; | ||
|
|
@@ -643,6 +651,35 @@ private boolean deleteObject(String key) | |
| } | ||
| } | ||
|
|
||
| @Override | ||
| public void deleteFiles(Collection<Path> paths) | ||
| throws IOException | ||
| { | ||
| try { | ||
| Iterable<List<Path>> partitions = Iterables.partition(paths, DELETE_BATCH_SIZE); | ||
| for (List<Path> currentBatch : partitions) { | ||
| deletePaths(currentBatch); | ||
| } | ||
| } | ||
| catch (AmazonClientException e) { | ||
| throw new IOException("Exception while batch deleting paths", e); | ||
| } | ||
| } | ||
|
|
||
| private void deletePaths(List<Path> paths) | ||
| { | ||
| List<KeyVersion> keys = paths.stream() | ||
| .map(TrinoS3FileSystem::keyFromPath) | ||
| .map(KeyVersion::new) | ||
| .collect(toImmutableList()); | ||
| DeleteObjectsRequest deleteObjectsRequest = new DeleteObjectsRequest(getBucketName(uri)) | ||
| .withRequesterPays(requesterPaysEnabled) | ||
| .withKeys(keys) | ||
| .withQuiet(true); | ||
|
|
||
| s3.deleteObjects(deleteObjectsRequest); | ||
| } | ||
|
|
||
| @Override | ||
| public boolean mkdirs(Path f, FsPermission permission) | ||
| { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -152,6 +152,7 @@ | |
| import java.util.OptionalLong; | ||
| import java.util.Set; | ||
| import java.util.concurrent.ConcurrentHashMap; | ||
| import java.util.function.Consumer; | ||
| import java.util.function.Supplier; | ||
| import java.util.regex.Matcher; | ||
| import java.util.regex.Pattern; | ||
|
|
@@ -269,6 +270,8 @@ public class IcebergMetadata | |
| private static final String NUMBER_OF_DISTINCT_VALUES_NAME = "NUMBER_OF_DISTINCT_VALUES"; | ||
| private static final FunctionName NUMBER_OF_DISTINCT_VALUES_FUNCTION = new FunctionName(IcebergThetaSketchForStats.NAME); | ||
|
|
||
| private static final Integer DELETE_BATCH_SIZE = 1000; | ||
|
|
||
| private final TypeManager typeManager; | ||
| private final JsonCodec<CommitTaskData> commitTaskCodec; | ||
| private final TrinoCatalog catalog; | ||
|
|
@@ -1176,10 +1179,32 @@ private void executeExpireSnapshots(ConnectorSession session, IcebergTableExecut | |
| IcebergSessionProperties.EXPIRE_SNAPSHOTS_MIN_RETENTION); | ||
|
|
||
| long expireTimestampMillis = session.getStart().toEpochMilli() - retention.toMillis(); | ||
| TrinoFileSystem fileSystem = fileSystemFactory.create(session); | ||
| List<String> pathsToDelete = new ArrayList<>(); | ||
| // deleteFunction is not accessed from multiple threads unless .executeDeleteWith() is used | ||
| Consumer<String> deleteFunction = path -> { | ||
| pathsToDelete.add(path); | ||
| if (pathsToDelete.size() == DELETE_BATCH_SIZE) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is this check (and the deletion code associated to it) still needed?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. yes it is still needed, I don't think we want to create a very huge list here and store in memory, we can discuss the value of DELETE_BATCH_SIZE but I think in general this check is needed
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. From You may end up in a situation where you add stuff with
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. not true, current implementation in the worst case deletes file at the same time as the old one. The old one deletes files when
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. in a threaded context many things go wrong
|
||
| try { | ||
| fileSystem.deleteFiles(pathsToDelete); | ||
| pathsToDelete.clear(); | ||
| } | ||
| catch (IOException e) { | ||
| throw new TrinoException(ICEBERG_FILESYSTEM_ERROR, "Failed to delete files during snapshot expiration", e); | ||
| } | ||
| } | ||
| }; | ||
|
|
||
| table.expireSnapshots() | ||
| .expireOlderThan(expireTimestampMillis) | ||
| .deleteWith(deleteFunction) | ||
|
findepi marked this conversation as resolved.
Outdated
|
||
| .commit(); | ||
| try { | ||
| fileSystem.deleteFiles(pathsToDelete); | ||
| } | ||
| catch (IOException e) { | ||
| throw new TrinoException(ICEBERG_FILESYSTEM_ERROR, "Failed to delete files during snapshot expiration", e); | ||
| } | ||
| } | ||
|
|
||
| private static void validateTableExecuteParameters( | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Does this work if the file is in the root directory of the file system?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Then it will be grouped by
Path("" )- is it incorrect ?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Probably something more like
s3://bucket/in the s3 case at least. I think it's fineThere was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
yes it should be fine imho