Skip to content

Commit

Permalink
backward compatibility for template remote keys
Browse files Browse the repository at this point in the history
  • Loading branch information
CheSema committed Oct 17, 2023
1 parent ebfad5c commit 6b93fb1
Show file tree
Hide file tree
Showing 43 changed files with 737 additions and 264 deletions.
68 changes: 68 additions & 0 deletions src/Common/ObjectStorageKey.cpp
@@ -0,0 +1,68 @@
#include "ObjectStorageKey.h"

#include <Common/Exception.h>

#include <filesystem>

namespace fs = std::filesystem;

namespace DB
{

namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}

const String & ObjectStorageKey::getPrefix() const
{
if (!is_relative)
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "object key has no prefix, key: {}", key);

return prefix;
}

const String & ObjectStorageKey::getSuffix() const
{
if (!is_relative)
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "object key has no suffix, key: {}", key);
return suffix;
}

const String & ObjectStorageKey::serialize() const
{
return key;
}

ObjectStorageKey ObjectStorageKey::createAsRelativeAnyway(String key_)
{
ObjectStorageKey object_key;
object_key.suffix = std::move(key_);
object_key.key = object_key.suffix;
object_key.is_relative = true;
return object_key;
}

ObjectStorageKey ObjectStorageKey::createAsRelative(String prefix_, String suffix_)
{
ObjectStorageKey object_key;
object_key.prefix = std::move(prefix_);
object_key.suffix = std::move(suffix_);

if (object_key.prefix.empty())
object_key.key = object_key.suffix;
else
object_key.key = fs::path(object_key.prefix) / object_key.suffix;

object_key.is_relative = true;
return object_key;
}

ObjectStorageKey ObjectStorageKey::createAsAbsolute(String key_)
{
ObjectStorageKey object_key;
object_key.key = std::move(key_);
object_key.is_relative = true;
return object_key;
}
}
29 changes: 29 additions & 0 deletions src/Common/ObjectStorageKey.h
@@ -0,0 +1,29 @@
#pragma once

#include <base/types.h>

#include <memory>

namespace DB
{
struct ObjectStorageKey
{
ObjectStorageKey() = default;

bool hasPrefix() const { return is_relative; }
const String & getPrefix() const;
const String & getSuffix() const;
const String & serialize() const;

static ObjectStorageKey createAsRelative(String prefix_, String suffix_);
static ObjectStorageKey createAsRelativeAnyway(String key_);
static ObjectStorageKey createAsAbsolute(String key_);

private:
String prefix;
String suffix;
String key;
bool is_relative = false;
};

}
3 changes: 2 additions & 1 deletion src/Core/Settings.h
Expand Up @@ -283,7 +283,8 @@ class IColumn;
M(Bool, http_write_exception_in_output_format, true, "Write exception in output format to produce valid output. Works with JSON and XML formats.", 0) \
M(UInt64, http_response_buffer_size, 0, "The number of bytes to buffer in the server memory before sending a HTTP response to the client or flushing to disk (when http_wait_end_of_query is enabled).", 0) \
\
M(Bool, fsync_metadata, true, "Do fsync after changing metadata for tables and databases (.sql files). Could be disabled in case of poor latency on server with high load of DDL queries and high load of disk subsystem.", 0) \
M(Bool, fsync_metadata, true, "Do fsync after changing metadata for tables and databases (.sql files). Could be disabled in case of poor latency on server with high load of DDL queries and high load of disk subsystem.", 0) \
M(Bool, storage_metadaata_write_full_object_key, false, "Enable write metadata files with VERSION_FULL_OBJECT_KEY format", 0) \
\
M(Bool, join_use_nulls, false, "Use NULLs for non-joined rows of outer JOINs for types that can be inside Nullable. If false, use default value of corresponding columns data type.", IMPORTANT) \
\
Expand Down
8 changes: 5 additions & 3 deletions src/Disks/IDisk.h
Expand Up @@ -302,12 +302,14 @@ class IDisk : public Space
struct LocalPathWithObjectStoragePaths
{
std::string local_path;
std::string common_prefix_for_objects;
StoredObjects objects;

LocalPathWithObjectStoragePaths(
const std::string & local_path_, const std::string & common_prefix_for_objects_, StoredObjects && objects_)
: local_path(local_path_), common_prefix_for_objects(common_prefix_for_objects_), objects(std::move(objects_)) {}
const std::string & local_path_,
StoredObjects && objects_)
: local_path(local_path_)
, objects(std::move(objects_))
{}
};

virtual void getRemotePathsRecursive(const String &, std::vector<LocalPathWithObjectStoragePaths> &)
Expand Down
17 changes: 3 additions & 14 deletions src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp
Expand Up @@ -102,9 +102,9 @@ AzureObjectStorage::AzureObjectStorage(
data_source_description.is_encrypted = false;
}

std::string AzureObjectStorage::generateBlobNameForPath(const std::string & /* path */)
ObjectStorageKey AzureObjectStorage::generateBlobNameForPath(const std::string & /* path */) const
{
return getRandomASCIIString(32);
return ObjectStorageKey::createAsRelativeAnyway(getRandomASCIIString(32));
}

bool AzureObjectStorage::exists(const StoredObject & object) const
Expand Down Expand Up @@ -320,18 +320,7 @@ void AzureObjectStorage::removeObjectsIfExist(const StoredObjects & objects)
auto client_ptr = client.get();
for (const auto & object : objects)
{
try
{
auto delete_info = client_ptr->DeleteBlob(object.remote_path);
}
catch (const Azure::Storage::StorageException & e)
{
/// If object doesn't exist...
if (e.StatusCode == Azure::Core::Http::HttpStatusCode::NotFound)
return;
tryLogCurrentException(__PRETTY_FUNCTION__);
throw;
}
removeObjectIfExists(object);
}

}
Expand Down
Expand Up @@ -121,7 +121,7 @@ class AzureObjectStorage : public IObjectStorage
const std::string & config_prefix,
ContextPtr context) override;

std::string generateBlobNameForPath(const std::string & path) override;
ObjectStorageKey generateBlobNameForPath(const std::string & path) const override;

bool isRemote() const override { return true; }

Expand Down
Expand Up @@ -31,11 +31,12 @@ void registerDiskAzureBlobStorage(DiskFactory & factory, bool global_skip_access
getAzureBlobContainerClient(config, config_prefix),
getAzureBlobStorageSettings(config, config_prefix, context));

auto metadata_storage = std::make_shared<MetadataStorageFromDisk>(metadata_disk, "");
String key_prefix;
auto metadata_storage = std::make_shared<MetadataStorageFromDisk>(metadata_disk, key_prefix);

std::shared_ptr<IDisk> azure_blob_storage_disk = std::make_shared<DiskObjectStorage>(
name,
/* no namespaces */"",
/* no namespaces */ key_prefix,
"DiskAzureBlobStorage",
std::move(metadata_storage),
std::move(azure_object_storage),
Expand Down
2 changes: 1 addition & 1 deletion src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp
Expand Up @@ -42,7 +42,7 @@ FileCache::Key CachedObjectStorage::getCacheKey(const std::string & path) const
return cache->createKeyForPath(path);
}

std::string CachedObjectStorage::generateBlobNameForPath(const std::string & path)
ObjectStorageKey CachedObjectStorage::generateBlobNameForPath(const std::string & path) const
{
return object_storage->generateBlobNameForPath(path);
}
Expand Down
2 changes: 1 addition & 1 deletion src/Disks/ObjectStorages/Cached/CachedObjectStorage.h
Expand Up @@ -92,7 +92,7 @@ class CachedObjectStorage final : public IObjectStorage

const std::string & getCacheName() const override { return cache_config_name; }

std::string generateBlobNameForPath(const std::string & path) override;
ObjectStorageKey generateBlobNameForPath(const std::string & path) const override;

bool isRemote() const override { return object_storage->isRemote(); }

Expand Down
15 changes: 8 additions & 7 deletions src/Disks/ObjectStorages/DiskObjectStorage.cpp
Expand Up @@ -48,14 +48,14 @@ DiskTransactionPtr DiskObjectStorage::createObjectStorageTransaction()

DiskObjectStorage::DiskObjectStorage(
const String & name_,
const String & object_storage_root_path_,
const String & object_key_prefix_,
const String & log_name,
MetadataStoragePtr metadata_storage_,
ObjectStoragePtr object_storage_,
const Poco::Util::AbstractConfiguration & config,
const String & config_prefix)
: IDisk(name_, config, config_prefix)
, object_storage_root_path(object_storage_root_path_)
, object_key_prefix(object_key_prefix_)
, log (&Poco::Logger::get("DiskObjectStorage(" + log_name + ")"))
, metadata_storage(std::move(metadata_storage_))
, object_storage(std::move(object_storage_))
Expand All @@ -80,7 +80,8 @@ void DiskObjectStorage::getRemotePathsRecursive(const String & local_path, std::
{
try
{
paths_map.emplace_back(local_path, metadata_storage->getObjectStorageRootPath(), getStorageObjects(local_path));
/// fix LocalPathWithObjectStoragePaths
paths_map.emplace_back(local_path, getStorageObjects(local_path));
}
catch (const Exception & e)
{
Expand Down Expand Up @@ -243,9 +244,9 @@ String DiskObjectStorage::getUniqueId(const String & path) const

bool DiskObjectStorage::checkUniqueId(const String & id) const
{
if (!id.starts_with(object_storage_root_path))
if (!id.starts_with(object_key_prefix))
{
LOG_DEBUG(log, "Blob with id {} doesn't start with blob storage prefix {}, Stack {}", id, object_storage_root_path, StackTrace().toString());
LOG_DEBUG(log, "Blob with id {} doesn't start with blob storage prefix {}, Stack {}", id, object_key_prefix, StackTrace().toString());
return false;
}

Expand Down Expand Up @@ -470,7 +471,7 @@ DiskObjectStoragePtr DiskObjectStorage::createDiskObjectStorage()
const auto config_prefix = "storage_configuration.disks." + name;
return std::make_shared<DiskObjectStorage>(
getName(),
object_storage_root_path,
object_key_prefix,
getName(),
metadata_storage,
object_storage,
Expand Down Expand Up @@ -586,7 +587,7 @@ void DiskObjectStorage::restoreMetadataIfNeeded(
{
metadata_helper->restore(config, config_prefix, context);

auto current_schema_version = metadata_helper->readSchemaVersion(object_storage.get(), object_storage_root_path);
auto current_schema_version = metadata_helper->readSchemaVersion(object_storage.get(), object_key_prefix);
if (current_schema_version < DiskObjectStorageRemoteMetadataRestoreHelper::RESTORABLE_SCHEMA_VERSION)
metadata_helper->migrateToRestorableSchema();

Expand Down
4 changes: 2 additions & 2 deletions src/Disks/ObjectStorages/DiskObjectStorage.h
Expand Up @@ -37,7 +37,7 @@ friend class DiskObjectStorageRemoteMetadataRestoreHelper;
public:
DiskObjectStorage(
const String & name_,
const String & object_storage_root_path_,
const String & object_key_prefix_,
const String & log_name,
MetadataStoragePtr metadata_storage_,
ObjectStoragePtr object_storage_,
Expand Down Expand Up @@ -224,7 +224,7 @@ friend class DiskObjectStorageRemoteMetadataRestoreHelper;
String getReadResourceName() const;
String getWriteResourceName() const;

const String object_storage_root_path;
const String object_key_prefix;
Poco::Logger * log;

MetadataStoragePtr metadata_storage;
Expand Down

0 comments on commit 6b93fb1

Please sign in to comment.