2022-10-12 14:58:13 +00:00
|
|
|
#include <Backups/BackupIO_S3.h>
|
|
|
|
|
|
|
|
#if USE_AWS_S3
|
|
|
|
#include <Common/quoteString.h>
|
|
|
|
#include <Interpreters/threadPoolCallbackRunner.h>
|
|
|
|
#include <Interpreters/Context.h>
|
2023-05-03 16:56:45 +00:00
|
|
|
#include <IO/SharedThreadPools.h>
|
2022-10-12 14:58:13 +00:00
|
|
|
#include <IO/ReadBufferFromS3.h>
|
|
|
|
#include <IO/WriteBufferFromS3.h>
|
2022-12-16 22:57:09 +00:00
|
|
|
#include <IO/HTTPHeaderEntries.h>
|
2023-01-21 14:34:09 +00:00
|
|
|
#include <IO/S3/copyS3File.h>
|
2023-02-03 13:30:52 +00:00
|
|
|
#include <IO/S3/Client.h>
|
2023-03-10 10:06:32 +00:00
|
|
|
#include <IO/S3/Credentials.h>
|
2023-04-28 11:15:29 +00:00
|
|
|
#include <Disks/IDisk.h>
|
2023-02-03 13:30:52 +00:00
|
|
|
|
2022-10-12 14:58:13 +00:00
|
|
|
#include <Poco/Util/AbstractConfiguration.h>
|
2023-01-10 00:32:37 +00:00
|
|
|
|
2022-10-12 14:58:13 +00:00
|
|
|
#include <aws/core/auth/AWSCredentials.h>
|
2023-02-03 13:30:52 +00:00
|
|
|
|
2023-01-10 00:32:37 +00:00
|
|
|
#include <filesystem>
|
2022-10-12 14:58:13 +00:00
|
|
|
|
|
|
|
|
|
|
|
namespace fs = std::filesystem;
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int S3_ERROR;
|
|
|
|
extern const int LOGICAL_ERROR;
|
|
|
|
}
|
|
|
|
|
|
|
|
namespace
|
|
|
|
{
|
2023-02-03 13:30:52 +00:00
|
|
|
std::shared_ptr<S3::Client>
|
2022-10-12 14:58:13 +00:00
|
|
|
makeS3Client(const S3::URI & s3_uri, const String & access_key_id, const String & secret_access_key, const ContextPtr & context)
|
|
|
|
{
|
|
|
|
auto settings = context->getStorageS3Settings().getSettings(s3_uri.uri.toString());
|
|
|
|
|
|
|
|
Aws::Auth::AWSCredentials credentials(access_key_id, secret_access_key);
|
2022-12-16 22:57:09 +00:00
|
|
|
HTTPHeaderEntries headers;
|
2022-10-12 14:58:13 +00:00
|
|
|
if (access_key_id.empty())
|
|
|
|
{
|
|
|
|
credentials = Aws::Auth::AWSCredentials(settings.auth_settings.access_key_id, settings.auth_settings.secret_access_key);
|
|
|
|
headers = settings.auth_settings.headers;
|
|
|
|
}
|
|
|
|
|
|
|
|
S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration(
|
|
|
|
settings.auth_settings.region,
|
|
|
|
context->getRemoteHostFilter(),
|
2022-10-07 10:46:45 +00:00
|
|
|
static_cast<unsigned>(context->getGlobalContext()->getSettingsRef().s3_max_redirects),
|
2022-10-12 14:58:13 +00:00
|
|
|
context->getGlobalContext()->getSettingsRef().enable_s3_requests_logging,
|
2022-11-17 16:35:04 +00:00
|
|
|
/* for_disk_s3 = */ false, /* get_request_throttler = */ {}, /* put_request_throttler = */ {});
|
2022-10-12 14:58:13 +00:00
|
|
|
|
|
|
|
client_configuration.endpointOverride = s3_uri.endpoint;
|
2022-10-07 10:46:45 +00:00
|
|
|
client_configuration.maxConnections = static_cast<unsigned>(context->getSettingsRef().s3_max_connections);
|
2022-10-12 14:58:13 +00:00
|
|
|
/// Increase connect timeout
|
|
|
|
client_configuration.connectTimeoutMs = 10 * 1000;
|
|
|
|
/// Requests in backups can be extremely long, set to one hour
|
|
|
|
client_configuration.requestTimeoutMs = 60 * 60 * 1000;
|
|
|
|
|
|
|
|
return S3::ClientFactory::instance().create(
|
|
|
|
client_configuration,
|
|
|
|
s3_uri.is_virtual_hosted_style,
|
|
|
|
credentials.GetAWSAccessKeyId(),
|
|
|
|
credentials.GetAWSSecretKey(),
|
|
|
|
settings.auth_settings.server_side_encryption_customer_key_base64,
|
2023-04-12 17:01:07 +00:00
|
|
|
settings.auth_settings.server_side_encryption_kms_config,
|
2022-10-12 14:58:13 +00:00
|
|
|
std::move(headers),
|
2023-03-27 14:44:34 +00:00
|
|
|
S3::CredentialsConfiguration
|
|
|
|
{
|
|
|
|
settings.auth_settings.use_environment_credentials.value_or(
|
2023-03-31 09:11:01 +00:00
|
|
|
context->getConfigRef().getBool("s3.use_environment_credentials", true)),
|
2023-03-27 14:44:34 +00:00
|
|
|
settings.auth_settings.use_insecure_imds_request.value_or(
|
|
|
|
context->getConfigRef().getBool("s3.use_insecure_imds_request", false)),
|
|
|
|
settings.auth_settings.expiration_window_seconds.value_or(
|
|
|
|
context->getConfigRef().getUInt64("s3.expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS)),
|
|
|
|
settings.auth_settings.no_sign_request.value_or(
|
|
|
|
context->getConfigRef().getBool("s3.no_sign_request", false)),
|
|
|
|
});
|
2022-10-12 14:58:13 +00:00
|
|
|
}
|
|
|
|
|
2023-02-03 13:30:52 +00:00
|
|
|
Aws::Vector<Aws::S3::Model::Object> listObjects(S3::Client & client, const S3::URI & s3_uri, const String & file_name)
|
2022-10-12 14:58:13 +00:00
|
|
|
{
|
2023-02-03 13:30:52 +00:00
|
|
|
S3::ListObjectsRequest request;
|
2022-10-12 14:58:13 +00:00
|
|
|
request.SetBucket(s3_uri.bucket);
|
|
|
|
request.SetPrefix(fs::path{s3_uri.key} / file_name);
|
|
|
|
request.SetMaxKeys(1);
|
|
|
|
auto outcome = client.ListObjects(request);
|
|
|
|
if (!outcome.IsSuccess())
|
2023-01-23 13:16:14 +00:00
|
|
|
throw Exception::createDeprecated(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR);
|
2022-10-12 14:58:13 +00:00
|
|
|
return outcome.GetResult().GetContents();
|
|
|
|
}
|
2022-12-15 10:11:46 +00:00
|
|
|
|
|
|
|
bool isNotFoundError(Aws::S3::S3Errors error)
|
|
|
|
{
|
|
|
|
return error == Aws::S3::S3Errors::RESOURCE_NOT_FOUND
|
|
|
|
|| error == Aws::S3::S3Errors::NO_SUCH_KEY;
|
|
|
|
}
|
2022-10-12 14:58:13 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
BackupReaderS3::BackupReaderS3(
|
2023-06-28 15:16:02 +00:00
|
|
|
const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool allow_s3_native_copy, const ContextPtr & context_)
|
2023-04-28 11:15:29 +00:00
|
|
|
: BackupReaderDefault(&Poco::Logger::get("BackupReaderS3"), context_)
|
2023-04-15 18:43:06 +00:00
|
|
|
, s3_uri(s3_uri_)
|
2022-10-12 14:58:13 +00:00
|
|
|
, client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_))
|
2022-11-17 16:35:04 +00:00
|
|
|
, request_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()).request_settings)
|
2023-04-25 17:44:03 +00:00
|
|
|
, data_source_description{DataSourceType::S3, s3_uri.endpoint, false, false}
|
2022-10-12 14:58:13 +00:00
|
|
|
{
|
2023-06-27 14:20:27 +00:00
|
|
|
request_settings.updateFromSettings(context_->getSettingsRef());
|
2022-11-17 16:35:04 +00:00
|
|
|
request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint
|
2023-06-28 15:16:02 +00:00
|
|
|
request_settings.allow_native_copy = allow_s3_native_copy;
|
2022-10-12 14:58:13 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
BackupReaderS3::~BackupReaderS3() = default;
|
|
|
|
|
|
|
|
bool BackupReaderS3::fileExists(const String & file_name)
|
|
|
|
{
|
|
|
|
return !listObjects(*client, s3_uri, file_name).empty();
|
|
|
|
}
|
|
|
|
|
|
|
|
UInt64 BackupReaderS3::getFileSize(const String & file_name)
|
|
|
|
{
|
|
|
|
auto objects = listObjects(*client, s3_uri, file_name);
|
|
|
|
if (objects.empty())
|
|
|
|
throw Exception(ErrorCodes::S3_ERROR, "Object {} must exist");
|
|
|
|
return objects[0].GetSize();
|
|
|
|
}
|
|
|
|
|
|
|
|
std::unique_ptr<SeekableReadBuffer> BackupReaderS3::readFile(const String & file_name)
|
|
|
|
{
|
|
|
|
return std::make_unique<ReadBufferFromS3>(
|
2022-11-17 16:35:04 +00:00
|
|
|
client, s3_uri.bucket, fs::path(s3_uri.key) / file_name, s3_uri.version_id, request_settings, read_settings);
|
2022-10-12 14:58:13 +00:00
|
|
|
}
|
|
|
|
|
2023-04-25 17:44:03 +00:00
|
|
|
void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup,
|
2023-04-28 11:15:29 +00:00
|
|
|
DiskPtr destination_disk, const String & destination_path, WriteMode write_mode)
|
2023-03-13 22:43:15 +00:00
|
|
|
{
|
2023-05-03 13:56:28 +00:00
|
|
|
/// Use the native copy as a more optimal way to copy a file from S3 to S3 if it's possible.
|
|
|
|
/// We don't check for `has_throttling` here because the native copy almost doesn't use network.
|
2023-04-25 17:44:03 +00:00
|
|
|
auto destination_data_source_description = destination_disk->getDataSourceDescription();
|
2023-06-27 14:20:27 +00:00
|
|
|
if (destination_data_source_description.sameKind(data_source_description)
|
2023-04-25 17:44:03 +00:00
|
|
|
&& (destination_data_source_description.is_encrypted == encrypted_in_backup))
|
|
|
|
{
|
2023-06-27 14:20:27 +00:00
|
|
|
LOG_TRACE(log, "Copying {} from S3 to disk {}", path_in_backup, destination_disk->getName());
|
2023-04-25 17:44:03 +00:00
|
|
|
auto write_blob_function = [&](const Strings & blob_path, WriteMode mode, const std::optional<ObjectAttributes> & object_attributes) -> size_t
|
|
|
|
{
|
|
|
|
/// Object storage always uses mode `Rewrite` because it simulates append using metadata and different files.
|
|
|
|
if (blob_path.size() != 2 || mode != WriteMode::Rewrite)
|
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
|
|
|
"Blob writing function called with unexpected blob_path.size={} or mode={}",
|
|
|
|
blob_path.size(), mode);
|
|
|
|
|
2023-06-27 14:20:27 +00:00
|
|
|
auto create_read_buffer = [this, path_in_backup]
|
|
|
|
{
|
|
|
|
return readFile(path_in_backup);
|
|
|
|
};
|
|
|
|
|
2023-04-25 17:44:03 +00:00
|
|
|
copyS3File(
|
2023-06-27 14:20:27 +00:00
|
|
|
create_read_buffer,
|
2023-04-25 17:44:03 +00:00
|
|
|
client,
|
|
|
|
s3_uri.bucket,
|
|
|
|
fs::path(s3_uri.key) / path_in_backup,
|
|
|
|
0,
|
|
|
|
file_size,
|
2023-05-05 09:40:33 +00:00
|
|
|
/* dest_bucket= */ blob_path[1],
|
|
|
|
/* dest_key= */ blob_path[0],
|
2023-04-25 17:44:03 +00:00
|
|
|
request_settings,
|
|
|
|
object_attributes,
|
2023-06-06 12:42:56 +00:00
|
|
|
threadPoolCallbackRunner<void>(getBackupsIOThreadPool().get(), "BackupReaderS3"),
|
2023-04-25 17:44:03 +00:00
|
|
|
/* for_disk_s3= */ true);
|
|
|
|
|
|
|
|
return file_size;
|
|
|
|
};
|
|
|
|
|
|
|
|
destination_disk->writeFileUsingBlobWritingFunction(destination_path, write_mode, write_blob_function);
|
2023-05-03 13:56:28 +00:00
|
|
|
return; /// copied!
|
2023-04-25 17:44:03 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
/// Fallback to copy through buffers.
|
2023-04-28 11:15:29 +00:00
|
|
|
BackupReaderDefault::copyFileToDisk(path_in_backup, file_size, encrypted_in_backup, destination_disk, destination_path, write_mode);
|
2023-03-13 22:43:15 +00:00
|
|
|
}
|
|
|
|
|
2022-10-12 14:58:13 +00:00
|
|
|
|
|
|
|
BackupWriterS3::BackupWriterS3(
|
2023-06-28 15:16:02 +00:00
|
|
|
const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool allow_s3_native_copy, const ContextPtr & context_)
|
2023-04-28 11:15:29 +00:00
|
|
|
: BackupWriterDefault(&Poco::Logger::get("BackupWriterS3"), context_)
|
2023-04-04 09:57:55 +00:00
|
|
|
, s3_uri(s3_uri_)
|
2022-10-12 14:58:13 +00:00
|
|
|
, client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_))
|
2022-11-17 16:35:04 +00:00
|
|
|
, request_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()).request_settings)
|
2023-04-25 17:44:03 +00:00
|
|
|
, data_source_description{DataSourceType::S3, s3_uri.endpoint, false, false}
|
2022-10-12 14:58:13 +00:00
|
|
|
{
|
2022-12-17 16:02:34 +00:00
|
|
|
request_settings.updateFromSettings(context_->getSettingsRef());
|
2022-11-17 16:35:04 +00:00
|
|
|
request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint
|
2023-06-28 15:16:02 +00:00
|
|
|
request_settings.allow_native_copy = allow_s3_native_copy;
|
2022-10-12 14:58:13 +00:00
|
|
|
}
|
|
|
|
|
2023-04-25 17:44:03 +00:00
|
|
|
void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path,
|
|
|
|
bool copy_encrypted, UInt64 start_pos, UInt64 length)
|
2022-10-12 14:58:13 +00:00
|
|
|
{
|
2023-05-03 13:56:28 +00:00
|
|
|
/// Use the native copy as a more optimal way to copy a file from S3 to S3 if it's possible.
|
|
|
|
/// We don't check for `has_throttling` here because the native copy almost doesn't use network.
|
2023-04-25 17:44:03 +00:00
|
|
|
auto source_data_source_description = src_disk->getDataSourceDescription();
|
2023-06-27 14:20:27 +00:00
|
|
|
if (source_data_source_description.sameKind(data_source_description) && (source_data_source_description.is_encrypted == copy_encrypted))
|
2023-04-25 17:44:03 +00:00
|
|
|
{
|
2023-05-05 09:40:33 +00:00
|
|
|
/// getBlobPath() can return more than 3 elements if the file is stored as multiple objects in S3 bucket.
|
2023-04-28 11:15:29 +00:00
|
|
|
/// In this case we can't use the native copy.
|
2023-04-25 17:44:03 +00:00
|
|
|
if (auto blob_path = src_disk->getBlobPath(src_path); blob_path.size() == 2)
|
|
|
|
{
|
2023-06-27 14:20:27 +00:00
|
|
|
auto create_read_buffer = [src_disk, src_path, copy_encrypted, settings = read_settings.adjustBufferSize(start_pos + length)]
|
|
|
|
{
|
|
|
|
if (copy_encrypted)
|
|
|
|
return src_disk->readEncryptedFile(src_path, settings);
|
|
|
|
else
|
|
|
|
return src_disk->readFile(src_path, settings);
|
|
|
|
};
|
|
|
|
|
|
|
|
LOG_TRACE(log, "Copying file {} from disk {} to S3", src_path, src_disk->getName());
|
2023-04-25 17:44:03 +00:00
|
|
|
copyS3File(
|
2023-06-27 14:20:27 +00:00
|
|
|
create_read_buffer,
|
2023-04-25 17:44:03 +00:00
|
|
|
client,
|
2023-05-05 09:40:33 +00:00
|
|
|
/* src_bucket */ blob_path[1],
|
|
|
|
/* src_key= */ blob_path[0],
|
2023-04-25 17:44:03 +00:00
|
|
|
start_pos,
|
|
|
|
length,
|
2023-05-01 16:43:20 +00:00
|
|
|
s3_uri.bucket,
|
2023-04-25 17:44:03 +00:00
|
|
|
fs::path(s3_uri.key) / path_in_backup,
|
|
|
|
request_settings,
|
|
|
|
{},
|
2023-06-06 12:42:56 +00:00
|
|
|
threadPoolCallbackRunner<void>(getBackupsIOThreadPool().get(), "BackupWriterS3"));
|
2023-05-03 13:56:28 +00:00
|
|
|
return; /// copied!
|
2023-04-25 17:44:03 +00:00
|
|
|
}
|
|
|
|
}
|
2022-10-12 14:58:13 +00:00
|
|
|
|
2023-04-25 17:44:03 +00:00
|
|
|
/// Fallback to copy through buffers.
|
2023-04-28 11:15:29 +00:00
|
|
|
BackupWriterDefault::copyFileFromDisk(path_in_backup, src_disk, src_path, copy_encrypted, start_pos, length);
|
2022-10-12 14:58:13 +00:00
|
|
|
}
|
|
|
|
|
2023-04-25 17:44:03 +00:00
|
|
|
void BackupWriterS3::copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length)
|
2023-01-10 00:32:37 +00:00
|
|
|
{
|
2023-04-25 17:44:03 +00:00
|
|
|
copyDataToS3File(create_read_buffer, start_pos, length, client, s3_uri.bucket, fs::path(s3_uri.key) / path_in_backup, request_settings, {},
|
2023-06-06 12:42:56 +00:00
|
|
|
threadPoolCallbackRunner<void>(getBackupsIOThreadPool().get(), "BackupWriterS3"));
|
2023-01-10 00:32:37 +00:00
|
|
|
}
|
2022-10-12 14:58:13 +00:00
|
|
|
|
|
|
|
BackupWriterS3::~BackupWriterS3() = default;
|
|
|
|
|
|
|
|
bool BackupWriterS3::fileExists(const String & file_name)
|
|
|
|
{
|
|
|
|
return !listObjects(*client, s3_uri, file_name).empty();
|
|
|
|
}
|
|
|
|
|
|
|
|
UInt64 BackupWriterS3::getFileSize(const String & file_name)
|
|
|
|
{
|
|
|
|
auto objects = listObjects(*client, s3_uri, file_name);
|
|
|
|
if (objects.empty())
|
|
|
|
throw Exception(ErrorCodes::S3_ERROR, "Object {} must exist");
|
|
|
|
return objects[0].GetSize();
|
|
|
|
}
|
|
|
|
|
2023-04-28 11:15:29 +00:00
|
|
|
std::unique_ptr<ReadBuffer> BackupWriterS3::readFile(const String & file_name, size_t expected_file_size)
|
2022-10-12 14:58:13 +00:00
|
|
|
{
|
2023-04-28 11:15:29 +00:00
|
|
|
return std::make_unique<ReadBufferFromS3>(
|
|
|
|
client, s3_uri.bucket, fs::path(s3_uri.key) / file_name, s3_uri.version_id, request_settings, read_settings,
|
|
|
|
false, 0, 0, false, expected_file_size);
|
2022-10-12 14:58:13 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
std::unique_ptr<WriteBuffer> BackupWriterS3::writeFile(const String & file_name)
|
|
|
|
{
|
|
|
|
return std::make_unique<WriteBufferFromS3>(
|
|
|
|
client,
|
2023-06-19 23:35:35 +00:00
|
|
|
client, // already has long timeout
|
2022-10-12 14:58:13 +00:00
|
|
|
s3_uri.bucket,
|
|
|
|
fs::path(s3_uri.key) / file_name,
|
2023-05-19 15:22:57 +00:00
|
|
|
DBMS_DEFAULT_BUFFER_SIZE,
|
2022-11-17 16:35:04 +00:00
|
|
|
request_settings,
|
2022-10-12 14:58:13 +00:00
|
|
|
std::nullopt,
|
2023-06-06 12:42:56 +00:00
|
|
|
threadPoolCallbackRunner<void>(getBackupsIOThreadPool().get(), "BackupWriterS3"),
|
2023-04-28 11:15:29 +00:00
|
|
|
write_settings);
|
2022-10-12 14:58:13 +00:00
|
|
|
}
|
|
|
|
|
2022-12-05 12:22:05 +00:00
|
|
|
void BackupWriterS3::removeFile(const String & file_name)
|
|
|
|
{
|
2023-02-03 13:30:52 +00:00
|
|
|
S3::DeleteObjectRequest request;
|
2022-12-05 12:22:05 +00:00
|
|
|
request.SetBucket(s3_uri.bucket);
|
|
|
|
request.SetKey(fs::path(s3_uri.key) / file_name);
|
|
|
|
auto outcome = client->DeleteObject(request);
|
2022-12-15 10:11:46 +00:00
|
|
|
if (!outcome.IsSuccess() && !isNotFoundError(outcome.GetError().GetErrorType()))
|
2023-01-23 13:16:14 +00:00
|
|
|
throw Exception::createDeprecated(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR);
|
2022-12-05 12:22:05 +00:00
|
|
|
}
|
|
|
|
|
2022-10-12 14:58:13 +00:00
|
|
|
void BackupWriterS3::removeFiles(const Strings & file_names)
|
2022-12-05 12:22:05 +00:00
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
|
|
|
if (!supports_batch_delete.has_value() || supports_batch_delete.value() == true)
|
|
|
|
{
|
|
|
|
removeFilesBatch(file_names);
|
|
|
|
supports_batch_delete = true;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
for (const auto & file_name : file_names)
|
|
|
|
removeFile(file_name);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
catch (const Exception &)
|
|
|
|
{
|
|
|
|
if (!supports_batch_delete.has_value())
|
|
|
|
{
|
|
|
|
supports_batch_delete = false;
|
|
|
|
LOG_TRACE(log, "DeleteObjects is not supported. Retrying with plain DeleteObject.");
|
|
|
|
|
|
|
|
for (const auto & file_name : file_names)
|
|
|
|
removeFile(file_name);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
throw;
|
|
|
|
}
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
void BackupWriterS3::removeFilesBatch(const Strings & file_names)
|
2022-10-12 14:58:13 +00:00
|
|
|
{
|
|
|
|
/// One call of DeleteObjects() cannot remove more than 1000 keys.
|
|
|
|
size_t chunk_size_limit = 1000;
|
|
|
|
|
|
|
|
size_t current_position = 0;
|
|
|
|
while (current_position < file_names.size())
|
|
|
|
{
|
|
|
|
std::vector<Aws::S3::Model::ObjectIdentifier> current_chunk;
|
|
|
|
for (; current_position < file_names.size() && current_chunk.size() < chunk_size_limit; ++current_position)
|
|
|
|
{
|
|
|
|
Aws::S3::Model::ObjectIdentifier obj;
|
|
|
|
obj.SetKey(fs::path(s3_uri.key) / file_names[current_position]);
|
|
|
|
current_chunk.push_back(obj);
|
|
|
|
}
|
|
|
|
|
|
|
|
Aws::S3::Model::Delete delkeys;
|
|
|
|
delkeys.SetObjects(current_chunk);
|
2023-02-03 13:30:52 +00:00
|
|
|
S3::DeleteObjectsRequest request;
|
2022-10-12 14:58:13 +00:00
|
|
|
request.SetBucket(s3_uri.bucket);
|
|
|
|
request.SetDelete(delkeys);
|
|
|
|
|
|
|
|
auto outcome = client->DeleteObjects(request);
|
2022-12-15 10:11:46 +00:00
|
|
|
if (!outcome.IsSuccess() && !isNotFoundError(outcome.GetError().GetErrorType()))
|
2023-01-23 13:16:14 +00:00
|
|
|
throw Exception::createDeprecated(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR);
|
2022-10-12 14:58:13 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
#endif
|