mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
Dictionaries update file path restrictions
This commit is contained in:
parent
91f910a612
commit
b26783b308
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
||||
|
@ -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;
|
||||
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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";
|
||||
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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'");
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
|
||||
}
|
||||
|
||||
|
@ -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'");
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
|
@ -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,
|
||||
|
@ -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)
|
||||
|
@ -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");
|
||||
|
@ -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";
|
||||
|
@ -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'");
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
|
@ -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"));
|
||||
|
@ -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);
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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';
|
||||
|
@ -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';
|
||||
|
@ -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));
|
||||
|
Loading…
Reference in New Issue
Block a user