Merge remote-tracking branch 'origin/master' into HEAD

This commit is contained in:
Alexander Kuzmenkov 2021-05-27 21:42:18 +03:00
commit 3d459633d7
54 changed files with 793 additions and 679 deletions

View File

@ -338,7 +338,7 @@ or
``` sql
LAYOUT(SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 16777216 READ_BUFFER_SIZE 1048576
PATH /var/lib/clickhouse/clickhouse_dictionaries/test_dict))
PATH ./user_files/test_dict))
```
### complex_key_ssd_cache {#complex-key-ssd-cache}

View File

@ -53,7 +53,7 @@ optional settings are available:
or
``` sql
SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated'))
SOURCE(FILE(path './user_files/os.tsv' format 'TabSeparated'))
SETTINGS(format_csv_allow_single_quotes = 0)
```
@ -70,7 +70,7 @@ Types of sources (`source_type`):
- [MongoDB](#dicts-external_dicts_dict_sources-mongodb)
- [Redis](#dicts-external_dicts_dict_sources-redis)
- [Cassandra](#dicts-external_dicts_dict_sources-cassandra)
- [PostgreSQL](#dicts-external_dicts_dict_sources-postgresql)
- [PostgreSQL](#dicts-external_dicts_dict_sources-postgresql)
## Local File {#dicts-external_dicts_dict_sources-local_file}
@ -88,7 +88,7 @@ Example of settings:
or
``` sql
SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated'))
SOURCE(FILE(path './user_files/os.tsv' format 'TabSeparated'))
```
Setting fields:

View File

@ -55,7 +55,7 @@ SOURCE(SOURCE_TYPE(param1 val1 ... paramN valN)) -- Source configuration
または
``` sql
SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated'))
SOURCE(FILE(path './user_files/os.tsv' format 'TabSeparated'))
SETTINGS(format_csv_allow_single_quotes = 0)
```
@ -87,7 +87,7 @@ SETTINGS(format_csv_allow_single_quotes = 0)
または
``` sql
SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated'))
SOURCE(FILE(path './user_files/os.tsv' format 'TabSeparated'))
```
フィールドの設定:

View File

@ -121,7 +121,7 @@ LAYOUT(HASHED(PREALLOCATE 0))
Аналогичен `hashed`, но при этом занимает меньше места в памяти и генерирует более высокую загрузку CPU.
Для этого типа размещения также можно задать `preallocate` в значении `true`. В данном случае это более важно, чем для типа `hashed`.
Для этого типа размещения также можно задать `preallocate` в значении `true`. В данном случае это более важно, чем для типа `hashed`.
Пример конфигурации:
@ -338,7 +338,7 @@ LAYOUT(CACHE(SIZE_IN_CELLS 1000000000))
``` sql
LAYOUT(SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 16777216 READ_BUFFER_SIZE 1048576
PATH /var/lib/clickhouse/clickhouse_dictionaries/test_dict))
PATH ./user_files/test_dict))
```
### complex_key_ssd_cache {#complex-key-ssd-cache}

View File

@ -53,7 +53,7 @@ SOURCE(SOURCE_TYPE(param1 val1 ... paramN valN)) -- Source configuration
или
``` sql
SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated'))
SOURCE(FILE(path './user_files/os.tsv' format 'TabSeparated'))
SETTINGS(format_csv_allow_single_quotes = 0)
```
@ -69,7 +69,7 @@ SETTINGS(format_csv_allow_single_quotes = 0)
- [ClickHouse](#dicts-external_dicts_dict_sources-clickhouse)
- [MongoDB](#dicts-external_dicts_dict_sources-mongodb)
- [Redis](#dicts-external_dicts_dict_sources-redis)
- [PostgreSQL](#dicts-external_dicts_dict_sources-postgresql)
- [PostgreSQL](#dicts-external_dicts_dict_sources-postgresql)
## Локальный файл {#dicts-external_dicts_dict_sources-local_file}
@ -87,7 +87,7 @@ SETTINGS(format_csv_allow_single_quotes = 0)
или
``` sql
SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated'))
SOURCE(FILE(path './user_files/os.tsv' format 'TabSeparated'))
```
Поля настройки:

View File

@ -55,7 +55,7 @@ SOURCE(SOURCE_TYPE(param1 val1 ... paramN valN)) -- Source configuration
``` sql
SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated'))
SOURCE(FILE(path './user_files/os.tsv' format 'TabSeparated'))
SETTINGS(format_csv_allow_single_quotes = 0)
```
@ -87,7 +87,7 @@ SETTINGS(format_csv_allow_single_quotes = 0)
``` sql
SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated'))
SOURCE(FILE(path './user_files/os.tsv' format 'TabSeparated'))
```
设置字段:

View File

@ -108,4 +108,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->getDictionariesLibPath()))
throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "File path {} is not inside {}", path, context->getDictionariesLibPath());
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

@ -2523,7 +2523,7 @@ private:
if (!from_type)
{
throw Exception(ErrorCodes::TYPE_MISMATCH,
"CAST AS Array can only be perforamed between same-dimensional Array or String types");
"CAST AS Array can only be performed between same-dimensional Array or String types");
}
DataTypePtr from_nested_type = from_type->getNestedType();
@ -2533,7 +2533,7 @@ private:
if (from_type->getNumberOfDimensions() != to_type.getNumberOfDimensions() && !from_empty_array)
throw Exception(ErrorCodes::TYPE_MISMATCH,
"CAST AS Array can only be perforamed between same-dimensional array types");
"CAST AS Array can only be performed between same-dimensional array types");
const DataTypePtr & to_nested_type = to_type.getNestedType();

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

@ -824,34 +824,48 @@ bool ParserCastOperator::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
/// Parse numbers (including decimals), strings and arrays of them.
const char * data_begin = pos->begin;
const char * data_end = pos->end;
bool is_string_literal = pos->type == TokenType::StringLiteral;
if (pos->type == TokenType::Number || is_string_literal)
{
++pos;
}
else if (pos->type == TokenType::OpeningSquareBracket)
else if (isOneOf<TokenType::OpeningSquareBracket, TokenType::OpeningRoundBracket>(pos->type))
{
TokenType last_token = TokenType::OpeningSquareBracket;
std::vector<TokenType> stack;
while (pos.isValid())
{
if (pos->type == TokenType::OpeningSquareBracket)
if (isOneOf<TokenType::OpeningSquareBracket, TokenType::OpeningRoundBracket>(pos->type))
{
if (!isOneOf<TokenType::OpeningSquareBracket, TokenType::Comma>(last_token))
stack.push_back(pos->type);
if (!isOneOf<TokenType::OpeningSquareBracket, TokenType::OpeningRoundBracket, TokenType::Comma>(last_token))
return false;
}
else if (pos->type == TokenType::ClosingSquareBracket)
{
if (last_token == TokenType::Comma)
if (isOneOf<TokenType::Comma, TokenType::OpeningRoundBracket>(last_token))
return false;
if (stack.empty() || stack.back() != TokenType::OpeningSquareBracket)
return false;
stack.pop_back();
}
else if (pos->type == TokenType::ClosingRoundBracket)
{
if (isOneOf<TokenType::Comma, TokenType::OpeningSquareBracket>(last_token))
return false;
if (stack.empty() || stack.back() != TokenType::OpeningRoundBracket)
return false;
stack.pop_back();
}
else if (pos->type == TokenType::Comma)
{
if (isOneOf<TokenType::OpeningSquareBracket, TokenType::Comma>(last_token))
if (isOneOf<TokenType::OpeningSquareBracket, TokenType::OpeningRoundBracket, TokenType::Comma>(last_token))
return false;
}
else if (isOneOf<TokenType::Number, TokenType::StringLiteral>(pos->type))
{
if (!isOneOf<TokenType::OpeningSquareBracket, TokenType::Comma>(last_token))
if (!isOneOf<TokenType::OpeningSquareBracket, TokenType::OpeningRoundBracket, TokenType::Comma>(last_token))
return false;
}
else
@ -859,14 +873,18 @@ bool ParserCastOperator::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
break;
}
/// Update data_end on every iteration to avoid appearances of extra trailing
/// whitespaces into data. Whitespaces are skipped at operator '++' of Pos.
data_end = pos->end;
last_token = pos->type;
++pos;
}
if (!stack.empty())
return false;
}
ASTPtr type_ast;
const char * data_end = pos->begin;
if (ParserToken(TokenType::DoubleColon).ignore(pos, expected)
&& ParserDataType().parse(pos, type_ast, expected))
{

View File

@ -8,7 +8,7 @@
<format>TabSeparated</format>
</file>
</source>
<lifetime>1</lifetime>
<lifetime>5</lifetime>
<layout><flat/></layout>
<structure><id><name>key</name> </id>
<attribute><name>a</name><type>Int32</type>

View File

@ -335,9 +335,9 @@ def get_random_string(length):
def get_used_disks_for_table(node, table_name):
return node.query(
return tuple(node.query(
"select disk_name from system.parts where table == '{}' and active=1 order by modification_time".format(
table_name)).strip().split('\n')
table_name)).strip().split('\n'))
def get_used_parts_for_table(node, table_name):
@ -450,7 +450,7 @@ def test_jbod_overflow(start_cluster, name, engine):
node1.query("INSERT INTO {} VALUES {}".format(name, ','.join(["('" + x + "')" for x in data])))
used_disks = get_used_disks_for_table(node1, name)
assert all(disk == 'jbod1' for disk in used_disks)
assert used_disks == tuple('jbod1' for _ in used_disks)
# should go to the external disk (jbod is overflown)
data = [] # 10MB in total
@ -469,11 +469,11 @@ def test_jbod_overflow(start_cluster, name, engine):
node1.query("OPTIMIZE TABLE {} FINAL".format(name))
time.sleep(2)
disks_for_merges = node1.query(
disks_for_merges = tuple(node1.query(
"SELECT disk_name FROM system.parts WHERE table == '{}' AND level >= 1 and active = 1 ORDER BY modification_time".format(
name)).strip().split('\n')
name)).strip().split('\n'))
assert all(disk == 'external' for disk in disks_for_merges)
assert disks_for_merges == tuple('external' for _ in disks_for_merges)
finally:
node1.query(f"DROP TABLE IF EXISTS {name} SYNC")

View File

@ -0,0 +1,155 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}')
$CLICKHOUSE_CLIENT -n --query="
DROP DATABASE IF EXISTS 01053_db;
CREATE DATABASE 01053_db Engine = Ordinary;
DROP TABLE IF EXISTS 01053_db.table_for_dict;
CREATE TABLE 01053_db.table_for_dict
(
id UInt64,
a UInt64,
b Int32,
c String
)
ENGINE = MergeTree()
ORDER BY id;
INSERT INTO 01053_db.table_for_dict VALUES (1, 100, -100, 'clickhouse'), (2, 3, 4, 'database'), (5, 6, 7, 'columns'), (10, 9, 8, '');
INSERT INTO 01053_db.table_for_dict SELECT number, 0, -1, 'a' FROM system.numbers WHERE number NOT IN (1, 2, 5, 10) LIMIT 370;
INSERT INTO 01053_db.table_for_dict SELECT number, 0, -1, 'b' FROM system.numbers WHERE number NOT IN (1, 2, 5, 10) LIMIT 370, 370;
INSERT INTO 01053_db.table_for_dict SELECT number, 0, -1, 'c' FROM system.numbers WHERE number NOT IN (1, 2, 5, 10) LIMIT 700, 370;
DROP DICTIONARY IF EXISTS 01053_db.ssd_dict;
-- Probably we need rewrite it to integration test
CREATE DICTIONARY 01053_db.ssd_dict
(
id UInt64,
a UInt64 DEFAULT 0,
b Int32 DEFAULT -1,
c String DEFAULT 'none'
)
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 '$USER_FILES_PATH/0d'));
SELECT 'TEST_SMALL';
SELECT dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(1));
SELECT dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(4));
SELECT dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(5));
SELECT dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(6));
SELECT dictGetString('01053_db.ssd_dict', 'c', toUInt64(2));
SELECT dictGetString('01053_db.ssd_dict', 'c', toUInt64(3));
SELECT * FROM 01053_db.ssd_dict ORDER BY id;
DROP DICTIONARY 01053_db.ssd_dict;
DROP TABLE IF EXISTS 01053_db.keys_table;
CREATE TABLE 01053_db.keys_table
(
id UInt64
)
ENGINE = StripeLog();
INSERT INTO 01053_db.keys_table VALUES (1);
INSERT INTO 01053_db.keys_table SELECT 11 + intHash64(number) % 1200 FROM system.numbers LIMIT 370;
INSERT INTO 01053_db.keys_table VALUES (2);
INSERT INTO 01053_db.keys_table SELECT 11 + intHash64(number) % 1200 FROM system.numbers LIMIT 370, 370;
INSERT INTO 01053_db.keys_table VALUES (5);
INSERT INTO 01053_db.keys_table SELECT 11 + intHash64(number) % 1200 FROM system.numbers LIMIT 700, 370;
INSERT INTO 01053_db.keys_table VALUES (10);
DROP DICTIONARY IF EXISTS 01053_db.ssd_dict;
CREATE DICTIONARY 01053_db.ssd_dict
(
id UInt64,
a UInt64 DEFAULT 0,
b Int32 DEFAULT -1,
c String DEFAULT 'none'
)
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 '$USER_FILES_PATH/1d' BLOCK_SIZE 512 WRITE_BUFFER_SIZE 4096));
SELECT 'UPDATE DICTIONARY';
SELECT sum(dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(id))) FROM 01053_db.keys_table;
SELECT 'VALUE FROM DISK';
SELECT dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(1));
SELECT dictGetString('01053_db.ssd_dict', 'c', toUInt64(1));
SELECT 'VALUE FROM RAM BUFFER';
SELECT dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(10));
SELECT dictGetString('01053_db.ssd_dict', 'c', toUInt64(10));
SELECT 'VALUES FROM DISK AND RAM BUFFER';
SELECT sum(dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(id))) FROM 01053_db.keys_table;
SELECT 'HAS';
SELECT count() FROM 01053_db.keys_table WHERE dictHas('01053_db.ssd_dict', toUInt64(id));
SELECT 'VALUES NOT FROM TABLE';
SELECT dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(1000000)), dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(1000000)), dictGetString('01053_db.ssd_dict', 'c', toUInt64(1000000));
SELECT dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(1000000)), dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(1000000)), dictGetString('01053_db.ssd_dict', 'c', toUInt64(1000000));
SELECT 'DUPLICATE KEYS';
SELECT arrayJoin([1, 2, 3, 3, 2, 1]) AS id, dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(id));
--SELECT
DROP DICTIONARY IF EXISTS 01053_db.ssd_dict;
DROP TABLE IF EXISTS 01053_db.keys_table;
CREATE TABLE 01053_db.keys_table
(
id UInt64
)
ENGINE = MergeTree()
ORDER BY id;
INSERT INTO 01053_db.keys_table VALUES (1);
INSERT INTO 01053_db.keys_table SELECT intHash64(number) FROM system.numbers LIMIT 370;
INSERT INTO 01053_db.keys_table VALUES (2);
INSERT INTO 01053_db.keys_table SELECT intHash64(number) FROM system.numbers LIMIT 370, 370;
INSERT INTO 01053_db.keys_table VALUES (5);
INSERT INTO 01053_db.keys_table SELECT intHash64(number) FROM system.numbers LIMIT 700, 370;
INSERT INTO 01053_db.keys_table VALUES (10);
OPTIMIZE TABLE 01053_db.keys_table;
CREATE DICTIONARY 01053_db.ssd_dict
(
id UInt64,
a UInt64 DEFAULT 0,
b Int32 DEFAULT -1
)
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 '$USER_FILES_PATH/2d' BLOCK_SIZE 512 WRITE_BUFFER_SIZE 1024));
SELECT 'UPDATE DICTIONARY (MT)';
SELECT sum(dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(id))) FROM 01053_db.keys_table;
SELECT 'VALUES FROM DISK AND RAM BUFFER (MT)';
SELECT sum(dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(id))) FROM 01053_db.keys_table;
DROP DICTIONARY IF EXISTS 01053_db.ssd_dict;
DROP TABLE IF EXISTS 01053_db.table_for_dict;
DROP DATABASE IF EXISTS 01053_db;"

View File

@ -1,159 +0,0 @@
SET send_logs_level = 'none';
DROP DATABASE IF EXISTS 01053_db;
CREATE DATABASE 01053_db Engine = Ordinary;
DROP TABLE IF EXISTS 01053_db.table_for_dict;
CREATE TABLE 01053_db.table_for_dict
(
id UInt64,
a UInt64,
b Int32,
c String
)
ENGINE = MergeTree()
ORDER BY id;
INSERT INTO 01053_db.table_for_dict VALUES (1, 100, -100, 'clickhouse'), (2, 3, 4, 'database'), (5, 6, 7, 'columns'), (10, 9, 8, '');
INSERT INTO 01053_db.table_for_dict SELECT number, 0, -1, 'a' FROM system.numbers WHERE number NOT IN (1, 2, 5, 10) LIMIT 370;
INSERT INTO 01053_db.table_for_dict SELECT number, 0, -1, 'b' FROM system.numbers WHERE number NOT IN (1, 2, 5, 10) LIMIT 370, 370;
INSERT INTO 01053_db.table_for_dict SELECT number, 0, -1, 'c' FROM system.numbers WHERE number NOT IN (1, 2, 5, 10) LIMIT 700, 370;
DROP DICTIONARY IF EXISTS 01053_db.ssd_dict;
-- FIXME filesystem error: in create_directory: Permission denied [/var/lib/clickhouse]
-- Probably we need rewrite it to integration test
CREATE DICTIONARY 01053_db.ssd_dict
(
id UInt64,
a UInt64 DEFAULT 0,
b Int32 DEFAULT -1,
c String DEFAULT 'none'
)
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'));
SELECT 'TEST_SMALL';
SELECT dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(1));
SELECT dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(4));
SELECT dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(5));
SELECT dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(6));
SELECT dictGetString('01053_db.ssd_dict', 'c', toUInt64(2));
SELECT dictGetString('01053_db.ssd_dict', 'c', toUInt64(3));
SELECT * FROM 01053_db.ssd_dict ORDER BY id;
DROP DICTIONARY 01053_db.ssd_dict;
DROP TABLE IF EXISTS 01053_db.keys_table;
CREATE TABLE 01053_db.keys_table
(
id UInt64
)
ENGINE = StripeLog();
INSERT INTO 01053_db.keys_table VALUES (1);
INSERT INTO 01053_db.keys_table SELECT 11 + intHash64(number) % 1200 FROM system.numbers LIMIT 370;
INSERT INTO 01053_db.keys_table VALUES (2);
INSERT INTO 01053_db.keys_table SELECT 11 + intHash64(number) % 1200 FROM system.numbers LIMIT 370, 370;
INSERT INTO 01053_db.keys_table VALUES (5);
INSERT INTO 01053_db.keys_table SELECT 11 + intHash64(number) % 1200 FROM system.numbers LIMIT 700, 370;
INSERT INTO 01053_db.keys_table VALUES (10);
DROP DICTIONARY IF EXISTS 01053_db.ssd_dict;
CREATE DICTIONARY 01053_db.ssd_dict
(
id UInt64,
a UInt64 DEFAULT 0,
b Int32 DEFAULT -1,
c String DEFAULT 'none'
)
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));
SELECT 'UPDATE DICTIONARY';
-- 118
SELECT sum(dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(id))) FROM 01053_db.keys_table;
SELECT 'VALUE FROM DISK';
-- -100
SELECT dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(1));
-- 'clickhouse'
SELECT dictGetString('01053_db.ssd_dict', 'c', toUInt64(1));
SELECT 'VALUE FROM RAM BUFFER';
-- 8
SELECT dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(10));
-- ''
SELECT dictGetString('01053_db.ssd_dict', 'c', toUInt64(10));
SELECT 'VALUES FROM DISK AND RAM BUFFER';
-- 118
SELECT sum(dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(id))) FROM 01053_db.keys_table;
SELECT 'HAS';
-- 1006
SELECT count() FROM 01053_db.keys_table WHERE dictHas('01053_db.ssd_dict', toUInt64(id));
SELECT 'VALUES NOT FROM TABLE';
-- 0 -1 none
SELECT dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(1000000)), dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(1000000)), dictGetString('01053_db.ssd_dict', 'c', toUInt64(1000000));
SELECT dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(1000000)), dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(1000000)), dictGetString('01053_db.ssd_dict', 'c', toUInt64(1000000));
SELECT 'DUPLICATE KEYS';
SELECT arrayJoin([1, 2, 3, 3, 2, 1]) AS id, dictGetInt32('01053_db.ssd_dict', 'b', toUInt64(id));
--SELECT
DROP DICTIONARY IF EXISTS 01053_db.ssd_dict;
DROP TABLE IF EXISTS 01053_db.keys_table;
CREATE TABLE 01053_db.keys_table
(
id UInt64
)
ENGINE = MergeTree()
ORDER BY id;
INSERT INTO 01053_db.keys_table VALUES (1);
INSERT INTO 01053_db.keys_table SELECT intHash64(number) FROM system.numbers LIMIT 370;
INSERT INTO 01053_db.keys_table VALUES (2);
INSERT INTO 01053_db.keys_table SELECT intHash64(number) FROM system.numbers LIMIT 370, 370;
INSERT INTO 01053_db.keys_table VALUES (5);
INSERT INTO 01053_db.keys_table SELECT intHash64(number) FROM system.numbers LIMIT 700, 370;
INSERT INTO 01053_db.keys_table VALUES (10);
OPTIMIZE TABLE 01053_db.keys_table;
CREATE DICTIONARY 01053_db.ssd_dict
(
id UInt64,
a UInt64 DEFAULT 0,
b Int32 DEFAULT -1
)
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));
SELECT 'UPDATE DICTIONARY (MT)';
-- 118
SELECT sum(dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(id))) FROM 01053_db.keys_table;
SELECT 'VALUES FROM DISK AND RAM BUFFER (MT)';
-- 118
SELECT sum(dictGetUInt64('01053_db.ssd_dict', 'a', toUInt64(id))) FROM 01053_db.keys_table;
DROP DICTIONARY IF EXISTS 01053_db.ssd_dict;
DROP TABLE IF EXISTS 01053_db.table_for_dict;
DROP DATABASE IF EXISTS 01053_db;

View File

@ -0,0 +1,124 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}')
$CLICKHOUSE_CLIENT -n --query="
DROP DATABASE IF EXISTS 01280_db;
CREATE DATABASE 01280_db Engine = Ordinary;
DROP TABLE IF EXISTS 01280_db.table_for_dict;
CREATE TABLE 01280_db.table_for_dict
(
k1 String,
k2 Int32,
a UInt64,
b Int32,
c String
)
ENGINE = MergeTree()
ORDER BY (k1, k2);
INSERT INTO 01280_db.table_for_dict VALUES (toString(1), 3, 100, -100, 'clickhouse'), (toString(2), -1, 3, 4, 'database'), (toString(5), -3, 6, 7, 'columns'), (toString(10), -20, 9, 8, '');
INSERT INTO 01280_db.table_for_dict SELECT toString(number), number + 1, 0, -1, 'a' FROM system.numbers WHERE number NOT IN (1, 2, 5, 10) LIMIT 370;
INSERT INTO 01280_db.table_for_dict SELECT toString(number), number + 10, 0, -1, 'b' FROM system.numbers WHERE number NOT IN (1, 2, 5, 10) LIMIT 370, 370;
INSERT INTO 01280_db.table_for_dict SELECT toString(number), number + 100, 0, -1, 'c' FROM system.numbers WHERE number NOT IN (1, 2, 5, 10) LIMIT 700, 370;
DROP DICTIONARY IF EXISTS 01280_db.ssd_dict;
CREATE DICTIONARY 01280_db.ssd_dict
(
k1 String,
k2 Int32,
a UInt64 DEFAULT 0,
b Int32 DEFAULT -1,
c String DEFAULT 'none'
)
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 '$USER_FILES_PATH/0d'));"
$CLICKHOUSE_CLIENT --testmode -nq "SELECT dictHas('01280_db.ssd_dict', 'a', tuple('1')); -- { serverError 43 }"
$CLICKHOUSE_CLIENT -n --query="
SELECT 'TEST_SMALL';
SELECT 'VALUE FROM RAM BUFFER';
SELECT dictGetUInt64('01280_db.ssd_dict', 'a', tuple('1', toInt32(3)));
SELECT dictGetInt32('01280_db.ssd_dict', 'b', tuple('1', toInt32(3)));
SELECT dictGetString('01280_db.ssd_dict', 'c', tuple('1', toInt32(3)));
SELECT dictGetUInt64('01280_db.ssd_dict', 'a', tuple('1', toInt32(3)));
SELECT dictGetInt32('01280_db.ssd_dict', 'b', tuple('1', toInt32(3)));
SELECT dictGetString('01280_db.ssd_dict', 'c', tuple('1', toInt32(3)));
SELECT dictGetUInt64('01280_db.ssd_dict', 'a', tuple('2', toInt32(-1)));
SELECT dictGetInt32('01280_db.ssd_dict', 'b', tuple('2', toInt32(-1)));
SELECT dictGetString('01280_db.ssd_dict', 'c', tuple('2', toInt32(-1)));
SELECT dictGetUInt64('01280_db.ssd_dict', 'a', tuple('5', toInt32(-3)));
SELECT dictGetInt32('01280_db.ssd_dict', 'b', tuple('5', toInt32(-3)));
SELECT dictGetString('01280_db.ssd_dict', 'c', tuple('5', toInt32(-3)));
SELECT dictGetUInt64('01280_db.ssd_dict', 'a', tuple('10', toInt32(-20)));
SELECT dictGetInt32('01280_db.ssd_dict', 'b', tuple('10', toInt32(-20)));
SELECT dictGetString('01280_db.ssd_dict', 'c', tuple('10', toInt32(-20)));"
$CLICKHOUSE_CLIENT --testmode -nq "SELECT dictGetUInt64('01280_db.ssd_dict', 'a', tuple(toInt32(3))); -- { serverError 53 }"
$CLICKHOUSE_CLIENT -n --query="DROP DICTIONARY 01280_db.ssd_dict;
DROP TABLE IF EXISTS 01280_db.keys_table;
CREATE TABLE 01280_db.keys_table
(
k1 String,
k2 Int32
)
ENGINE = StripeLog();
INSERT INTO 01280_db.keys_table VALUES ('1', 3);
INSERT INTO 01280_db.keys_table SELECT toString(intHash64(number + 1) % 1200), 11 + intHash64(number) % 1200 FROM system.numbers LIMIT 370;
INSERT INTO 01280_db.keys_table VALUES ('2', -1);
INSERT INTO 01280_db.keys_table SELECT toString(intHash64(number + 1) % 1200), 11 + intHash64(number) % 1200 FROM system.numbers LIMIT 370, 370;
INSERT INTO 01280_db.keys_table VALUES ('5', -3);
INSERT INTO 01280_db.keys_table SELECT toString(intHash64(number + 1) % 1200), 11 + intHash64(number) % 1200 FROM system.numbers LIMIT 700, 370;
INSERT INTO 01280_db.keys_table VALUES ('10', -20);
DROP DICTIONARY IF EXISTS 01280_db.ssd_dict;CREATE DICTIONARY 01280_db.ssd_dict
(
k1 String,
k2 Int32,
a UInt64 DEFAULT 0,
b Int32 DEFAULT -1,
c String DEFAULT 'none'
)
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 '$USER_FILES_PATH/1d' BLOCK_SIZE 512 WRITE_BUFFER_SIZE 4096));
SELECT 'UPDATE DICTIONARY';
SELECT sum(dictGetUInt64('01280_db.ssd_dict', 'a', (k1, k2))) FROM 01280_db.keys_table;
SELECT 'VALUE FROM DISK';
SELECT dictGetInt32('01280_db.ssd_dict', 'b', ('1', toInt32(3)));
SELECT dictGetString('01280_db.ssd_dict', 'c', ('1', toInt32(3)));
SELECT 'VALUE FROM RAM BUFFER';
SELECT dictGetInt32('01280_db.ssd_dict', 'b', ('10', toInt32(-20)));
SELECT dictGetString('01280_db.ssd_dict', 'c', ('10', toInt32(-20)));
SELECT 'VALUES FROM DISK AND RAM BUFFER';
SELECT sum(dictGetUInt64('01280_db.ssd_dict', 'a', (k1, k2))) FROM 01280_db.keys_table;
SELECT 'HAS';
SELECT count() FROM 01280_db.keys_table WHERE dictHas('01280_db.ssd_dict', (k1, k2));
SELECT 'VALUES NOT FROM TABLE';
SELECT dictGetUInt64('01280_db.ssd_dict', 'a', ('unknown', toInt32(0))), dictGetInt32('01280_db.ssd_dict', 'b', ('unknown', toInt32(0))), dictGetString('01280_db.ssd_dict', 'c', ('unknown', toInt32(0)));
SELECT dictGetUInt64('01280_db.ssd_dict', 'a', ('unknown', toInt32(0))), dictGetInt32('01280_db.ssd_dict', 'b', ('unknown', toInt32(0))), dictGetString('01280_db.ssd_dict', 'c', ('unknown', toInt32(0)));
SELECT 'DUPLICATE KEYS';
SELECT arrayJoin([('1', toInt32(3)), ('2', toInt32(-1)), ('', toInt32(0)), ('', toInt32(0)), ('2', toInt32(-1)), ('1', toInt32(3))]) AS keys, dictGetInt32('01280_db.ssd_dict', 'b', keys);
DROP DICTIONARY IF EXISTS database_for_dict.ssd_dict;
DROP TABLE IF EXISTS database_for_dict.keys_table;"

View File

@ -1,139 +0,0 @@
SET send_logs_level = 'none';
DROP DATABASE IF EXISTS 01280_db;
CREATE DATABASE 01280_db Engine = Ordinary;
DROP TABLE IF EXISTS 01280_db.table_for_dict;
CREATE TABLE 01280_db.table_for_dict
(
k1 String,
k2 Int32,
a UInt64,
b Int32,
c String
)
ENGINE = MergeTree()
ORDER BY (k1, k2);
INSERT INTO 01280_db.table_for_dict VALUES (toString(1), 3, 100, -100, 'clickhouse'), (toString(2), -1, 3, 4, 'database'), (toString(5), -3, 6, 7, 'columns'), (toString(10), -20, 9, 8, '');
INSERT INTO 01280_db.table_for_dict SELECT toString(number), number + 1, 0, -1, 'a' FROM system.numbers WHERE number NOT IN (1, 2, 5, 10) LIMIT 370;
INSERT INTO 01280_db.table_for_dict SELECT toString(number), number + 10, 0, -1, 'b' FROM system.numbers WHERE number NOT IN (1, 2, 5, 10) LIMIT 370, 370;
INSERT INTO 01280_db.table_for_dict SELECT toString(number), number + 100, 0, -1, 'c' FROM system.numbers WHERE number NOT IN (1, 2, 5, 10) LIMIT 700, 370;
DROP DICTIONARY IF EXISTS 01280_db.ssd_dict;
-- FIXME filesystem error: in create_directory: Permission denied [/var/lib/clickhouse]
-- Probably we need rewrite it to integration test
CREATE DICTIONARY 01280_db.ssd_dict
(
k1 String,
k2 Int32,
a UInt64 DEFAULT 0,
b Int32 DEFAULT -1,
c String DEFAULT 'none'
)
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'));
SELECT 'TEST_SMALL';
SELECT 'VALUE FROM RAM BUFFER';
SELECT dictHas('01280_db.ssd_dict', 'a', tuple('1')); -- { serverError 43 }
SELECT dictGetUInt64('01280_db.ssd_dict', 'a', tuple('1', toInt32(3)));
SELECT dictGetInt32('01280_db.ssd_dict', 'b', tuple('1', toInt32(3)));
SELECT dictGetString('01280_db.ssd_dict', 'c', tuple('1', toInt32(3)));
SELECT dictGetUInt64('01280_db.ssd_dict', 'a', tuple('1', toInt32(3)));
SELECT dictGetInt32('01280_db.ssd_dict', 'b', tuple('1', toInt32(3)));
SELECT dictGetString('01280_db.ssd_dict', 'c', tuple('1', toInt32(3)));
SELECT dictGetUInt64('01280_db.ssd_dict', 'a', tuple('2', toInt32(-1)));
SELECT dictGetInt32('01280_db.ssd_dict', 'b', tuple('2', toInt32(-1)));
SELECT dictGetString('01280_db.ssd_dict', 'c', tuple('2', toInt32(-1)));
SELECT dictGetUInt64('01280_db.ssd_dict', 'a', tuple('5', toInt32(-3)));
SELECT dictGetInt32('01280_db.ssd_dict', 'b', tuple('5', toInt32(-3)));
SELECT dictGetString('01280_db.ssd_dict', 'c', tuple('5', toInt32(-3)));
SELECT dictGetUInt64('01280_db.ssd_dict', 'a', tuple('10', toInt32(-20)));
SELECT dictGetInt32('01280_db.ssd_dict', 'b', tuple('10', toInt32(-20)));
SELECT dictGetString('01280_db.ssd_dict', 'c', tuple('10', toInt32(-20)));
SELECT dictGetUInt64('01280_db.ssd_dict', 'a', tuple(toInt32(3))); --{serverError 53}
DROP DICTIONARY 01280_db.ssd_dict;
DROP TABLE IF EXISTS 01280_db.keys_table;
CREATE TABLE 01280_db.keys_table
(
k1 String,
k2 Int32
)
ENGINE = StripeLog();
INSERT INTO 01280_db.keys_table VALUES ('1', 3);
INSERT INTO 01280_db.keys_table SELECT toString(intHash64(number + 1) % 1200), 11 + intHash64(number) % 1200 FROM system.numbers LIMIT 370;
INSERT INTO 01280_db.keys_table VALUES ('2', -1);
INSERT INTO 01280_db.keys_table SELECT toString(intHash64(number + 1) % 1200), 11 + intHash64(number) % 1200 FROM system.numbers LIMIT 370, 370;
INSERT INTO 01280_db.keys_table VALUES ('5', -3);
INSERT INTO 01280_db.keys_table SELECT toString(intHash64(number + 1) % 1200), 11 + intHash64(number) % 1200 FROM system.numbers LIMIT 700, 370;
INSERT INTO 01280_db.keys_table VALUES ('10', -20);
DROP DICTIONARY IF EXISTS 01280_db.ssd_dict;
CREATE DICTIONARY 01280_db.ssd_dict
(
k1 String,
k2 Int32,
a UInt64 DEFAULT 0,
b Int32 DEFAULT -1,
c String DEFAULT 'none'
)
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));
SELECT 'UPDATE DICTIONARY';
-- 118
SELECT sum(dictGetUInt64('01280_db.ssd_dict', 'a', (k1, k2))) FROM 01280_db.keys_table;
SELECT 'VALUE FROM DISK';
-- -100
SELECT dictGetInt32('01280_db.ssd_dict', 'b', ('1', toInt32(3)));
-- 'clickhouse'
SELECT dictGetString('01280_db.ssd_dict', 'c', ('1', toInt32(3)));
SELECT 'VALUE FROM RAM BUFFER';
-- 8
SELECT dictGetInt32('01280_db.ssd_dict', 'b', ('10', toInt32(-20)));
-- ''
SELECT dictGetString('01280_db.ssd_dict', 'c', ('10', toInt32(-20)));
SELECT 'VALUES FROM DISK AND RAM BUFFER';
-- 118
SELECT sum(dictGetUInt64('01280_db.ssd_dict', 'a', (k1, k2))) FROM 01280_db.keys_table;
SELECT 'HAS';
-- 6
SELECT count() FROM 01280_db.keys_table WHERE dictHas('01280_db.ssd_dict', (k1, k2));
SELECT 'VALUES NOT FROM TABLE';
-- 0 -1 none
SELECT dictGetUInt64('01280_db.ssd_dict', 'a', ('unknown', toInt32(0))), dictGetInt32('01280_db.ssd_dict', 'b', ('unknown', toInt32(0))), dictGetString('01280_db.ssd_dict', 'c', ('unknown', toInt32(0)));
SELECT dictGetUInt64('01280_db.ssd_dict', 'a', ('unknown', toInt32(0))), dictGetInt32('01280_db.ssd_dict', 'b', ('unknown', toInt32(0))), dictGetString('01280_db.ssd_dict', 'c', ('unknown', toInt32(0)));
SELECT 'DUPLICATE KEYS';
SELECT arrayJoin([('1', toInt32(3)), ('2', toInt32(-1)), ('', toInt32(0)), ('', toInt32(0)), ('2', toInt32(-1)), ('1', toInt32(3))]) AS keys, dictGetInt32('01280_db.ssd_dict', 'b', keys);
DROP DICTIONARY IF EXISTS database_for_dict.ssd_dict;
DROP TABLE IF EXISTS database_for_dict.keys_table;

View File

@ -0,0 +1,132 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}')
$CLICKHOUSE_CLIENT -n --query="
DROP DATABASE IF EXISTS 01684_database_for_cache_dictionary;
CREATE DATABASE 01684_database_for_cache_dictionary;
CREATE TABLE 01684_database_for_cache_dictionary.simple_key_simple_attributes_source_table
(
id UInt64,
value_first String,
value_second String
)
ENGINE = TinyLog;
INSERT INTO 01684_database_for_cache_dictionary.simple_key_simple_attributes_source_table VALUES(0, 'value_0', 'value_second_0');
INSERT INTO 01684_database_for_cache_dictionary.simple_key_simple_attributes_source_table VALUES(1, 'value_1', 'value_second_1');
INSERT INTO 01684_database_for_cache_dictionary.simple_key_simple_attributes_source_table VALUES(2, 'value_2', 'value_second_2');
CREATE DICTIONARY 01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes
(
id UInt64,
value_first String DEFAULT 'value_first_default',
value_second String DEFAULT 'value_second_default'
)
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 '$USER_FILES_PATH/0d'));
SELECT 'Dictionary cache_dictionary_simple_key_simple_attributes';
SELECT 'dictGet existing value';
SELECT dictGet('01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first,
dictGet('01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3;
SELECT 'dictGet with non existing value';
SELECT dictGet('01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first,
dictGet('01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4;
SELECT 'dictGetOrDefault existing value';
SELECT dictGetOrDefault('01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first,
dictGetOrDefault('01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3;
SELECT 'dictGetOrDefault non existing value';
SELECT dictGetOrDefault('01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first,
dictGetOrDefault('01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4;
SELECT 'dictHas';
SELECT dictHas('01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', number) FROM system.numbers LIMIT 4;
SELECT 'select all values as input stream';
SELECT * FROM 01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes ORDER BY id;
DROP DICTIONARY 01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes;
DROP TABLE 01684_database_for_cache_dictionary.simple_key_simple_attributes_source_table;
CREATE TABLE 01684_database_for_cache_dictionary.simple_key_complex_attributes_source_table
(
id UInt64,
value_first String,
value_second Nullable(String)
)
ENGINE = TinyLog;
INSERT INTO 01684_database_for_cache_dictionary.simple_key_complex_attributes_source_table VALUES(0, 'value_0', 'value_second_0');
INSERT INTO 01684_database_for_cache_dictionary.simple_key_complex_attributes_source_table VALUES(1, 'value_1', NULL);
INSERT INTO 01684_database_for_cache_dictionary.simple_key_complex_attributes_source_table VALUES(2, 'value_2', 'value_second_2');
CREATE DICTIONARY 01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes
(
id UInt64,
value_first String DEFAULT 'value_first_default',
value_second Nullable(String) DEFAULT 'value_second_default'
)
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 '$USER_FILES_PATH/1d'));
SELECT 'Dictionary cache_dictionary_simple_key_complex_attributes';
SELECT 'dictGet existing value';
SELECT dictGet('01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first,
dictGet('01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3;
SELECT 'dictGet with non existing value';
SELECT dictGet('01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first,
dictGet('01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4;
SELECT 'dictGetOrDefault existing value';
SELECT dictGetOrDefault('01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first,
dictGetOrDefault('01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3;
SELECT 'dictGetOrDefault non existing value';
SELECT dictGetOrDefault('01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first,
dictGetOrDefault('01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4;
SELECT 'dictHas';
SELECT dictHas('01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', number) FROM system.numbers LIMIT 4;
SELECT 'select all values as input stream';
SELECT * FROM 01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes ORDER BY id;
DROP DICTIONARY 01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes;
DROP TABLE 01684_database_for_cache_dictionary.simple_key_complex_attributes_source_table;
CREATE TABLE 01684_database_for_cache_dictionary.simple_key_hierarchy_table
(
id UInt64,
parent_id UInt64
) ENGINE = TinyLog();
INSERT INTO 01684_database_for_cache_dictionary.simple_key_hierarchy_table VALUES (1, 0);
INSERT INTO 01684_database_for_cache_dictionary.simple_key_hierarchy_table VALUES (2, 1);
INSERT INTO 01684_database_for_cache_dictionary.simple_key_hierarchy_table VALUES (3, 1);
INSERT INTO 01684_database_for_cache_dictionary.simple_key_hierarchy_table VALUES (4, 2);
CREATE DICTIONARY 01684_database_for_cache_dictionary.cache_dictionary_simple_key_hierarchy
(
id UInt64,
parent_id UInt64 HIERARCHICAL
)
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 '$USER_FILES_PATH/2d'));
SELECT 'Dictionary cache_dictionary_simple_key_hierarchy';
SELECT 'dictGet';
SELECT dictGet('01684_database_for_cache_dictionary.cache_dictionary_simple_key_hierarchy', 'parent_id', number) FROM system.numbers LIMIT 5;
SELECT 'dictGetHierarchy';
SELECT dictGetHierarchy('01684_database_for_cache_dictionary.cache_dictionary_simple_key_hierarchy', toUInt64(1));
SELECT dictGetHierarchy('01684_database_for_cache_dictionary.cache_dictionary_simple_key_hierarchy', toUInt64(4));
DROP DICTIONARY 01684_database_for_cache_dictionary.cache_dictionary_simple_key_hierarchy;
DROP TABLE 01684_database_for_cache_dictionary.simple_key_hierarchy_table;
DROP DATABASE 01684_database_for_cache_dictionary;"

View File

@ -1,123 +0,0 @@
DROP DATABASE IF EXISTS 01684_database_for_cache_dictionary;
CREATE DATABASE 01684_database_for_cache_dictionary;
CREATE TABLE 01684_database_for_cache_dictionary.simple_key_simple_attributes_source_table
(
id UInt64,
value_first String,
value_second String
)
ENGINE = TinyLog;
INSERT INTO 01684_database_for_cache_dictionary.simple_key_simple_attributes_source_table VALUES(0, 'value_0', 'value_second_0');
INSERT INTO 01684_database_for_cache_dictionary.simple_key_simple_attributes_source_table VALUES(1, 'value_1', 'value_second_1');
INSERT INTO 01684_database_for_cache_dictionary.simple_key_simple_attributes_source_table VALUES(2, 'value_2', 'value_second_2');
CREATE DICTIONARY 01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes
(
id UInt64,
value_first String DEFAULT 'value_first_default',
value_second String DEFAULT 'value_second_default'
)
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'));
SELECT 'Dictionary cache_dictionary_simple_key_simple_attributes';
SELECT 'dictGet existing value';
SELECT dictGet('01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first,
dictGet('01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3;
SELECT 'dictGet with non existing value';
SELECT dictGet('01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first,
dictGet('01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4;
SELECT 'dictGetOrDefault existing value';
SELECT dictGetOrDefault('01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first,
dictGetOrDefault('01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3;
SELECT 'dictGetOrDefault non existing value';
SELECT dictGetOrDefault('01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first,
dictGetOrDefault('01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4;
SELECT 'dictHas';
SELECT dictHas('01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes', number) FROM system.numbers LIMIT 4;
SELECT 'select all values as input stream';
SELECT * FROM 01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes ORDER BY id;
DROP DICTIONARY 01684_database_for_cache_dictionary.cache_dictionary_simple_key_simple_attributes;
DROP TABLE 01684_database_for_cache_dictionary.simple_key_simple_attributes_source_table;
CREATE TABLE 01684_database_for_cache_dictionary.simple_key_complex_attributes_source_table
(
id UInt64,
value_first String,
value_second Nullable(String)
)
ENGINE = TinyLog;
INSERT INTO 01684_database_for_cache_dictionary.simple_key_complex_attributes_source_table VALUES(0, 'value_0', 'value_second_0');
INSERT INTO 01684_database_for_cache_dictionary.simple_key_complex_attributes_source_table VALUES(1, 'value_1', NULL);
INSERT INTO 01684_database_for_cache_dictionary.simple_key_complex_attributes_source_table VALUES(2, 'value_2', 'value_second_2');
CREATE DICTIONARY 01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes
(
id UInt64,
value_first String DEFAULT 'value_first_default',
value_second Nullable(String) DEFAULT 'value_second_default'
)
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'));
SELECT 'Dictionary cache_dictionary_simple_key_complex_attributes';
SELECT 'dictGet existing value';
SELECT dictGet('01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first,
dictGet('01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3;
SELECT 'dictGet with non existing value';
SELECT dictGet('01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first,
dictGet('01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4;
SELECT 'dictGetOrDefault existing value';
SELECT dictGetOrDefault('01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first,
dictGetOrDefault('01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3;
SELECT 'dictGetOrDefault non existing value';
SELECT dictGetOrDefault('01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first,
dictGetOrDefault('01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4;
SELECT 'dictHas';
SELECT dictHas('01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes', number) FROM system.numbers LIMIT 4;
SELECT 'select all values as input stream';
SELECT * FROM 01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes ORDER BY id;
DROP DICTIONARY 01684_database_for_cache_dictionary.cache_dictionary_simple_key_complex_attributes;
DROP TABLE 01684_database_for_cache_dictionary.simple_key_complex_attributes_source_table;
CREATE TABLE 01684_database_for_cache_dictionary.simple_key_hierarchy_table
(
id UInt64,
parent_id UInt64
) ENGINE = TinyLog();
INSERT INTO 01684_database_for_cache_dictionary.simple_key_hierarchy_table VALUES (1, 0);
INSERT INTO 01684_database_for_cache_dictionary.simple_key_hierarchy_table VALUES (2, 1);
INSERT INTO 01684_database_for_cache_dictionary.simple_key_hierarchy_table VALUES (3, 1);
INSERT INTO 01684_database_for_cache_dictionary.simple_key_hierarchy_table VALUES (4, 2);
CREATE DICTIONARY 01684_database_for_cache_dictionary.cache_dictionary_simple_key_hierarchy
(
id UInt64,
parent_id UInt64 HIERARCHICAL
)
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'));
SELECT 'Dictionary cache_dictionary_simple_key_hierarchy';
SELECT 'dictGet';
SELECT dictGet('01684_database_for_cache_dictionary.cache_dictionary_simple_key_hierarchy', 'parent_id', number) FROM system.numbers LIMIT 5;
SELECT 'dictGetHierarchy';
SELECT dictGetHierarchy('01684_database_for_cache_dictionary.cache_dictionary_simple_key_hierarchy', toUInt64(1));
SELECT dictGetHierarchy('01684_database_for_cache_dictionary.cache_dictionary_simple_key_hierarchy', toUInt64(4));
DROP DICTIONARY 01684_database_for_cache_dictionary.cache_dictionary_simple_key_hierarchy;
DROP TABLE 01684_database_for_cache_dictionary.simple_key_hierarchy_table;
DROP DATABASE 01684_database_for_cache_dictionary;

View File

@ -0,0 +1,106 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}')
$CLICKHOUSE_CLIENT -n --query="
DROP DATABASE IF EXISTS 01685_database_for_cache_dictionary;
CREATE DATABASE 01685_database_for_cache_dictionary;
CREATE TABLE 01685_database_for_cache_dictionary.complex_key_simple_attributes_source_table
(
id UInt64,
id_key String,
value_first String,
value_second String
)
ENGINE = TinyLog;
INSERT INTO 01685_database_for_cache_dictionary.complex_key_simple_attributes_source_table VALUES(0, 'id_key_0', 'value_0', 'value_second_0');
INSERT INTO 01685_database_for_cache_dictionary.complex_key_simple_attributes_source_table VALUES(1, 'id_key_1', 'value_1', 'value_second_1');
INSERT INTO 01685_database_for_cache_dictionary.complex_key_simple_attributes_source_table VALUES(2, 'id_key_2', 'value_2', 'value_second_2');
CREATE DICTIONARY 01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes
(
id UInt64,
id_key String,
value_first String DEFAULT 'value_first_default',
value_second String DEFAULT 'value_second_default'
)
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 '$USER_FILES_PATH/0d'));
SELECT 'Dictionary cache_dictionary_complex_key_simple_attributes';
SELECT 'dictGet existing value';
SELECT dictGet('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
dictGet('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 3;
SELECT 'dictGet with non existing value';
SELECT dictGet('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
dictGet('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 4;
SELECT 'dictGetOrDefault existing value';
SELECT dictGetOrDefault('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
dictGetOrDefault('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 3;
SELECT 'dictGetOrDefault non existing value';
SELECT dictGetOrDefault('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
dictGetOrDefault('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 4;
SELECT 'dictHas';
SELECT dictHas('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4;
SELECT 'select all values as input stream';
SELECT * FROM 01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes ORDER BY id;
DROP DICTIONARY 01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes;
DROP TABLE 01685_database_for_cache_dictionary.complex_key_simple_attributes_source_table;
CREATE TABLE 01685_database_for_cache_dictionary.complex_key_complex_attributes_source_table
(
id UInt64,
id_key String,
value_first String,
value_second Nullable(String)
)
ENGINE = TinyLog;
INSERT INTO 01685_database_for_cache_dictionary.complex_key_complex_attributes_source_table VALUES(0, 'id_key_0', 'value_0', 'value_second_0');
INSERT INTO 01685_database_for_cache_dictionary.complex_key_complex_attributes_source_table VALUES(1, 'id_key_1', 'value_1', NULL);
INSERT INTO 01685_database_for_cache_dictionary.complex_key_complex_attributes_source_table VALUES(2, 'id_key_2', 'value_2', 'value_second_2');
CREATE DICTIONARY 01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes
(
id UInt64,
id_key String,
value_first String DEFAULT 'value_first_default',
value_second Nullable(String) DEFAULT 'value_second_default'
)
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 '$USER_FILES_PATH/1d'));
SELECT 'Dictionary cache_dictionary_complex_key_complex_attributes';
SELECT 'dictGet existing value';
SELECT dictGet('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
dictGet('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 3;
SELECT 'dictGet with non existing value';
SELECT dictGet('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
dictGet('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 4;
SELECT 'dictGetOrDefault existing value';
SELECT dictGetOrDefault('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
dictGetOrDefault('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 3;
SELECT 'dictGetOrDefault non existing value';
SELECT dictGetOrDefault('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
dictGetOrDefault('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 4;
SELECT 'dictHas';
SELECT dictHas('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4;
SELECT 'select all values as input stream';
SELECT * FROM 01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes ORDER BY id;
DROP DICTIONARY 01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes;
DROP TABLE 01685_database_for_cache_dictionary.complex_key_complex_attributes_source_table;
DROP DATABASE 01685_database_for_cache_dictionary;"

View File

@ -1,98 +0,0 @@
DROP DATABASE IF EXISTS 01685_database_for_cache_dictionary;
CREATE DATABASE 01685_database_for_cache_dictionary;
CREATE TABLE 01685_database_for_cache_dictionary.complex_key_simple_attributes_source_table
(
id UInt64,
id_key String,
value_first String,
value_second String
)
ENGINE = TinyLog;
INSERT INTO 01685_database_for_cache_dictionary.complex_key_simple_attributes_source_table VALUES(0, 'id_key_0', 'value_0', 'value_second_0');
INSERT INTO 01685_database_for_cache_dictionary.complex_key_simple_attributes_source_table VALUES(1, 'id_key_1', 'value_1', 'value_second_1');
INSERT INTO 01685_database_for_cache_dictionary.complex_key_simple_attributes_source_table VALUES(2, 'id_key_2', 'value_2', 'value_second_2');
CREATE DICTIONARY 01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes
(
id UInt64,
id_key String,
value_first String DEFAULT 'value_first_default',
value_second String DEFAULT 'value_second_default'
)
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'));
SELECT 'Dictionary cache_dictionary_complex_key_simple_attributes';
SELECT 'dictGet existing value';
SELECT dictGet('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
dictGet('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 3;
SELECT 'dictGet with non existing value';
SELECT dictGet('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
dictGet('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 4;
SELECT 'dictGetOrDefault existing value';
SELECT dictGetOrDefault('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
dictGetOrDefault('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 3;
SELECT 'dictGetOrDefault non existing value';
SELECT dictGetOrDefault('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
dictGetOrDefault('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 4;
SELECT 'dictHas';
SELECT dictHas('01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4;
SELECT 'select all values as input stream';
SELECT * FROM 01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes ORDER BY id;
DROP DICTIONARY 01685_database_for_cache_dictionary.cache_dictionary_complex_key_simple_attributes;
DROP TABLE 01685_database_for_cache_dictionary.complex_key_simple_attributes_source_table;
CREATE TABLE 01685_database_for_cache_dictionary.complex_key_complex_attributes_source_table
(
id UInt64,
id_key String,
value_first String,
value_second Nullable(String)
)
ENGINE = TinyLog;
INSERT INTO 01685_database_for_cache_dictionary.complex_key_complex_attributes_source_table VALUES(0, 'id_key_0', 'value_0', 'value_second_0');
INSERT INTO 01685_database_for_cache_dictionary.complex_key_complex_attributes_source_table VALUES(1, 'id_key_1', 'value_1', NULL);
INSERT INTO 01685_database_for_cache_dictionary.complex_key_complex_attributes_source_table VALUES(2, 'id_key_2', 'value_2', 'value_second_2');
CREATE DICTIONARY 01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes
(
id UInt64,
id_key String,
value_first String DEFAULT 'value_first_default',
value_second Nullable(String) DEFAULT 'value_second_default'
)
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'));
SELECT 'Dictionary cache_dictionary_complex_key_complex_attributes';
SELECT 'dictGet existing value';
SELECT dictGet('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
dictGet('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 3;
SELECT 'dictGet with non existing value';
SELECT dictGet('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first,
dictGet('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 4;
SELECT 'dictGetOrDefault existing value';
SELECT dictGetOrDefault('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
dictGetOrDefault('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 3;
SELECT 'dictGetOrDefault non existing value';
SELECT dictGetOrDefault('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first,
dictGetOrDefault('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 4;
SELECT 'dictHas';
SELECT dictHas('01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4;
SELECT 'select all values as input stream';
SELECT * FROM 01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes ORDER BY id;
DROP DICTIONARY 01685_database_for_cache_dictionary.cache_dictionary_complex_key_complex_attributes;
DROP TABLE 01685_database_for_cache_dictionary.complex_key_complex_attributes_source_table;
DROP DATABASE 01685_database_for_cache_dictionary;

View File

@ -4,8 +4,6 @@ Hashed dictionary
5.00000
Cache dictionary
5.00000
SSDCache dictionary
5.00000
Direct dictionary
5.00000
IPTrie dictionary

View File

@ -55,22 +55,6 @@ SELECT dictGet('cache_dictionary', 'decimal_value', toUInt64(1));
DROP DICTIONARY cache_dictionary;
DROP DICTIONARY IF EXISTS ssd_cache_dictionary;
CREATE DICTIONARY ssd_cache_dictionary
(
id UInt64,
decimal_value Decimal256(5)
)
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'));
SELECT 'SSDCache dictionary';
SELECT dictGet('ssd_cache_dictionary', 'decimal_value', toUInt64(1));
DROP DICTIONARY ssd_cache_dictionary;
DROP DICTIONARY IF EXISTS direct_dictionary;
CREATE DICTIONARY direct_dictionary
(

View File

@ -0,0 +1,13 @@
(0.1000000000000000000000000000000000000000000000000000000000000000000000,0.2000000000000000000000000000000000000000000000000000000000000000000000)
SELECT CAST(\'(0.1, 0.2)\', \'Tuple(Decimal(75, 70), Decimal(75, 70))\')
0.1000
SELECT CAST(\'0.1\', \'Decimal(4, 4)\')
[1,2,3]
SELECT CAST(\'[1, 2, 3]\', \'Array(Int32)\')
[1,2]
SELECT CAST([CAST(\'1\', \'UInt32\'), CAST(\'2\', \'UInt32\')], \'Array(UInt64)\')
[[1,2],[3]]
SELECT CAST([CAST(\'[1, 2]\', \'Array(UInt32)\'), [3]], \'Array(Array(UInt64))\')
[[1,2],[3]]
SELECT CAST([CAST([CAST(\'1\', \'UInt16\'), CAST(\'2\', \'UInt16\')], \'Array(UInt32)\'), [3]], \'Array(Array(UInt64))\')
[(1,'a'),(3,'b')] Nested(u UInt8, s String)

View File

@ -0,0 +1,19 @@
SELECT (0.1, 0.2)::Tuple(Decimal(75, 70), Decimal(75, 70));
EXPLAIN SYNTAX SELECT (0.1, 0.2)::Tuple(Decimal(75, 70), Decimal(75, 70));
SELECT 0.1 :: Decimal(4, 4);
EXPLAIN SYNTAX SELECT 0.1 :: Decimal(4, 4);
SELECT [1, 2, 3] :: Array(Int32);
EXPLAIN SYNTAX SELECT [1, 2, 3] :: Array(Int32);
SELECT [1::UInt32, 2::UInt32]::Array(UInt64);
EXPLAIN SYNTAX SELECT [1::UInt32, 2::UInt32]::Array(UInt64);
SELECT [[1, 2]::Array(UInt32), [3]]::Array(Array(UInt64));
EXPLAIN SYNTAX SELECT [[1, 2]::Array(UInt32), [3]]::Array(Array(UInt64));
SELECT [[1::UInt16, 2::UInt16]::Array(UInt32), [3]]::Array(Array(UInt64));
EXPLAIN SYNTAX SELECT [[1::UInt16, 2::UInt16]::Array(UInt32), [3]]::Array(Array(UInt64));
SELECT [(1, 'a'), (3, 'b')]::Nested(u UInt8, s String) AS t, toTypeName(t);

View File

@ -0,0 +1,2 @@
SSDCache dictionary
5.00000

View File

@ -0,0 +1,35 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}')
$CLICKHOUSE_CLIENT -n --query="
SET allow_experimental_bigint_types = 1;
DROP TABLE IF EXISTS dictionary_decimal_source_table;
CREATE TABLE dictionary_decimal_source_table
(
id UInt64,
decimal_value Decimal256(5)
) ENGINE = TinyLog;
INSERT INTO dictionary_decimal_source_table VALUES (1, 5.0);
DROP DICTIONARY IF EXISTS ssd_cache_dictionary;
CREATE DICTIONARY ssd_cache_dictionary
(
id UInt64,
decimal_value Decimal256(5)
)
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 '$USER_FILES_PATH/0d'));
SELECT 'SSDCache dictionary';
SELECT dictGet('ssd_cache_dictionary', 'decimal_value', toUInt64(1));
DROP DICTIONARY ssd_cache_dictionary;"

View File

@ -238,3 +238,4 @@
01850_dist_INSERT_preserve_error
01870_modulo_partition_key
01880_remote_ipv6
01882_check_max_parts_to_merge_at_once

View File

@ -403,7 +403,11 @@
"01802_test_postgresql_protocol_with_row_policy", /// It cannot parse DROP ROW POLICY
"01823_explain_json",
"01852_cast_operator",
"01852_cast_operator_bad_cases"
"01852_cast_operator_bad_cases",
"01053_ssd_dictionary",
"01280_ssd_complex_key_dictionary",
"01684_ssd_cache_dictionary_simple_key",
"01685_ssd_cache_dictionary_complex_key"
],
"parallel":
[