Dictionaries update file path restrictions

This commit is contained in:
Maksim Kita 2021-05-25 00:27:24 +03:00
parent 91f910a612
commit b26783b308
33 changed files with 165 additions and 122 deletions

View File

@ -20,6 +20,7 @@ namespace ErrorCodes
extern const int SYSTEM_ERROR;
extern const int NOT_IMPLEMENTED;
extern const int CANNOT_STATVFS;
extern const int PATH_ACCESS_DENIED;
}
@ -108,4 +109,23 @@ String getFilesystemName([[maybe_unused]] const String & mount_point)
#endif
}
bool pathStartsWith(const std::filesystem::path & path, const std::filesystem::path & prefix_path)
{
auto absolute_path = std::filesystem::weakly_canonical(path);
auto absolute_prefix_path = std::filesystem::weakly_canonical(prefix_path);
auto [_, prefix_path_mismatch_it] = std::mismatch(absolute_path.begin(), absolute_path.end(), absolute_prefix_path.begin(), absolute_prefix_path.end());
bool path_starts_with_prefix_path = (prefix_path_mismatch_it == absolute_prefix_path.end());
return path_starts_with_prefix_path;
}
bool pathStartsWith(const String & path, const String & prefix_path)
{
auto filesystem_path = std::filesystem::path(path);
auto filesystem_prefix_path = std::filesystem::path(prefix_path);
return pathStartsWith(filesystem_path, filesystem_prefix_path);
}
}

View File

@ -29,4 +29,10 @@ String getFilesystemName([[maybe_unused]] const String & mount_point);
struct statvfs getStatVFS(const String & path);
/// Returns true if path starts with prefix path
bool pathStartsWith(const std::filesystem::path & path, const std::filesystem::path & prefix_path);
/// Returns true if path starts with prefix path
bool pathStartsWith(const String & path, const String & prefix_path);
}

View File

@ -19,7 +19,7 @@ void registerDictionarySourceCassandra(DictionarySourceFactory & factory)
[[maybe_unused]] Block & sample_block,
ContextPtr /* context */,
const std::string & /* default_database */,
bool /*check_config*/) -> DictionarySourcePtr
bool /*created_from_ddl*/) -> DictionarySourcePtr
{
#if USE_CASSANDRA
setupCassandraDriverLibraryLogging(CASS_LOG_INFO);

View File

@ -211,7 +211,7 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory)
Block & sample_block,
ContextPtr context,
const std::string & default_database [[maybe_unused]],
bool /* check_config */) -> DictionarySourcePtr
bool /* created_from_ddl */) -> DictionarySourcePtr
{
bool secure = config.getBool(config_prefix + ".secure", false);
auto context_copy = Context::createCopy(context);

View File

@ -17,22 +17,22 @@ namespace ErrorCodes
extern const int UNKNOWN_ELEMENT_IN_CONFIG;
}
void DictionaryFactory::registerLayout(const std::string & layout_type, Creator create_layout, bool is_complex)
void DictionaryFactory::registerLayout(const std::string & layout_type, LayoutCreateFunction create_layout, bool is_layout_complex)
{
if (!registered_layouts.emplace(layout_type, std::move(create_layout)).second)
auto it = registered_layouts.find(layout_type);
if (it != registered_layouts.end())
throw Exception(ErrorCodes::LOGICAL_ERROR, "DictionaryFactory: the layout name '{}' is not unique", layout_type);
layout_complexity[layout_type] = is_complex;
RegisteredLayout layout { .layout_create_function = create_layout, .is_layout_complex = is_layout_complex };
registered_layouts.emplace(layout_type, std::move(layout));
}
DictionaryPtr DictionaryFactory::create(
const std::string & name,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
ContextPtr context,
bool check_source_config) const
bool created_from_ddl) const
{
Poco::Util::AbstractConfiguration::Keys keys;
const auto & layout_prefix = config_prefix + ".layout";
@ -45,7 +45,7 @@ DictionaryPtr DictionaryFactory::create(
const DictionaryStructure dict_struct{config, config_prefix};
DictionarySourcePtr source_ptr = DictionarySourceFactory::instance().create(
name, config, config_prefix + ".source", dict_struct, context, config.getString(config_prefix + ".database", ""), check_source_config);
name, config, config_prefix + ".source", dict_struct, context, config.getString(config_prefix + ".database", ""), created_from_ddl);
LOG_TRACE(&Poco::Logger::get("DictionaryFactory"), "Created dictionary source '{}' for dictionary '{}'", source_ptr->toString(), name);
if (context->hasQueryContext() && context->getSettingsRef().log_queries)
@ -57,8 +57,8 @@ DictionaryPtr DictionaryFactory::create(
const auto found = registered_layouts.find(layout_type);
if (found != registered_layouts.end())
{
const auto & layout_creator = found->second;
return layout_creator(name, dict_struct, config, config_prefix, std::move(source_ptr));
const auto & layout_creator = found->second.layout_create_function;
return layout_creator(name, dict_struct, config, config_prefix, std::move(source_ptr), context, created_from_ddl);
}
}
@ -76,14 +76,16 @@ DictionaryPtr DictionaryFactory::create(const std::string & name, const ASTCreat
bool DictionaryFactory::isComplex(const std::string & layout_type) const
{
auto found = layout_complexity.find(layout_type);
auto it = registered_layouts.find(layout_type);
if (found != layout_complexity.end())
return found->second;
if (it == registered_layouts.end())
{
throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG,
"Unknown dictionary layout type: {}",
layout_type);
}
throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG,
"Unknown dictionary layout type: {}",
layout_type);
return it->second.is_layout_complex;
}

View File

@ -37,29 +37,36 @@ public:
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
ContextPtr context,
bool check_source_config = false) const;
bool created_from_ddl) const;
/// Create dictionary from DDL-query
DictionaryPtr create(const std::string & name,
const ASTCreateQuery & ast,
ContextPtr context) const;
using Creator = std::function<DictionaryPtr(
using LayoutCreateFunction = std::function<DictionaryPtr(
const std::string & name,
const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
DictionarySourcePtr source_ptr)>;
DictionarySourcePtr source_ptr,
ContextPtr context,
bool created_from_ddl)>;
bool isComplex(const std::string & layout_type) const;
void registerLayout(const std::string & layout_type, Creator create_layout, bool is_complex);
void registerLayout(const std::string & layout_type, LayoutCreateFunction create_layout, bool is_layout_complex);
private:
using LayoutRegistry = std::unordered_map<std::string, Creator>;
struct RegisteredLayout
{
LayoutCreateFunction layout_create_function;
bool is_layout_complex;
};
using LayoutRegistry = std::unordered_map<std::string, RegisteredLayout>;
LayoutRegistry registered_layouts;
using LayoutComplexity = std::unordered_map<std::string, bool>;
LayoutComplexity layout_complexity;
};
}

View File

@ -305,7 +305,9 @@ namespace
const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
DictionarySourcePtr source_ptr)
DictionarySourcePtr source_ptr,
ContextPtr /* context */,
bool /* created_from_ddl */)
{
const auto * layout_name = dictionary_key_type == DictionaryKeyType::simple ? "direct" : "complex_key_direct";

View File

@ -20,6 +20,7 @@
namespace DB
{
static const UInt64 max_block_size = 8192;
namespace ErrorCodes
@ -266,7 +267,7 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory)
Block & sample_block,
ContextPtr context,
const std::string & /* default_database */,
bool check_config) -> DictionarySourcePtr
bool created_from_ddl) -> DictionarySourcePtr
{
if (dict_struct.has_expressions)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Dictionary source of type `executable` does not support attribute expressions");
@ -274,7 +275,7 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory)
/// Executable dictionaries may execute arbitrary commands.
/// It's OK for dictionaries created by administrator from xml-file, but
/// maybe dangerous for dictionaries created from DDL-queries.
if (check_config)
if (created_from_ddl)
throw Exception(ErrorCodes::DICTIONARY_ACCESS_DENIED, "Dictionaries with executable dictionary source are not allowed to be created from DDL query");
auto context_local_copy = copyContextAndApplySettings(config_prefix, context, config);

View File

@ -275,7 +275,7 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory)
Block & sample_block,
ContextPtr context,
const std::string & /* default_database */,
bool check_config) -> DictionarySourcePtr
bool created_from_ddl) -> DictionarySourcePtr
{
if (dict_struct.has_expressions)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Dictionary source of type `executable_pool` does not support attribute expressions");
@ -283,7 +283,7 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory)
/// Executable dictionaries may execute arbitrary commands.
/// It's OK for dictionaries created by administrator from xml-file, but
/// maybe dangerous for dictionaries created from DDL-queries.
if (check_config)
if (created_from_ddl)
throw Exception(ErrorCodes::DICTIONARY_ACCESS_DENIED, "Dictionaries with executable pool dictionary source are not allowed to be created from DDL query");
auto context_local_copy = copyContextAndApplySettings(config_prefix, context, config);

View File

@ -1,13 +1,15 @@
#include "FileDictionarySource.h"
#include <Poco/File.h>
#include <filesystem>
#include <Poco/File.h>
#include <common/logger_useful.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/filesystemHelpers.h>
#include <DataStreams/OwningBlockInputStream.h>
#include <IO/ReadBufferFromFile.h>
#include <Interpreters/Context.h>
#include <Common/StringUtils/StringUtils.h>
#include <common/logger_useful.h>
#include "DictionarySourceFactory.h"
#include "DictionaryStructure.h"
#include "registerDictionaries.h"
@ -26,28 +28,14 @@ namespace ErrorCodes
FileDictionarySource::FileDictionarySource(
const std::string & filepath_, const std::string & format_,
Block & sample_block_, ContextPtr context_, bool check_config)
Block & sample_block_, ContextPtr context_, bool created_from_ddl)
: filepath{filepath_}
, format{format_}
, sample_block{sample_block_}
, context(context_)
{
if (check_config)
{
auto source_file_path = std::filesystem::path(filepath);
auto source_file_absolute_path = std::filesystem::canonical(source_file_path);
String user_files_path_string_value = context->getUserFilesPath();
auto user_files_path = std::filesystem::path(user_files_path_string_value);
auto user_files_absolute_path = std::filesystem::canonical(user_files_path);
auto [_, user_files_absolute_path_mismatch_it] = std::mismatch(source_file_absolute_path.begin(), source_file_absolute_path.end(), user_files_absolute_path.begin(), user_files_absolute_path.end());
bool user_files_absolute_path_include_source_file_absolute_path = user_files_absolute_path_mismatch_it == user_files_absolute_path.end();
if (!user_files_absolute_path_include_source_file_absolute_path)
throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "File path {} is not inside {}", filepath, user_files_path_string_value);
}
if (created_from_ddl && !pathStartsWith(filepath, context->getUserFilesPath()))
throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "File path {} is not inside {}", filepath, context->getUserFilesPath());
}
@ -91,7 +79,7 @@ void registerDictionarySourceFile(DictionarySourceFactory & factory)
Block & sample_block,
ContextPtr context,
const std::string & /* default_database */,
bool check_config) -> DictionarySourcePtr
bool created_from_ddl) -> DictionarySourcePtr
{
if (dict_struct.has_expressions)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Dictionary source of type `file` does not support attribute expressions");
@ -101,7 +89,7 @@ void registerDictionarySourceFile(DictionarySourceFactory & factory)
auto context_local_copy = copyContextAndApplySettings(config_prefix, context, config);
return std::make_unique<FileDictionarySource>(filepath, format, sample_block, context_local_copy, check_config);
return std::make_unique<FileDictionarySource>(filepath, format, sample_block, context_local_copy, created_from_ddl);
};
factory.registerSource("file", create_table_source);

View File

@ -17,7 +17,7 @@ class FileDictionarySource final : public IDictionarySource
{
public:
FileDictionarySource(const std::string & filepath_, const std::string & format_,
Block & sample_block_, ContextPtr context_, bool check_config);
Block & sample_block_, ContextPtr context_, bool created_from_ddl);
FileDictionarySource(const FileDictionarySource & other);

View File

@ -502,10 +502,12 @@ BlockInputStreamPtr FlatDictionary::getBlockInputStream(const Names & column_nam
void registerDictionaryFlat(DictionaryFactory & factory)
{
auto create_layout = [=](const std::string & full_name,
const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
DictionarySourcePtr source_ptr) -> DictionaryPtr
const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
DictionarySourcePtr source_ptr,
ContextPtr /* context */,
bool /* created_from_ddl */) -> DictionaryPtr
{
if (dict_struct.key)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'key' is not supported for dictionary of layout 'flat'");

View File

@ -22,6 +22,7 @@ namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
static const UInt64 max_block_size = 8192;
@ -31,7 +32,7 @@ HTTPDictionarySource::HTTPDictionarySource(
const std::string & config_prefix,
Block & sample_block_,
ContextPtr context_,
bool check_config)
bool created_from_ddl)
: log(&Poco::Logger::get("HTTPDictionarySource"))
, update_time{std::chrono::system_clock::from_time_t(0)}
, dict_struct{dict_struct_}
@ -42,7 +43,7 @@ HTTPDictionarySource::HTTPDictionarySource(
, context(context_)
, timeouts(ConnectionTimeouts::getHTTPTimeouts(context))
{
if (check_config)
if (created_from_ddl)
context->getRemoteHostFilter().checkURL(Poco::URI(url));
const auto & credentials_prefix = config_prefix + ".credentials";
@ -232,14 +233,14 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory)
Block & sample_block,
ContextPtr context,
const std::string & /* default_database */,
bool check_config) -> DictionarySourcePtr {
bool created_from_ddl) -> DictionarySourcePtr {
if (dict_struct.has_expressions)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Dictionary source of type `http` does not support attribute expressions");
auto context_local_copy = copyContextAndApplySettings(config_prefix, context, config);
return std::make_unique<HTTPDictionarySource>(
dict_struct, config, config_prefix + ".http", sample_block, context_local_copy, check_config);
dict_struct, config, config_prefix + ".http", sample_block, context_local_copy, created_from_ddl);
};
factory.registerSource("http", create_table_source);
}

View File

@ -28,7 +28,7 @@ public:
const std::string & config_prefix,
Block & sample_block_,
ContextPtr context_,
bool check_config);
bool created_from_ddl);
HTTPDictionarySource(const HTTPDictionarySource & other);
HTTPDictionarySource & operator=(const HTTPDictionarySource &) = delete;

View File

@ -751,13 +751,13 @@ void registerDictionaryHashed(DictionaryFactory & factory)
using namespace std::placeholders;
factory.registerLayout("hashed",
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::simple, /* sparse = */ false); }, false);
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::simple, /* sparse = */ false); }, false);
factory.registerLayout("sparse_hashed",
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::simple, /* sparse = */ true); }, false);
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::simple, /* sparse = */ true); }, false);
factory.registerLayout("complex_key_hashed",
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::complex, /* sparse = */ false); }, true);
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::complex, /* sparse = */ false); }, true);
factory.registerLayout("complex_key_sparse_hashed",
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::complex, /* sparse = */ true); }, true);
[=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::complex, /* sparse = */ true); }, true);
}

View File

@ -934,7 +934,9 @@ void registerDictionaryTrie(DictionaryFactory & factory)
const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
DictionarySourcePtr source_ptr) -> DictionaryPtr
DictionarySourcePtr source_ptr,
ContextPtr /* context */,
bool /*created_from_ddl*/) -> DictionaryPtr
{
if (!dict_struct.key || dict_struct.key->size() != 1)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Dictionary of layout 'ip_trie' has to have one 'key'");

View File

@ -1,18 +1,18 @@
#include "LibraryDictionarySource.h"
#include <DataStreams/OneBlockInputStream.h>
#include <Interpreters/Context.h>
#include <Poco/File.h>
#include <common/logger_useful.h>
#include <ext/bit_cast.h>
#include <ext/range.h>
#include <ext/scope_guard.h>
#include "DictionarySourceFactory.h"
#include "DictionarySourceHelpers.h"
#include "DictionaryStructure.h"
#include "registerDictionaries.h"
#include <Common/filesystemHelpers.h>
#include <IO/WriteBufferFromString.h>
#include <IO/WriteHelpers.h>
#include <DataStreams/OneBlockInputStream.h>
#include <Interpreters/Context.h>
#include <Dictionaries/DictionarySourceFactory.h>
#include <Dictionaries/DictionarySourceHelpers.h>
#include <Dictionaries/DictionaryStructure.h>
#include <Dictionaries/registerDictionaries.h>
namespace DB
@ -32,7 +32,7 @@ LibraryDictionarySource::LibraryDictionarySource(
const std::string & config_prefix_,
Block & sample_block_,
ContextPtr context_,
bool check_config)
bool created_from_ddl)
: log(&Poco::Logger::get("LibraryDictionarySource"))
, dict_struct{dict_struct_}
, config_prefix{config_prefix_}
@ -41,13 +41,8 @@ LibraryDictionarySource::LibraryDictionarySource(
, sample_block{sample_block_}
, context(Context::createCopy(context_))
{
if (check_config)
{
const String dictionaries_lib_path = context->getDictionariesLibPath();
if (!startsWith(path, dictionaries_lib_path))
throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "LibraryDictionarySource: Library path {} is not inside {}", path, dictionaries_lib_path);
}
if (created_from_ddl && !pathStartsWith(path, context->getUserFilesPath()))
throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "File path {} is not inside {}", path, context->getUserFilesPath());
if (!Poco::File(path).exists())
throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "LibraryDictionarySource: Can't load library {}: file doesn't exist", Poco::File(path).path());
@ -179,9 +174,9 @@ void registerDictionarySourceLibrary(DictionarySourceFactory & factory)
Block & sample_block,
ContextPtr context,
const std::string & /* default_database */,
bool check_config) -> DictionarySourcePtr
bool created_from_ddl) -> DictionarySourcePtr
{
return std::make_unique<LibraryDictionarySource>(dict_struct, config, config_prefix + ".library", sample_block, context, check_config);
return std::make_unique<LibraryDictionarySource>(dict_struct, config, config_prefix + ".library", sample_block, context, created_from_ddl);
};
factory.registerSource("library", create_table_source);

View File

@ -40,7 +40,7 @@ public:
const std::string & config_prefix_,
Block & sample_block_,
ContextPtr context_,
bool check_config);
bool created_from_ddl);
LibraryDictionarySource(const LibraryDictionarySource & other);
LibraryDictionarySource & operator=(const LibraryDictionarySource &) = delete;

View File

@ -15,7 +15,7 @@ void registerDictionarySourceMongoDB(DictionarySourceFactory & factory)
Block & sample_block,
ContextPtr,
const std::string & /* default_database */,
bool /* check_config */)
bool /* created_from_ddl */)
{
const auto config_prefix = root_config_prefix + ".mongodb";
return std::make_unique<MongoDBDictionarySource>(dict_struct,

View File

@ -26,7 +26,7 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory)
[[maybe_unused]] Block & sample_block,
[[maybe_unused]] ContextPtr context,
const std::string & /* default_database */,
bool /* check_config */) -> DictionarySourcePtr {
bool /* created_from_ddl */) -> DictionarySourcePtr {
#if USE_MYSQL
StreamSettings mysql_input_stream_settings(context->getSettingsRef()
, config.getBool(config_prefix + ".mysql.close_connection", false) || config.getBool(config_prefix + ".mysql.share_connection", false)

View File

@ -166,7 +166,9 @@ DictionaryPtr createLayout(const std::string & ,
const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
DictionarySourcePtr source_ptr)
DictionarySourcePtr source_ptr,
ContextPtr /* context */,
bool /*created_from_ddl*/)
{
const String database = config.getString(config_prefix + ".database", "");
const String name = config.getString(config_prefix + ".name");

View File

@ -190,7 +190,7 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory)
Block & sample_block,
ContextPtr context,
const std::string & /* default_database */,
bool /* check_config */) -> DictionarySourcePtr
bool /* created_from_ddl */) -> DictionarySourcePtr
{
#if USE_LIBPQXX
const auto config_prefix = root_config_prefix + ".postgresql";

View File

@ -636,7 +636,9 @@ void registerDictionaryRangeHashed(DictionaryFactory & factory)
const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
DictionarySourcePtr source_ptr) -> DictionaryPtr
DictionarySourcePtr source_ptr,
ContextPtr /* context */,
bool /*created_from_ddl*/) -> DictionaryPtr
{
if (dict_struct.key)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'key' is not supported for dictionary of layout 'range_hashed'");

View File

@ -14,7 +14,7 @@ void registerDictionarySourceRedis(DictionarySourceFactory & factory)
Block & sample_block,
ContextPtr /* context */,
const std::string & /* default_database */,
bool /* check_config */) -> DictionarySourcePtr {
bool /* created_from_ddl */) -> DictionarySourcePtr {
return std::make_unique<RedisDictionarySource>(dict_struct, config, config_prefix + ".redis", sample_block);
};
factory.registerSource("redis", create_table_source);

View File

@ -53,7 +53,6 @@ struct SSDCacheDictionaryStorageConfiguration
{
const size_t strict_max_lifetime_seconds;
const DictionaryLifetime lifetime;
const std::string file_path;
const size_t max_partitions_count;
const size_t block_size;

View File

@ -307,7 +307,7 @@ void registerDictionarySourceJDBC(DictionarySourceFactory & factory)
Block & /* sample_block */,
ContextPtr /* context */,
const std::string & /* default_database */,
bool /* check_config */) -> DictionarySourcePtr {
bool /* created_from_ddl */) -> DictionarySourcePtr {
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED,
"Dictionary source of type `jdbc` is disabled until consistent support for nullable fields.");
// BridgeHelperPtr bridge = std::make_shared<XDBCBridgeHelper<JDBCBridgeMixin>>(config, context.getSettings().http_receive_timeout, config.getString(config_prefix + ".connection_string"));

View File

@ -1,7 +1,9 @@
#include "CacheDictionary.h"
#include "CacheDictionaryStorage.h"
#include "SSDCacheDictionaryStorage.h"
#include <Common/filesystemHelpers.h>
#include <Dictionaries/DictionaryFactory.h>
#include <Interpreters/Context.h>
namespace DB
{
@ -11,6 +13,7 @@ namespace ErrorCodes
extern const int TOO_SMALL_BUFFER_SIZE;
extern const int UNSUPPORTED_METHOD;
extern const int BAD_ARGUMENTS;
extern const int PATH_ACCESS_DENIED;
}
CacheDictionaryStorageConfiguration parseCacheStorageConfiguration(
@ -84,19 +87,16 @@ SSDCacheDictionaryStorageConfiguration parseSSDCacheStorageConfiguration(
"{}: write_buffer_size must be a multiple of block_size",
full_name);
auto directory_path = config.getString(dictionary_configuration_prefix + "path");
if (directory_path.empty())
auto file_path = config.getString(dictionary_configuration_prefix + "path");
if (file_path.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"{}: ssd cache dictionary cannot have empty path",
full_name);
if (directory_path.at(0) != '/')
directory_path = std::filesystem::path{config.getString("path")}.concat(directory_path).string();
SSDCacheDictionaryStorageConfiguration configuration{
strict_max_lifetime_seconds,
dict_lifetime,
directory_path,
file_path,
max_partitions_count,
block_size,
file_size / block_size,
@ -205,7 +205,9 @@ DictionaryPtr createSSDCacheDictionaryLayout(
const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
DictionarySourcePtr source_ptr)
DictionarySourcePtr source_ptr,
ContextPtr context,
bool created_from_ddl)
{
static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by CacheDictionary");
@ -241,10 +243,14 @@ DictionaryPtr createSSDCacheDictionaryLayout(
const bool allow_read_expired_keys = config.getBool(layout_prefix + ".cache.allow_read_expired_keys", false);
auto storage_configuration = parseSSDCacheStorageConfiguration(full_name, config, layout_prefix, dict_lifetime, dictionary_key_type);
if (created_from_ddl && !pathStartsWith(storage_configuration.file_path, context->getUserFilesPath()))
throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "File path {} is not inside {}", storage_configuration.file_path, context->getUserFilesPath());
auto storage = std::make_shared<SSDCacheDictionaryStorage<dictionary_key_type>>(storage_configuration);
auto update_queue_configuration
= parseCacheDictionaryUpdateQueueConfiguration(full_name, config, layout_prefix, dictionary_key_type);
auto update_queue_configuration = parseCacheDictionaryUpdateQueueConfiguration(full_name, config, layout_prefix, dictionary_key_type);
return std::make_unique<CacheDictionary<dictionary_key_type>>(
dict_id, dict_struct, std::move(source_ptr), storage, update_queue_configuration, dict_lifetime, allow_read_expired_keys);
@ -258,7 +264,9 @@ void registerDictionaryCache(DictionaryFactory & factory)
const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
DictionarySourcePtr source_ptr) -> DictionaryPtr
DictionarySourcePtr source_ptr,
ContextPtr /* context */,
bool /* created_from_ddl */) -> DictionaryPtr
{
return createCacheDictionaryLayout<DictionaryKeyType::simple>(full_name, dict_struct, config, config_prefix, std::move(source_ptr));
};
@ -269,7 +277,9 @@ void registerDictionaryCache(DictionaryFactory & factory)
const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
DictionarySourcePtr source_ptr) -> DictionaryPtr
DictionarySourcePtr source_ptr,
ContextPtr /* context */,
bool /* created_from_ddl */) -> DictionaryPtr
{
return createCacheDictionaryLayout<DictionaryKeyType::complex>(full_name, dict_struct, config, config_prefix, std::move(source_ptr));
};
@ -282,9 +292,11 @@ void registerDictionaryCache(DictionaryFactory & factory)
const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
DictionarySourcePtr source_ptr) -> DictionaryPtr
DictionarySourcePtr source_ptr,
ContextPtr context,
bool created_from_ddl) -> DictionaryPtr
{
return createSSDCacheDictionaryLayout<DictionaryKeyType::simple>(full_name, dict_struct, config, config_prefix, std::move(source_ptr));
return createSSDCacheDictionaryLayout<DictionaryKeyType::simple>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), context, created_from_ddl);
};
factory.registerLayout("ssd_cache", create_simple_ssd_cache_layout, false);
@ -293,8 +305,10 @@ void registerDictionaryCache(DictionaryFactory & factory)
const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
DictionarySourcePtr source_ptr) -> DictionaryPtr {
return createSSDCacheDictionaryLayout<DictionaryKeyType::complex>(full_name, dict_struct, config, config_prefix, std::move(source_ptr));
DictionarySourcePtr source_ptr,
ContextPtr context,
bool created_from_ddl) -> DictionaryPtr {
return createSSDCacheDictionaryLayout<DictionaryKeyType::complex>(full_name, dict_struct, config, config_prefix, std::move(source_ptr), context, created_from_ddl);
};
factory.registerLayout("complex_key_ssd_cache", create_complex_key_ssd_cache_layout, true);

View File

@ -38,8 +38,8 @@ ExternalLoader::LoadablePtr ExternalDictionariesLoader::create(
{
/// For dictionaries from databases (created with DDL queries) we have to perform
/// additional checks, so we identify them here.
bool dictionary_from_database = !repository_name.empty();
return DictionaryFactory::instance().create(name, config, key_in_config, getContext(), dictionary_from_database);
bool created_from_ddl = !repository_name.empty();
return DictionaryFactory::instance().create(name, config, key_in_config, getContext(), created_from_ddl);
}
ExternalDictionariesLoader::DictPtr ExternalDictionariesLoader::getDictionary(const std::string & dictionary_name, ContextPtr local_context) const

View File

@ -35,7 +35,7 @@ CREATE DICTIONARY 01053_db.ssd_dict
PRIMARY KEY id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' PASSWORD '' DB '01053_db'))
LIFETIME(MIN 1000 MAX 2000)
LAYOUT(SSD_CACHE(FILE_SIZE 8192 PATH '/var/lib/clickhouse/clickhouse_dicts/0d'));
LAYOUT(SSD_CACHE(FILE_SIZE 8192 PATH './user_files/0d'));
SELECT 'TEST_SMALL';
SELECT dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(1));
@ -76,7 +76,7 @@ CREATE DICTIONARY 01053_db.ssd_dict
PRIMARY KEY id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' PASSWORD '' DB '01053_db'))
LIFETIME(MIN 1000 MAX 2000)
LAYOUT(SSD_CACHE(FILE_SIZE 8192 PATH '/var/lib/clickhouse/clickhouse_dicts/1d' BLOCK_SIZE 512 WRITE_BUFFER_SIZE 4096));
LAYOUT(SSD_CACHE(FILE_SIZE 8192 PATH './user_files/1d' BLOCK_SIZE 512 WRITE_BUFFER_SIZE 4096));
SELECT 'UPDATE DICTIONARY';
-- 118
@ -142,7 +142,7 @@ CREATE DICTIONARY 01053_db.ssd_dict
PRIMARY KEY id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' PASSWORD '' DB '01053_db'))
LIFETIME(MIN 1000 MAX 2000)
LAYOUT(SSD_CACHE(FILE_SIZE 8192 PATH '/var/lib/clickhouse/clickhouse_dicts/2d' BLOCK_SIZE 512 WRITE_BUFFER_SIZE 1024));
LAYOUT(SSD_CACHE(FILE_SIZE 8192 PATH './user_files/2d' BLOCK_SIZE 512 WRITE_BUFFER_SIZE 1024));
SELECT 'UPDATE DICTIONARY (MT)';
-- 118

View File

@ -37,7 +37,7 @@ CREATE DICTIONARY 01280_db.ssd_dict
PRIMARY KEY k1, k2
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' PASSWORD '' DB '01280_db'))
LIFETIME(MIN 1000 MAX 2000)
LAYOUT(COMPLEX_KEY_SSD_CACHE(FILE_SIZE 8192 PATH '/var/lib/clickhouse/clickhouse_dicts/0d'));
LAYOUT(COMPLEX_KEY_SSD_CACHE(FILE_SIZE 8192 PATH './user_files/0d'));
SELECT 'TEST_SMALL';
SELECT 'VALUE FROM RAM BUFFER';
@ -98,7 +98,7 @@ CREATE DICTIONARY 01280_db.ssd_dict
PRIMARY KEY k1, k2
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' PASSWORD '' DB '01280_db'))
LIFETIME(MIN 1000 MAX 2000)
LAYOUT(COMPLEX_KEY_SSD_CACHE(FILE_SIZE 8192 PATH '/var/lib/clickhouse/clickhouse_dicts/1d' BLOCK_SIZE 512 WRITE_BUFFER_SIZE 4096));
LAYOUT(COMPLEX_KEY_SSD_CACHE(FILE_SIZE 8192 PATH './user_files/1d' BLOCK_SIZE 512 WRITE_BUFFER_SIZE 4096));
SELECT 'UPDATE DICTIONARY';
-- 118

View File

@ -22,7 +22,7 @@ CREATE DICTIONARY 01684_database_for_cache_dictionary.cache_dictionary_simple_ke
PRIMARY KEY id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'simple_key_simple_attributes_source_table'))
LIFETIME(MIN 1 MAX 1000)
LAYOUT(SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 8192 PATH '/var/lib/clickhouse/clickhouse_dicts/0d'));
LAYOUT(SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 8192 PATH './user_files/0d'));
SELECT 'Dictionary cache_dictionary_simple_key_simple_attributes';
SELECT 'dictGet existing value';
@ -66,7 +66,7 @@ CREATE DICTIONARY 01684_database_for_cache_dictionary.cache_dictionary_simple_ke
PRIMARY KEY id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'simple_key_complex_attributes_source_table'))
LIFETIME(MIN 1 MAX 1000)
LAYOUT(SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 8192 PATH '/var/lib/clickhouse/clickhouse_dicts/1d'));
LAYOUT(SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 8192 PATH './user_files/1d'));
SELECT 'Dictionary cache_dictionary_simple_key_complex_attributes';
SELECT 'dictGet existing value';
@ -108,7 +108,7 @@ CREATE DICTIONARY 01684_database_for_cache_dictionary.cache_dictionary_simple_ke
PRIMARY KEY id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'simple_key_hierarchy_table'))
LIFETIME(MIN 1 MAX 1000)
LAYOUT(SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 8192 PATH '/var/lib/clickhouse/clickhouse_dicts/2d'));
LAYOUT(SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 8192 PATH './user_files/2d'));
SELECT 'Dictionary cache_dictionary_simple_key_hierarchy';
SELECT 'dictGet';

View File

@ -24,7 +24,7 @@ CREATE DICTIONARY 01685_database_for_cache_dictionary.cache_dictionary_complex_k
PRIMARY KEY id, id_key
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'complex_key_simple_attributes_source_table' DB '01685_database_for_cache_dictionary'))
LIFETIME(MIN 1 MAX 1000)
LAYOUT(COMPLEX_KEY_SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 8192 PATH '/var/lib/clickhouse/clickhouse_dicts/0d'));
LAYOUT(COMPLEX_KEY_SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 8192 PATH './user_files/0d'));
SELECT 'Dictionary cache_dictionary_complex_key_simple_attributes';
SELECT 'dictGet existing value';
@ -71,7 +71,7 @@ CREATE DICTIONARY 01685_database_for_cache_dictionary.cache_dictionary_complex_k
PRIMARY KEY id, id_key
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'complex_key_complex_attributes_source_table' DB '01685_database_for_cache_dictionary'))
LIFETIME(MIN 1 MAX 1000)
LAYOUT(COMPLEX_KEY_SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 8192 PATH '/var/lib/clickhouse/clickhouse_dicts/1d'));
LAYOUT(COMPLEX_KEY_SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 8192 PATH './user_files/1d'));
SELECT 'Dictionary cache_dictionary_complex_key_complex_attributes';
SELECT 'dictGet existing value';

View File

@ -64,7 +64,7 @@ CREATE DICTIONARY ssd_cache_dictionary
PRIMARY KEY id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'dictionary_decimal_source_table'))
LIFETIME(MIN 1 MAX 1000)
LAYOUT(SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 8192 PATH '/var/lib/clickhouse/clickhouse_dicts/0d'));
LAYOUT(SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 8192 PATH './user_files/0d'));
SELECT 'SSDCache dictionary';
SELECT dictGet('ssd_cache_dictionary', 'decimal_value', toUInt64(1));