mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge pull request #7360 from ClickHouse/dictionaries_ddl_loader
Dictionaries ddl loader
This commit is contained in:
commit
c7cd911305
@ -130,8 +130,8 @@ list (APPEND dbms_headers
|
||||
|
||||
list (APPEND dbms_sources src/TableFunctions/ITableFunction.cpp src/TableFunctions/TableFunctionFactory.cpp)
|
||||
list (APPEND dbms_headers src/TableFunctions/ITableFunction.h src/TableFunctions/TableFunctionFactory.h)
|
||||
list (APPEND dbms_sources src/Dictionaries/DictionaryFactory.cpp src/Dictionaries/DictionarySourceFactory.cpp src/Dictionaries/DictionaryStructure.cpp)
|
||||
list (APPEND dbms_headers src/Dictionaries/DictionaryFactory.h src/Dictionaries/DictionarySourceFactory.h src/Dictionaries/DictionaryStructure.h)
|
||||
list (APPEND dbms_sources src/Dictionaries/DictionaryFactory.cpp src/Dictionaries/DictionarySourceFactory.cpp src/Dictionaries/DictionaryStructure.cpp src/Dictionaries/getDictionaryConfigurationFromAST.cpp)
|
||||
list (APPEND dbms_headers src/Dictionaries/DictionaryFactory.h src/Dictionaries/DictionarySourceFactory.h src/Dictionaries/DictionaryStructure.h src/Dictionaries/getDictionaryConfigurationFromAST.h)
|
||||
|
||||
if (NOT ENABLE_SSL)
|
||||
list (REMOVE_ITEM clickhouse_common_io_sources src/Common/OpenSSLHelpers.cpp)
|
||||
|
@ -44,7 +44,7 @@ void ReplicasStatusHandler::handleRequest(Poco::Net::HTTPServerRequest & request
|
||||
if (db.second->getEngineName() == "Lazy")
|
||||
continue;
|
||||
|
||||
for (auto iterator = db.second->getIterator(context); iterator->isValid(); iterator->next())
|
||||
for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next())
|
||||
{
|
||||
auto & table = iterator->table();
|
||||
StorageReplicatedMergeTree * table_replicated = dynamic_cast<StorageReplicatedMergeTree *>(table.get());
|
||||
|
@ -37,10 +37,12 @@
|
||||
#include <Interpreters/AsynchronousMetrics.h>
|
||||
#include <Interpreters/DDLWorker.h>
|
||||
#include <Interpreters/ExternalDictionariesLoader.h>
|
||||
#include <Interpreters/ExternalModelsLoader.h>
|
||||
#include <Interpreters/ProcessList.h>
|
||||
#include <Interpreters/loadMetadata.h>
|
||||
#include <Interpreters/DNSCacheUpdater.h>
|
||||
#include <Interpreters/SystemLog.cpp>
|
||||
#include <Interpreters/ExternalLoaderXMLConfigRepository.h>
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
#include <Storages/System/attachSystemTables.h>
|
||||
#include <AggregateFunctions/registerAggregateFunctions.h>
|
||||
@ -920,6 +922,12 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
global_context->tryCreateEmbeddedDictionaries();
|
||||
global_context->getExternalDictionariesLoader().enableAlwaysLoadEverything(true);
|
||||
}
|
||||
|
||||
auto dictionaries_repository = std::make_unique<ExternalLoaderXMLConfigRepository>(config(), "dictionaries_config");
|
||||
global_context->getExternalDictionariesLoader().addConfigRepository("", std::move(dictionaries_repository));
|
||||
|
||||
auto models_repository = std::make_unique<ExternalLoaderXMLConfigRepository>(config(), "models_config");
|
||||
global_context->getExternalModelsLoader().addConfigRepository("", std::move(models_repository));
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
|
@ -155,7 +155,7 @@ namespace ErrorCodes
|
||||
extern const int NOT_FOUND_FUNCTION_ELEMENT_FOR_AGGREGATE = 147;
|
||||
extern const int NOT_FOUND_RELATION_ELEMENT_FOR_CONDITION = 148;
|
||||
extern const int NOT_FOUND_RHS_ELEMENT_FOR_CONDITION = 149;
|
||||
extern const int NO_ATTRIBUTES_LISTED = 150;
|
||||
extern const int EMPTY_LIST_OF_ATTRIBUTES_PASSED = 150;
|
||||
extern const int INDEX_OF_COLUMN_IN_SORT_CLAUSE_IS_OUT_OF_RANGE = 151;
|
||||
extern const int UNKNOWN_DIRECTION_OF_SORTING = 152;
|
||||
extern const int ILLEGAL_DIVISION = 153;
|
||||
@ -361,7 +361,7 @@ namespace ErrorCodes
|
||||
extern const int PART_IS_TEMPORARILY_LOCKED = 384;
|
||||
extern const int MULTIPLE_STREAMS_REQUIRED = 385;
|
||||
extern const int NO_COMMON_TYPE = 386;
|
||||
extern const int EXTERNAL_LOADABLE_ALREADY_EXISTS = 387;
|
||||
extern const int DICTIONARY_ALREADY_EXISTS = 387;
|
||||
extern const int CANNOT_ASSIGN_OPTIMIZE = 388;
|
||||
extern const int INSERT_WAS_DEDUPLICATED = 389;
|
||||
extern const int CANNOT_GET_CREATE_TABLE_QUERY = 390;
|
||||
@ -461,6 +461,9 @@ namespace ErrorCodes
|
||||
extern const int INTERNAL_REDIS_ERROR = 484;
|
||||
extern const int SCALAR_ALREADY_EXISTS = 485;
|
||||
extern const int UNKNOWN_SCALAR = 486;
|
||||
extern const int CANNOT_GET_CREATE_DICTIONARY_QUERY = 487;
|
||||
extern const int UNKNOWN_DICTIONARY = 488;
|
||||
extern const int INCORRECT_DICTIONARY_DEFINITION = 489;
|
||||
|
||||
extern const int KEEPER_EXCEPTION = 999;
|
||||
extern const int POCO_EXCEPTION = 1000;
|
||||
|
@ -68,7 +68,7 @@ protected:
|
||||
public:
|
||||
using Configuration = Poco::Util::AbstractConfiguration;
|
||||
|
||||
Context & context;
|
||||
const Context & context;
|
||||
const Configuration & config;
|
||||
|
||||
static constexpr inline auto DEFAULT_HOST = "localhost";
|
||||
@ -79,7 +79,7 @@ public:
|
||||
static constexpr inline auto IDENTIFIER_QUOTE_HANDLER = "/identifier_quote";
|
||||
static constexpr inline auto PING_OK_ANSWER = "Ok.";
|
||||
|
||||
XDBCBridgeHelper(Context & global_context_, const Poco::Timespan & http_timeout_, const std::string & connection_string_)
|
||||
XDBCBridgeHelper(const Context & global_context_, const Poco::Timespan & http_timeout_, const std::string & connection_string_)
|
||||
: http_timeout(http_timeout_), connection_string(connection_string_), context(global_context_), config(context.getConfigRef())
|
||||
{
|
||||
size_t bridge_port = config.getUInt(BridgeHelperMixin::configPrefix() + ".port", DEFAULT_PORT);
|
||||
|
@ -19,6 +19,7 @@ namespace ErrorCodes
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int CANNOT_GET_CREATE_TABLE_QUERY;
|
||||
extern const int SYNTAX_ERROR;
|
||||
extern const int UNSUPPORTED_METHOD;
|
||||
}
|
||||
|
||||
DatabaseDictionary::DatabaseDictionary(const String & name_)
|
||||
@ -27,32 +28,36 @@ DatabaseDictionary::DatabaseDictionary(const String & name_)
|
||||
{
|
||||
}
|
||||
|
||||
void DatabaseDictionary::loadTables(Context &, bool)
|
||||
void DatabaseDictionary::loadStoredObjects(Context &, bool)
|
||||
{
|
||||
}
|
||||
|
||||
Tables DatabaseDictionary::listTables(const Context & context, const FilterByNameFunction & filter_by_name)
|
||||
{
|
||||
Tables tables;
|
||||
ExternalLoader::Loadables loadables;
|
||||
ExternalLoader::LoadResults load_results;
|
||||
if (filter_by_name)
|
||||
{
|
||||
/// If `filter_by_name` is set, we iterate through all dictionaries with such names. That's why we need to load all of them.
|
||||
loadables = context.getExternalDictionariesLoader().loadAndGet(filter_by_name);
|
||||
context.getExternalDictionariesLoader().load(filter_by_name, load_results);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// If `filter_by_name` isn't set, we iterate through only already loaded dictionaries. We don't try to load all dictionaries in this case.
|
||||
loadables = context.getExternalDictionariesLoader().getCurrentlyLoadedObjects();
|
||||
load_results = context.getExternalDictionariesLoader().getCurrentLoadResults();
|
||||
}
|
||||
|
||||
for (const auto & loadable : loadables)
|
||||
for (const auto & [object_name, info]: load_results)
|
||||
{
|
||||
auto dict_ptr = std::static_pointer_cast<const IDictionaryBase>(loadable);
|
||||
auto dict_name = dict_ptr->getName();
|
||||
const DictionaryStructure & dictionary_structure = dict_ptr->getStructure();
|
||||
auto columns = StorageDictionary::getNamesAndTypes(dictionary_structure);
|
||||
tables[dict_name] = StorageDictionary::create(getDatabaseName(), dict_name, ColumnsDescription{columns}, context, true, dict_name);
|
||||
/// Load tables only from XML dictionaries, don't touch other
|
||||
if (info.object != nullptr && info.repository_name.empty())
|
||||
{
|
||||
auto dict_ptr = std::static_pointer_cast<const IDictionaryBase>(info.object);
|
||||
auto dict_name = dict_ptr->getName();
|
||||
const DictionaryStructure & dictionary_structure = dict_ptr->getStructure();
|
||||
auto columns = StorageDictionary::getNamesAndTypes(dictionary_structure);
|
||||
tables[dict_name] = StorageDictionary::create(getDatabaseName(), dict_name, ColumnsDescription{columns}, context, true, dict_name);
|
||||
}
|
||||
}
|
||||
return tables;
|
||||
}
|
||||
@ -64,6 +69,66 @@ bool DatabaseDictionary::isTableExist(
|
||||
return context.getExternalDictionariesLoader().getCurrentStatus(table_name) != ExternalLoader::Status::NOT_EXIST;
|
||||
}
|
||||
|
||||
|
||||
bool DatabaseDictionary::isDictionaryExist(
|
||||
const Context & /*context*/,
|
||||
const String & /*table_name*/) const
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
DatabaseDictionariesIteratorPtr DatabaseDictionary::getDictionariesIterator(
|
||||
const Context & /*context*/,
|
||||
const FilterByNameFunction & /*filter_by_dictionary_name*/)
|
||||
{
|
||||
return std::make_unique<DatabaseDictionariesSnapshotIterator>();
|
||||
}
|
||||
|
||||
|
||||
void DatabaseDictionary::createDictionary(
|
||||
const Context & /*context*/,
|
||||
const String & /*dictionary_name*/,
|
||||
const ASTPtr & /*query*/)
|
||||
{
|
||||
throw Exception("Dictionary engine doesn't support dictionaries.", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
||||
void DatabaseDictionary::removeDictionary(
|
||||
const Context & /*context*/,
|
||||
const String & /*table_name*/)
|
||||
{
|
||||
throw Exception("Dictionary engine doesn't support dictionaries.", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
||||
void DatabaseDictionary::attachDictionary(
|
||||
const String & /*dictionary_name*/, const Context & /*context*/, bool /*reload*/)
|
||||
{
|
||||
throw Exception("Dictionary engine doesn't support dictionaries.", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
||||
void DatabaseDictionary::detachDictionary(
|
||||
const String & /*dictionary_name*/, const Context & /*context*/, bool /*reload*/)
|
||||
{
|
||||
throw Exception("Dictionary engine doesn't support dictionaries.", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
||||
|
||||
ASTPtr DatabaseDictionary::tryGetCreateDictionaryQuery(
|
||||
const Context & /*context*/,
|
||||
const String & /*table_name*/) const
|
||||
{
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
|
||||
ASTPtr DatabaseDictionary::getCreateDictionaryQuery(
|
||||
const Context & /*context*/,
|
||||
const String & /*table_name*/) const
|
||||
{
|
||||
throw Exception("Dictionary engine doesn't support dictionaries.", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
||||
StoragePtr DatabaseDictionary::tryGetTable(
|
||||
const Context & context,
|
||||
const String & table_name) const
|
||||
@ -79,9 +144,9 @@ StoragePtr DatabaseDictionary::tryGetTable(
|
||||
return {};
|
||||
}
|
||||
|
||||
DatabaseIteratorPtr DatabaseDictionary::getIterator(const Context & context, const FilterByNameFunction & filter_by_name)
|
||||
DatabaseTablesIteratorPtr DatabaseDictionary::getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_name)
|
||||
{
|
||||
return std::make_unique<DatabaseSnapshotIterator>(listTables(context, filter_by_name));
|
||||
return std::make_unique<DatabaseTablesSnapshotIterator>(listTables(context, filter_by_name));
|
||||
}
|
||||
|
||||
bool DatabaseDictionary::empty(const Context & context) const
|
||||
@ -115,7 +180,7 @@ void DatabaseDictionary::removeTable(
|
||||
throw Exception("DatabaseDictionary: removeTable() is not supported", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
time_t DatabaseDictionary::getTableMetadataModificationTime(
|
||||
time_t DatabaseDictionary::getObjectMetadataModificationTime(
|
||||
const Context &,
|
||||
const String &)
|
||||
{
|
||||
|
@ -31,7 +31,7 @@ public:
|
||||
return "Dictionary";
|
||||
}
|
||||
|
||||
void loadTables(
|
||||
void loadStoredObjects(
|
||||
Context & context,
|
||||
bool has_force_restore_data_flag) override;
|
||||
|
||||
@ -39,11 +39,15 @@ public:
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
|
||||
bool isDictionaryExist(const Context & context, const String & table_name) const override;
|
||||
|
||||
StoragePtr tryGetTable(
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
|
||||
DatabaseIteratorPtr getIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) override;
|
||||
DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) override;
|
||||
|
||||
DatabaseDictionariesIteratorPtr getDictionariesIterator(const Context & context, const FilterByNameFunction & filter_by_dictionary_name = {}) override;
|
||||
|
||||
bool empty(const Context & context) const override;
|
||||
|
||||
@ -53,14 +57,20 @@ public:
|
||||
const StoragePtr & table,
|
||||
const ASTPtr & query) override;
|
||||
|
||||
void createDictionary(
|
||||
const Context & context, const String & dictionary_name, const ASTPtr & query) override;
|
||||
|
||||
void removeTable(
|
||||
const Context & context,
|
||||
const String & table_name) override;
|
||||
|
||||
void removeDictionary(const Context & context, const String & table_name) override;
|
||||
|
||||
void attachTable(const String & table_name, const StoragePtr & table) override;
|
||||
|
||||
StoragePtr detachTable(const String & table_name) override;
|
||||
|
||||
time_t getTableMetadataModificationTime(
|
||||
time_t getObjectMetadataModificationTime(
|
||||
const Context & context,
|
||||
const String & table_name) override;
|
||||
|
||||
@ -74,6 +84,15 @@ public:
|
||||
|
||||
ASTPtr getCreateDatabaseQuery(const Context & context) const override;
|
||||
|
||||
ASTPtr getCreateDictionaryQuery(const Context & context, const String & table_name) const override;
|
||||
|
||||
ASTPtr tryGetCreateDictionaryQuery(const Context & context, const String & table_name) const override;
|
||||
|
||||
|
||||
void attachDictionary(const String & dictionary_name, const Context & context, bool reload) override;
|
||||
|
||||
void detachDictionary(const String & dictionary_name, const Context & context, bool reload) override;
|
||||
|
||||
void shutdown() override;
|
||||
|
||||
private:
|
||||
|
@ -53,11 +53,13 @@ DatabasePtr DatabaseFactory::get(
|
||||
else if (engine_name == "MySQL")
|
||||
{
|
||||
const ASTFunction * engine = engine_define->engine;
|
||||
const auto & arguments = engine->arguments->children;
|
||||
|
||||
if (arguments.size() != 4)
|
||||
throw Exception("MySQL Database require mysql_hostname, mysql_database_name, mysql_username, mysql_password arguments.",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
if (!engine->arguments || engine->arguments->children.size() != 4)
|
||||
throw Exception(
|
||||
"MySQL Database require mysql_hostname, mysql_database_name, mysql_username, mysql_password arguments.",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
const auto & arguments = engine->arguments->children;
|
||||
|
||||
const auto & mysql_host_name = arguments[0]->as<ASTLiteral>()->value.safeGet<String>();
|
||||
const auto & mysql_database_name = arguments[1]->as<ASTLiteral>()->value.safeGet<String>();
|
||||
@ -74,11 +76,11 @@ DatabasePtr DatabaseFactory::get(
|
||||
else if (engine_name == "Lazy")
|
||||
{
|
||||
const ASTFunction * engine = engine_define->engine;
|
||||
const auto & arguments = engine->arguments->children;
|
||||
|
||||
if (arguments.size() != 1)
|
||||
throw Exception("Lazy database require cache_expiration_time_seconds argument.",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
if (!engine->arguments || engine->arguments->children.size() != 1)
|
||||
throw Exception("Lazy database require cache_expiration_time_seconds argument", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
const auto & arguments = engine->arguments->children;
|
||||
|
||||
const auto cache_expiration_time_seconds = arguments[0]->as<ASTLiteral>()->value.safeGet<UInt64>();
|
||||
return std::make_shared<DatabaseLazy>(database_name, metadata_path, cache_expiration_time_seconds, context);
|
||||
|
@ -29,8 +29,6 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
|
||||
static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768;
|
||||
|
||||
|
||||
DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_, time_t expiration_time_, const Context & context)
|
||||
: name(name_)
|
||||
@ -43,11 +41,11 @@ DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_,
|
||||
}
|
||||
|
||||
|
||||
void DatabaseLazy::loadTables(
|
||||
void DatabaseLazy::loadStoredObjects(
|
||||
Context & /* context */,
|
||||
bool /* has_force_restore_data_flag */)
|
||||
{
|
||||
DatabaseOnDisk::iterateTableFiles(*this, log, [this](const String & file_name)
|
||||
DatabaseOnDisk::iterateMetadataFiles(*this, log, [this](const String & file_name)
|
||||
{
|
||||
const std::string table_name = file_name.substr(0, file_name.size() - 4);
|
||||
attachTable(table_name, nullptr);
|
||||
@ -70,7 +68,16 @@ void DatabaseLazy::createTable(
|
||||
std::lock_guard lock(tables_mutex);
|
||||
auto it = tables_cache.find(table_name);
|
||||
if (it != tables_cache.end())
|
||||
it->second.metadata_modification_time = DatabaseOnDisk::getTableMetadataModificationTime(*this, table_name);
|
||||
it->second.metadata_modification_time = DatabaseOnDisk::getObjectMetadataModificationTime(*this, table_name);
|
||||
}
|
||||
|
||||
|
||||
void DatabaseLazy::createDictionary(
|
||||
const Context & /*context*/,
|
||||
const String & /*dictionary_name*/,
|
||||
const ASTPtr & /*query*/)
|
||||
{
|
||||
throw Exception("Lazy engine can be used only with *Log tables.", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
||||
|
||||
@ -82,6 +89,51 @@ void DatabaseLazy::removeTable(
|
||||
DatabaseOnDisk::removeTable(*this, context, table_name, log);
|
||||
}
|
||||
|
||||
void DatabaseLazy::removeDictionary(
|
||||
const Context & /*context*/,
|
||||
const String & /*table_name*/)
|
||||
{
|
||||
throw Exception("Lazy engine can be used only with *Log tables.", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
||||
ASTPtr DatabaseLazy::getCreateDictionaryQuery(
|
||||
const Context & /*context*/,
|
||||
const String & /*table_name*/) const
|
||||
{
|
||||
throw Exception("Lazy engine can be used only with *Log tables.", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
||||
ASTPtr DatabaseLazy::tryGetCreateDictionaryQuery(const Context & /*context*/, const String & /*table_name*/) const
|
||||
{
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
bool DatabaseLazy::isDictionaryExist(const Context & /*context*/, const String & /*table_name*/) const
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
DatabaseDictionariesIteratorPtr DatabaseLazy::getDictionariesIterator(
|
||||
const Context & /*context*/,
|
||||
const FilterByNameFunction & /*filter_by_dictionary_name*/)
|
||||
{
|
||||
return std::make_unique<DatabaseDictionariesSnapshotIterator>();
|
||||
}
|
||||
|
||||
void DatabaseLazy::attachDictionary(
|
||||
const String & /*dictionary_name*/,
|
||||
const Context & /*context*/,
|
||||
bool /*load*/)
|
||||
{
|
||||
throw Exception("Lazy engine can be used only with *Log tables.", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
||||
void DatabaseLazy::detachDictionary(const String & /*dictionary_name*/, const Context & /*context*/, bool /*reload*/)
|
||||
{
|
||||
throw Exception("Lazy engine can be used only with *Log tables.", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
||||
void DatabaseLazy::renameTable(
|
||||
const Context & context,
|
||||
const String & table_name,
|
||||
@ -94,7 +146,7 @@ void DatabaseLazy::renameTable(
|
||||
}
|
||||
|
||||
|
||||
time_t DatabaseLazy::getTableMetadataModificationTime(
|
||||
time_t DatabaseLazy::getObjectMetadataModificationTime(
|
||||
const Context & /* context */,
|
||||
const String & table_name)
|
||||
{
|
||||
@ -172,7 +224,7 @@ StoragePtr DatabaseLazy::tryGetTable(
|
||||
return loadTable(context, table_name);
|
||||
}
|
||||
|
||||
DatabaseIteratorPtr DatabaseLazy::getIterator(const Context & context, const FilterByNameFunction & filter_by_table_name)
|
||||
DatabaseTablesIteratorPtr DatabaseLazy::getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name)
|
||||
{
|
||||
std::lock_guard lock(tables_mutex);
|
||||
Strings filtered_tables;
|
||||
@ -198,7 +250,7 @@ void DatabaseLazy::attachTable(const String & table_name, const StoragePtr & tab
|
||||
|
||||
auto [it, inserted] = tables_cache.emplace(std::piecewise_construct,
|
||||
std::forward_as_tuple(table_name),
|
||||
std::forward_as_tuple(table, current_time, DatabaseOnDisk::getTableMetadataModificationTime(*this, table_name)));
|
||||
std::forward_as_tuple(table, current_time, DatabaseOnDisk::getObjectMetadataModificationTime(*this, table_name)));
|
||||
if (!inserted)
|
||||
throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
|
||||
|
||||
@ -267,9 +319,9 @@ String DatabaseLazy::getDatabaseName() const
|
||||
return name;
|
||||
}
|
||||
|
||||
String DatabaseLazy::getTableMetadataPath(const String & table_name) const
|
||||
String DatabaseLazy::getObjectMetadataPath(const String & table_name) const
|
||||
{
|
||||
return DatabaseOnDisk::getTableMetadataPath(*this, table_name);
|
||||
return DatabaseOnDisk::getObjectMetadataPath(*this, table_name);
|
||||
}
|
||||
|
||||
StoragePtr DatabaseLazy::loadTable(const Context & context, const String & table_name) const
|
||||
@ -280,31 +332,18 @@ StoragePtr DatabaseLazy::loadTable(const Context & context, const String & table
|
||||
|
||||
const String table_metadata_path = getMetadataPath() + "/" + escapeForFileName(table_name) + ".sql";
|
||||
|
||||
String s;
|
||||
{
|
||||
char in_buf[METADATA_FILE_BUFFER_SIZE];
|
||||
ReadBufferFromFile in(table_metadata_path, METADATA_FILE_BUFFER_SIZE, -1, in_buf);
|
||||
readStringUntilEOF(s, in);
|
||||
}
|
||||
|
||||
/** Empty files with metadata are generated after a rough restart of the server.
|
||||
* Remove these files to slightly reduce the work of the admins on startup.
|
||||
*/
|
||||
if (s.empty())
|
||||
{
|
||||
LOG_ERROR(log, "LoadTable: File " << table_metadata_path << " is empty. Removing.");
|
||||
Poco::File(table_metadata_path).remove();
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
try
|
||||
{
|
||||
String table_name_;
|
||||
StoragePtr table;
|
||||
Context context_copy(context); /// some tables can change context, but not LogTables
|
||||
std::tie(table_name_, table) = createTableFromDefinition(
|
||||
s, name, getDataPath(), context_copy, false, "in file " + table_metadata_path);
|
||||
if (!endsWith(table->getName(), "Log"))
|
||||
|
||||
auto ast = parseCreateQueryFromMetadataFile(table_metadata_path, log);
|
||||
if (ast)
|
||||
std::tie(table_name_, table) = createTableFromAST(
|
||||
ast->as<const ASTCreateQuery &>(), name, getDataPath(), context_copy, false);
|
||||
|
||||
if (!ast || !endsWith(table->getName(), "Log"))
|
||||
throw Exception("Only *Log tables can be used with Lazy database engine.", ErrorCodes::LOGICAL_ERROR);
|
||||
{
|
||||
std::lock_guard lock(tables_mutex);
|
||||
|
@ -2,11 +2,13 @@
|
||||
|
||||
#include <Databases/DatabasesCommon.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
class DatabaseLazyIterator;
|
||||
|
||||
/** Lazy engine of databases.
|
||||
@ -20,7 +22,7 @@ public:
|
||||
|
||||
String getEngineName() const override { return "Lazy"; }
|
||||
|
||||
void loadTables(
|
||||
void loadStoredObjects(
|
||||
Context & context,
|
||||
bool has_force_restore_data_flag) override;
|
||||
|
||||
@ -30,10 +32,19 @@ public:
|
||||
const StoragePtr & table,
|
||||
const ASTPtr & query) override;
|
||||
|
||||
void createDictionary(
|
||||
const Context & context,
|
||||
const String & dictionary_name,
|
||||
const ASTPtr & query) override;
|
||||
|
||||
void removeTable(
|
||||
const Context & context,
|
||||
const String & table_name) override;
|
||||
|
||||
void removeDictionary(
|
||||
const Context & context,
|
||||
const String & table_name) override;
|
||||
|
||||
void renameTable(
|
||||
const Context & context,
|
||||
const String & table_name,
|
||||
@ -49,7 +60,7 @@ public:
|
||||
const ConstraintsDescription & constraints,
|
||||
const ASTModifier & engine_modifier) override;
|
||||
|
||||
time_t getTableMetadataModificationTime(
|
||||
time_t getObjectMetadataModificationTime(
|
||||
const Context & context,
|
||||
const String & table_name) override;
|
||||
|
||||
@ -61,12 +72,20 @@ public:
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
|
||||
ASTPtr getCreateDictionaryQuery(
|
||||
const Context & context,
|
||||
const String & dictionary_name) const override;
|
||||
|
||||
ASTPtr tryGetCreateDictionaryQuery(
|
||||
const Context & context,
|
||||
const String & dictionary_name) const override;
|
||||
|
||||
ASTPtr getCreateDatabaseQuery(const Context & context) const override;
|
||||
|
||||
String getDataPath() const override;
|
||||
String getDatabaseName() const override;
|
||||
String getMetadataPath() const override;
|
||||
String getTableMetadataPath(const String & table_name) const override;
|
||||
String getObjectMetadataPath(const String & table_name) const override;
|
||||
|
||||
void drop() override;
|
||||
|
||||
@ -74,18 +93,28 @@ public:
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
|
||||
bool isDictionaryExist(
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
|
||||
StoragePtr tryGetTable(
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
|
||||
bool empty(const Context & context) const override;
|
||||
|
||||
DatabaseIteratorPtr getIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) override;
|
||||
DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) override;
|
||||
|
||||
DatabaseDictionariesIteratorPtr getDictionariesIterator(const Context & context, const FilterByNameFunction & filter_by_dictionary_name = {}) override;
|
||||
|
||||
void attachTable(const String & table_name, const StoragePtr & table) override;
|
||||
|
||||
StoragePtr detachTable(const String & table_name) override;
|
||||
|
||||
void attachDictionary(const String & dictionary_name, const Context & context, bool reload) override;
|
||||
|
||||
void detachDictionary(const String & dictionary_name, const Context & context, bool reload) override;
|
||||
|
||||
void shutdown() override;
|
||||
|
||||
~DatabaseLazy() override;
|
||||
@ -138,10 +167,13 @@ private:
|
||||
};
|
||||
|
||||
|
||||
class DatabaseLazyIterator final : public IDatabaseIterator
|
||||
class DatabaseLazyIterator final : public IDatabaseTablesIterator
|
||||
{
|
||||
public:
|
||||
DatabaseLazyIterator(DatabaseLazy & database_, const Context & context_, Strings && table_names_);
|
||||
DatabaseLazyIterator(
|
||||
DatabaseLazy & database_,
|
||||
const Context & context_,
|
||||
Strings && table_names_);
|
||||
|
||||
void next() override;
|
||||
bool isValid() const override;
|
||||
@ -155,5 +187,4 @@ private:
|
||||
Strings::const_iterator iterator;
|
||||
mutable StoragePtr current_storage;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -9,6 +9,8 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_GET_CREATE_TABLE_QUERY;
|
||||
extern const int CANNOT_GET_CREATE_DICTIONARY_QUERY;
|
||||
extern const int UNSUPPORTED_METHOD;
|
||||
}
|
||||
|
||||
DatabaseMemory::DatabaseMemory(String name_)
|
||||
@ -16,7 +18,7 @@ DatabaseMemory::DatabaseMemory(String name_)
|
||||
, log(&Logger::get("DatabaseMemory(" + name + ")"))
|
||||
{}
|
||||
|
||||
void DatabaseMemory::loadTables(
|
||||
void DatabaseMemory::loadStoredObjects(
|
||||
Context & /*context*/,
|
||||
bool /*has_force_restore_data_flag*/)
|
||||
{
|
||||
@ -32,6 +34,21 @@ void DatabaseMemory::createTable(
|
||||
attachTable(table_name, table);
|
||||
}
|
||||
|
||||
|
||||
void DatabaseMemory::attachDictionary(const String & /*name*/, const Context & /*context*/, bool /*reload*/)
|
||||
{
|
||||
throw Exception("There is no ATTACH DICTIONARY query for DatabaseMemory", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
||||
void DatabaseMemory::createDictionary(
|
||||
const Context & /*context*/,
|
||||
const String & /*dictionary_name*/,
|
||||
const ASTPtr & /*query*/)
|
||||
{
|
||||
throw Exception("There is no CREATE DICTIONARY query for DatabaseMemory", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
||||
|
||||
void DatabaseMemory::removeTable(
|
||||
const Context & /*context*/,
|
||||
const String & table_name)
|
||||
@ -39,9 +56,23 @@ void DatabaseMemory::removeTable(
|
||||
detachTable(table_name);
|
||||
}
|
||||
|
||||
time_t DatabaseMemory::getTableMetadataModificationTime(
|
||||
const Context &,
|
||||
const String &)
|
||||
|
||||
void DatabaseMemory::detachDictionary(const String & /*name*/, const Context & /*context*/, bool /*reload*/)
|
||||
{
|
||||
throw Exception("There is no DETACH DICTIONARY query for DatabaseMemory", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
||||
|
||||
void DatabaseMemory::removeDictionary(
|
||||
const Context & /*context*/,
|
||||
const String & /*dictionary_name*/)
|
||||
{
|
||||
throw Exception("There is no DROP DICTIONARY query for DatabaseMemory", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
||||
|
||||
time_t DatabaseMemory::getObjectMetadataModificationTime(
|
||||
const Context &, const String &)
|
||||
{
|
||||
return static_cast<time_t>(0);
|
||||
}
|
||||
@ -53,6 +84,15 @@ ASTPtr DatabaseMemory::getCreateTableQuery(
|
||||
throw Exception("There is no CREATE TABLE query for DatabaseMemory tables", ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY);
|
||||
}
|
||||
|
||||
|
||||
ASTPtr DatabaseMemory::getCreateDictionaryQuery(
|
||||
const Context &,
|
||||
const String &) const
|
||||
{
|
||||
throw Exception("There is no CREATE DICTIONARY query for DatabaseMemory dictionaries", ErrorCodes::CANNOT_GET_CREATE_DICTIONARY_QUERY);
|
||||
}
|
||||
|
||||
|
||||
ASTPtr DatabaseMemory::getCreateDatabaseQuery(
|
||||
const Context &) const
|
||||
{
|
||||
|
@ -23,7 +23,7 @@ public:
|
||||
|
||||
String getEngineName() const override { return "Memory"; }
|
||||
|
||||
void loadTables(
|
||||
void loadStoredObjects(
|
||||
Context & context,
|
||||
bool has_force_restore_data_flag) override;
|
||||
|
||||
@ -33,16 +33,35 @@ public:
|
||||
const StoragePtr & table,
|
||||
const ASTPtr & query) override;
|
||||
|
||||
void createDictionary(
|
||||
const Context & context,
|
||||
const String & dictionary_name,
|
||||
const ASTPtr & query) override;
|
||||
|
||||
void attachDictionary(
|
||||
const String & name,
|
||||
const Context & context,
|
||||
bool reload) override;
|
||||
|
||||
void removeTable(
|
||||
const Context & context,
|
||||
const String & table_name) override;
|
||||
|
||||
time_t getTableMetadataModificationTime(
|
||||
void removeDictionary(
|
||||
const Context & context,
|
||||
const String & table_name) override;
|
||||
const String & dictionary_name) override;
|
||||
|
||||
void detachDictionary(
|
||||
const String & name,
|
||||
const Context & context,
|
||||
bool reload) override;
|
||||
|
||||
time_t getObjectMetadataModificationTime(const Context & context, const String & table_name) override;
|
||||
|
||||
ASTPtr getCreateTableQuery(const Context & context, const String & table_name) const override;
|
||||
ASTPtr getCreateDictionaryQuery(const Context & context, const String & table_name) const override;
|
||||
ASTPtr tryGetCreateTableQuery(const Context &, const String &) const override { return nullptr; }
|
||||
ASTPtr tryGetCreateDictionaryQuery(const Context &, const String &) const override { return nullptr; }
|
||||
|
||||
ASTPtr getCreateDatabaseQuery(const Context & context) const override;
|
||||
|
||||
|
@ -64,7 +64,7 @@ bool DatabaseMySQL::empty(const Context &) const
|
||||
return local_tables_cache.empty();
|
||||
}
|
||||
|
||||
DatabaseIteratorPtr DatabaseMySQL::getIterator(const Context &, const FilterByNameFunction & filter_by_table_name)
|
||||
DatabaseTablesIteratorPtr DatabaseMySQL::getTablesIterator(const Context &, const FilterByNameFunction & filter_by_table_name)
|
||||
{
|
||||
Tables tables;
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
@ -75,7 +75,7 @@ DatabaseIteratorPtr DatabaseMySQL::getIterator(const Context &, const FilterByNa
|
||||
if (!filter_by_table_name || filter_by_table_name(local_table.first))
|
||||
tables[local_table.first] = local_table.second.storage;
|
||||
|
||||
return std::make_unique<DatabaseSnapshotIterator>(tables);
|
||||
return std::make_unique<DatabaseTablesSnapshotIterator>(tables);
|
||||
}
|
||||
|
||||
bool DatabaseMySQL::isTableExist(const Context & context, const String & name) const
|
||||
@ -107,7 +107,7 @@ ASTPtr DatabaseMySQL::tryGetCreateTableQuery(const Context &, const String & tab
|
||||
return local_tables_cache[table_name].create_table_query;
|
||||
}
|
||||
|
||||
time_t DatabaseMySQL::getTableMetadataModificationTime(const Context &, const String & table_name)
|
||||
time_t DatabaseMySQL::getObjectMetadataModificationTime(const Context &, const String & table_name)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <mysqlxx/Pool.h>
|
||||
#include <Databases/DatabasesCommon.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <memory>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -29,17 +30,32 @@ public:
|
||||
|
||||
bool empty(const Context & context) const override;
|
||||
|
||||
DatabaseIteratorPtr getIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) override;
|
||||
DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) override;
|
||||
|
||||
DatabaseDictionariesIteratorPtr getDictionariesIterator(const Context &, const FilterByNameFunction & = {}) override
|
||||
{
|
||||
return std::make_unique<DatabaseDictionariesSnapshotIterator>();
|
||||
}
|
||||
|
||||
ASTPtr getCreateDatabaseQuery(const Context & context) const override;
|
||||
|
||||
bool isTableExist(const Context & context, const String & name) const override;
|
||||
|
||||
bool isDictionaryExist(const Context &, const String &) const override { return false; }
|
||||
|
||||
StoragePtr tryGetTable(const Context & context, const String & name) const override;
|
||||
|
||||
ASTPtr tryGetCreateTableQuery(const Context & context, const String & name) const override;
|
||||
|
||||
time_t getTableMetadataModificationTime(const Context & context, const String & name) override;
|
||||
ASTPtr getCreateDictionaryQuery(const Context &, const String &) const override
|
||||
{
|
||||
throw Exception("MySQL database engine does not support dictionaries.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
ASTPtr tryGetCreateDictionaryQuery(const Context &, const String &) const override { return nullptr; }
|
||||
|
||||
|
||||
time_t getObjectMetadataModificationTime(const Context & context, const String & name) override;
|
||||
|
||||
void shutdown() override;
|
||||
|
||||
@ -48,7 +64,12 @@ public:
|
||||
throw Exception("MySQL database engine does not support detach table.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
void loadTables(Context &, bool) override
|
||||
void detachDictionary(const String &, const Context &, bool) override
|
||||
{
|
||||
throw Exception("MySQL database engine does not support detach dictionary.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
void loadStoredObjects(Context &, bool) override
|
||||
{
|
||||
/// do nothing
|
||||
}
|
||||
@ -58,16 +79,33 @@ public:
|
||||
throw Exception("MySQL database engine does not support remove table.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
void removeDictionary(const Context &, const String &) override
|
||||
{
|
||||
throw Exception("MySQL database engine does not support remove dictionary.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
|
||||
void attachTable(const String &, const StoragePtr &) override
|
||||
{
|
||||
throw Exception("MySQL database engine does not support attach table.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
void attachDictionary(const String &, const Context &, bool) override
|
||||
{
|
||||
throw Exception("MySQL database engine does not support attach dictionary.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
void createTable(const Context &, const String &, const StoragePtr &, const ASTPtr &) override
|
||||
{
|
||||
throw Exception("MySQL database engine does not support create table.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
void createDictionary(const Context &, const String &, const ASTPtr &) override
|
||||
{
|
||||
throw Exception("MySQL database engine does not support create dictionary.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
|
||||
private:
|
||||
struct MySQLStorageInfo
|
||||
{
|
||||
|
@ -1,37 +1,49 @@
|
||||
#include <Databases/DatabaseOnDisk.h>
|
||||
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
#include <Interpreters/ExternalDictionariesLoader.h>
|
||||
#include <Dictionaries/getDictionaryConfigurationFromAST.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
|
||||
#include <common/logger_useful.h>
|
||||
#include <Poco/DirectoryIterator.h>
|
||||
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768;
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_GET_CREATE_TABLE_QUERY;
|
||||
extern const int CANNOT_GET_CREATE_DICTIONARY_QUERY;
|
||||
extern const int FILE_DOESNT_EXIST;
|
||||
extern const int INCORRECT_FILE_NAME;
|
||||
extern const int SYNTAX_ERROR;
|
||||
extern const int TABLE_ALREADY_EXISTS;
|
||||
extern const int UNKNOWN_TABLE;
|
||||
extern const int DICTIONARY_ALREADY_EXISTS;
|
||||
extern const int EMPTY_LIST_OF_COLUMNS_PASSED;
|
||||
}
|
||||
|
||||
|
||||
namespace detail
|
||||
{
|
||||
String getTableMetadataPath(const String & base_path, const String & table_name)
|
||||
String getObjectMetadataPath(const String & base_path, const String & table_name)
|
||||
{
|
||||
return base_path + (endsWith(base_path, "/") ? "" : "/") + escapeForFileName(table_name) + ".sql";
|
||||
}
|
||||
@ -85,6 +97,107 @@ namespace detail
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
ASTPtr parseCreateQueryFromMetadataFile(const String & filepath, Poco::Logger * log)
|
||||
{
|
||||
String definition;
|
||||
{
|
||||
char in_buf[METADATA_FILE_BUFFER_SIZE];
|
||||
ReadBufferFromFile in(filepath, METADATA_FILE_BUFFER_SIZE, -1, in_buf);
|
||||
readStringUntilEOF(definition, in);
|
||||
}
|
||||
|
||||
/** Empty files with metadata are generated after a rough restart of the server.
|
||||
* Remove these files to slightly reduce the work of the admins on startup.
|
||||
*/
|
||||
if (definition.empty())
|
||||
{
|
||||
LOG_ERROR(log, "File " << filepath << " is empty. Removing.");
|
||||
Poco::File(filepath).remove();
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
ParserCreateQuery parser_create;
|
||||
ASTPtr result = parseQuery(parser_create, definition, "in file " + filepath, 0);
|
||||
return result;
|
||||
}
|
||||
|
||||
|
||||
|
||||
std::pair<String, StoragePtr> createTableFromAST(
|
||||
ASTCreateQuery ast_create_query,
|
||||
const String & database_name,
|
||||
const String & database_data_path,
|
||||
Context & context,
|
||||
bool has_force_restore_data_flag)
|
||||
{
|
||||
ast_create_query.attach = true;
|
||||
ast_create_query.database = database_name;
|
||||
|
||||
if (ast_create_query.as_table_function)
|
||||
{
|
||||
const auto & table_function = ast_create_query.as_table_function->as<ASTFunction &>();
|
||||
const auto & factory = TableFunctionFactory::instance();
|
||||
StoragePtr storage = factory.get(table_function.name, context)->execute(ast_create_query.as_table_function, context, ast_create_query.table);
|
||||
return {ast_create_query.table, storage};
|
||||
}
|
||||
/// We do not directly use `InterpreterCreateQuery::execute`, because
|
||||
/// - the database has not been created yet;
|
||||
/// - the code is simpler, since the query is already brought to a suitable form.
|
||||
if (!ast_create_query.columns_list || !ast_create_query.columns_list->columns)
|
||||
throw Exception("Missing definition of columns.", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED);
|
||||
|
||||
ColumnsDescription columns = InterpreterCreateQuery::getColumnsDescription(*ast_create_query.columns_list->columns, context);
|
||||
ConstraintsDescription constraints = InterpreterCreateQuery::getConstraintsDescription(ast_create_query.columns_list->constraints);
|
||||
|
||||
return
|
||||
{
|
||||
ast_create_query.table,
|
||||
StorageFactory::instance().get(
|
||||
ast_create_query,
|
||||
database_data_path, ast_create_query.table, database_name, context, context.getGlobalContext(),
|
||||
columns, constraints,
|
||||
true, has_force_restore_data_flag)
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
String getObjectDefinitionFromCreateQuery(const ASTPtr & query)
|
||||
{
|
||||
ASTPtr query_clone = query->clone();
|
||||
auto * create = query_clone->as<ASTCreateQuery>();
|
||||
|
||||
if (!create)
|
||||
{
|
||||
std::ostringstream query_stream;
|
||||
formatAST(*create, query_stream, true);
|
||||
throw Exception("Query '" + query_stream.str() + "' is not CREATE query", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
if (!create->is_dictionary)
|
||||
create->attach = true;
|
||||
|
||||
/// We remove everything that is not needed for ATTACH from the query.
|
||||
create->database.clear();
|
||||
create->as_database.clear();
|
||||
create->as_table.clear();
|
||||
create->if_not_exists = false;
|
||||
create->is_populate = false;
|
||||
create->replace_view = false;
|
||||
|
||||
/// For views it is necessary to save the SELECT query itself, for the rest - on the contrary
|
||||
if (!create->is_view && !create->is_materialized_view && !create->is_live_view)
|
||||
create->select = nullptr;
|
||||
|
||||
create->format = nullptr;
|
||||
create->out_file = nullptr;
|
||||
|
||||
std::ostringstream statement_stream;
|
||||
formatAST(*create, statement_stream, false);
|
||||
statement_stream << '\n';
|
||||
return statement_stream.str();
|
||||
}
|
||||
|
||||
void DatabaseOnDisk::createTable(
|
||||
IDatabase & database,
|
||||
const Context & context,
|
||||
@ -106,15 +219,19 @@ void DatabaseOnDisk::createTable(
|
||||
/// A race condition would be possible if a table with the same name is simultaneously created using CREATE and using ATTACH.
|
||||
/// But there is protection from it - see using DDLGuard in InterpreterCreateQuery.
|
||||
|
||||
if (database.isDictionaryExist(context, table_name))
|
||||
throw Exception("Dictionary " + backQuote(database.getDatabaseName()) + "." + backQuote(table_name) + " already exists.",
|
||||
ErrorCodes::DICTIONARY_ALREADY_EXISTS);
|
||||
|
||||
if (database.isTableExist(context, table_name))
|
||||
throw Exception("Table " + backQuote(database.getDatabaseName()) + "." + backQuote(table_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
|
||||
|
||||
String table_metadata_path = database.getTableMetadataPath(table_name);
|
||||
String table_metadata_path = database.getObjectMetadataPath(table_name);
|
||||
String table_metadata_tmp_path = table_metadata_path + ".tmp";
|
||||
String statement;
|
||||
|
||||
{
|
||||
statement = getTableDefinitionFromCreateQuery(query);
|
||||
statement = getObjectDefinitionFromCreateQuery(query);
|
||||
|
||||
/// Exclusive flags guarantees, that table is not created right now in another thread. Otherwise, exception will be thrown.
|
||||
WriteBufferFromFile out(table_metadata_tmp_path, statement.size(), O_WRONLY | O_CREAT | O_EXCL);
|
||||
@ -141,6 +258,70 @@ void DatabaseOnDisk::createTable(
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void DatabaseOnDisk::createDictionary(
|
||||
IDatabase & database,
|
||||
const Context & context,
|
||||
const String & dictionary_name,
|
||||
const ASTPtr & query)
|
||||
{
|
||||
const auto & settings = context.getSettingsRef();
|
||||
|
||||
/** The code is based on the assumption that all threads share the same order of operations
|
||||
* - creating the .sql.tmp file;
|
||||
* - adding a dictionary to `dictionaries`;
|
||||
* - rename .sql.tmp to .sql.
|
||||
*/
|
||||
|
||||
/// A race condition would be possible if a dictionary with the same name is simultaneously created using CREATE and using ATTACH.
|
||||
/// But there is protection from it - see using DDLGuard in InterpreterCreateQuery.
|
||||
if (database.isDictionaryExist(context, dictionary_name))
|
||||
throw Exception("Dictionary " + backQuote(database.getDatabaseName()) + "." + backQuote(dictionary_name) + " already exists.", ErrorCodes::DICTIONARY_ALREADY_EXISTS);
|
||||
|
||||
if (database.isTableExist(context, dictionary_name))
|
||||
throw Exception("Table " + backQuote(database.getDatabaseName()) + "." + backQuote(dictionary_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
|
||||
|
||||
|
||||
String dictionary_metadata_path = database.getObjectMetadataPath(dictionary_name);
|
||||
String dictionary_metadata_tmp_path = dictionary_metadata_path + ".tmp";
|
||||
String statement;
|
||||
|
||||
{
|
||||
statement = getObjectDefinitionFromCreateQuery(query);
|
||||
|
||||
/// Exclusive flags guarantees, that table is not created right now in another thread. Otherwise, exception will be thrown.
|
||||
WriteBufferFromFile out(dictionary_metadata_tmp_path, statement.size(), O_WRONLY | O_CREAT | O_EXCL);
|
||||
writeString(statement, out);
|
||||
out.next();
|
||||
if (settings.fsync_metadata)
|
||||
out.sync();
|
||||
out.close();
|
||||
}
|
||||
|
||||
try
|
||||
{
|
||||
/// Do not load it now because we want more strict loading
|
||||
database.attachDictionary(dictionary_name, context, false);
|
||||
/// Load dictionary
|
||||
bool lazy_load = context.getConfigRef().getBool("dictionaries_lazy_load", true);
|
||||
String dict_name = database.getDatabaseName() + "." + dictionary_name;
|
||||
context.getExternalDictionariesLoader().addDictionaryWithConfig(
|
||||
dict_name, database.getDatabaseName(), query->as<const ASTCreateQuery &>(), !lazy_load);
|
||||
|
||||
/// If it was ATTACH query and file with dictionary metadata already exist
|
||||
/// (so, ATTACH is done after DETACH), then rename atomically replaces old file with new one.
|
||||
Poco::File(dictionary_metadata_tmp_path).renameTo(dictionary_metadata_path);
|
||||
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
database.detachDictionary(dictionary_name, context);
|
||||
Poco::File(dictionary_metadata_tmp_path).remove();
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void DatabaseOnDisk::removeTable(
|
||||
IDatabase & database,
|
||||
const Context & /* context */,
|
||||
@ -149,7 +330,7 @@ void DatabaseOnDisk::removeTable(
|
||||
{
|
||||
StoragePtr res = database.detachTable(table_name);
|
||||
|
||||
String table_metadata_path = database.getTableMetadataPath(table_name);
|
||||
String table_metadata_path = database.getObjectMetadataPath(table_name);
|
||||
|
||||
try
|
||||
{
|
||||
@ -171,12 +352,39 @@ void DatabaseOnDisk::removeTable(
|
||||
}
|
||||
}
|
||||
|
||||
ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(const IDatabase & database, const Context & context,
|
||||
const String & table_name, bool throw_on_error)
|
||||
|
||||
void DatabaseOnDisk::removeDictionary(
|
||||
IDatabase & database,
|
||||
const Context & context,
|
||||
const String & dictionary_name,
|
||||
Poco::Logger * /*log*/)
|
||||
{
|
||||
database.detachDictionary(dictionary_name, context);
|
||||
|
||||
String dictionary_metadata_path = database.getObjectMetadataPath(dictionary_name);
|
||||
|
||||
try
|
||||
{
|
||||
Poco::File(dictionary_metadata_path).remove();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// If remove was not possible for some reason
|
||||
database.attachDictionary(dictionary_name, context);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(
|
||||
const IDatabase & database,
|
||||
const Context & context,
|
||||
const String & table_name,
|
||||
bool throw_on_error)
|
||||
{
|
||||
ASTPtr ast;
|
||||
|
||||
auto table_metadata_path = detail::getTableMetadataPath(database.getMetadataPath(), table_name);
|
||||
auto table_metadata_path = detail::getObjectMetadataPath(database.getMetadataPath(), table_name);
|
||||
ast = detail::getCreateQueryFromMetadata(table_metadata_path, database.getDatabaseName(), throw_on_error);
|
||||
if (!ast && throw_on_error)
|
||||
{
|
||||
@ -193,6 +401,30 @@ ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(const IDatabase & database, const
|
||||
return ast;
|
||||
}
|
||||
|
||||
|
||||
ASTPtr DatabaseOnDisk::getCreateDictionaryQueryImpl(
|
||||
const IDatabase & database,
|
||||
const Context & context,
|
||||
const String & dictionary_name,
|
||||
bool throw_on_error)
|
||||
{
|
||||
ASTPtr ast;
|
||||
|
||||
auto dictionary_metadata_path = detail::getObjectMetadataPath(database.getMetadataPath(), dictionary_name);
|
||||
ast = detail::getCreateQueryFromMetadata(dictionary_metadata_path, database.getDatabaseName(), throw_on_error);
|
||||
if (!ast && throw_on_error)
|
||||
{
|
||||
/// Handle system.* tables for which there are no table.sql files.
|
||||
bool has_dictionary = database.isDictionaryExist(context, dictionary_name);
|
||||
|
||||
auto msg = has_dictionary ? "There is no CREATE DICTIONARY query for table " : "There is no metadata file for dictionary ";
|
||||
|
||||
throw Exception(msg + backQuote(dictionary_name), ErrorCodes::CANNOT_GET_CREATE_DICTIONARY_QUERY);
|
||||
}
|
||||
|
||||
return ast;
|
||||
}
|
||||
|
||||
ASTPtr DatabaseOnDisk::getCreateTableQuery(const IDatabase & database, const Context & context, const String & table_name)
|
||||
{
|
||||
return getCreateTableQueryImpl(database, context, table_name, true);
|
||||
@ -203,6 +435,17 @@ ASTPtr DatabaseOnDisk::tryGetCreateTableQuery(const IDatabase & database, const
|
||||
return getCreateTableQueryImpl(database, context, table_name, false);
|
||||
}
|
||||
|
||||
|
||||
ASTPtr DatabaseOnDisk::getCreateDictionaryQuery(const IDatabase & database, const Context & context, const String & dictionary_name)
|
||||
{
|
||||
return getCreateDictionaryQueryImpl(database, context, dictionary_name, true);
|
||||
}
|
||||
|
||||
ASTPtr DatabaseOnDisk::tryGetCreateDictionaryQuery(const IDatabase & database, const Context & context, const String & dictionary_name)
|
||||
{
|
||||
return getCreateDictionaryQueryImpl(database, context, dictionary_name, false);
|
||||
}
|
||||
|
||||
ASTPtr DatabaseOnDisk::getCreateDatabaseQuery(const IDatabase & database, const Context & /*context*/)
|
||||
{
|
||||
ASTPtr ast;
|
||||
@ -226,29 +469,25 @@ void DatabaseOnDisk::drop(const IDatabase & database)
|
||||
Poco::File(database.getMetadataPath()).remove(false);
|
||||
}
|
||||
|
||||
String DatabaseOnDisk::getTableMetadataPath(const IDatabase & database, const String & table_name)
|
||||
String DatabaseOnDisk::getObjectMetadataPath(const IDatabase & database, const String & table_name)
|
||||
{
|
||||
return detail::getTableMetadataPath(database.getMetadataPath(), table_name);
|
||||
return detail::getObjectMetadataPath(database.getMetadataPath(), table_name);
|
||||
}
|
||||
|
||||
time_t DatabaseOnDisk::getTableMetadataModificationTime(
|
||||
time_t DatabaseOnDisk::getObjectMetadataModificationTime(
|
||||
const IDatabase & database,
|
||||
const String & table_name)
|
||||
{
|
||||
String table_metadata_path = getTableMetadataPath(database, table_name);
|
||||
String table_metadata_path = getObjectMetadataPath(database, table_name);
|
||||
Poco::File meta_file(table_metadata_path);
|
||||
|
||||
if (meta_file.exists())
|
||||
{
|
||||
return meta_file.getLastModified().epochTime();
|
||||
}
|
||||
else
|
||||
{
|
||||
return static_cast<time_t>(0);
|
||||
}
|
||||
}
|
||||
|
||||
void DatabaseOnDisk::iterateTableFiles(const IDatabase & database, Poco::Logger * log, const IteratingFunction & iterating_function)
|
||||
void DatabaseOnDisk::iterateMetadataFiles(const IDatabase & database, Poco::Logger * log, const IteratingFunction & iterating_function)
|
||||
{
|
||||
Poco::DirectoryIterator dir_end;
|
||||
for (Poco::DirectoryIterator dir_it(database.getMetadataPath()); dir_it != dir_end; ++dir_it)
|
||||
@ -265,11 +504,11 @@ void DatabaseOnDisk::iterateTableFiles(const IDatabase & database, Poco::Logger
|
||||
static const char * tmp_drop_ext = ".sql.tmp_drop";
|
||||
if (endsWith(dir_it.name(), tmp_drop_ext))
|
||||
{
|
||||
const std::string table_name = dir_it.name().substr(0, dir_it.name().size() - strlen(tmp_drop_ext));
|
||||
if (Poco::File(database.getDataPath() + '/' + table_name).exists())
|
||||
const std::string object_name = dir_it.name().substr(0, dir_it.name().size() - strlen(tmp_drop_ext));
|
||||
if (Poco::File(database.getDataPath() + '/' + object_name).exists())
|
||||
{
|
||||
Poco::File(dir_it->path()).renameTo(table_name + ".sql");
|
||||
LOG_WARNING(log, "Table " << backQuote(table_name) << " was not dropped previously");
|
||||
Poco::File(dir_it->path()).renameTo(object_name + ".sql");
|
||||
LOG_WARNING(log, "Object " << backQuote(object_name) << " was not dropped previously");
|
||||
}
|
||||
else
|
||||
{
|
||||
|
@ -13,12 +13,28 @@ namespace DB
|
||||
|
||||
namespace detail
|
||||
{
|
||||
String getTableMetadataPath(const String & base_path, const String & table_name);
|
||||
String getObjectMetadataPath(const String & base_path, const String & dictionary_name);
|
||||
String getDatabaseMetadataPath(const String & base_path);
|
||||
ASTPtr getQueryFromMetadata(const String & metadata_path, bool throw_on_error = true);
|
||||
ASTPtr getCreateQueryFromMetadata(const String & metadata_path, const String & database, bool throw_on_error);
|
||||
}
|
||||
|
||||
ASTPtr parseCreateQueryFromMetadataFile(const String & filepath, Poco::Logger * log);
|
||||
|
||||
std::pair<String, StoragePtr> createTableFromAST(
|
||||
ASTCreateQuery ast_create_query,
|
||||
const String & database_name,
|
||||
const String & database_data_path,
|
||||
Context & context,
|
||||
bool has_force_restore_data_flag);
|
||||
|
||||
/** Get the row with the table definition based on the CREATE query.
|
||||
* It is an ATTACH query that you can execute to create a table from the correspondent database.
|
||||
* See the implementation.
|
||||
*/
|
||||
String getObjectDefinitionFromCreateQuery(const ASTPtr & query);
|
||||
|
||||
|
||||
/* Class to provide basic operations with tables when metadata is stored on disk in .sql files.
|
||||
*/
|
||||
class DatabaseOnDisk
|
||||
@ -31,12 +47,24 @@ public:
|
||||
const StoragePtr & table,
|
||||
const ASTPtr & query);
|
||||
|
||||
static void createDictionary(
|
||||
IDatabase & database,
|
||||
const Context & context,
|
||||
const String & dictionary_name,
|
||||
const ASTPtr & query);
|
||||
|
||||
static void removeTable(
|
||||
IDatabase & database,
|
||||
const Context & context,
|
||||
const String & table_name,
|
||||
Poco::Logger * log);
|
||||
|
||||
static void removeDictionary(
|
||||
IDatabase & database,
|
||||
const Context & context,
|
||||
const String & dictionary_name,
|
||||
Poco::Logger * log);
|
||||
|
||||
template <typename Database>
|
||||
static void renameTable(
|
||||
IDatabase & database,
|
||||
@ -56,23 +84,33 @@ public:
|
||||
const Context & context,
|
||||
const String & table_name);
|
||||
|
||||
static ASTPtr getCreateDictionaryQuery(
|
||||
const IDatabase & database,
|
||||
const Context & context,
|
||||
const String & dictionary_name);
|
||||
|
||||
static ASTPtr tryGetCreateDictionaryQuery(
|
||||
const IDatabase & database,
|
||||
const Context & context,
|
||||
const String & dictionary_name);
|
||||
|
||||
static ASTPtr getCreateDatabaseQuery(
|
||||
const IDatabase & database,
|
||||
const Context & context);
|
||||
|
||||
static void drop(const IDatabase & database);
|
||||
|
||||
static String getTableMetadataPath(
|
||||
static String getObjectMetadataPath(
|
||||
const IDatabase & database,
|
||||
const String & table_name);
|
||||
const String & object_name);
|
||||
|
||||
static time_t getTableMetadataModificationTime(
|
||||
static time_t getObjectMetadataModificationTime(
|
||||
const IDatabase & database,
|
||||
const String & table_name);
|
||||
const String & object_name);
|
||||
|
||||
|
||||
using IteratingFunction = std::function<void(const String &)>;
|
||||
static void iterateTableFiles(const IDatabase & database, Poco::Logger * log, const IteratingFunction & iterating_function);
|
||||
static void iterateMetadataFiles(const IDatabase & database, Poco::Logger * log, const IteratingFunction & iterating_function);
|
||||
|
||||
private:
|
||||
static ASTPtr getCreateTableQueryImpl(
|
||||
@ -80,6 +118,12 @@ private:
|
||||
const Context & context,
|
||||
const String & table_name,
|
||||
bool throw_on_error);
|
||||
|
||||
static ASTPtr getCreateDictionaryQueryImpl(
|
||||
const IDatabase & database,
|
||||
const Context & context,
|
||||
const String & dictionary_name,
|
||||
bool throw_on_error);
|
||||
};
|
||||
|
||||
|
||||
@ -126,7 +170,7 @@ void DatabaseOnDisk::renameTable(
|
||||
throw Exception{Exception::CreateFromPoco, e};
|
||||
}
|
||||
|
||||
ASTPtr ast = detail::getQueryFromMetadata(detail::getTableMetadataPath(database.getMetadataPath(), table_name));
|
||||
ASTPtr ast = detail::getQueryFromMetadata(detail::getObjectMetadataPath(database.getMetadataPath(), table_name));
|
||||
if (!ast)
|
||||
throw Exception("There is no metadata file for table " + backQuote(table_name) + ".", ErrorCodes::FILE_DOESNT_EXIST);
|
||||
ast->as<ASTCreateQuery &>().table = to_table_name;
|
||||
|
@ -11,10 +11,17 @@
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
#include <Interpreters/ExternalLoaderDatabaseConfigRepository.h>
|
||||
#include <Interpreters/ExternalDictionariesLoader.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/ParserDictionary.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Dictionaries/DictionaryFactory.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
|
||||
#include <Poco/DirectoryIterator.h>
|
||||
#include <Poco/Event.h>
|
||||
@ -33,59 +40,64 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_CREATE_TABLE_FROM_METADATA;
|
||||
extern const int CANNOT_CREATE_DICTIONARY_FROM_METADATA;
|
||||
extern const int EMPTY_LIST_OF_COLUMNS_PASSED;
|
||||
extern const int CANNOT_PARSE_TEXT;
|
||||
extern const int EMPTY_LIST_OF_ATTRIBUTES_PASSED;
|
||||
}
|
||||
|
||||
|
||||
static constexpr size_t PRINT_MESSAGE_EACH_N_TABLES = 256;
|
||||
static constexpr size_t PRINT_MESSAGE_EACH_N_OBJECTS = 256;
|
||||
static constexpr size_t PRINT_MESSAGE_EACH_N_SECONDS = 5;
|
||||
static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768;
|
||||
|
||||
static void loadTable(
|
||||
Context & context,
|
||||
const String & database_metadata_path,
|
||||
DatabaseOrdinary & database,
|
||||
const String & database_name,
|
||||
const String & database_data_path,
|
||||
const String & file_name,
|
||||
bool has_force_restore_data_flag)
|
||||
namespace
|
||||
{
|
||||
Logger * log = &Logger::get("loadTable");
|
||||
|
||||
const String table_metadata_path = database_metadata_path + "/" + file_name;
|
||||
|
||||
String s;
|
||||
void loadObject(
|
||||
Context & context,
|
||||
const ASTCreateQuery & query,
|
||||
DatabaseOrdinary & database,
|
||||
const String database_data_path,
|
||||
const String & database_name,
|
||||
bool has_force_restore_data_flag)
|
||||
try
|
||||
{
|
||||
if (query.is_dictionary)
|
||||
{
|
||||
char in_buf[METADATA_FILE_BUFFER_SIZE];
|
||||
ReadBufferFromFile in(table_metadata_path, METADATA_FILE_BUFFER_SIZE, -1, in_buf);
|
||||
readStringUntilEOF(s, in);
|
||||
String dictionary_name = query.table;
|
||||
database.attachDictionary(dictionary_name, context, false);
|
||||
}
|
||||
|
||||
/** Empty files with metadata are generated after a rough restart of the server.
|
||||
* Remove these files to slightly reduce the work of the admins on startup.
|
||||
*/
|
||||
if (s.empty())
|
||||
{
|
||||
LOG_ERROR(log, "File " << table_metadata_path << " is empty. Removing.");
|
||||
Poco::File(table_metadata_path).remove();
|
||||
return;
|
||||
}
|
||||
|
||||
try
|
||||
else
|
||||
{
|
||||
String table_name;
|
||||
StoragePtr table;
|
||||
std::tie(table_name, table) = createTableFromDefinition(
|
||||
s, database_name, database_data_path, context, has_force_restore_data_flag, "in file " + table_metadata_path);
|
||||
std::tie(table_name, table)
|
||||
= createTableFromAST(query, database_name, database_data_path, context, has_force_restore_data_flag);
|
||||
database.attachTable(table_name, table);
|
||||
}
|
||||
catch (const Exception & e)
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
throw Exception(
|
||||
"Cannot create object '" + query.table + "' from query " + serializeAST(query) + ", error: " + e.displayText() + ", stack trace:\n"
|
||||
+ e.getStackTrace().toString(),
|
||||
ErrorCodes::CANNOT_CREATE_TABLE_FROM_METADATA);
|
||||
}
|
||||
|
||||
|
||||
void logAboutProgress(Poco::Logger * log, size_t processed, size_t total, AtomicStopwatch & watch)
|
||||
{
|
||||
if (processed % PRINT_MESSAGE_EACH_N_OBJECTS == 0 || watch.compareAndRestart(PRINT_MESSAGE_EACH_N_SECONDS))
|
||||
{
|
||||
throw Exception("Cannot create table from metadata file " + table_metadata_path + ", error: " + e.displayText() +
|
||||
", stack trace:\n" + e.getStackTrace().toString(),
|
||||
ErrorCodes::CANNOT_CREATE_TABLE_FROM_METADATA);
|
||||
LOG_INFO(log, std::fixed << std::setprecision(2) << processed * 100.0 / total << "%");
|
||||
watch.restart();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
DatabaseOrdinary::DatabaseOrdinary(String name_, const String & metadata_path_, const Context & context)
|
||||
: DatabaseWithOwnTablesBase(std::move(name_))
|
||||
@ -97,57 +109,78 @@ DatabaseOrdinary::DatabaseOrdinary(String name_, const String & metadata_path_,
|
||||
}
|
||||
|
||||
|
||||
void DatabaseOrdinary::loadTables(
|
||||
void DatabaseOrdinary::loadStoredObjects(
|
||||
Context & context,
|
||||
bool has_force_restore_data_flag)
|
||||
{
|
||||
using FileNames = std::vector<std::string>;
|
||||
FileNames file_names;
|
||||
|
||||
DatabaseOnDisk::iterateTableFiles(*this, log, [&file_names](const String & file_name)
|
||||
{
|
||||
file_names.push_back(file_name);
|
||||
});
|
||||
|
||||
if (file_names.empty())
|
||||
return;
|
||||
|
||||
/** Tables load faster if they are loaded in sorted (by name) order.
|
||||
* Otherwise (for the ext4 filesystem), `DirectoryIterator` iterates through them in some order,
|
||||
* which does not correspond to order tables creation and does not correspond to order of their location on disk.
|
||||
*/
|
||||
std::sort(file_names.begin(), file_names.end());
|
||||
using FileNames = std::map<std::string, ASTPtr>;
|
||||
FileNames file_names;
|
||||
|
||||
const size_t total_tables = file_names.size();
|
||||
LOG_INFO(log, "Total " << total_tables << " tables.");
|
||||
size_t total_dictionaries = 0;
|
||||
DatabaseOnDisk::iterateMetadataFiles(*this, log, [&file_names, &total_dictionaries, this](const String & file_name)
|
||||
{
|
||||
String full_path = metadata_path + "/" + file_name;
|
||||
try
|
||||
{
|
||||
auto ast = parseCreateQueryFromMetadataFile(full_path, log);
|
||||
if (ast)
|
||||
{
|
||||
auto * create_query = ast->as<ASTCreateQuery>();
|
||||
file_names[file_name] = ast;
|
||||
total_dictionaries += create_query->is_dictionary;
|
||||
}
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
throw Exception(
|
||||
"Cannot parse definition from metadata file " + full_path + ", error: " + e.displayText() + ", stack trace:\n"
|
||||
+ e.getStackTrace().toString(), ErrorCodes::CANNOT_PARSE_TEXT);
|
||||
}
|
||||
|
||||
});
|
||||
|
||||
size_t total_tables = file_names.size() - total_dictionaries;
|
||||
|
||||
LOG_INFO(log, "Total " << total_tables << " tables and " << total_dictionaries << " dictionaries.");
|
||||
|
||||
AtomicStopwatch watch;
|
||||
std::atomic<size_t> tables_processed {0};
|
||||
std::atomic<size_t> tables_processed{0};
|
||||
std::atomic<size_t> dictionaries_processed{0};
|
||||
|
||||
auto loadOneTable = [&](const String & table)
|
||||
auto loadOneObject = [&](const ASTCreateQuery & query)
|
||||
{
|
||||
loadTable(context, getMetadataPath(), *this, getDatabaseName(), getDataPath(), table, has_force_restore_data_flag);
|
||||
loadObject(context, query, *this, getDataPath(), getDatabaseName(), has_force_restore_data_flag);
|
||||
|
||||
/// Messages, so that it's not boring to wait for the server to load for a long time.
|
||||
if (++tables_processed % PRINT_MESSAGE_EACH_N_TABLES == 0
|
||||
|| watch.compareAndRestart(PRINT_MESSAGE_EACH_N_SECONDS))
|
||||
{
|
||||
LOG_INFO(log, std::fixed << std::setprecision(2) << tables_processed * 100.0 / total_tables << "%");
|
||||
watch.restart();
|
||||
}
|
||||
if (query.is_dictionary)
|
||||
logAboutProgress(log, ++dictionaries_processed, total_dictionaries, watch);
|
||||
else
|
||||
logAboutProgress(log, ++tables_processed, total_tables, watch);
|
||||
};
|
||||
|
||||
ThreadPool pool(SettingMaxThreads().getAutoValue());
|
||||
|
||||
for (const auto & file_name : file_names)
|
||||
for (const auto & name_with_query : file_names)
|
||||
{
|
||||
pool.scheduleOrThrowOnError([&]() { loadOneTable(file_name); });
|
||||
pool.scheduleOrThrowOnError([&]() { loadOneObject(name_with_query.second->as<const ASTCreateQuery &>()); });
|
||||
}
|
||||
|
||||
pool.wait();
|
||||
|
||||
/// After all tables was basically initialized, startup them.
|
||||
startupTables(pool);
|
||||
|
||||
/// Add database as repository
|
||||
auto dictionaries_repository = std::make_unique<ExternalLoaderDatabaseConfigRepository>(shared_from_this(), context);
|
||||
auto & external_loader = context.getExternalDictionariesLoader();
|
||||
external_loader.addConfigRepository(getDatabaseName(), std::move(dictionaries_repository));
|
||||
bool lazy_load = context.getConfigRef().getBool("dictionaries_lazy_load", true);
|
||||
external_loader.reload(!lazy_load);
|
||||
}
|
||||
|
||||
|
||||
@ -160,18 +193,12 @@ void DatabaseOrdinary::startupTables(ThreadPool & thread_pool)
|
||||
return;
|
||||
|
||||
AtomicStopwatch watch;
|
||||
std::atomic<size_t> tables_processed {0};
|
||||
std::atomic<size_t> tables_processed{0};
|
||||
|
||||
auto startupOneTable = [&](const StoragePtr & table)
|
||||
{
|
||||
table->startup();
|
||||
|
||||
if (++tables_processed % PRINT_MESSAGE_EACH_N_TABLES == 0
|
||||
|| watch.compareAndRestart(PRINT_MESSAGE_EACH_N_SECONDS))
|
||||
{
|
||||
LOG_INFO(log, std::fixed << std::setprecision(2) << tables_processed * 100.0 / total_tables << "%");
|
||||
watch.restart();
|
||||
}
|
||||
logAboutProgress(log, ++tables_processed, total_tables, watch);
|
||||
};
|
||||
|
||||
try
|
||||
@ -187,7 +214,6 @@ void DatabaseOrdinary::startupTables(ThreadPool & thread_pool)
|
||||
thread_pool.wait();
|
||||
}
|
||||
|
||||
|
||||
void DatabaseOrdinary::createTable(
|
||||
const Context & context,
|
||||
const String & table_name,
|
||||
@ -197,6 +223,13 @@ void DatabaseOrdinary::createTable(
|
||||
DatabaseOnDisk::createTable(*this, context, table_name, table, query);
|
||||
}
|
||||
|
||||
void DatabaseOrdinary::createDictionary(
|
||||
const Context & context,
|
||||
const String & dictionary_name,
|
||||
const ASTPtr & query)
|
||||
{
|
||||
DatabaseOnDisk::createDictionary(*this, context, dictionary_name, query);
|
||||
}
|
||||
|
||||
void DatabaseOrdinary::removeTable(
|
||||
const Context & context,
|
||||
@ -205,6 +238,13 @@ void DatabaseOrdinary::removeTable(
|
||||
DatabaseOnDisk::removeTable(*this, context, table_name, log);
|
||||
}
|
||||
|
||||
void DatabaseOrdinary::removeDictionary(
|
||||
const Context & context,
|
||||
const String & table_name)
|
||||
{
|
||||
DatabaseOnDisk::removeDictionary(*this, context, table_name, log);
|
||||
}
|
||||
|
||||
void DatabaseOrdinary::renameTable(
|
||||
const Context & context,
|
||||
const String & table_name,
|
||||
@ -216,11 +256,11 @@ void DatabaseOrdinary::renameTable(
|
||||
}
|
||||
|
||||
|
||||
time_t DatabaseOrdinary::getTableMetadataModificationTime(
|
||||
time_t DatabaseOrdinary::getObjectMetadataModificationTime(
|
||||
const Context & /* context */,
|
||||
const String & table_name)
|
||||
{
|
||||
return DatabaseOnDisk::getTableMetadataModificationTime(*this, table_name);
|
||||
return DatabaseOnDisk::getObjectMetadataModificationTime(*this, table_name);
|
||||
}
|
||||
|
||||
ASTPtr DatabaseOrdinary::getCreateTableQuery(const Context & context, const String & table_name) const
|
||||
@ -233,6 +273,17 @@ ASTPtr DatabaseOrdinary::tryGetCreateTableQuery(const Context & context, const S
|
||||
return DatabaseOnDisk::tryGetCreateTableQuery(*this, context, table_name);
|
||||
}
|
||||
|
||||
|
||||
ASTPtr DatabaseOrdinary::getCreateDictionaryQuery(const Context & context, const String & dictionary_name) const
|
||||
{
|
||||
return DatabaseOnDisk::getCreateDictionaryQuery(*this, context, dictionary_name);
|
||||
}
|
||||
|
||||
ASTPtr DatabaseOrdinary::tryGetCreateDictionaryQuery(const Context & context, const String & dictionary_name) const
|
||||
{
|
||||
return DatabaseOnDisk::tryGetCreateTableQuery(*this, context, dictionary_name);
|
||||
}
|
||||
|
||||
ASTPtr DatabaseOrdinary::getCreateDatabaseQuery(const Context & context) const
|
||||
{
|
||||
return DatabaseOnDisk::getCreateDatabaseQuery(*this, context);
|
||||
@ -283,7 +334,7 @@ void DatabaseOrdinary::alterTable(
|
||||
if (storage_modifier)
|
||||
storage_modifier(*ast_create_query.storage);
|
||||
|
||||
statement = getTableDefinitionFromCreateQuery(ast);
|
||||
statement = getObjectDefinitionFromCreateQuery(ast);
|
||||
|
||||
{
|
||||
WriteBufferFromFile out(table_metadata_tmp_path, statement.size(), O_WRONLY | O_CREAT | O_EXCL);
|
||||
@ -328,9 +379,9 @@ String DatabaseOrdinary::getDatabaseName() const
|
||||
return name;
|
||||
}
|
||||
|
||||
String DatabaseOrdinary::getTableMetadataPath(const String & table_name) const
|
||||
String DatabaseOrdinary::getObjectMetadataPath(const String & table_name) const
|
||||
{
|
||||
return detail::getTableMetadataPath(getMetadataPath(), table_name);
|
||||
return DatabaseOnDisk::getObjectMetadataPath(*this, table_name);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -18,7 +18,7 @@ public:
|
||||
|
||||
String getEngineName() const override { return "Ordinary"; }
|
||||
|
||||
void loadTables(
|
||||
void loadStoredObjects(
|
||||
Context & context,
|
||||
bool has_force_restore_data_flag) override;
|
||||
|
||||
@ -28,10 +28,19 @@ public:
|
||||
const StoragePtr & table,
|
||||
const ASTPtr & query) override;
|
||||
|
||||
void createDictionary(
|
||||
const Context & context,
|
||||
const String & dictionary_name,
|
||||
const ASTPtr & query) override;
|
||||
|
||||
void removeTable(
|
||||
const Context & context,
|
||||
const String & table_name) override;
|
||||
|
||||
void removeDictionary(
|
||||
const Context & context,
|
||||
const String & table_name) override;
|
||||
|
||||
void renameTable(
|
||||
const Context & context,
|
||||
const String & table_name,
|
||||
@ -47,7 +56,7 @@ public:
|
||||
const ConstraintsDescription & constraints,
|
||||
const ASTModifier & engine_modifier) override;
|
||||
|
||||
time_t getTableMetadataModificationTime(
|
||||
time_t getObjectMetadataModificationTime(
|
||||
const Context & context,
|
||||
const String & table_name) override;
|
||||
|
||||
@ -59,12 +68,20 @@ public:
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
|
||||
ASTPtr tryGetCreateDictionaryQuery(
|
||||
const Context & context,
|
||||
const String & name) const override;
|
||||
|
||||
ASTPtr getCreateDictionaryQuery(
|
||||
const Context & context,
|
||||
const String & name) const override;
|
||||
|
||||
ASTPtr getCreateDatabaseQuery(const Context & context) const override;
|
||||
|
||||
String getDataPath() const override;
|
||||
String getDatabaseName() const override;
|
||||
String getMetadataPath() const override;
|
||||
String getTableMetadataPath(const String & table_name) const override;
|
||||
String getObjectMetadataPath(const String & table_name) const override;
|
||||
|
||||
void drop() override;
|
||||
|
||||
@ -74,8 +91,6 @@ private:
|
||||
Poco::Logger * log;
|
||||
|
||||
void startupTables(ThreadPool & thread_pool);
|
||||
|
||||
ASTPtr getCreateTableQueryImpl(const Context & context, const String & table_name, bool throw_on_error) const;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -1,15 +1,20 @@
|
||||
#include <Databases/DatabasesCommon.h>
|
||||
|
||||
#include <Interpreters/ExternalDictionariesLoader.h>
|
||||
#include <Interpreters/ExternalLoaderDatabaseConfigRepository.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/ParserDictionary.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Storages/StorageDictionary.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <Dictionaries/DictionaryFactory.h>
|
||||
|
||||
#include <sstream>
|
||||
|
||||
@ -23,115 +28,119 @@ namespace ErrorCodes
|
||||
extern const int TABLE_ALREADY_EXISTS;
|
||||
extern const int UNKNOWN_TABLE;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int DICTIONARY_ALREADY_EXISTS;
|
||||
}
|
||||
|
||||
|
||||
String getTableDefinitionFromCreateQuery(const ASTPtr & query)
|
||||
namespace
|
||||
{
|
||||
ASTPtr query_clone = query->clone();
|
||||
auto & create = query_clone->as<ASTCreateQuery &>();
|
||||
|
||||
/// We remove everything that is not needed for ATTACH from the query.
|
||||
create.attach = true;
|
||||
create.database.clear();
|
||||
create.as_database.clear();
|
||||
create.as_table.clear();
|
||||
create.if_not_exists = false;
|
||||
create.is_populate = false;
|
||||
create.replace_view = false;
|
||||
|
||||
/// For views it is necessary to save the SELECT query itself, for the rest - on the contrary
|
||||
if (!create.is_view && !create.is_materialized_view && !create.is_live_view)
|
||||
create.select = nullptr;
|
||||
|
||||
create.format = nullptr;
|
||||
create.out_file = nullptr;
|
||||
|
||||
std::ostringstream statement_stream;
|
||||
formatAST(create, statement_stream, false);
|
||||
statement_stream << '\n';
|
||||
return statement_stream.str();
|
||||
}
|
||||
|
||||
|
||||
std::pair<String, StoragePtr> createTableFromDefinition(
|
||||
const String & definition,
|
||||
const String & database_name,
|
||||
const String & database_data_path,
|
||||
Context & context,
|
||||
bool has_force_restore_data_flag,
|
||||
const String & description_for_error_message)
|
||||
StoragePtr getDictionaryStorage(const Context & context, const String & table_name, const String & db_name)
|
||||
{
|
||||
ParserCreateQuery parser;
|
||||
ASTPtr ast = parseQuery(parser, definition.data(), definition.data() + definition.size(), description_for_error_message, 0);
|
||||
|
||||
auto & ast_create_query = ast->as<ASTCreateQuery &>();
|
||||
ast_create_query.attach = true;
|
||||
ast_create_query.database = database_name;
|
||||
|
||||
if (ast_create_query.as_table_function)
|
||||
auto dict_name = db_name + "." + table_name;
|
||||
const auto & external_loader = context.getExternalDictionariesLoader();
|
||||
auto dict_ptr = external_loader.tryGetDictionary(dict_name);
|
||||
if (dict_ptr)
|
||||
{
|
||||
const auto & table_function = ast_create_query.as_table_function->as<ASTFunction &>();
|
||||
const auto & factory = TableFunctionFactory::instance();
|
||||
StoragePtr storage = factory.get(table_function.name, context)->execute(ast_create_query.as_table_function, context, ast_create_query.table);
|
||||
return {ast_create_query.table, storage};
|
||||
const DictionaryStructure & dictionary_structure = dict_ptr->getStructure();
|
||||
auto columns = StorageDictionary::getNamesAndTypes(dictionary_structure);
|
||||
return StorageDictionary::create(db_name, table_name, ColumnsDescription{columns}, context, true, dict_name);
|
||||
}
|
||||
/// We do not directly use `InterpreterCreateQuery::execute`, because
|
||||
/// - the database has not been created yet;
|
||||
/// - the code is simpler, since the query is already brought to a suitable form.
|
||||
if (!ast_create_query.columns_list || !ast_create_query.columns_list->columns)
|
||||
throw Exception("Missing definition of columns.", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED);
|
||||
|
||||
ColumnsDescription columns = InterpreterCreateQuery::getColumnsDescription(*ast_create_query.columns_list->columns, context);
|
||||
ConstraintsDescription constraints = InterpreterCreateQuery::getConstraintsDescription(ast_create_query.columns_list->constraints);
|
||||
|
||||
return
|
||||
{
|
||||
ast_create_query.table,
|
||||
StorageFactory::instance().get(
|
||||
ast_create_query,
|
||||
database_data_path, ast_create_query.table, database_name, context, context.getGlobalContext(),
|
||||
columns, constraints,
|
||||
true, has_force_restore_data_flag)
|
||||
};
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
bool DatabaseWithOwnTablesBase::isTableExist(
|
||||
const Context & /*context*/,
|
||||
const String & table_name) const
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
return tables.find(table_name) != tables.end();
|
||||
return tables.find(table_name) != tables.end() || dictionaries.find(table_name) != dictionaries.end();
|
||||
}
|
||||
|
||||
bool DatabaseWithOwnTablesBase::isDictionaryExist(
|
||||
const Context & /*context*/,
|
||||
const String & dictionary_name) const
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
return dictionaries.find(dictionary_name) != dictionaries.end();
|
||||
}
|
||||
|
||||
StoragePtr DatabaseWithOwnTablesBase::tryGetTable(
|
||||
const Context & /*context*/,
|
||||
const Context & context,
|
||||
const String & table_name) const
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
auto it = tables.find(table_name);
|
||||
if (it == tables.end())
|
||||
return {};
|
||||
return it->second;
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
auto it = tables.find(table_name);
|
||||
if (it != tables.end())
|
||||
return it->second;
|
||||
}
|
||||
|
||||
if (isDictionaryExist(context, table_name))
|
||||
/// We don't need lock database here, because database doesn't store dictionary itself
|
||||
/// just metadata
|
||||
return getDictionaryStorage(context, table_name, getDatabaseName());
|
||||
|
||||
return {};
|
||||
}
|
||||
|
||||
DatabaseIteratorPtr DatabaseWithOwnTablesBase::getIterator(const Context & /*context*/, const FilterByNameFunction & filter_by_table_name)
|
||||
DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesWithDictionaryTablesIterator(const Context & context, const FilterByNameFunction & filter_by_name)
|
||||
{
|
||||
auto tables_it = getTablesIterator(context, filter_by_name);
|
||||
auto dictionaries_it = getDictionariesIterator(context, filter_by_name);
|
||||
|
||||
Tables result;
|
||||
while (tables_it && tables_it->isValid())
|
||||
{
|
||||
result.emplace(tables_it->name(), tables_it->table());
|
||||
tables_it->next();
|
||||
}
|
||||
|
||||
while (dictionaries_it && dictionaries_it->isValid())
|
||||
{
|
||||
auto table_name = dictionaries_it->name();
|
||||
auto table_ptr = getDictionaryStorage(context, table_name, getDatabaseName());
|
||||
if (table_ptr)
|
||||
result.emplace(table_name, table_ptr);
|
||||
dictionaries_it->next();
|
||||
}
|
||||
|
||||
return std::make_unique<DatabaseTablesSnapshotIterator>(result);
|
||||
}
|
||||
|
||||
DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesIterator(const Context & /*context*/, const FilterByNameFunction & filter_by_table_name)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
if (!filter_by_table_name)
|
||||
return std::make_unique<DatabaseSnapshotIterator>(tables);
|
||||
return std::make_unique<DatabaseTablesSnapshotIterator>(tables);
|
||||
|
||||
Tables filtered_tables;
|
||||
for (const auto & [table_name, storage] : tables)
|
||||
if (filter_by_table_name(table_name))
|
||||
filtered_tables.emplace(table_name, storage);
|
||||
return std::make_unique<DatabaseSnapshotIterator>(std::move(filtered_tables));
|
||||
|
||||
return std::make_unique<DatabaseTablesSnapshotIterator>(std::move(filtered_tables));
|
||||
}
|
||||
|
||||
|
||||
DatabaseDictionariesIteratorPtr DatabaseWithOwnTablesBase::getDictionariesIterator(const Context & /*context*/, const FilterByNameFunction & filter_by_dictionary_name)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
if (!filter_by_dictionary_name)
|
||||
return std::make_unique<DatabaseDictionariesSnapshotIterator>(dictionaries);
|
||||
|
||||
Dictionaries filtered_dictionaries;
|
||||
for (const auto & dictionary_name : dictionaries)
|
||||
if (filter_by_dictionary_name(dictionary_name))
|
||||
filtered_dictionaries.emplace(dictionary_name);
|
||||
return std::make_unique<DatabaseDictionariesSnapshotIterator>(std::move(filtered_dictionaries));
|
||||
}
|
||||
|
||||
bool DatabaseWithOwnTablesBase::empty(const Context & /*context*/) const
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
return tables.empty();
|
||||
return tables.empty() && dictionaries.empty();
|
||||
}
|
||||
|
||||
StoragePtr DatabaseWithOwnTablesBase::detachTable(const String & table_name)
|
||||
@ -139,6 +148,9 @@ StoragePtr DatabaseWithOwnTablesBase::detachTable(const String & table_name)
|
||||
StoragePtr res;
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
if (dictionaries.count(table_name))
|
||||
throw Exception("Cannot detach dictionary " + name + "." + table_name + " as table, use DETACH DICTIONARY query.", ErrorCodes::UNKNOWN_TABLE);
|
||||
|
||||
auto it = tables.find(table_name);
|
||||
if (it == tables.end())
|
||||
throw Exception("Table " + name + "." + table_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
|
||||
@ -149,6 +161,21 @@ StoragePtr DatabaseWithOwnTablesBase::detachTable(const String & table_name)
|
||||
return res;
|
||||
}
|
||||
|
||||
void DatabaseWithOwnTablesBase::detachDictionary(const String & dictionary_name, const Context & context, bool reload)
|
||||
{
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
auto it = dictionaries.find(dictionary_name);
|
||||
if (it == dictionaries.end())
|
||||
throw Exception("Dictionary " + name + "." + dictionary_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
|
||||
dictionaries.erase(it);
|
||||
}
|
||||
|
||||
if (reload)
|
||||
context.getExternalDictionariesLoader().reload(getDatabaseName() + "." + dictionary_name);
|
||||
|
||||
}
|
||||
|
||||
void DatabaseWithOwnTablesBase::attachTable(const String & table_name, const StoragePtr & table)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
@ -156,6 +183,25 @@ void DatabaseWithOwnTablesBase::attachTable(const String & table_name, const Sto
|
||||
throw Exception("Table " + name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
|
||||
}
|
||||
|
||||
|
||||
void DatabaseWithOwnTablesBase::attachDictionary(const String & dictionary_name, const Context & context, bool load)
|
||||
{
|
||||
const auto & external_loader = context.getExternalDictionariesLoader();
|
||||
|
||||
String full_name = getDatabaseName() + "." + dictionary_name;
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
auto status = external_loader.getCurrentStatus(full_name);
|
||||
if (status != ExternalLoader::Status::NOT_EXIST || !dictionaries.emplace(dictionary_name).second)
|
||||
throw Exception(
|
||||
"Dictionary " + full_name + " already exists.",
|
||||
ErrorCodes::DICTIONARY_ALREADY_EXISTS);
|
||||
}
|
||||
|
||||
if (load)
|
||||
external_loader.reload(full_name, true);
|
||||
}
|
||||
|
||||
void DatabaseWithOwnTablesBase::shutdown()
|
||||
{
|
||||
/// You can not hold a lock during shutdown.
|
||||
@ -174,6 +220,7 @@ void DatabaseWithOwnTablesBase::shutdown()
|
||||
|
||||
std::lock_guard lock(mutex);
|
||||
tables.clear();
|
||||
dictionaries.clear();
|
||||
}
|
||||
|
||||
DatabaseWithOwnTablesBase::~DatabaseWithOwnTablesBase()
|
||||
|
@ -15,63 +15,6 @@ namespace DB
|
||||
|
||||
class Context;
|
||||
|
||||
|
||||
/** Get the row with the table definition based on the CREATE query.
|
||||
* It is an ATTACH query that you can execute to create a table from the correspondent database.
|
||||
* See the implementation.
|
||||
*/
|
||||
String getTableDefinitionFromCreateQuery(const ASTPtr & query);
|
||||
|
||||
|
||||
/** Create a table by its definition, without using InterpreterCreateQuery.
|
||||
* (InterpreterCreateQuery has more complex functionality, and it can not be used if the database has not been created yet)
|
||||
* Returns the table name and the table itself.
|
||||
* You must subsequently call IStorage::startup method to use the table.
|
||||
*/
|
||||
std::pair<String, StoragePtr> createTableFromDefinition(
|
||||
const String & definition,
|
||||
const String & database_name,
|
||||
const String & database_data_path,
|
||||
Context & context,
|
||||
bool has_force_restore_data_flag,
|
||||
const String & description_for_error_message);
|
||||
|
||||
|
||||
/// Copies list of tables and iterates through such snapshot.
|
||||
class DatabaseSnapshotIterator final : public IDatabaseIterator
|
||||
{
|
||||
private:
|
||||
Tables tables;
|
||||
Tables::iterator it;
|
||||
|
||||
public:
|
||||
DatabaseSnapshotIterator(Tables & tables_)
|
||||
: tables(tables_), it(tables.begin()) {}
|
||||
|
||||
DatabaseSnapshotIterator(Tables && tables_)
|
||||
: tables(tables_), it(tables.begin()) {}
|
||||
|
||||
void next() override
|
||||
{
|
||||
++it;
|
||||
}
|
||||
|
||||
bool isValid() const override
|
||||
{
|
||||
return it != tables.end();
|
||||
}
|
||||
|
||||
const String & name() const override
|
||||
{
|
||||
return it->first;
|
||||
}
|
||||
|
||||
const StoragePtr & table() const override
|
||||
{
|
||||
return it->second;
|
||||
}
|
||||
};
|
||||
|
||||
/// A base class for databases that manage their own list of tables.
|
||||
class DatabaseWithOwnTablesBase : public IDatabase
|
||||
{
|
||||
@ -80,18 +23,27 @@ public:
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
|
||||
bool isDictionaryExist(const Context & context, const String & dictionary_name) const override;
|
||||
|
||||
StoragePtr tryGetTable(
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
|
||||
bool empty(const Context & context) const override;
|
||||
|
||||
|
||||
void attachTable(const String & table_name, const StoragePtr & table) override;
|
||||
|
||||
void attachDictionary(const String & name, const Context & context, bool reload) override;
|
||||
|
||||
StoragePtr detachTable(const String & table_name) override;
|
||||
|
||||
DatabaseIteratorPtr getIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) override;
|
||||
void detachDictionary(const String & name, const Context & context, bool reload) override;
|
||||
|
||||
DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) override;
|
||||
|
||||
DatabaseDictionariesIteratorPtr getDictionariesIterator(const Context & context, const FilterByNameFunction & filter_by_dictionary_name = {}) override;
|
||||
|
||||
DatabaseTablesIteratorPtr getTablesWithDictionaryTablesIterator(const Context & context, const FilterByNameFunction & filter_by_dictionary_name = {}) override;
|
||||
|
||||
void shutdown() override;
|
||||
|
||||
@ -102,6 +54,7 @@ protected:
|
||||
|
||||
mutable std::mutex mutex;
|
||||
Tables tables;
|
||||
Dictionaries dictionaries;
|
||||
|
||||
DatabaseWithOwnTablesBase(String name_) : name(std::move(name_)) { }
|
||||
};
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Core/Types.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
#include <Dictionaries/IDictionary.h>
|
||||
#include <Common/Exception.h>
|
||||
|
||||
#include <ctime>
|
||||
@ -19,16 +20,14 @@ struct ConstraintsDescription;
|
||||
class ColumnsDescription;
|
||||
struct IndicesDescription;
|
||||
struct TableStructureWriteLockHolder;
|
||||
using Dictionaries = std::set<String>;
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
|
||||
/** Allows to iterate over tables.
|
||||
*/
|
||||
class IDatabaseIterator
|
||||
class IDatabaseTablesIterator
|
||||
{
|
||||
public:
|
||||
virtual void next() = 0;
|
||||
@ -37,15 +36,57 @@ public:
|
||||
virtual const String & name() const = 0;
|
||||
virtual const StoragePtr & table() const = 0;
|
||||
|
||||
virtual ~IDatabaseIterator() {}
|
||||
virtual ~IDatabaseTablesIterator() = default;
|
||||
};
|
||||
|
||||
using DatabaseIteratorPtr = std::unique_ptr<IDatabaseIterator>;
|
||||
/// Copies list of tables and iterates through such snapshot.
|
||||
class DatabaseTablesSnapshotIterator : public IDatabaseTablesIterator
|
||||
{
|
||||
private:
|
||||
Tables tables;
|
||||
Tables::iterator it;
|
||||
|
||||
public:
|
||||
DatabaseTablesSnapshotIterator(Tables & tables_) : tables(tables_), it(tables.begin()) {}
|
||||
|
||||
DatabaseTablesSnapshotIterator(Tables && tables_) : tables(tables_), it(tables.begin()) {}
|
||||
|
||||
void next() { ++it; }
|
||||
|
||||
bool isValid() const { return it != tables.end(); }
|
||||
|
||||
const String & name() const { return it->first; }
|
||||
|
||||
const StoragePtr & table() const { return it->second; }
|
||||
};
|
||||
|
||||
/// Copies list of dictionaries and iterates through such snapshot.
|
||||
class DatabaseDictionariesSnapshotIterator
|
||||
{
|
||||
private:
|
||||
Dictionaries dictionaries;
|
||||
Dictionaries::iterator it;
|
||||
|
||||
public:
|
||||
DatabaseDictionariesSnapshotIterator() = default;
|
||||
DatabaseDictionariesSnapshotIterator(Dictionaries & dictionaries_) : dictionaries(dictionaries_), it(dictionaries.begin()) {}
|
||||
|
||||
DatabaseDictionariesSnapshotIterator(Dictionaries && dictionaries_) : dictionaries(dictionaries_), it(dictionaries.begin()) {}
|
||||
|
||||
void next() { ++it; }
|
||||
|
||||
bool isValid() const { return !dictionaries.empty() && it != dictionaries.end(); }
|
||||
|
||||
const String & name() const { return *it; }
|
||||
};
|
||||
|
||||
using DatabaseTablesIteratorPtr = std::unique_ptr<IDatabaseTablesIterator>;
|
||||
using DatabaseDictionariesIteratorPtr = std::unique_ptr<DatabaseDictionariesSnapshotIterator>;
|
||||
|
||||
|
||||
/** Database engine.
|
||||
* It is responsible for:
|
||||
* - initialization of set of known tables;
|
||||
* - initialization of set of known tables and dictionaries;
|
||||
* - checking existence of a table and getting a table object;
|
||||
* - retrieving a list of all tables;
|
||||
* - creating and dropping tables;
|
||||
@ -60,7 +101,7 @@ public:
|
||||
|
||||
/// Load a set of existing tables.
|
||||
/// You can call only once, right after the object is created.
|
||||
virtual void loadTables(
|
||||
virtual void loadStoredObjects(
|
||||
Context & context,
|
||||
bool has_force_restore_data_flag) = 0;
|
||||
|
||||
@ -69,6 +110,11 @@ public:
|
||||
const Context & context,
|
||||
const String & name) const = 0;
|
||||
|
||||
/// Check the existence of the dictionary
|
||||
virtual bool isDictionaryExist(
|
||||
const Context & context,
|
||||
const String & name) const = 0;
|
||||
|
||||
/// Get the table for work. Return nullptr if there is no table.
|
||||
virtual StoragePtr tryGetTable(
|
||||
const Context & context,
|
||||
@ -78,7 +124,16 @@ public:
|
||||
|
||||
/// Get an iterator that allows you to pass through all the tables.
|
||||
/// It is possible to have "hidden" tables that are not visible when passing through, but are visible if you get them by name using the functions above.
|
||||
virtual DatabaseIteratorPtr getIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) = 0;
|
||||
virtual DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) = 0;
|
||||
|
||||
/// Get an iterator to pass through all the dictionaries.
|
||||
virtual DatabaseDictionariesIteratorPtr getDictionariesIterator(const Context & context, const FilterByNameFunction & filter_by_dictionary_name = {}) = 0;
|
||||
|
||||
/// Get an iterator to pass through all the tables and dictionary tables.
|
||||
virtual DatabaseTablesIteratorPtr getTablesWithDictionaryTablesIterator(const Context & context, const FilterByNameFunction & filter_by_name = {})
|
||||
{
|
||||
return getTablesIterator(context, filter_by_name);
|
||||
}
|
||||
|
||||
/// Is the database empty.
|
||||
virtual bool empty(const Context & context) const = 0;
|
||||
@ -90,17 +145,35 @@ public:
|
||||
const StoragePtr & table,
|
||||
const ASTPtr & query) = 0;
|
||||
|
||||
/// Add the dictionary to the database. Record its presence in the metadata.
|
||||
virtual void createDictionary(
|
||||
const Context & context,
|
||||
const String & dictionary_name,
|
||||
const ASTPtr & query) = 0;
|
||||
|
||||
/// Delete the table from the database. Delete the metadata.
|
||||
virtual void removeTable(
|
||||
const Context & context,
|
||||
const String & name) = 0;
|
||||
|
||||
/// Delete the dictionary from the database. Delete the metadata.
|
||||
virtual void removeDictionary(
|
||||
const Context & context,
|
||||
const String & dictionary_name) = 0;
|
||||
|
||||
/// Add a table to the database, but do not add it to the metadata. The database may not support this method.
|
||||
virtual void attachTable(const String & name, const StoragePtr & table) = 0;
|
||||
|
||||
/// Add dictionary to the database, but do not add it to the metadata. The database may not support this method.
|
||||
/// load is false when we starting up and lazy_load is true, so we don't want to load dictionaries synchronously.
|
||||
virtual void attachDictionary(const String & name, const Context & context, bool reload = true) = 0;
|
||||
|
||||
/// Forget about the table without deleting it, and return it. The database may not support this method.
|
||||
virtual StoragePtr detachTable(const String & name) = 0;
|
||||
|
||||
/// Forget about the dictionary without deleting it, and return it. The database may not support this method.
|
||||
virtual void detachDictionary(const String & name, const Context & context, bool reload = true) = 0;
|
||||
|
||||
/// Rename the table and possibly move the table to another database.
|
||||
virtual void renameTable(
|
||||
const Context & /*context*/,
|
||||
@ -128,7 +201,7 @@ public:
|
||||
}
|
||||
|
||||
/// Returns time of table's metadata change, 0 if there is no corresponding metadata file.
|
||||
virtual time_t getTableMetadataModificationTime(
|
||||
virtual time_t getObjectMetadataModificationTime(
|
||||
const Context & context,
|
||||
const String & name) = 0;
|
||||
|
||||
@ -140,6 +213,14 @@ public:
|
||||
return tryGetCreateTableQuery(context, name);
|
||||
}
|
||||
|
||||
/// Get the CREATE DICTIONARY query for the dictionary. Returns nullptr if dictionary doesn't exists.
|
||||
virtual ASTPtr tryGetCreateDictionaryQuery(const Context & context, const String & name) const = 0;
|
||||
|
||||
virtual ASTPtr getCreateDictionaryQuery(const Context & context, const String & name) const
|
||||
{
|
||||
return tryGetCreateDictionaryQuery(context, name);
|
||||
}
|
||||
|
||||
/// Get the CREATE DATABASE query for current database.
|
||||
virtual ASTPtr getCreateDatabaseQuery(const Context & context) const = 0;
|
||||
|
||||
@ -150,7 +231,7 @@ public:
|
||||
/// Returns metadata path if the database supports it, empty string otherwise
|
||||
virtual String getMetadataPath() const { return {}; }
|
||||
/// Returns metadata path of a concrete table if the database supports it, empty string otherwise
|
||||
virtual String getTableMetadataPath(const String & /*table_name*/) const { return {}; }
|
||||
virtual String getObjectMetadataPath(const String & /*table_name*/) const { return {}; }
|
||||
|
||||
/// Ask all tables to complete the background threads they are using and delete all table objects.
|
||||
virtual void shutdown() = 0;
|
||||
|
@ -11,8 +11,8 @@ generate_code(CacheDictionary_generate2 UInt8 UInt16 UInt32 UInt64 UInt128 Int8
|
||||
generate_code(CacheDictionary_generate3 UInt8 UInt16 UInt32 UInt64 UInt128 Int8 Int16 Int32 Int64 Float32 Float64 Decimal32 Decimal64 Decimal128)
|
||||
add_headers_and_sources(clickhouse_dictionaries ${CMAKE_CURRENT_BINARY_DIR}/generated/)
|
||||
|
||||
list(REMOVE_ITEM clickhouse_dictionaries_sources DictionaryFactory.cpp DictionarySourceFactory.cpp DictionaryStructure.cpp)
|
||||
list(REMOVE_ITEM clickhouse_dictionaries_headers DictionaryFactory.h DictionarySourceFactory.h DictionaryStructure.h)
|
||||
list(REMOVE_ITEM clickhouse_dictionaries_sources DictionaryFactory.cpp DictionarySourceFactory.cpp DictionaryStructure.cpp getDictionaryConfigurationFromAST.cpp)
|
||||
list(REMOVE_ITEM clickhouse_dictionaries_headers DictionaryFactory.h DictionarySourceFactory.h DictionaryStructure.h getDictionaryConfigurationFromAST.h)
|
||||
|
||||
add_library(clickhouse_dictionaries ${clickhouse_dictionaries_sources})
|
||||
target_link_libraries(clickhouse_dictionaries PRIVATE dbms clickhouse_common_io ${BTRIE_LIBRARIES})
|
||||
|
@ -611,7 +611,7 @@ void registerDictionaryCache(DictionaryFactory & factory)
|
||||
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};
|
||||
return std::make_unique<CacheDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, size);
|
||||
};
|
||||
factory.registerLayout("cache", create_layout);
|
||||
factory.registerLayout("cache", create_layout, false);
|
||||
}
|
||||
|
||||
|
||||
|
@ -52,7 +52,7 @@ ClickHouseDictionarySource::ClickHouseDictionarySource(
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
const Block & sample_block_,
|
||||
Context & context_)
|
||||
const Context & context_)
|
||||
: update_time{std::chrono::system_clock::from_time_t(0)}
|
||||
, dict_struct{dict_struct_}
|
||||
, host{config.getString(config_prefix + ".host")}
|
||||
@ -206,7 +206,7 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
Context & context) -> DictionarySourcePtr
|
||||
const Context & context) -> DictionarySourcePtr
|
||||
{
|
||||
return std::make_unique<ClickHouseDictionarySource>(dict_struct, config, config_prefix + ".clickhouse", sample_block, context);
|
||||
};
|
||||
|
@ -23,7 +23,7 @@ public:
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
const Block & sample_block_,
|
||||
Context & context);
|
||||
const Context & context);
|
||||
|
||||
/// copy-constructor is provided in order to support cloneability
|
||||
ClickHouseDictionarySource(const ClickHouseDictionarySource & other);
|
||||
|
@ -415,7 +415,7 @@ void registerDictionaryComplexKeyCache(DictionaryFactory & factory)
|
||||
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};
|
||||
return std::make_unique<ComplexKeyCacheDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, size);
|
||||
};
|
||||
factory.registerLayout("complex_key_cache", create_layout);
|
||||
factory.registerLayout("complex_key_cache", create_layout, true);
|
||||
}
|
||||
|
||||
|
||||
|
@ -755,7 +755,7 @@ void registerDictionaryComplexKeyHashed(DictionaryFactory & factory)
|
||||
const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false);
|
||||
return std::make_unique<ComplexKeyHashedDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
|
||||
};
|
||||
factory.registerLayout("complex_key_hashed", create_layout);
|
||||
factory.registerLayout("complex_key_hashed", create_layout, true);
|
||||
}
|
||||
|
||||
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <memory>
|
||||
#include "DictionarySourceFactory.h"
|
||||
#include "DictionaryStructure.h"
|
||||
#include "getDictionaryConfigurationFromAST.h"
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -12,15 +13,21 @@ namespace ErrorCodes
|
||||
extern const int UNKNOWN_ELEMENT_IN_CONFIG;
|
||||
}
|
||||
|
||||
void DictionaryFactory::registerLayout(const std::string & layout_type, Creator create_layout)
|
||||
void DictionaryFactory::registerLayout(const std::string & layout_type, Creator create_layout, bool is_complex)
|
||||
{
|
||||
if (!registered_layouts.emplace(layout_type, std::move(create_layout)).second)
|
||||
throw Exception("DictionaryFactory: the layout name '" + layout_type + "' is not unique", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
layout_complexity[layout_type] = is_complex;
|
||||
|
||||
}
|
||||
|
||||
|
||||
DictionaryPtr DictionaryFactory::create(
|
||||
const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Context & context) const
|
||||
const std::string & name,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
const Context & context) const
|
||||
{
|
||||
Poco::Util::AbstractConfiguration::Keys keys;
|
||||
const auto & layout_prefix = config_prefix + ".layout";
|
||||
@ -31,7 +38,7 @@ DictionaryPtr DictionaryFactory::create(
|
||||
|
||||
const DictionaryStructure dict_struct{config, config_prefix + ".structure"};
|
||||
|
||||
auto source_ptr = DictionarySourceFactory::instance().create(name, config, config_prefix + ".source", dict_struct, context);
|
||||
DictionarySourcePtr source_ptr = DictionarySourceFactory::instance().create(name, config, config_prefix + ".source", dict_struct, context);
|
||||
|
||||
const auto & layout_type = keys.front();
|
||||
|
||||
@ -39,14 +46,21 @@ DictionaryPtr DictionaryFactory::create(
|
||||
const auto found = registered_layouts.find(layout_type);
|
||||
if (found != registered_layouts.end())
|
||||
{
|
||||
const auto & create_layout = found->second;
|
||||
return create_layout(name, dict_struct, config, config_prefix, std::move(source_ptr));
|
||||
const auto & layout_creator = found->second;
|
||||
return layout_creator(name, dict_struct, config, config_prefix, std::move(source_ptr));
|
||||
}
|
||||
}
|
||||
|
||||
throw Exception{name + ": unknown dictionary layout type: " + layout_type, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG};
|
||||
}
|
||||
|
||||
DictionaryPtr DictionaryFactory::create(const std::string & name, const ASTCreateQuery & ast, const Context & context) const
|
||||
{
|
||||
auto configurationFromAST = getDictionaryConfigurationFromAST(ast);
|
||||
return DictionaryFactory::create(name, *configurationFromAST, "dictionary", context);
|
||||
}
|
||||
|
||||
|
||||
DictionaryFactory & DictionaryFactory::instance()
|
||||
{
|
||||
static DictionaryFactory ret;
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include "IDictionary.h"
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
|
||||
|
||||
namespace Poco
|
||||
@ -27,7 +28,15 @@ public:
|
||||
|
||||
static DictionaryFactory & instance();
|
||||
|
||||
DictionaryPtr create(const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Context & context) const;
|
||||
DictionaryPtr create(
|
||||
const std::string & name,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
const Context & context) const;
|
||||
|
||||
DictionaryPtr create(const std::string & name,
|
||||
const ASTCreateQuery & ast,
|
||||
const Context & context) const;
|
||||
|
||||
using Creator = std::function<DictionaryPtr(
|
||||
const std::string & name,
|
||||
@ -36,11 +45,15 @@ public:
|
||||
const std::string & config_prefix,
|
||||
DictionarySourcePtr source_ptr)>;
|
||||
|
||||
void registerLayout(const std::string & layout_type, Creator create_layout);
|
||||
bool isComplex(const std::string & layout_type) const { return layout_complexity.at(layout_type); }
|
||||
|
||||
void registerLayout(const std::string & layout_type, Creator create_layout, bool is_complex);
|
||||
|
||||
private:
|
||||
using LayoutRegistry = std::unordered_map<std::string, Creator>;
|
||||
LayoutRegistry registered_layouts;
|
||||
using LayoutComplexity = std::unordered_map<std::string, bool>;
|
||||
LayoutComplexity layout_complexity;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -80,7 +80,7 @@ DictionarySourcePtr DictionarySourceFactory::create(
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
const DictionaryStructure & dict_struct,
|
||||
Context & context) const
|
||||
const Context & context) const
|
||||
{
|
||||
Poco::Util::AbstractConfiguration::Keys keys;
|
||||
config.keys(config_prefix, keys);
|
||||
|
@ -31,7 +31,7 @@ public:
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
Context & context)>;
|
||||
const Context & context)>;
|
||||
|
||||
DictionarySourceFactory();
|
||||
|
||||
@ -42,7 +42,7 @@ public:
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
const DictionaryStructure & dict_struct,
|
||||
Context & context) const;
|
||||
const Context & context) const;
|
||||
|
||||
private:
|
||||
using SourceRegistry = std::unordered_map<std::string, Creator>;
|
||||
|
@ -217,7 +217,7 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
Context & context) -> DictionarySourcePtr
|
||||
const Context & context) -> DictionarySourcePtr
|
||||
{
|
||||
if (dict_struct.has_expressions)
|
||||
throw Exception{"Dictionary source of type `executable` does not support attribute expressions", ErrorCodes::LOGICAL_ERROR};
|
||||
|
@ -56,7 +56,7 @@ void registerDictionarySourceFile(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
Context & context) -> DictionarySourcePtr
|
||||
const Context & context) -> DictionarySourcePtr
|
||||
{
|
||||
if (dict_struct.has_expressions)
|
||||
throw Exception{"Dictionary source of type `file` does not support attribute expressions", ErrorCodes::LOGICAL_ERROR};
|
||||
|
@ -724,7 +724,7 @@ void registerDictionaryFlat(DictionaryFactory & factory)
|
||||
const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false);
|
||||
return std::make_unique<FlatDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
|
||||
};
|
||||
factory.registerLayout("flat", create_layout);
|
||||
factory.registerLayout("flat", create_layout, false);
|
||||
}
|
||||
|
||||
|
||||
|
@ -188,7 +188,7 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
Context & context) -> DictionarySourcePtr
|
||||
const Context & context) -> DictionarySourcePtr
|
||||
{
|
||||
if (dict_struct.has_expressions)
|
||||
throw Exception{"Dictionary source of type `http` does not support attribute expressions", ErrorCodes::LOGICAL_ERROR};
|
||||
|
@ -787,8 +787,8 @@ void registerDictionaryHashed(DictionaryFactory & factory)
|
||||
const bool sparse = name == "sparse_hashed";
|
||||
return std::make_unique<HashedDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty, sparse);
|
||||
};
|
||||
factory.registerLayout("hashed", create_layout);
|
||||
factory.registerLayout("sparse_hashed", create_layout);
|
||||
factory.registerLayout("hashed", create_layout, false);
|
||||
factory.registerLayout("sparse_hashed", create_layout, false);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -691,7 +691,7 @@ void registerDictionaryRangeHashed(DictionaryFactory & factory)
|
||||
const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false);
|
||||
return std::make_unique<RangeHashedDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
|
||||
};
|
||||
factory.registerLayout("range_hashed", create_layout);
|
||||
factory.registerLayout("range_hashed", create_layout, false);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -767,7 +767,7 @@ void registerDictionaryTrie(DictionaryFactory & factory)
|
||||
// This is specialised trie for storing IPv4 and IPv6 prefixes.
|
||||
return std::make_unique<TrieDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
|
||||
};
|
||||
factory.registerLayout("ip_trie", create_layout);
|
||||
factory.registerLayout("ip_trie", create_layout, true);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -238,7 +238,7 @@ void registerDictionarySourceXDBC(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
Context & context) -> DictionarySourcePtr {
|
||||
const Context & context) -> DictionarySourcePtr {
|
||||
#if USE_POCO_SQLODBC || USE_POCO_DATAODBC
|
||||
BridgeHelperPtr bridge = std::make_shared<XDBCBridgeHelper<ODBCBridgeMixin>>(
|
||||
context, context.getSettings().http_receive_timeout, config.getString(config_prefix + ".odbc.connection_string"));
|
||||
|
444
dbms/src/Dictionaries/getDictionaryConfigurationFromAST.cpp
Normal file
444
dbms/src/Dictionaries/getDictionaryConfigurationFromAST.cpp
Normal file
@ -0,0 +1,444 @@
|
||||
#include <Dictionaries/getDictionaryConfigurationFromAST.h>
|
||||
|
||||
#include <Poco/DOM/AutoPtr.h>
|
||||
#include <Poco/DOM/Document.h>
|
||||
#include <Poco/DOM/Element.h>
|
||||
#include <Poco/DOM/Text.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include <Poco/Util/XMLConfiguration.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Core/Names.h>
|
||||
#include <Parsers/ASTFunctionWithKeyValueArguments.h>
|
||||
#include <Parsers/ASTDictionaryAttributeDeclaration.h>
|
||||
#include <Dictionaries/DictionaryFactory.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int INCORRECT_DICTIONARY_DEFINITION;
|
||||
}
|
||||
|
||||
/// There are a lot of code, but it's very simple and straightforward
|
||||
/// We just convert
|
||||
namespace
|
||||
{
|
||||
|
||||
/// Get value from field and convert it to string.
|
||||
/// Also remove quotes from strings.
|
||||
String getUnescapedFieldString(const Field & field)
|
||||
{
|
||||
String string = applyVisitor(FieldVisitorToString(), field);
|
||||
if (!string.empty() && string.front() == '\'' && string.back() == '\'')
|
||||
return string.substr(1, string.size() - 2);
|
||||
return string;
|
||||
}
|
||||
|
||||
|
||||
using namespace Poco;
|
||||
using namespace Poco::XML;
|
||||
/*
|
||||
* Transforms next definition
|
||||
* LIFETIME(MIN 10, MAX 100)
|
||||
* to the next configuration
|
||||
* <lifetime>
|
||||
* <min>10</min>
|
||||
* <max>100</max>
|
||||
* </lifetime>
|
||||
*/
|
||||
void buildLifetimeConfiguration(
|
||||
AutoPtr<Document> doc,
|
||||
AutoPtr<Element> root,
|
||||
const ASTDictionaryLifetime * lifetime)
|
||||
{
|
||||
|
||||
AutoPtr<Element> lifetime_element(doc->createElement("lifetime"));
|
||||
AutoPtr<Element> min_element(doc->createElement("min"));
|
||||
AutoPtr<Element> max_element(doc->createElement("max"));
|
||||
AutoPtr<Text> min_sec(doc->createTextNode(toString(lifetime->min_sec)));
|
||||
min_element->appendChild(min_sec);
|
||||
AutoPtr<Text> max_sec(doc->createTextNode(toString(lifetime->max_sec)));
|
||||
max_element->appendChild(max_sec);
|
||||
lifetime_element->appendChild(min_element);
|
||||
lifetime_element->appendChild(max_element);
|
||||
root->appendChild(lifetime_element);
|
||||
}
|
||||
|
||||
/*
|
||||
* Transforms next definition
|
||||
* LAYOUT(FLAT())
|
||||
* to the next configuration
|
||||
* <layout>
|
||||
* <flat/>
|
||||
* </layout>
|
||||
*
|
||||
* And next definition
|
||||
* LAYOUT(CACHE(SIZE_IN_CELLS 1000))
|
||||
* to the next one
|
||||
* <layout>
|
||||
* <cache>
|
||||
* <size_in_cells>1000</size_in_cells>
|
||||
* </cache>
|
||||
* </layout>
|
||||
*/
|
||||
void buildLayoutConfiguration(
|
||||
AutoPtr<Document> doc,
|
||||
AutoPtr<Element> root,
|
||||
const ASTDictionaryLayout * layout)
|
||||
{
|
||||
AutoPtr<Element> layout_element(doc->createElement("layout"));
|
||||
root->appendChild(layout_element);
|
||||
AutoPtr<Element> layout_type_element(doc->createElement(layout->layout_type));
|
||||
layout_element->appendChild(layout_type_element);
|
||||
if (layout->parameter.has_value())
|
||||
{
|
||||
const auto & param = layout->parameter;
|
||||
AutoPtr<Element> layout_type_parameter_element(doc->createElement(param->first));
|
||||
const ASTLiteral & literal = param->second->as<const ASTLiteral &>();
|
||||
AutoPtr<Text> value(doc->createTextNode(toString(literal.value.get<UInt64>())));
|
||||
layout_type_parameter_element->appendChild(value);
|
||||
layout_type_element->appendChild(layout_type_parameter_element);
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* Transforms next definition
|
||||
* RANGE(MIN StartDate, MAX EndDate)
|
||||
* to the next configuration
|
||||
* <range_min><name>StartDate</name></range_min>
|
||||
* <range_max><name>EndDate</name></range_max>
|
||||
*/
|
||||
void buildRangeConfiguration(AutoPtr<Document> doc, AutoPtr<Element> root, const ASTDictionaryRange * range)
|
||||
{
|
||||
// appends <key><name>value</name></key> to root
|
||||
auto appendElem = [&doc, &root](const std::string & key, const std::string & value)
|
||||
{
|
||||
AutoPtr<Element> element(doc->createElement(key));
|
||||
AutoPtr<Element> name(doc->createElement("name"));
|
||||
AutoPtr<Text> text(doc->createTextNode(value));
|
||||
name->appendChild(text);
|
||||
element->appendChild(name);
|
||||
root->appendChild(element);
|
||||
};
|
||||
|
||||
appendElem("range_min", range->min_attr_name);
|
||||
appendElem("range_max", range->max_attr_name);
|
||||
}
|
||||
|
||||
|
||||
/// Get primary key columns names from AST
|
||||
Names getPrimaryKeyColumns(const ASTExpressionList * primary_key)
|
||||
{
|
||||
Names result;
|
||||
const auto & children = primary_key->children;
|
||||
|
||||
for (size_t index = 0; index != children.size(); ++index)
|
||||
{
|
||||
const ASTIdentifier * key_part = children[index]->as<const ASTIdentifier>();
|
||||
result.push_back(key_part->name);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Transofrms single dictionary attribute to configuration
|
||||
* third_column UInt8 DEFAULT 2 EXPRESSION rand() % 100 * 77
|
||||
* to
|
||||
* <attribute>
|
||||
* <name>third_column</name>
|
||||
* <type>UInt8</type>
|
||||
* <null_value>2</null_value>
|
||||
* <expression>(rand() % 100) * 77</expression>
|
||||
* </attribute>
|
||||
*/
|
||||
void buildSingleAttribute(
|
||||
AutoPtr<Document> doc,
|
||||
AutoPtr<Element> root,
|
||||
const ASTDictionaryAttributeDeclaration * dict_attr)
|
||||
{
|
||||
AutoPtr<Element> attribute_element(doc->createElement("attribute"));
|
||||
root->appendChild(attribute_element);
|
||||
|
||||
AutoPtr<Element> name_element(doc->createElement("name"));
|
||||
AutoPtr<Text> name(doc->createTextNode(dict_attr->name));
|
||||
name_element->appendChild(name);
|
||||
attribute_element->appendChild(name_element);
|
||||
|
||||
AutoPtr<Element> type_element(doc->createElement("type"));
|
||||
AutoPtr<Text> type(doc->createTextNode(queryToString(dict_attr->type)));
|
||||
type_element->appendChild(type);
|
||||
attribute_element->appendChild(type_element);
|
||||
|
||||
AutoPtr<Element> null_value_element(doc->createElement("null_value"));
|
||||
String null_value_str;
|
||||
if (dict_attr->default_value)
|
||||
null_value_str = queryToString(dict_attr->default_value);
|
||||
AutoPtr<Text> null_value(doc->createTextNode(null_value_str));
|
||||
null_value_element->appendChild(null_value);
|
||||
attribute_element->appendChild(null_value_element);
|
||||
|
||||
if (dict_attr->expression != nullptr)
|
||||
{
|
||||
AutoPtr<Element> expression_element(doc->createElement("expression"));
|
||||
AutoPtr<Text> expression(doc->createTextNode(queryToString(dict_attr->expression)));
|
||||
expression_element->appendChild(expression);
|
||||
attribute_element->appendChild(expression_element);
|
||||
}
|
||||
|
||||
if (dict_attr->hierarchical)
|
||||
{
|
||||
AutoPtr<Element> hierarchical_element(doc->createElement("hierarchical"));
|
||||
AutoPtr<Text> hierarchical(doc->createTextNode("true"));
|
||||
hierarchical_element->appendChild(hierarchical);
|
||||
attribute_element->appendChild(hierarchical_element);
|
||||
}
|
||||
|
||||
if (dict_attr->injective)
|
||||
{
|
||||
AutoPtr<Element> injective_element(doc->createElement("injective"));
|
||||
AutoPtr<Text> injective(doc->createTextNode("true"));
|
||||
injective_element->appendChild(injective);
|
||||
attribute_element->appendChild(injective_element);
|
||||
}
|
||||
|
||||
if (dict_attr->is_object_id)
|
||||
{
|
||||
AutoPtr<Element> is_object_id_element(doc->createElement("is_object_id"));
|
||||
AutoPtr<Text> is_object_id(doc->createTextNode("true"));
|
||||
is_object_id_element->appendChild(is_object_id);
|
||||
attribute_element->appendChild(is_object_id_element);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Transforms
|
||||
* PRIMARY KEY Attr1 ,..., AttrN
|
||||
* to the next configuration
|
||||
* <id><name>Attr1</name></id>
|
||||
* or
|
||||
* <key>
|
||||
* <attribute>
|
||||
* <name>Attr1</name>
|
||||
* <type>UInt8</type>
|
||||
* </attribute>
|
||||
* ...
|
||||
* <attribute> fe
|
||||
* </key>
|
||||
*
|
||||
*/
|
||||
void buildPrimaryKeyConfiguration(
|
||||
AutoPtr<Document> doc,
|
||||
AutoPtr<Element> root,
|
||||
bool complex,
|
||||
const Names & key_names,
|
||||
const ASTExpressionList * dictionary_attributes)
|
||||
{
|
||||
if (!complex)
|
||||
{
|
||||
if (key_names.size() != 1)
|
||||
throw Exception("Primary key for simple dictionary must contain exactly one element",
|
||||
ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
|
||||
AutoPtr<Element> id_element(doc->createElement("id"));
|
||||
root->appendChild(id_element);
|
||||
AutoPtr<Element> name_element(doc->createElement("name"));
|
||||
id_element->appendChild(name_element);
|
||||
AutoPtr<Text> name(doc->createTextNode(*key_names.begin()));
|
||||
name_element->appendChild(name);
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto & children = dictionary_attributes->children;
|
||||
if (children.size() < key_names.size())
|
||||
throw Exception(
|
||||
"Primary key fields count is more, than dictionary attributes count.", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
|
||||
AutoPtr<Element> key_element(doc->createElement("key"));
|
||||
root->appendChild(key_element);
|
||||
for (const auto & key_name : key_names)
|
||||
{
|
||||
bool found = false;
|
||||
for (const auto & attr : children)
|
||||
{
|
||||
const ASTDictionaryAttributeDeclaration * dict_attr = attr->as<const ASTDictionaryAttributeDeclaration>();
|
||||
if (dict_attr->name == key_name)
|
||||
{
|
||||
found = true;
|
||||
buildSingleAttribute(doc, key_element, dict_attr);
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (!found)
|
||||
throw Exception(
|
||||
"Primary key field '" + key_name + "' not found among attributes.", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Transforms list of ASTDictionaryAttributeDeclarations to list of dictionary attributes
|
||||
*/
|
||||
std::unordered_set<std::string> buildDictionaryAttributesConfiguration(
|
||||
AutoPtr<Document> doc,
|
||||
AutoPtr<Element> root,
|
||||
const ASTExpressionList * dictionary_attributes,
|
||||
const Names & key_columns)
|
||||
{
|
||||
const auto & children = dictionary_attributes->children;
|
||||
std::unordered_set<std::string> dictionary_attributes_names;
|
||||
for (size_t i = 0; i < children.size(); ++i)
|
||||
{
|
||||
const ASTDictionaryAttributeDeclaration * dict_attr = children[i]->as<const ASTDictionaryAttributeDeclaration>();
|
||||
if (!dict_attr->type)
|
||||
throw Exception("Dictionary attribute must has type", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
|
||||
dictionary_attributes_names.insert(dict_attr->name);
|
||||
if (std::find(key_columns.begin(), key_columns.end(), dict_attr->name) == key_columns.end())
|
||||
buildSingleAttribute(doc, root, dict_attr);
|
||||
}
|
||||
return dictionary_attributes_names;
|
||||
}
|
||||
|
||||
/** Transform function with key-value arguments to configuration
|
||||
* (used for source transformation)
|
||||
*/
|
||||
void buildConfigurationFromFunctionWithKeyValueArguments(
|
||||
AutoPtr<Document> doc,
|
||||
AutoPtr<Element> root,
|
||||
const ASTExpressionList * ast_expr_list)
|
||||
{
|
||||
const auto & children = ast_expr_list->children;
|
||||
for (size_t i = 0; i != children.size(); ++i)
|
||||
{
|
||||
const ASTPair * pair = children[i]->as<const ASTPair>();
|
||||
AutoPtr<Element> current_xml_element(doc->createElement(pair->first));
|
||||
root->appendChild(current_xml_element);
|
||||
|
||||
if (auto identifier = pair->second->as<const ASTIdentifier>(); identifier)
|
||||
{
|
||||
AutoPtr<Text> value(doc->createTextNode(identifier->name));
|
||||
current_xml_element->appendChild(value);
|
||||
}
|
||||
else if (auto literal = pair->second->as<const ASTLiteral>(); literal)
|
||||
{
|
||||
AutoPtr<Text> value(doc->createTextNode(getUnescapedFieldString(literal->value)));
|
||||
current_xml_element->appendChild(value);
|
||||
}
|
||||
else if (auto list = pair->second->as<const ASTExpressionList>(); list)
|
||||
{
|
||||
buildConfigurationFromFunctionWithKeyValueArguments(doc, current_xml_element, list);
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception(
|
||||
"Incorrect ASTPair contains wrong value, should be literal, identifier or list",
|
||||
ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Build source definition from ast.
|
||||
* SOURCE(MYSQL(HOST 'localhost' PORT 9000 USER 'default' REPLICA(HOST '127.0.0.1' PRIORITY 1) PASSWORD ''))
|
||||
* to
|
||||
* <source>
|
||||
* <mysql>
|
||||
* <host>localhost</host>
|
||||
* ...
|
||||
* <replica>
|
||||
* <host>127.0.0.1</host>
|
||||
* ...
|
||||
* </replica>
|
||||
* </mysql>
|
||||
* </source>
|
||||
*/
|
||||
void buildSourceConfiguration(AutoPtr<Document> doc, AutoPtr<Element> root, const ASTFunctionWithKeyValueArguments * source)
|
||||
{
|
||||
AutoPtr<Element> outer_element(doc->createElement("source"));
|
||||
root->appendChild(outer_element);
|
||||
AutoPtr<Element> source_element(doc->createElement(source->name));
|
||||
outer_element->appendChild(source_element);
|
||||
buildConfigurationFromFunctionWithKeyValueArguments(doc, source_element, source->elements->as<const ASTExpressionList>());
|
||||
}
|
||||
|
||||
/** Check all AST fields are filled, throws exception
|
||||
* in other case
|
||||
*/
|
||||
void checkAST(const ASTCreateQuery & query)
|
||||
{
|
||||
if (!query.is_dictionary || query.dictionary == nullptr)
|
||||
throw Exception("Cannot convert dictionary to configuration from non-dictionary AST.", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
|
||||
if (query.dictionary_attributes_list == nullptr || query.dictionary_attributes_list->children.empty())
|
||||
throw Exception("Cannot create dictionary with empty attributes list", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
|
||||
if (query.dictionary->layout == nullptr)
|
||||
throw Exception("Cannot create dictionary with empty layout", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
|
||||
if (query.dictionary->lifetime == nullptr)
|
||||
throw Exception("Cannot create dictionary with empty lifetime", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
|
||||
if (query.dictionary->primary_key == nullptr)
|
||||
throw Exception("Cannot create dictionary without primary key", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
|
||||
if (query.dictionary->source == nullptr)
|
||||
throw Exception("Cannot create dictionary with empty source", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
|
||||
/// Range can be empty
|
||||
}
|
||||
|
||||
void checkPrimaryKey(const std::unordered_set<std::string> & all_attrs, const Names & key_attrs)
|
||||
{
|
||||
for (const auto & key_attr : key_attrs)
|
||||
if (all_attrs.count(key_attr) == 0)
|
||||
throw Exception("Unknown key attribute '" + key_attr + "'", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
DictionaryConfigurationPtr getDictionaryConfigurationFromAST(const ASTCreateQuery & query)
|
||||
{
|
||||
checkAST(query);
|
||||
|
||||
AutoPtr<Poco::XML::Document> xml_document(new Poco::XML::Document());
|
||||
AutoPtr<Poco::XML::Element> document_root(xml_document->createElement("dictionaries"));
|
||||
xml_document->appendChild(document_root);
|
||||
AutoPtr<Poco::XML::Element> current_dictionary(xml_document->createElement("dictionary"));
|
||||
document_root->appendChild(current_dictionary);
|
||||
AutoPtr<Poco::Util::XMLConfiguration> conf(new Poco::Util::XMLConfiguration());
|
||||
|
||||
AutoPtr<Poco::XML::Element> name_element(xml_document->createElement("name"));
|
||||
current_dictionary->appendChild(name_element);
|
||||
AutoPtr<Text> name(xml_document->createTextNode(query.database + "." + query.table));
|
||||
name_element->appendChild(name);
|
||||
|
||||
AutoPtr<Element> structure_element(xml_document->createElement("structure"));
|
||||
current_dictionary->appendChild(structure_element);
|
||||
Names pk_attrs = getPrimaryKeyColumns(query.dictionary->primary_key);
|
||||
auto dictionary_layout = query.dictionary->layout;
|
||||
|
||||
bool complex = DictionaryFactory::instance().isComplex(dictionary_layout->layout_type);
|
||||
|
||||
auto all_attr_names = buildDictionaryAttributesConfiguration(xml_document, structure_element, query.dictionary_attributes_list, pk_attrs);
|
||||
checkPrimaryKey(all_attr_names, pk_attrs);
|
||||
|
||||
buildPrimaryKeyConfiguration(xml_document, structure_element, complex, pk_attrs, query.dictionary_attributes_list);
|
||||
|
||||
buildLayoutConfiguration(xml_document, current_dictionary, dictionary_layout);
|
||||
buildSourceConfiguration(xml_document, current_dictionary, query.dictionary->source);
|
||||
buildLifetimeConfiguration(xml_document, current_dictionary, query.dictionary->lifetime);
|
||||
|
||||
if (query.dictionary->range)
|
||||
buildRangeConfiguration(xml_document, structure_element, query.dictionary->range);
|
||||
|
||||
conf->load(xml_document);
|
||||
return conf;
|
||||
}
|
||||
|
||||
}
|
15
dbms/src/Dictionaries/getDictionaryConfigurationFromAST.h
Normal file
15
dbms/src/Dictionaries/getDictionaryConfigurationFromAST.h
Normal file
@ -0,0 +1,15 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
using DictionaryConfigurationPtr = Poco::AutoPtr<Poco::Util::AbstractConfiguration>;
|
||||
|
||||
/// Convert dictionary AST to Poco::AbstractConfiguration
|
||||
/// This function is necessary because all loadable objects configuration are Poco::AbstractConfiguration
|
||||
/// Can throw exception if query is ill-formed
|
||||
DictionaryConfigurationPtr getDictionaryConfigurationFromAST(const ASTCreateQuery & query);
|
||||
|
||||
}
|
224
dbms/src/Dictionaries/tests/gtest_dictionary_configuration.cpp
Normal file
224
dbms/src/Dictionaries/tests/gtest_dictionary_configuration.cpp
Normal file
@ -0,0 +1,224 @@
|
||||
#include <iostream>
|
||||
|
||||
#include <sstream>
|
||||
#include <Core/Types.h>
|
||||
#include <Poco/Util/XMLConfiguration.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTDropQuery.h>
|
||||
#include <Parsers/DumpASTNode.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/ParserDictionary.h>
|
||||
#include <Parsers/ParserDropQuery.h>
|
||||
#include <Parsers/ParserTablePropertiesQuery.h>
|
||||
#include <Parsers/TablePropertiesQueriesASTs.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Dictionaries/getDictionaryConfigurationFromAST.h>
|
||||
#include <Dictionaries/registerDictionaries.h>
|
||||
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
using namespace DB;
|
||||
|
||||
static bool registered = false;
|
||||
/// For debug
|
||||
std::string configurationToString(const DictionaryConfigurationPtr & config)
|
||||
{
|
||||
const Poco::Util::XMLConfiguration * xml_config = dynamic_cast<const Poco::Util::XMLConfiguration *>(config.get());
|
||||
std::ostringstream oss;
|
||||
xml_config->save(oss);
|
||||
return oss.str();
|
||||
}
|
||||
|
||||
TEST(ConvertDictionaryAST, SimpleDictConfiguration)
|
||||
{
|
||||
if (!registered)
|
||||
{
|
||||
registerDictionaries();
|
||||
registered = true;
|
||||
}
|
||||
|
||||
String input = " CREATE DICTIONARY test.dict1"
|
||||
" ("
|
||||
" key_column UInt64 DEFAULT 0,"
|
||||
" second_column UInt8 DEFAULT 1,"
|
||||
" third_column UInt8 DEFAULT 2"
|
||||
" )"
|
||||
" PRIMARY KEY key_column"
|
||||
" SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' PASSWORD '' DB 'test' TABLE 'table_for_dict'))"
|
||||
" LAYOUT(FLAT())"
|
||||
" LIFETIME(MIN 1 MAX 10)"
|
||||
" RANGE(MIN second_column MAX third_column)";
|
||||
|
||||
ParserCreateDictionaryQuery parser;
|
||||
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
|
||||
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
|
||||
DictionaryConfigurationPtr config = getDictionaryConfigurationFromAST(*create);
|
||||
|
||||
/// name
|
||||
EXPECT_EQ(config->getString("dictionary.name"), "test.dict1");
|
||||
|
||||
/// lifetime
|
||||
EXPECT_EQ(config->getInt("dictionary.lifetime.min"), 1);
|
||||
EXPECT_EQ(config->getInt("dictionary.lifetime.max"), 10);
|
||||
|
||||
/// range
|
||||
EXPECT_EQ(config->getString("dictionary.structure.range_min"), "second_column");
|
||||
EXPECT_EQ(config->getString("dictionary.structure.range_max"), "third_column");
|
||||
|
||||
/// source
|
||||
EXPECT_EQ(config->getString("dictionary.source.clickhouse.host"), "localhost");
|
||||
EXPECT_EQ(config->getInt("dictionary.source.clickhouse.port"), 9000);
|
||||
EXPECT_EQ(config->getString("dictionary.source.clickhouse.user"), "default");
|
||||
EXPECT_EQ(config->getString("dictionary.source.clickhouse.password"), "");
|
||||
EXPECT_EQ(config->getString("dictionary.source.clickhouse.db"), "test");
|
||||
EXPECT_EQ(config->getString("dictionary.source.clickhouse.table"), "table_for_dict");
|
||||
|
||||
/// attributes and key
|
||||
Poco::Util::AbstractConfiguration::Keys keys;
|
||||
config->keys("dictionary.structure", keys);
|
||||
|
||||
EXPECT_EQ(keys.size(), 5); /// + ranged keys
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + keys[0] + ".name"), "second_column");
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + keys[0] + ".type"), "UInt8");
|
||||
EXPECT_EQ(config->getInt("dictionary.structure." + keys[0] + ".null_value"), 1);
|
||||
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + keys[1] + ".name"), "third_column");
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + keys[1] + ".type"), "UInt8");
|
||||
EXPECT_EQ(config->getInt("dictionary.structure." + keys[1] + ".null_value"), 2);
|
||||
|
||||
EXPECT_EQ(keys[2], "id");
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + keys[2] + ".name"), "key_column");
|
||||
|
||||
/// layout
|
||||
EXPECT_TRUE(config->has("dictionary.layout.flat"));
|
||||
}
|
||||
|
||||
|
||||
TEST(ConvertDictionaryAST, TrickyAttributes)
|
||||
{
|
||||
if (!registered)
|
||||
{
|
||||
registerDictionaries();
|
||||
registered = true;
|
||||
}
|
||||
|
||||
String input = " CREATE DICTIONARY dict2"
|
||||
" ("
|
||||
" key_column UInt64 IS_OBJECT_ID,"
|
||||
" second_column UInt8 HIERARCHICAL INJECTIVE,"
|
||||
" third_column UInt8 DEFAULT 2 EXPRESSION rand() % 100 * 77"
|
||||
" )"
|
||||
" PRIMARY KEY key_column"
|
||||
" LAYOUT(hashed())"
|
||||
" LIFETIME(MIN 1 MAX 10)"
|
||||
" SOURCE(CLICKHOUSE(HOST 'localhost'))";
|
||||
|
||||
ParserCreateDictionaryQuery parser;
|
||||
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
|
||||
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
|
||||
DictionaryConfigurationPtr config = getDictionaryConfigurationFromAST(*create);
|
||||
|
||||
Poco::Util::AbstractConfiguration::Keys keys;
|
||||
config->keys("dictionary.structure", keys);
|
||||
|
||||
EXPECT_EQ(keys.size(), 3);
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + keys[0] + ".name"), "second_column");
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + keys[0] + ".type"), "UInt8");
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + keys[0] + ".null_value"), "");
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + keys[0] + ".hierarchical"), "true");
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + keys[0] + ".injective"), "true");
|
||||
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + keys[1] + ".name"), "third_column");
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + keys[1] + ".type"), "UInt8");
|
||||
EXPECT_EQ(config->getInt("dictionary.structure." + keys[1] + ".null_value"), 2);
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + keys[1] + ".expression"), "(rand() % 100) * 77");
|
||||
|
||||
EXPECT_EQ(keys[2], "id");
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + keys[2] + ".name"), "key_column");
|
||||
}
|
||||
|
||||
|
||||
TEST(ConvertDictionaryAST, ComplexKeyAndLayoutWithParams)
|
||||
{
|
||||
if (!registered)
|
||||
{
|
||||
registerDictionaries();
|
||||
registered = true;
|
||||
}
|
||||
|
||||
String input = " CREATE DICTIONARY dict4"
|
||||
" ("
|
||||
" key_column1 String,"
|
||||
" key_column2 UInt64,"
|
||||
" third_column UInt8,"
|
||||
" fourth_column UInt8"
|
||||
" )"
|
||||
" PRIMARY KEY key_column1, key_column2"
|
||||
" SOURCE(MYSQL())"
|
||||
" LAYOUT(COMPLEX_KEY_CACHE(size_in_cells 50))"
|
||||
" LIFETIME(MIN 1 MAX 10)";
|
||||
|
||||
ParserCreateDictionaryQuery parser;
|
||||
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
|
||||
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
|
||||
DictionaryConfigurationPtr config = getDictionaryConfigurationFromAST(*create);
|
||||
|
||||
Poco::Util::AbstractConfiguration::Keys keys;
|
||||
config->keys("dictionary.structure.key", keys);
|
||||
|
||||
EXPECT_EQ(keys.size(), 2);
|
||||
EXPECT_EQ(config->getString("dictionary.structure.key." + keys[0] + ".name"), "key_column1");
|
||||
EXPECT_EQ(config->getString("dictionary.structure.key." + keys[0] + ".type"), "String");
|
||||
|
||||
EXPECT_EQ(config->getString("dictionary.structure.key." + keys[1] + ".name"), "key_column2");
|
||||
EXPECT_EQ(config->getString("dictionary.structure.key." + keys[1] + ".type"), "UInt64");
|
||||
|
||||
Poco::Util::AbstractConfiguration::Keys attrs;
|
||||
config->keys("dictionary.structure", attrs);
|
||||
|
||||
EXPECT_EQ(attrs.size(), 3);
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + attrs[0] + ".name"), "third_column");
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + attrs[0] + ".type"), "UInt8");
|
||||
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + attrs[1] + ".name"), "fourth_column");
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + attrs[1] + ".type"), "UInt8");
|
||||
|
||||
EXPECT_EQ(attrs[2], "key");
|
||||
|
||||
EXPECT_EQ(config->getInt("dictionary.layout.complex_key_cache.size_in_cells"), 50);
|
||||
}
|
||||
|
||||
|
||||
TEST(ConvertDictionaryAST, ComplexSource)
|
||||
{
|
||||
if (!registered)
|
||||
{
|
||||
registerDictionaries();
|
||||
registered = true;
|
||||
}
|
||||
|
||||
String input = " CREATE DICTIONARY dict4"
|
||||
" ("
|
||||
" key_column UInt64,"
|
||||
" second_column UInt8,"
|
||||
" third_column UInt8"
|
||||
" )"
|
||||
" PRIMARY KEY key_column"
|
||||
" SOURCE(MYSQL(HOST 'localhost' PORT 9000 USER 'default' REPLICA(HOST '127.0.0.1' PRIORITY 1) PASSWORD ''))"
|
||||
" LAYOUT(CACHE(size_in_cells 50))"
|
||||
" LIFETIME(MIN 1 MAX 10)"
|
||||
" RANGE(MIN second_column MAX third_column)";
|
||||
|
||||
ParserCreateDictionaryQuery parser;
|
||||
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
|
||||
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
|
||||
DictionaryConfigurationPtr config = getDictionaryConfigurationFromAST(*create);
|
||||
/// source
|
||||
EXPECT_EQ(config->getString("dictionary.source.mysql.host"), "localhost");
|
||||
EXPECT_EQ(config->getInt("dictionary.source.mysql.port"), 9000);
|
||||
EXPECT_EQ(config->getString("dictionary.source.mysql.user"), "default");
|
||||
EXPECT_EQ(config->getString("dictionary.source.mysql.password"), "");
|
||||
EXPECT_EQ(config->getString("dictionary.source.mysql.replica.host"), "127.0.0.1");
|
||||
EXPECT_EQ(config->getInt("dictionary.source.mysql.replica.priority"), 1);
|
||||
}
|
@ -23,7 +23,7 @@ template <typename F>
|
||||
inline void forEachTable(Context & context, F && f)
|
||||
{
|
||||
for (auto & elem : context.getDatabases())
|
||||
for (auto iterator = elem.second->getIterator(context); iterator->isValid(); iterator->next())
|
||||
for (auto iterator = elem.second->getTablesIterator(context); iterator->isValid(); iterator->next())
|
||||
f(iterator->table());
|
||||
|
||||
}
|
||||
|
@ -167,7 +167,7 @@ void AsynchronousMetrics::update()
|
||||
/// Lazy database can not contain MergeTree tables
|
||||
if (db.second->getEngineName() == "Lazy")
|
||||
continue;
|
||||
for (auto iterator = db.second->getIterator(context); iterator->isValid(); iterator->next())
|
||||
for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next())
|
||||
{
|
||||
++total_number_of_tables;
|
||||
auto & table = iterator->table();
|
||||
|
@ -30,6 +30,7 @@
|
||||
#include <Dictionaries/Embedded/GeoDictionariesLoader.h>
|
||||
#include <Interpreters/EmbeddedDictionaries.h>
|
||||
#include <Interpreters/ExternalLoaderXMLConfigRepository.h>
|
||||
#include <Interpreters/ExternalLoaderDatabaseConfigRepository.h>
|
||||
#include <Interpreters/ExternalDictionariesLoader.h>
|
||||
#include <Interpreters/ExternalModelsLoader.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
@ -193,7 +194,7 @@ struct ContextShared
|
||||
bool shutdown_called = false;
|
||||
|
||||
/// Do not allow simultaneous execution of DDL requests on the same table.
|
||||
/// database -> table -> (mutex, counter), counter: how many threads are running a query on the table at the same time
|
||||
/// database -> object -> (mutex, counter), counter: how many threads are running a query on the table at the same time
|
||||
/// For the duration of the operation, an element is placed here, and an object is returned,
|
||||
/// which deletes the element in the destructor when counter becomes zero.
|
||||
/// In case the element already exists, waits, when query will be executed in other thread. See class DDLGuard below.
|
||||
@ -794,6 +795,16 @@ bool Context::isTableExist(const String & database_name, const String & table_na
|
||||
&& it->second->isTableExist(*this, table_name);
|
||||
}
|
||||
|
||||
bool Context::isDictionaryExists(const String & database_name, const String & dictionary_name) const
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
String db = resolveDatabase(database_name, current_database);
|
||||
checkDatabaseAccessRightsImpl(db);
|
||||
|
||||
Databases::const_iterator it = shared->databases.find(db);
|
||||
return shared->databases.end() != it && it->second->isDictionaryExist(*this, dictionary_name);
|
||||
}
|
||||
|
||||
bool Context::isDatabaseExist(const String & database_name) const
|
||||
{
|
||||
@ -809,22 +820,6 @@ bool Context::isExternalTableExist(const String & table_name) const
|
||||
}
|
||||
|
||||
|
||||
void Context::assertTableExists(const String & database_name, const String & table_name) const
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
String db = resolveDatabase(database_name, current_database);
|
||||
checkDatabaseAccessRightsImpl(db);
|
||||
|
||||
Databases::const_iterator it = shared->databases.find(db);
|
||||
if (shared->databases.end() == it)
|
||||
throw Exception("Database " + backQuoteIfNeed(db) + " doesn't exist", ErrorCodes::UNKNOWN_DATABASE);
|
||||
|
||||
if (!it->second->isTableExist(*this, table_name))
|
||||
throw Exception("Table " + backQuoteIfNeed(db) + "." + backQuoteIfNeed(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
|
||||
}
|
||||
|
||||
|
||||
void Context::assertTableDoesntExist(const String & database_name, const String & table_name, bool check_database_access_rights) const
|
||||
{
|
||||
auto lock = getLock();
|
||||
@ -1076,9 +1071,10 @@ void Context::addDatabase(const String & database_name, const DatabasePtr & data
|
||||
DatabasePtr Context::detachDatabase(const String & database_name)
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
auto res = getDatabase(database_name);
|
||||
getExternalDictionariesLoader().removeConfigRepository(database_name);
|
||||
shared->databases.erase(database_name);
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
@ -1093,6 +1089,17 @@ ASTPtr Context::getCreateTableQuery(const String & database_name, const String &
|
||||
return shared->databases[db]->getCreateTableQuery(*this, table_name);
|
||||
}
|
||||
|
||||
|
||||
ASTPtr Context::getCreateDictionaryQuery(const String & database_name, const String & dictionary_name) const
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
String db = resolveDatabase(database_name, current_database);
|
||||
assertDatabaseExists(db);
|
||||
|
||||
return shared->databases[db]->getCreateDictionaryQuery(*this, dictionary_name);
|
||||
}
|
||||
|
||||
ASTPtr Context::getCreateExternalTableQuery(const String & table_name) const
|
||||
{
|
||||
TableAndCreateASTs::const_iterator jt = external_tables.find(table_name);
|
||||
@ -1338,21 +1345,13 @@ EmbeddedDictionaries & Context::getEmbeddedDictionaries()
|
||||
|
||||
const ExternalDictionariesLoader & Context::getExternalDictionariesLoader() const
|
||||
{
|
||||
{
|
||||
std::lock_guard lock(shared->external_dictionaries_mutex);
|
||||
if (shared->external_dictionaries_loader)
|
||||
return *shared->external_dictionaries_loader;
|
||||
}
|
||||
|
||||
const auto & config = getConfigRef();
|
||||
std::lock_guard lock(shared->external_dictionaries_mutex);
|
||||
if (!shared->external_dictionaries_loader)
|
||||
{
|
||||
if (!this->global_context)
|
||||
throw Exception("Logical error: there is no global context", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
auto config_repository = std::make_unique<ExternalLoaderXMLConfigRepository>(config, "dictionaries_config");
|
||||
shared->external_dictionaries_loader.emplace(std::move(config_repository), *this->global_context);
|
||||
shared->external_dictionaries_loader.emplace(*this->global_context);
|
||||
}
|
||||
return *shared->external_dictionaries_loader;
|
||||
}
|
||||
@ -1371,8 +1370,7 @@ const ExternalModelsLoader & Context::getExternalModelsLoader() const
|
||||
if (!this->global_context)
|
||||
throw Exception("Logical error: there is no global context", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
auto config_repository = std::make_unique<ExternalLoaderXMLConfigRepository>(getConfigRef(), "models_config");
|
||||
shared->external_models_loader.emplace(std::move(config_repository), *this->global_context);
|
||||
shared->external_models_loader.emplace(*this->global_context);
|
||||
}
|
||||
return *shared->external_models_loader;
|
||||
}
|
||||
@ -2069,7 +2067,7 @@ void Context::dropCompiledExpressionCache() const
|
||||
#endif
|
||||
|
||||
|
||||
void Context::addXDBCBridgeCommand(std::unique_ptr<ShellCommand> cmd)
|
||||
void Context::addXDBCBridgeCommand(std::unique_ptr<ShellCommand> cmd) const
|
||||
{
|
||||
auto lock = getLock();
|
||||
shared->bridge_commands.emplace_back(std::move(cmd));
|
||||
|
@ -252,9 +252,9 @@ public:
|
||||
/// Checking the existence of the table/database. Database can be empty - in this case the current database is used.
|
||||
bool isTableExist(const String & database_name, const String & table_name) const;
|
||||
bool isDatabaseExist(const String & database_name) const;
|
||||
bool isDictionaryExists(const String & database_name, const String & dictionary_name) const;
|
||||
bool isExternalTableExist(const String & table_name) const;
|
||||
bool hasDatabaseAccessRights(const String & database_name) const;
|
||||
void assertTableExists(const String & database_name, const String & table_name) const;
|
||||
|
||||
bool hasDictionaryAccessRights(const String & dictionary_name) const;
|
||||
|
||||
@ -363,6 +363,7 @@ public:
|
||||
ASTPtr getCreateTableQuery(const String & database_name, const String & table_name) const;
|
||||
ASTPtr getCreateExternalTableQuery(const String & table_name) const;
|
||||
ASTPtr getCreateDatabaseQuery(const String & database_name) const;
|
||||
ASTPtr getCreateDictionaryQuery(const String & database_name, const String & dictionary_name) const;
|
||||
|
||||
const DatabasePtr getDatabase(const String & database_name) const;
|
||||
DatabasePtr getDatabase(const String & database_name);
|
||||
@ -552,7 +553,7 @@ public:
|
||||
#endif
|
||||
|
||||
/// Add started bridge command. It will be killed after context destruction
|
||||
void addXDBCBridgeCommand(std::unique_ptr<ShellCommand> cmd);
|
||||
void addXDBCBridgeCommand(std::unique_ptr<ShellCommand> cmd) const;
|
||||
|
||||
IHostContextPtr & getHostContext();
|
||||
const IHostContextPtr & getHostContext() const;
|
||||
|
@ -1,17 +1,16 @@
|
||||
#include <Interpreters/ExternalDictionariesLoader.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Dictionaries/DictionaryFactory.h>
|
||||
#include <Dictionaries/getDictionaryConfigurationFromAST.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Must not acquire Context lock in constructor to avoid possibility of deadlocks.
|
||||
ExternalDictionariesLoader::ExternalDictionariesLoader(
|
||||
ExternalLoaderConfigRepositoryPtr config_repository, Context & context_)
|
||||
ExternalDictionariesLoader::ExternalDictionariesLoader(Context & context_)
|
||||
: ExternalLoader("external dictionary", &Logger::get("ExternalDictionariesLoader"))
|
||||
, context(context_)
|
||||
{
|
||||
addConfigRepository(std::move(config_repository), {"dictionary", "name"});
|
||||
enableAsyncLoading(true);
|
||||
enablePeriodicUpdates(true);
|
||||
}
|
||||
@ -23,4 +22,21 @@ ExternalLoader::LoadablePtr ExternalDictionariesLoader::create(
|
||||
return DictionaryFactory::instance().create(name, config, key_in_config, context);
|
||||
}
|
||||
|
||||
void ExternalDictionariesLoader::addConfigRepository(
|
||||
const std::string & repository_name, std::unique_ptr<IExternalLoaderConfigRepository> config_repository)
|
||||
{
|
||||
ExternalLoader::addConfigRepository(repository_name, std::move(config_repository), {"dictionary", "name"});
|
||||
}
|
||||
|
||||
|
||||
void ExternalDictionariesLoader::addDictionaryWithConfig(
|
||||
const String & dictionary_name, const String & repo_name, const ASTCreateQuery & query, bool load_never_loading) const
|
||||
{
|
||||
ExternalLoader::addObjectAndLoad(
|
||||
dictionary_name, /// names are equal
|
||||
dictionary_name,
|
||||
repo_name,
|
||||
getDictionaryConfigurationFromAST(query),
|
||||
"dictionary", load_never_loading);
|
||||
}
|
||||
}
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Interpreters/IExternalLoaderConfigRepository.h>
|
||||
#include <Interpreters/ExternalLoader.h>
|
||||
#include <common/logger_useful.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <memory>
|
||||
|
||||
|
||||
@ -19,9 +20,7 @@ public:
|
||||
using DictPtr = std::shared_ptr<const IDictionaryBase>;
|
||||
|
||||
/// Dictionaries will be loaded immediately and then will be updated in separate thread, each 'reload_period' seconds.
|
||||
ExternalDictionariesLoader(
|
||||
ExternalLoaderConfigRepositoryPtr config_repository,
|
||||
Context & context_);
|
||||
ExternalDictionariesLoader(Context & context_);
|
||||
|
||||
DictPtr getDictionary(const std::string & name) const
|
||||
{
|
||||
@ -33,6 +32,18 @@ public:
|
||||
return std::static_pointer_cast<const IDictionaryBase>(tryGetLoadable(name));
|
||||
}
|
||||
|
||||
void addConfigRepository(
|
||||
const std::string & repository_name,
|
||||
std::unique_ptr<IExternalLoaderConfigRepository> config_repository);
|
||||
|
||||
/// Starts reloading of a specified object.
|
||||
void addDictionaryWithConfig(
|
||||
const String & dictionary_name,
|
||||
const String & repo_name,
|
||||
const ASTCreateQuery & query,
|
||||
bool load_never_loading = false) const;
|
||||
|
||||
|
||||
protected:
|
||||
LoadablePtr create(const std::string & name, const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & key_in_config) const override;
|
||||
|
@ -20,12 +20,29 @@ namespace ErrorCodes
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
/// Lock mutex only in async mode
|
||||
/// In other case does nothing
|
||||
struct LoadingGuardForAsyncLoad
|
||||
{
|
||||
std::unique_lock<std::mutex> lock;
|
||||
LoadingGuardForAsyncLoad(bool async, std::mutex & mutex)
|
||||
{
|
||||
if (async)
|
||||
lock = std::unique_lock(mutex);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
struct ExternalLoader::ObjectConfig
|
||||
{
|
||||
String config_path;
|
||||
Poco::AutoPtr<Poco::Util::AbstractConfiguration> config;
|
||||
String key_in_config;
|
||||
String repository_name;
|
||||
};
|
||||
|
||||
|
||||
@ -40,26 +57,84 @@ public:
|
||||
}
|
||||
~LoadablesConfigReader() = default;
|
||||
|
||||
void addConfigRepository(std::unique_ptr<IExternalLoaderConfigRepository> repository, const ExternalLoaderConfigSettings & settings)
|
||||
void addConfigRepository(
|
||||
const String & name,
|
||||
std::unique_ptr<IExternalLoaderConfigRepository> repository,
|
||||
const ExternalLoaderConfigSettings & settings)
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
repositories.emplace_back(std::move(repository), std::move(settings));
|
||||
repositories.emplace(name, std::make_pair(std::move(repository), settings));
|
||||
}
|
||||
|
||||
void removeConfigRepository(const String & name)
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
repositories.erase(name);
|
||||
}
|
||||
|
||||
using ObjectConfigsPtr = std::shared_ptr<const std::unordered_map<String /* object's name */, ObjectConfig>>;
|
||||
|
||||
/// Reads configuration files.
|
||||
|
||||
/// Reads configurations.
|
||||
ObjectConfigsPtr read()
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
|
||||
std::lock_guard lock(mutex);
|
||||
// Check last modification times of files and read those files which are new or changed.
|
||||
if (!readLoadablesInfos())
|
||||
return configs; // Nothing changed, so we can return the previous result.
|
||||
|
||||
return collectConfigs();
|
||||
}
|
||||
|
||||
ObjectConfig updateLoadableInfo(
|
||||
const String & external_name,
|
||||
const String & object_name,
|
||||
const String & repo_name,
|
||||
const Poco::AutoPtr<Poco::Util::AbstractConfiguration> & config,
|
||||
const String & key)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
auto it = loadables_infos.find(object_name);
|
||||
if (it == loadables_infos.end())
|
||||
{
|
||||
LoadablesInfos loadable_info;
|
||||
loadables_infos[object_name] = loadable_info;
|
||||
}
|
||||
auto & loadable_info = loadables_infos[object_name];
|
||||
ObjectConfig object_config{object_name, config, key, repo_name};
|
||||
bool found = false;
|
||||
for (auto iter = loadable_info.configs.begin(); iter != loadable_info.configs.end(); ++iter)
|
||||
{
|
||||
if (iter->first == external_name)
|
||||
{
|
||||
iter->second = object_config;
|
||||
found = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (!found)
|
||||
loadable_info.configs.emplace_back(external_name, object_config);
|
||||
loadable_info.last_update_time = Poco::Timestamp{}; /// now
|
||||
loadable_info.in_use = true;
|
||||
return object_config;
|
||||
}
|
||||
|
||||
private:
|
||||
struct LoadablesInfos
|
||||
{
|
||||
Poco::Timestamp last_update_time = 0;
|
||||
std::vector<std::pair<String, ObjectConfig>> configs; // Parsed loadable's contents.
|
||||
bool in_use = true; // Whether the `LoadablesInfos` should be destroyed because the correspondent loadable is deleted.
|
||||
};
|
||||
|
||||
/// Collect current configurations
|
||||
ObjectConfigsPtr collectConfigs()
|
||||
{
|
||||
// Generate new result.
|
||||
auto new_configs = std::make_shared<std::unordered_map<String /* object's name */, ObjectConfig>>();
|
||||
for (const auto & [path, loadable_info] : loadables_infos)
|
||||
for (const auto & [path, loadable_info] : loadables_infos)
|
||||
{
|
||||
for (const auto & [name, config] : loadable_info.configs)
|
||||
{
|
||||
@ -81,14 +156,6 @@ public:
|
||||
return configs;
|
||||
}
|
||||
|
||||
private:
|
||||
struct LoadablesInfos
|
||||
{
|
||||
Poco::Timestamp last_update_time = 0;
|
||||
std::vector<std::pair<String, ObjectConfig>> configs; // Parsed file's contents.
|
||||
bool in_use = true; // Whether the ` LoadablesInfos` should be destroyed because the correspondent file is deleted.
|
||||
};
|
||||
|
||||
/// Read files and store them to the map ` loadables_infos`.
|
||||
bool readLoadablesInfos()
|
||||
{
|
||||
@ -100,58 +167,59 @@ private:
|
||||
loadable_info.in_use = false;
|
||||
}
|
||||
|
||||
for (const auto & [repository, settings] : repositories)
|
||||
for (const auto & [repo_name, repo_with_settings] : repositories)
|
||||
{
|
||||
const auto names = repository->getAllLoadablesDefinitionNames();
|
||||
for (const auto & name : names)
|
||||
const auto names = repo_with_settings.first->getAllLoadablesDefinitionNames();
|
||||
for (const auto & loadable_name : names)
|
||||
{
|
||||
auto it = loadables_infos.find(name);
|
||||
if (it != loadables_infos.end())
|
||||
auto it = loadables_infos.find(loadable_name);
|
||||
if (it != loadables_infos.end())
|
||||
{
|
||||
LoadablesInfos & loadable_info = it->second;
|
||||
if (readLoadablesInfo(*repository, name, settings, loadable_info))
|
||||
if (readLoadablesInfo(repo_name, *repo_with_settings.first, loadable_name, repo_with_settings.second, loadable_info))
|
||||
changed = true;
|
||||
}
|
||||
else
|
||||
{
|
||||
LoadablesInfos loadable_info;
|
||||
if (readLoadablesInfo(*repository, name, settings, loadable_info))
|
||||
if (readLoadablesInfo(repo_name, *repo_with_settings.first, loadable_name, repo_with_settings.second, loadable_info))
|
||||
{
|
||||
loadables_infos.emplace(name, std::move(loadable_info));
|
||||
loadables_infos.emplace(loadable_name, std::move(loadable_info));
|
||||
changed = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
std::vector<String> deleted_files;
|
||||
std::vector<String> deleted_names;
|
||||
for (auto & [path, loadable_info] : loadables_infos)
|
||||
if (!loadable_info.in_use)
|
||||
deleted_files.emplace_back(path);
|
||||
if (!deleted_files.empty())
|
||||
deleted_names.emplace_back(path);
|
||||
if (!deleted_names.empty())
|
||||
{
|
||||
for (const String & deleted_file : deleted_files)
|
||||
loadables_infos.erase(deleted_file);
|
||||
for (const String & deleted_name : deleted_names)
|
||||
loadables_infos.erase(deleted_name);
|
||||
changed = true;
|
||||
}
|
||||
return changed;
|
||||
}
|
||||
|
||||
bool readLoadablesInfo(
|
||||
const String & repo_name,
|
||||
IExternalLoaderConfigRepository & repository,
|
||||
const String & path,
|
||||
const String & object_name,
|
||||
const ExternalLoaderConfigSettings & settings,
|
||||
LoadablesInfos & loadable_info) const
|
||||
{
|
||||
try
|
||||
{
|
||||
if (path.empty() || !repository.exists(path))
|
||||
if (object_name.empty() || !repository.exists(object_name))
|
||||
{
|
||||
LOG_WARNING(log, "config file '" + path + "' does not exist");
|
||||
LOG_WARNING(log, "Config file '" + object_name + "' does not exist");
|
||||
return false;
|
||||
}
|
||||
|
||||
auto update_time_from_repository = repository.getUpdateTime(path);
|
||||
auto update_time_from_repository = repository.getUpdateTime(object_name);
|
||||
|
||||
/// Actually it can't be less, but for sure we check less or equal
|
||||
if (update_time_from_repository <= loadable_info.last_update_time)
|
||||
@ -160,31 +228,31 @@ private:
|
||||
return false;
|
||||
}
|
||||
|
||||
auto file_contents = repository.load(path);
|
||||
auto file_contents = repository.load(object_name);
|
||||
|
||||
/// get all objects' definitions
|
||||
Poco::Util::AbstractConfiguration::Keys keys;
|
||||
file_contents->keys(keys);
|
||||
|
||||
/// for each object defined in xml config
|
||||
/// for each object defined in repositories
|
||||
std::vector<std::pair<String, ObjectConfig>> configs_from_file;
|
||||
for (const auto & key : keys)
|
||||
{
|
||||
if (!startsWith(key, settings.external_config))
|
||||
{
|
||||
if (!startsWith(key, "comment") && !startsWith(key, "include_from"))
|
||||
LOG_WARNING(log, path << ": file contains unknown node '" << key << "', expected '" << settings.external_config << "'");
|
||||
LOG_WARNING(log, object_name << ": file contains unknown node '" << key << "', expected '" << settings.external_config << "'");
|
||||
continue;
|
||||
}
|
||||
|
||||
String name = file_contents->getString(key + "." + settings.external_name);
|
||||
if (name.empty())
|
||||
String external_name = file_contents->getString(key + "." + settings.external_name);
|
||||
if (external_name.empty())
|
||||
{
|
||||
LOG_WARNING(log, path << ": node '" << key << "' defines " << type_name << " with an empty name. It's not allowed");
|
||||
LOG_WARNING(log, object_name << ": node '" << key << "' defines " << type_name << " with an empty name. It's not allowed");
|
||||
continue;
|
||||
}
|
||||
|
||||
configs_from_file.emplace_back(name, ObjectConfig{path, file_contents, key});
|
||||
configs_from_file.emplace_back(external_name, ObjectConfig{object_name, file_contents, key, repo_name});
|
||||
}
|
||||
|
||||
loadable_info.configs = std::move(configs_from_file);
|
||||
@ -194,16 +262,19 @@ private:
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(log, "Failed to read config file '" + path + "'");
|
||||
tryLogCurrentException(log, "Failed to load config for dictionary '" + object_name + "'");
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
const String type_name;
|
||||
Logger * log;
|
||||
|
||||
std::mutex mutex;
|
||||
std::vector<std::pair<std::unique_ptr<IExternalLoaderConfigRepository>, ExternalLoaderConfigSettings>> repositories;
|
||||
using RepositoryPtr = std::unique_ptr<IExternalLoaderConfigRepository>;
|
||||
using RepositoryWithSettings = std::pair<RepositoryPtr, ExternalLoaderConfigSettings>;
|
||||
std::unordered_map<String, RepositoryWithSettings> repositories;
|
||||
ObjectConfigsPtr configs;
|
||||
std::unordered_map<String /* config path */, LoadablesInfos> loadables_infos;
|
||||
};
|
||||
@ -267,10 +338,10 @@ public:
|
||||
else
|
||||
{
|
||||
const auto & new_config = new_config_it->second;
|
||||
if (!isSameConfiguration(*info.config.config, info.config.key_in_config, *new_config.config, new_config.key_in_config))
|
||||
if (!isSameConfiguration(*info.object_config.config, info.object_config.key_in_config, *new_config.config, new_config.key_in_config))
|
||||
{
|
||||
/// Configuration has been changed.
|
||||
info.config = new_config;
|
||||
info.object_config = new_config;
|
||||
info.config_changed = true;
|
||||
|
||||
if (info.wasLoading())
|
||||
@ -305,6 +376,12 @@ public:
|
||||
event.notify_all();
|
||||
}
|
||||
|
||||
void setSingleObjectConfigurationWithoutLoading(const String & external_name, const ObjectConfig & config)
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
infos.emplace(external_name, Info{config});
|
||||
}
|
||||
|
||||
/// Sets whether all the objects from the configuration should be always loaded (even if they aren't used).
|
||||
void enableAlwaysLoadEverything(bool enable)
|
||||
{
|
||||
@ -326,7 +403,6 @@ public:
|
||||
/// Sets whether the objects should be loaded asynchronously, each loading in a new thread (from the thread pool).
|
||||
void enableAsyncLoading(bool enable)
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
enable_async_loading = enable;
|
||||
}
|
||||
|
||||
@ -432,8 +508,17 @@ public:
|
||||
loaded_objects = collectLoadedObjects(filter_by_name);
|
||||
}
|
||||
|
||||
/// Tries to finish loading of the objects for which the specified function returns true.
|
||||
void load(const FilterByNameFunction & filter_by_name, LoadResults & loaded_results, Duration timeout = NO_TIMEOUT)
|
||||
{
|
||||
std::unique_lock lock{mutex};
|
||||
loadImpl(filter_by_name, timeout, lock);
|
||||
loaded_results = collectLoadResults(filter_by_name);
|
||||
}
|
||||
|
||||
/// Tries to finish loading of all the objects during the timeout.
|
||||
void load(Loadables & loaded_objects, Duration timeout = NO_TIMEOUT) { load(allNames, loaded_objects, timeout); }
|
||||
void load(LoadResults & loaded_results, Duration timeout = NO_TIMEOUT) { load(allNames, loaded_results, timeout); }
|
||||
|
||||
/// Starts reloading a specified object.
|
||||
void reload(const String & name, bool load_never_loading = false)
|
||||
@ -441,7 +526,9 @@ public:
|
||||
std::lock_guard lock{mutex};
|
||||
Info * info = getInfo(name);
|
||||
if (!info)
|
||||
{
|
||||
return;
|
||||
}
|
||||
|
||||
if (info->wasLoading() || load_never_loading)
|
||||
{
|
||||
@ -539,7 +626,7 @@ public:
|
||||
private:
|
||||
struct Info
|
||||
{
|
||||
Info(const ObjectConfig & config_) : config(config_) {}
|
||||
Info(const ObjectConfig & object_config_) : object_config(object_config_) {}
|
||||
|
||||
bool loaded() const { return object != nullptr; }
|
||||
bool failed() const { return !object && exception; }
|
||||
@ -571,11 +658,12 @@ private:
|
||||
result.exception = exception;
|
||||
result.loading_start_time = loading_start_time;
|
||||
result.loading_duration = loadingDuration();
|
||||
result.origin = config.config_path;
|
||||
result.origin = object_config.config_path;
|
||||
result.repository_name = object_config.repository_name;
|
||||
return result;
|
||||
}
|
||||
|
||||
ObjectConfig config;
|
||||
ObjectConfig object_config;
|
||||
LoadablePtr object;
|
||||
TimePoint loading_start_time;
|
||||
TimePoint loading_end_time;
|
||||
@ -618,8 +706,10 @@ private:
|
||||
LoadResults load_results;
|
||||
load_results.reserve(infos.size());
|
||||
for (const auto & [name, info] : infos)
|
||||
{
|
||||
if (filter_by_name(name))
|
||||
load_results.emplace_back(name, info.loadResult());
|
||||
}
|
||||
return load_results;
|
||||
}
|
||||
|
||||
@ -691,48 +781,15 @@ private:
|
||||
}
|
||||
}
|
||||
|
||||
/// Does the loading, possibly in the separate thread.
|
||||
void doLoading(const String & name, size_t loading_id, bool async)
|
||||
/// Load one object, returns object ptr or exception
|
||||
/// Do not require locking
|
||||
|
||||
std::pair<LoadablePtr, std::exception_ptr> loadOneObject(
|
||||
const String & name,
|
||||
const ObjectConfig & config,
|
||||
bool config_changed,
|
||||
LoadablePtr previous_version)
|
||||
{
|
||||
std::unique_lock<std::mutex> lock;
|
||||
if (async)
|
||||
{
|
||||
setThreadName("ExterLdrJob");
|
||||
lock = std::unique_lock{mutex}; /// If `async == false` the mutex is already locked.
|
||||
}
|
||||
|
||||
SCOPE_EXIT({
|
||||
if (async)
|
||||
{
|
||||
if (!lock.owns_lock())
|
||||
lock.lock();
|
||||
/// Remove the information about the thread after it finishes.
|
||||
auto it = loading_ids.find(loading_id);
|
||||
if (it != loading_ids.end())
|
||||
{
|
||||
it->second.detach();
|
||||
loading_ids.erase(it);
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
/// We check here if this is exactly the same loading as we planned to perform.
|
||||
/// This check is necessary because the object could be removed or load with another config before this thread even starts.
|
||||
Info * info = getInfo(name);
|
||||
if (!info || !info->loading() || (info->loading_id != loading_id))
|
||||
return;
|
||||
|
||||
ObjectConfig config = info->config;
|
||||
bool config_changed = info->config_changed;
|
||||
LoadablePtr previous_version = info->object;
|
||||
size_t error_count = info->error_count;
|
||||
|
||||
/// Use `create_function` to perform the actual loading.
|
||||
/// It's much better to do it with `mutex` unlocked because the loading can take a lot of time
|
||||
/// and require access to other objects.
|
||||
if (async)
|
||||
lock.unlock();
|
||||
|
||||
LoadablePtr new_object;
|
||||
std::exception_ptr new_exception;
|
||||
try
|
||||
@ -743,10 +800,45 @@ private:
|
||||
{
|
||||
new_exception = std::current_exception();
|
||||
}
|
||||
return std::make_pair(new_object, new_exception);
|
||||
|
||||
if (!new_object && !new_exception)
|
||||
throw Exception("No object created and no exception raised for " + type_name, ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
/// Return single object info, checks loading_id and name
|
||||
std::optional<Info> getSingleObjectInfo(const String & name, size_t loading_id, bool async)
|
||||
{
|
||||
LoadingGuardForAsyncLoad lock(async, mutex);
|
||||
Info * info = getInfo(name);
|
||||
if (!info || !info->loading() || (info->loading_id != loading_id))
|
||||
return {};
|
||||
|
||||
return *info;
|
||||
}
|
||||
|
||||
/// Removes object loading_id from loading_ids if it present
|
||||
/// in other case do nothin should by done with lock
|
||||
void finishObjectLoading(size_t loading_id, const LoadingGuardForAsyncLoad &)
|
||||
{
|
||||
auto it = loading_ids.find(loading_id);
|
||||
if (it != loading_ids.end())
|
||||
{
|
||||
it->second.detach();
|
||||
loading_ids.erase(it);
|
||||
}
|
||||
}
|
||||
|
||||
/// Process loading result
|
||||
/// Calculates next update time and process errors
|
||||
void processLoadResult(
|
||||
const String & name,
|
||||
size_t loading_id,
|
||||
LoadablePtr previous_version,
|
||||
LoadablePtr new_object,
|
||||
std::exception_ptr new_exception,
|
||||
size_t error_count,
|
||||
bool async)
|
||||
{
|
||||
LoadingGuardForAsyncLoad lock(async, mutex);
|
||||
/// Calculate a new update time.
|
||||
TimePoint next_update_time;
|
||||
try
|
||||
@ -755,7 +847,12 @@ private:
|
||||
++error_count;
|
||||
else
|
||||
error_count = 0;
|
||||
next_update_time = calculateNextUpdateTime(new_object, error_count);
|
||||
|
||||
LoadablePtr object = previous_version;
|
||||
if (new_object)
|
||||
object = new_object;
|
||||
|
||||
next_update_time = calculateNextUpdateTime(object, error_count);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
@ -763,10 +860,8 @@ private:
|
||||
next_update_time = TimePoint::max();
|
||||
}
|
||||
|
||||
/// Lock the mutex again to store the changes.
|
||||
if (async)
|
||||
lock.lock();
|
||||
info = getInfo(name);
|
||||
|
||||
Info * info = getInfo(name);
|
||||
|
||||
/// And again we should check if this is still the same loading as we were doing.
|
||||
/// This is necessary because the object could be removed or load with another config while the `mutex` was unlocked.
|
||||
@ -802,10 +897,38 @@ private:
|
||||
if (new_object)
|
||||
info->config_changed = false;
|
||||
|
||||
/// Notify `event` to recheck conditions in loadImpl() now.
|
||||
if (async)
|
||||
lock.unlock();
|
||||
event.notify_all();
|
||||
finishObjectLoading(loading_id, lock);
|
||||
}
|
||||
|
||||
|
||||
/// Does the loading, possibly in the separate thread.
|
||||
void doLoading(const String & name, size_t loading_id, bool async)
|
||||
{
|
||||
try
|
||||
{
|
||||
/// We check here if this is exactly the same loading as we planned to perform.
|
||||
/// This check is necessary because the object could be removed or load with another config before this thread even starts.
|
||||
std::optional<Info> info = getSingleObjectInfo(name, loading_id, async);
|
||||
if (!info)
|
||||
return;
|
||||
|
||||
/// Use `create_function` to perform the actual loading.
|
||||
/// It's much better to do it with `mutex` unlocked because the loading can take a lot of time
|
||||
/// and require access to other objects.
|
||||
auto [new_object, new_exception] = loadOneObject(name, info->object_config, info->config_changed, info->object);
|
||||
if (!new_object && !new_exception)
|
||||
throw Exception("No object created and no exception raised for " + type_name, ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
|
||||
processLoadResult(name, loading_id, info->object, new_object, new_exception, info->error_count, async);
|
||||
event.notify_all();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
LoadingGuardForAsyncLoad lock(async, mutex);
|
||||
finishObjectLoading(loading_id, lock);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
void cancelLoading(const String & name)
|
||||
@ -844,7 +967,8 @@ private:
|
||||
TimePoint calculateNextUpdateTime(const LoadablePtr & loaded_object, size_t error_count) const
|
||||
{
|
||||
static constexpr auto never = TimePoint::max();
|
||||
if (!error_count)
|
||||
|
||||
if (loaded_object)
|
||||
{
|
||||
if (!loaded_object->supportUpdates())
|
||||
return never;
|
||||
@ -854,8 +978,11 @@ private:
|
||||
if (lifetime.min_sec == 0 || lifetime.max_sec == 0)
|
||||
return never;
|
||||
|
||||
std::uniform_int_distribution<UInt64> distribution{lifetime.min_sec, lifetime.max_sec};
|
||||
return std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)};
|
||||
if (!error_count)
|
||||
{
|
||||
std::uniform_int_distribution<UInt64> distribution{lifetime.min_sec, lifetime.max_sec};
|
||||
return std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)};
|
||||
}
|
||||
}
|
||||
|
||||
return std::chrono::system_clock::now() + std::chrono::seconds(calculateDurationWithBackoff(rnd_engine, error_count));
|
||||
@ -870,7 +997,7 @@ private:
|
||||
ObjectConfigsPtr configs;
|
||||
std::unordered_map<String, Info> infos;
|
||||
bool always_load_everything = false;
|
||||
bool enable_async_loading = false;
|
||||
std::atomic<bool> enable_async_loading = false;
|
||||
std::unordered_map<size_t, ThreadFromGlobalPool> loading_ids;
|
||||
size_t next_loading_id = 1; /// should always be > 0
|
||||
mutable pcg64 rnd_engine{randomSeed()};
|
||||
@ -956,12 +1083,19 @@ ExternalLoader::ExternalLoader(const String & type_name_, Logger * log)
|
||||
ExternalLoader::~ExternalLoader() = default;
|
||||
|
||||
void ExternalLoader::addConfigRepository(
|
||||
std::unique_ptr<IExternalLoaderConfigRepository> config_repository, const ExternalLoaderConfigSettings & config_settings)
|
||||
const std::string & repository_name,
|
||||
std::unique_ptr<IExternalLoaderConfigRepository> config_repository,
|
||||
const ExternalLoaderConfigSettings & config_settings)
|
||||
{
|
||||
config_files_reader->addConfigRepository(std::move(config_repository), config_settings);
|
||||
config_files_reader->addConfigRepository(repository_name, std::move(config_repository), config_settings);
|
||||
loading_dispatcher->setConfiguration(config_files_reader->read());
|
||||
}
|
||||
|
||||
void ExternalLoader::removeConfigRepository(const std::string & repository_name)
|
||||
{
|
||||
config_files_reader->removeConfigRepository(repository_name);
|
||||
}
|
||||
|
||||
void ExternalLoader::enableAlwaysLoadEverything(bool enable)
|
||||
{
|
||||
loading_dispatcher->enableAlwaysLoadEverything(enable);
|
||||
@ -1035,23 +1169,52 @@ void ExternalLoader::load(const FilterByNameFunction & filter_by_name, Loadables
|
||||
loading_dispatcher->load(loaded_objects, timeout);
|
||||
}
|
||||
|
||||
|
||||
void ExternalLoader::load(const FilterByNameFunction & filter_by_name, LoadResults & loaded_objects, Duration timeout) const
|
||||
{
|
||||
if (filter_by_name)
|
||||
loading_dispatcher->load(filter_by_name, loaded_objects, timeout);
|
||||
else
|
||||
loading_dispatcher->load(loaded_objects, timeout);
|
||||
}
|
||||
|
||||
|
||||
void ExternalLoader::load(Loadables & loaded_objects, Duration timeout) const
|
||||
{
|
||||
return loading_dispatcher->load(loaded_objects, timeout);
|
||||
}
|
||||
|
||||
void ExternalLoader::reload(const String & name, bool load_never_loading)
|
||||
void ExternalLoader::reload(const String & name, bool load_never_loading) const
|
||||
{
|
||||
loading_dispatcher->setConfiguration(config_files_reader->read());
|
||||
auto configs = config_files_reader->read();
|
||||
loading_dispatcher->setConfiguration(configs);
|
||||
loading_dispatcher->reload(name, load_never_loading);
|
||||
}
|
||||
|
||||
void ExternalLoader::reload(bool load_never_loading)
|
||||
void ExternalLoader::reload(bool load_never_loading) const
|
||||
{
|
||||
loading_dispatcher->setConfiguration(config_files_reader->read());
|
||||
loading_dispatcher->reload(load_never_loading);
|
||||
}
|
||||
|
||||
void ExternalLoader::addObjectAndLoad(
|
||||
const String & name,
|
||||
const String & external_name,
|
||||
const String & repo_name,
|
||||
const Poco::AutoPtr<Poco::Util::AbstractConfiguration> & config,
|
||||
const String & key,
|
||||
bool load_never_loading) const
|
||||
{
|
||||
auto object_config = config_files_reader->updateLoadableInfo(external_name, name, repo_name, config, key);
|
||||
loading_dispatcher->setSingleObjectConfigurationWithoutLoading(external_name, object_config);
|
||||
LoadablePtr loaded_object;
|
||||
if (load_never_loading)
|
||||
loading_dispatcher->loadStrict(name, loaded_object);
|
||||
else
|
||||
loading_dispatcher->load(name, loaded_object, Duration::zero());
|
||||
}
|
||||
|
||||
|
||||
ExternalLoader::LoadablePtr ExternalLoader::createObject(
|
||||
const String & name, const ObjectConfig & config, bool config_changed, const LoadablePtr & previous_version) const
|
||||
{
|
||||
|
@ -72,6 +72,7 @@ public:
|
||||
TimePoint loading_start_time;
|
||||
Duration loading_duration;
|
||||
std::exception_ptr exception;
|
||||
std::string repository_name;
|
||||
};
|
||||
|
||||
using LoadResults = std::vector<std::pair<String, LoadResult>>;
|
||||
@ -81,7 +82,12 @@ public:
|
||||
|
||||
/// Adds a repository which will be used to read configurations from.
|
||||
void addConfigRepository(
|
||||
std::unique_ptr<IExternalLoaderConfigRepository> config_repository, const ExternalLoaderConfigSettings & config_settings);
|
||||
const std::string & repository_name,
|
||||
std::unique_ptr<IExternalLoaderConfigRepository> config_repository,
|
||||
const ExternalLoaderConfigSettings & config_settings);
|
||||
|
||||
/// Removes a repository which were used to read configurations.
|
||||
void removeConfigRepository(const std::string & repository_name);
|
||||
|
||||
/// Sets whether all the objects from the configuration should be always loaded (even those which are never used).
|
||||
void enableAlwaysLoadEverything(bool enable);
|
||||
@ -132,6 +138,7 @@ public:
|
||||
|
||||
/// Tries to finish loading of the objects for which the specified function returns true.
|
||||
void load(const FilterByNameFunction & filter_by_name, Loadables & loaded_objects, Duration timeout = NO_TIMEOUT) const;
|
||||
void load(const FilterByNameFunction & filter_by_name, LoadResults & load_results, Duration timeout = NO_TIMEOUT) const;
|
||||
Loadables loadAndGet(const FilterByNameFunction & filter_by_name, Duration timeout = NO_TIMEOUT) const { Loadables loaded_objects; load(filter_by_name, loaded_objects, timeout); return loaded_objects; }
|
||||
|
||||
/// Tries to finish loading of all the objects during the timeout.
|
||||
@ -140,16 +147,27 @@ public:
|
||||
/// Starts reloading of a specified object.
|
||||
/// `load_never_loading` specifies what to do if the object has never been loading before.
|
||||
/// The function can either skip it (false) or load for the first time (true).
|
||||
void reload(const String & name, bool load_never_loading = false);
|
||||
/// Also function can load dictionary synchronously
|
||||
void reload(const String & name, bool load_never_loading = false) const;
|
||||
|
||||
|
||||
/// Starts reloading of all the objects.
|
||||
/// `load_never_loading` specifies what to do with the objects which have never been loading before.
|
||||
/// The function can either skip them (false) or load for the first time (true).
|
||||
void reload(bool load_never_loading = false);
|
||||
void reload(bool load_never_loading = false) const;
|
||||
|
||||
protected:
|
||||
virtual LoadablePtr create(const String & name, const Poco::Util::AbstractConfiguration & config, const String & key_in_config) const = 0;
|
||||
|
||||
/// Reload object with already parsed configuration
|
||||
void addObjectAndLoad(
|
||||
const String & name, /// name of dictionary
|
||||
const String & external_name, /// name of source (example xml-file, may contain more than dictionary)
|
||||
const String & repo_name, /// name of repository (database name, or all xml files)
|
||||
const Poco::AutoPtr<Poco::Util::AbstractConfiguration> & config,
|
||||
const String & key_in_config, /// key where we can start search of loadables (<dictionary>, <model>, etc)
|
||||
bool load_never_loading = false) const;
|
||||
|
||||
private:
|
||||
struct ObjectConfig;
|
||||
|
||||
|
@ -0,0 +1,56 @@
|
||||
#include <Interpreters/ExternalLoaderDatabaseConfigRepository.h>
|
||||
#include <Dictionaries/getDictionaryConfigurationFromAST.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int UNKNOWN_DICTIONARY;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
String trimDatabaseName(const std::string & loadable_definition_name, const DatabasePtr database)
|
||||
{
|
||||
const auto & dbname = database->getDatabaseName();
|
||||
if (!startsWith(loadable_definition_name, dbname))
|
||||
throw Exception(
|
||||
"Loadable '" + loadable_definition_name + "' is not from database '" + database->getDatabaseName(), ErrorCodes::UNKNOWN_DICTIONARY);
|
||||
/// dbname.loadable_name
|
||||
///--> remove <---
|
||||
return loadable_definition_name.substr(dbname.length() + 1);
|
||||
}
|
||||
}
|
||||
|
||||
LoadablesConfigurationPtr ExternalLoaderDatabaseConfigRepository::load(const std::string & loadable_definition_name) const
|
||||
{
|
||||
String dictname = trimDatabaseName(loadable_definition_name, database);
|
||||
return getDictionaryConfigurationFromAST(database->getCreateDictionaryQuery(context, dictname)->as<const ASTCreateQuery &>());
|
||||
}
|
||||
|
||||
bool ExternalLoaderDatabaseConfigRepository::exists(const std::string & loadable_definition_name) const
|
||||
{
|
||||
return database->isDictionaryExist(
|
||||
context, trimDatabaseName(loadable_definition_name, database));
|
||||
}
|
||||
|
||||
Poco::Timestamp ExternalLoaderDatabaseConfigRepository::getUpdateTime(const std::string & loadable_definition_name)
|
||||
{
|
||||
return database->getObjectMetadataModificationTime(context, trimDatabaseName(loadable_definition_name, database));
|
||||
}
|
||||
|
||||
std::set<std::string> ExternalLoaderDatabaseConfigRepository::getAllLoadablesDefinitionNames() const
|
||||
{
|
||||
std::set<std::string> result;
|
||||
const auto & dbname = database->getDatabaseName();
|
||||
auto itr = database->getDictionariesIterator(context);
|
||||
while (itr && itr->isValid())
|
||||
{
|
||||
result.insert(dbname + "." + itr->name());
|
||||
itr->next();
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,34 @@
|
||||
#pragma once
|
||||
|
||||
#include <Interpreters/IExternalLoaderConfigRepository.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Repository from database, which stores dictionary definitions on disk.
|
||||
/// Tracks update time and existance of .sql files through IDatabase.
|
||||
class ExternalLoaderDatabaseConfigRepository : public IExternalLoaderConfigRepository
|
||||
{
|
||||
public:
|
||||
ExternalLoaderDatabaseConfigRepository(const DatabasePtr & database_, const Context & context_)
|
||||
: database(database_)
|
||||
, context(context_)
|
||||
{
|
||||
}
|
||||
|
||||
std::set<std::string> getAllLoadablesDefinitionNames() const override;
|
||||
|
||||
bool exists(const std::string & loadable_definition_name) const override;
|
||||
|
||||
Poco::Timestamp getUpdateTime(const std::string & loadable_definition_name) override;
|
||||
|
||||
LoadablesConfigurationPtr load(const std::string & loadable_definition_name) const override;
|
||||
|
||||
private:
|
||||
DatabasePtr database;
|
||||
Context context;
|
||||
};
|
||||
|
||||
}
|
@ -10,12 +10,10 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
|
||||
ExternalModelsLoader::ExternalModelsLoader(
|
||||
ExternalLoaderConfigRepositoryPtr config_repository, Context & context_)
|
||||
ExternalModelsLoader::ExternalModelsLoader(Context & context_)
|
||||
: ExternalLoader("external model", &Logger::get("ExternalModelsLoader"))
|
||||
, context(context_)
|
||||
{
|
||||
addConfigRepository(std::move(config_repository), {"model", "name"});
|
||||
enablePeriodicUpdates(true);
|
||||
}
|
||||
|
||||
@ -40,4 +38,8 @@ std::shared_ptr<const IExternalLoadable> ExternalModelsLoader::create(
|
||||
}
|
||||
}
|
||||
|
||||
void ExternalModelsLoader::addConfigRepository(const String & name, std::unique_ptr<IExternalLoaderConfigRepository> config_repository)
|
||||
{
|
||||
ExternalLoader::addConfigRepository(name, std::move(config_repository), {"models", "name"});
|
||||
}
|
||||
}
|
||||
|
@ -18,15 +18,17 @@ public:
|
||||
using ModelPtr = std::shared_ptr<const IModel>;
|
||||
|
||||
/// Models will be loaded immediately and then will be updated in separate thread, each 'reload_period' seconds.
|
||||
ExternalModelsLoader(
|
||||
ExternalLoaderConfigRepositoryPtr config_repository,
|
||||
Context & context_);
|
||||
ExternalModelsLoader(Context & context_);
|
||||
|
||||
ModelPtr getModel(const std::string & name) const
|
||||
{
|
||||
return std::static_pointer_cast<const IModel>(getLoadable(name));
|
||||
}
|
||||
|
||||
void addConfigRepository(const String & name,
|
||||
std::unique_ptr<IExternalLoaderConfigRepository> config_repository);
|
||||
|
||||
|
||||
protected:
|
||||
LoadablePtr create(const std::string & name, const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & key_in_config) const override;
|
||||
|
@ -24,6 +24,11 @@ UInt64 calculateDurationWithBackoff(pcg64 & rnd_engine, size_t error_count)
|
||||
|
||||
if (error_count < 1)
|
||||
error_count = 1;
|
||||
|
||||
/// max seconds is 600 and 2 ** 10 == 1024
|
||||
if (error_count > 11)
|
||||
error_count = 11;
|
||||
|
||||
std::uniform_int_distribution<UInt64> distribution(0, static_cast<UInt64>(std::exp2(error_count - 1)));
|
||||
return std::min<UInt64>(backoff_max_sec, backoff_initial_sec + distribution(rnd_engine));
|
||||
}
|
||||
|
@ -69,6 +69,7 @@ namespace ErrorCodes
|
||||
extern const int THERE_IS_NO_DEFAULT_VALUE;
|
||||
extern const int BAD_DATABASE_FOR_TEMPORARY_TABLE;
|
||||
extern const int SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY;
|
||||
extern const int DICTIONARY_ALREADY_EXISTS;
|
||||
}
|
||||
|
||||
|
||||
@ -163,7 +164,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
|
||||
if (need_write_metadata)
|
||||
Poco::File(metadata_file_tmp_path).renameTo(metadata_file_path);
|
||||
|
||||
database->loadTables(context, has_force_restore_data_flag);
|
||||
database->loadStoredObjects(context, has_force_restore_data_flag);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
@ -630,6 +631,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
|
||||
/// Table can be created before or it can be created concurrently in another thread, while we were waiting in DDLGuard.
|
||||
if (database->isTableExist(context, table_name))
|
||||
{
|
||||
/// TODO Check structure of table
|
||||
if (create.if_not_exists)
|
||||
return {};
|
||||
else if (create.replace_view)
|
||||
@ -704,6 +706,39 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
|
||||
return {};
|
||||
}
|
||||
|
||||
BlockIO InterpreterCreateQuery::createDictionary(ASTCreateQuery & create)
|
||||
{
|
||||
String dictionary_name = create.table;
|
||||
|
||||
String database_name = !create.database.empty() ? create.database : context.getCurrentDatabase();
|
||||
|
||||
auto guard = context.getDDLGuard(database_name, dictionary_name);
|
||||
DatabasePtr database = context.getDatabase(database_name);
|
||||
|
||||
if (database->isDictionaryExist(context, dictionary_name))
|
||||
{
|
||||
/// TODO Check structure of dictionary
|
||||
if (create.if_not_exists)
|
||||
return {};
|
||||
else
|
||||
throw Exception(
|
||||
"Dictionary " + database_name + "." + dictionary_name + " already exists.", ErrorCodes::DICTIONARY_ALREADY_EXISTS);
|
||||
}
|
||||
|
||||
if (create.attach)
|
||||
{
|
||||
auto query = context.getCreateDictionaryQuery(database_name, dictionary_name);
|
||||
create = query->as<ASTCreateQuery &>();
|
||||
create.attach = true;
|
||||
}
|
||||
|
||||
if (create.attach)
|
||||
database->attachDictionary(dictionary_name, context);
|
||||
else
|
||||
database->createDictionary(context, dictionary_name, query_ptr);
|
||||
|
||||
return {};
|
||||
}
|
||||
|
||||
BlockIO InterpreterCreateQuery::execute()
|
||||
{
|
||||
@ -713,11 +748,11 @@ BlockIO InterpreterCreateQuery::execute()
|
||||
|
||||
/// CREATE|ATTACH DATABASE
|
||||
if (!create.database.empty() && create.table.empty())
|
||||
{
|
||||
return createDatabase(create);
|
||||
}
|
||||
else
|
||||
else if (!create.is_dictionary)
|
||||
return createTable(create);
|
||||
else
|
||||
return createDictionary(create);
|
||||
}
|
||||
|
||||
|
||||
@ -742,13 +777,22 @@ void InterpreterCreateQuery::checkAccess(const ASTCreateQuery & create)
|
||||
|
||||
throw Exception("Cannot create database. DDL queries are prohibited for the user", ErrorCodes::QUERY_IS_PROHIBITED);
|
||||
}
|
||||
String object = "table";
|
||||
|
||||
if (create.is_dictionary)
|
||||
{
|
||||
if (readonly)
|
||||
throw Exception("Cannot create dictionary in readonly mode", ErrorCodes::READONLY);
|
||||
object = "dictionary";
|
||||
}
|
||||
|
||||
if (create.temporary && readonly >= 2)
|
||||
return;
|
||||
|
||||
if (readonly)
|
||||
throw Exception("Cannot create table in readonly mode", ErrorCodes::READONLY);
|
||||
throw Exception("Cannot create table or dictionary in readonly mode", ErrorCodes::READONLY);
|
||||
|
||||
throw Exception("Cannot create table. DDL queries are prohibited for the user", ErrorCodes::QUERY_IS_PROHIBITED);
|
||||
throw Exception("Cannot create " + object + ". DDL queries are prohibited for the user", ErrorCodes::QUERY_IS_PROHIBITED);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -51,6 +51,7 @@ public:
|
||||
private:
|
||||
BlockIO createDatabase(ASTCreateQuery & create);
|
||||
BlockIO createTable(ASTCreateQuery & create);
|
||||
BlockIO createDictionary(ASTCreateQuery & create);
|
||||
|
||||
/// Calculate list of columns, constraints, indices, etc... of table and return columns.
|
||||
ColumnsDescription setProperties(ASTCreateQuery & create, const Block & as_select_sample, const StoragePtr & as_storage) const;
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DDLWorker.h>
|
||||
#include <Interpreters/InterpreterDropQuery.h>
|
||||
#include <Interpreters/ExternalDictionariesLoader.h>
|
||||
#include <Parsers/ASTDropQuery.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
@ -24,6 +25,7 @@ namespace ErrorCodes
|
||||
extern const int SYNTAX_ERROR;
|
||||
extern const int UNKNOWN_TABLE;
|
||||
extern const int QUERY_IS_PROHIBITED;
|
||||
extern const int UNKNOWN_DICTIONARY;
|
||||
}
|
||||
|
||||
|
||||
@ -40,15 +42,26 @@ BlockIO InterpreterDropQuery::execute()
|
||||
return executeDDLQueryOnCluster(query_ptr, context, {drop.database});
|
||||
|
||||
if (!drop.table.empty())
|
||||
return executeToTable(drop.database, drop.table, drop.kind, drop.if_exists, drop.temporary, drop.no_ddl_lock);
|
||||
{
|
||||
if (!drop.is_dictionary)
|
||||
return executeToTable(drop.database, drop.table, drop.kind, drop.if_exists, drop.temporary, drop.no_ddl_lock);
|
||||
else
|
||||
return executeToDictionary(drop.database, drop.table, drop.kind, drop.if_exists, drop.temporary, drop.no_ddl_lock);
|
||||
}
|
||||
else if (!drop.database.empty())
|
||||
return executeToDatabase(drop.database, drop.kind, drop.if_exists);
|
||||
else
|
||||
throw Exception("Database and table names is empty.", ErrorCodes::LOGICAL_ERROR);
|
||||
throw Exception("Nothing to drop, both names are empty.", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
|
||||
BlockIO InterpreterDropQuery::executeToTable(String & database_name_, String & table_name, ASTDropQuery::Kind kind, bool if_exists, bool if_temporary, bool no_ddl_lock)
|
||||
BlockIO InterpreterDropQuery::executeToTable(
|
||||
String & database_name_,
|
||||
String & table_name,
|
||||
ASTDropQuery::Kind kind,
|
||||
bool if_exists,
|
||||
bool if_temporary,
|
||||
bool no_ddl_lock)
|
||||
{
|
||||
if (if_temporary || database_name_.empty())
|
||||
{
|
||||
@ -135,6 +148,50 @@ BlockIO InterpreterDropQuery::executeToTable(String & database_name_, String & t
|
||||
return {};
|
||||
}
|
||||
|
||||
|
||||
BlockIO InterpreterDropQuery::executeToDictionary(
|
||||
String & database_name_,
|
||||
String & dictionary_name,
|
||||
ASTDropQuery::Kind kind,
|
||||
bool if_exists,
|
||||
bool is_temporary,
|
||||
bool no_ddl_lock)
|
||||
{
|
||||
if (is_temporary)
|
||||
throw Exception("Temporary dictionaries are not possible.", ErrorCodes::SYNTAX_ERROR);
|
||||
|
||||
String database_name = database_name_.empty() ? context.getCurrentDatabase() : database_name_;
|
||||
|
||||
auto ddl_guard = (!no_ddl_lock ? context.getDDLGuard(database_name, dictionary_name) : nullptr);
|
||||
|
||||
DatabasePtr database = tryGetDatabase(database_name, false);
|
||||
|
||||
if (!database || !database->isDictionaryExist(context, dictionary_name))
|
||||
{
|
||||
if (!if_exists)
|
||||
throw Exception(
|
||||
"Dictionary " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(dictionary_name) + " doesn't exist.",
|
||||
ErrorCodes::UNKNOWN_DICTIONARY);
|
||||
else
|
||||
return {};
|
||||
}
|
||||
|
||||
if (kind == ASTDropQuery::Kind::Detach)
|
||||
{
|
||||
/// Drop dictionary from memory, don't touch data and metadata
|
||||
database->detachDictionary(dictionary_name, context);
|
||||
}
|
||||
else if (kind == ASTDropQuery::Kind::Truncate)
|
||||
{
|
||||
throw Exception("Cannot TRUNCATE dictionary", ErrorCodes::SYNTAX_ERROR);
|
||||
}
|
||||
else if (kind == ASTDropQuery::Kind::Drop)
|
||||
{
|
||||
database->removeDictionary(context, dictionary_name);
|
||||
}
|
||||
return {};
|
||||
}
|
||||
|
||||
BlockIO InterpreterDropQuery::executeToTemporaryTable(String & table_name, ASTDropQuery::Kind kind)
|
||||
{
|
||||
if (kind == ASTDropQuery::Kind::Detach)
|
||||
@ -185,12 +242,18 @@ BlockIO InterpreterDropQuery::executeToDatabase(String & database_name, ASTDropQ
|
||||
}
|
||||
else if (kind == ASTDropQuery::Kind::Drop)
|
||||
{
|
||||
for (auto iterator = database->getIterator(context); iterator->isValid(); iterator->next())
|
||||
for (auto iterator = database->getTablesIterator(context); iterator->isValid(); iterator->next())
|
||||
{
|
||||
String current_table_name = iterator->name();
|
||||
executeToTable(database_name, current_table_name, kind, false, false, false);
|
||||
}
|
||||
|
||||
for (auto iterator = database->getDictionariesIterator(context); iterator->isValid(); iterator->next())
|
||||
{
|
||||
String current_dictionary = iterator->name();
|
||||
executeToDictionary(database_name, current_dictionary, kind, false, false, false);
|
||||
}
|
||||
|
||||
auto context_lock = context.getLock();
|
||||
|
||||
/// Someone could have time to delete the database before us.
|
||||
|
@ -32,6 +32,8 @@ private:
|
||||
|
||||
BlockIO executeToTable(String & database_name, String & table_name, ASTDropQuery::Kind kind, bool if_exists, bool if_temporary, bool no_ddl_lock);
|
||||
|
||||
BlockIO executeToDictionary(String & database_name, String & table_name, ASTDropQuery::Kind kind, bool if_exists, bool if_temporary, bool no_ddl_lock);
|
||||
|
||||
DatabasePtr tryGetDatabase(String & database_name, bool exists);
|
||||
|
||||
DatabaseAndTable tryGetDatabaseAndTable(String & database_name, String & table_name, bool if_exists);
|
||||
|
@ -13,6 +13,11 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int SYNTAX_ERROR;
|
||||
}
|
||||
|
||||
BlockIO InterpreterExistsQuery::execute()
|
||||
{
|
||||
BlockIO res;
|
||||
@ -32,11 +37,24 @@ Block InterpreterExistsQuery::getSampleBlock()
|
||||
|
||||
BlockInputStreamPtr InterpreterExistsQuery::executeImpl()
|
||||
{
|
||||
const auto & ast = query_ptr->as<ASTExistsTableQuery &>();
|
||||
bool res = ast.temporary ? context.isExternalTableExist(ast.table) : context.isTableExist(ast.database, ast.table);
|
||||
ASTQueryWithTableAndOutput * exists_query;
|
||||
bool result = false;
|
||||
if (exists_query = query_ptr->as<ASTExistsTableQuery>(); exists_query)
|
||||
{
|
||||
if (exists_query->temporary)
|
||||
result = context.isExternalTableExist(exists_query->table);
|
||||
else
|
||||
result = context.isTableExist(exists_query->database, exists_query->table);
|
||||
}
|
||||
else if (exists_query = query_ptr->as<ASTExistsDictionaryQuery>(); exists_query)
|
||||
{
|
||||
if (exists_query->temporary)
|
||||
throw Exception("Temporary dictionaries are not possible.", ErrorCodes::SYNTAX_ERROR);
|
||||
result = context.isDictionaryExists(exists_query->database, exists_query->table);
|
||||
}
|
||||
|
||||
return std::make_shared<OneBlockInputStream>(Block{{
|
||||
ColumnUInt8::create(1, res),
|
||||
ColumnUInt8::create(1, result),
|
||||
std::make_shared<DataTypeUInt8>(),
|
||||
"result" }});
|
||||
}
|
||||
|
@ -137,6 +137,10 @@ std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, Context &
|
||||
{
|
||||
return std::make_unique<InterpreterExistsQuery>(query, context);
|
||||
}
|
||||
else if (query->as<ASTExistsDictionaryQuery>())
|
||||
{
|
||||
return std::make_unique<InterpreterExistsQuery>(query, context);
|
||||
}
|
||||
else if (query->as<ASTShowCreateTableQuery>())
|
||||
{
|
||||
return std::make_unique<InterpreterShowCreateQuery>(query, context);
|
||||
@ -145,6 +149,10 @@ std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, Context &
|
||||
{
|
||||
return std::make_unique<InterpreterShowCreateQuery>(query, context);
|
||||
}
|
||||
else if (query->as<ASTShowCreateDictionaryQuery>())
|
||||
{
|
||||
return std::make_unique<InterpreterShowCreateQuery>(query, context);
|
||||
}
|
||||
else if (query->as<ASTDescribeQuery>())
|
||||
{
|
||||
return std::make_unique<InterpreterDescribeQuery>(query, context);
|
||||
|
@ -42,22 +42,30 @@ Block InterpreterShowCreateQuery::getSampleBlock()
|
||||
|
||||
BlockInputStreamPtr InterpreterShowCreateQuery::executeImpl()
|
||||
{
|
||||
/// FIXME: try to prettify this cast using `as<>()`
|
||||
const auto & ast = dynamic_cast<const ASTQueryWithTableAndOutput &>(*query_ptr);
|
||||
|
||||
if (ast.temporary && !ast.database.empty())
|
||||
throw Exception("Temporary databases are not possible.", ErrorCodes::SYNTAX_ERROR);
|
||||
|
||||
ASTPtr create_query;
|
||||
if (ast.temporary)
|
||||
create_query = context.getCreateExternalTableQuery(ast.table);
|
||||
else if (ast.table.empty())
|
||||
create_query = context.getCreateDatabaseQuery(ast.database);
|
||||
else
|
||||
create_query = context.getCreateTableQuery(ast.database, ast.table);
|
||||
ASTQueryWithTableAndOutput * show_query;
|
||||
if (show_query = query_ptr->as<ASTShowCreateTableQuery>(); show_query)
|
||||
{
|
||||
if (show_query->temporary)
|
||||
create_query = context.getCreateExternalTableQuery(show_query->table);
|
||||
else
|
||||
create_query = context.getCreateTableQuery(show_query->database, show_query->table);
|
||||
}
|
||||
else if (show_query = query_ptr->as<ASTShowCreateDatabaseQuery>(); show_query)
|
||||
{
|
||||
if (show_query->temporary)
|
||||
throw Exception("Temporary databases are not possible.", ErrorCodes::SYNTAX_ERROR);
|
||||
create_query = context.getCreateDatabaseQuery(show_query->database);
|
||||
}
|
||||
else if (show_query = query_ptr->as<ASTShowCreateDictionaryQuery>(); show_query)
|
||||
{
|
||||
if (show_query->temporary)
|
||||
throw Exception("Temporary dictionaries are not possible.", ErrorCodes::SYNTAX_ERROR);
|
||||
create_query = context.getCreateDictionaryQuery(show_query->database, show_query->table);
|
||||
}
|
||||
|
||||
if (!create_query && ast.temporary)
|
||||
throw Exception("Unable to show the create query of " + ast.table + ". Maybe it was created by the system.", ErrorCodes::THERE_IS_NO_QUERY);
|
||||
if (!create_query && show_query->temporary)
|
||||
throw Exception("Unable to show the create query of " + show_query->table + ". Maybe it was created by the system.", ErrorCodes::THERE_IS_NO_QUERY);
|
||||
|
||||
std::stringstream stream;
|
||||
formatAST(*create_query, stream, false, true);
|
||||
|
@ -44,10 +44,21 @@ String InterpreterShowTablesQuery::getRewrittenQuery()
|
||||
context.assertDatabaseExists(database, false);
|
||||
|
||||
std::stringstream rewritten_query;
|
||||
rewritten_query << "SELECT name FROM system.tables WHERE ";
|
||||
rewritten_query << "SELECT name FROM system.";
|
||||
|
||||
if (query.dictionaries)
|
||||
rewritten_query << "dictionaries ";
|
||||
else
|
||||
rewritten_query << "tables ";
|
||||
|
||||
rewritten_query << "WHERE ";
|
||||
|
||||
if (query.temporary)
|
||||
{
|
||||
if (query.dictionaries)
|
||||
throw Exception("Temporary dictionaries are not possible.", ErrorCodes::SYNTAX_ERROR);
|
||||
rewritten_query << "is_temporary";
|
||||
}
|
||||
else
|
||||
rewritten_query << "database = " << std::quoted(database, '\'');
|
||||
|
||||
|
@ -315,7 +315,7 @@ void InterpreterSystemQuery::restartReplicas(Context & system_context)
|
||||
DatabasePtr & database = elem.second;
|
||||
const String & database_name = elem.first;
|
||||
|
||||
for (auto iterator = database->getIterator(system_context); iterator->isValid(); iterator->next())
|
||||
for (auto iterator = database->getTablesIterator(system_context); iterator->isValid(); iterator->next())
|
||||
{
|
||||
if (dynamic_cast<const StorageReplicatedMergeTree *>(iterator->table().get()))
|
||||
replica_names.emplace_back(database_name, iterator->name());
|
||||
|
@ -84,7 +84,7 @@ try
|
||||
context.setPath("./");
|
||||
auto database = std::make_shared<DatabaseOrdinary>("test", "./metadata/test/", context);
|
||||
context.addDatabase("test", database);
|
||||
database->loadTables(context, false);
|
||||
database->loadStoredObjects(context, false);
|
||||
context.setCurrentDatabase("test");
|
||||
|
||||
InterpreterCreateQuery interpreter(ast, context);
|
||||
|
@ -39,7 +39,7 @@ try
|
||||
|
||||
DatabasePtr system = std::make_shared<DatabaseOrdinary>("system", "./metadata/system/", context);
|
||||
context.addDatabase("system", system);
|
||||
system->loadTables(context, false);
|
||||
system->loadStoredObjects(context, false);
|
||||
attachSystemTablesLocal(*context.getDatabase("system"));
|
||||
context.setCurrentDatabase("default");
|
||||
|
||||
|
@ -238,10 +238,10 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Always CREATE and always DICTIONARY
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << "CREATE DICTIONARY " << (if_not_exists ? "IF NOT EXISTS " : "")
|
||||
<< (settings.hilite ? hilite_none : "") << (!database.empty() ? backQuoteIfNeed(database) + "." : "")
|
||||
<< backQuoteIfNeed(table);
|
||||
/// Always DICTIONARY
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << (attach ? "ATTACH " : "CREATE ") << "DICTIONARY "
|
||||
<< (if_not_exists ? "IF NOT EXISTS " : "") << (settings.hilite ? hilite_none : "")
|
||||
<< (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table);
|
||||
}
|
||||
|
||||
if (as_table_function)
|
||||
|
@ -24,8 +24,7 @@ void ASTDictionaryRange::formatImpl(const FormatSettings & settings,
|
||||
<< "("
|
||||
<< (settings.hilite ? hilite_keyword : "")
|
||||
<< "MIN "
|
||||
<< (settings.hilite ? hilite_none : "")
|
||||
<< min_attr_name << ", "
|
||||
<< min_attr_name << " "
|
||||
<< (settings.hilite ? hilite_keyword : "")
|
||||
<< "MAX "
|
||||
<< (settings.hilite ? hilite_none : "")
|
||||
@ -54,8 +53,7 @@ void ASTDictionaryLifetime::formatImpl(const FormatSettings & settings,
|
||||
<< "("
|
||||
<< (settings.hilite ? hilite_keyword : "")
|
||||
<< "MIN "
|
||||
<< (settings.hilite ? hilite_none : "")
|
||||
<< min_sec << ", "
|
||||
<< min_sec << " "
|
||||
<< (settings.hilite ? hilite_keyword : "")
|
||||
<< "MAX "
|
||||
<< (settings.hilite ? hilite_none : "")
|
||||
@ -133,14 +131,17 @@ void ASTDictionary::formatImpl(const FormatSettings & settings, FormatState & st
|
||||
if (primary_key)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << settings.nl_or_ws << "PRIMARY KEY "
|
||||
<< (settings.hilite ? hilite_none : "");
|
||||
<< (settings.hilite ? hilite_none : "");
|
||||
primary_key->formatImpl(settings, state, frame);
|
||||
}
|
||||
|
||||
if (source)
|
||||
settings.ostr << settings.nl_or_ws;
|
||||
|
||||
source->formatImpl(settings, state, frame);
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << settings.nl_or_ws << "SOURCE("
|
||||
<< (settings.hilite ? hilite_none : "");
|
||||
source->formatImpl(settings, state, frame);
|
||||
settings.ostr << ")";
|
||||
}
|
||||
|
||||
if (lifetime)
|
||||
{
|
||||
|
@ -22,7 +22,8 @@ void ASTShowTablesQuery::formatQueryImpl(const FormatSettings & settings, Format
|
||||
}
|
||||
else
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << "SHOW " << (temporary ? "TEMPORARY " : "") << "TABLES" << (settings.hilite ? hilite_none : "");
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << "SHOW " << (temporary ? "TEMPORARY " : "") <<
|
||||
(dictionaries ? "DICTIONARIES" : "TABLES") << (settings.hilite ? hilite_none : "");
|
||||
|
||||
if (!from.empty())
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : "")
|
||||
@ -41,4 +42,3 @@ void ASTShowTablesQuery::formatQueryImpl(const FormatSettings & settings, Format
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
@ -15,6 +15,7 @@ class ASTShowTablesQuery : public ASTQueryWithOutput
|
||||
{
|
||||
public:
|
||||
bool databases{false};
|
||||
bool dictionaries{false};
|
||||
bool temporary{false};
|
||||
String from;
|
||||
String like;
|
||||
|
@ -823,6 +823,7 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ParserKeyword s_create("CREATE");
|
||||
ParserKeyword s_attach("ATTACH");
|
||||
ParserKeyword s_dictionary("DICTIONARY");
|
||||
ParserKeyword s_if_not_exists("IF NOT EXISTS");
|
||||
ParserIdentifier name_p;
|
||||
@ -840,8 +841,14 @@ bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, E
|
||||
ASTPtr attributes;
|
||||
ASTPtr dictionary;
|
||||
|
||||
bool attach = false;
|
||||
if (!s_create.ignore(pos, expected))
|
||||
return false;
|
||||
{
|
||||
if (s_attach.ignore(pos, expected))
|
||||
attach = true;
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
if (s_if_not_exists.ignore(pos, expected))
|
||||
if_not_exists = true;
|
||||
@ -859,21 +866,25 @@ bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, E
|
||||
return false;
|
||||
}
|
||||
|
||||
if (!s_left_paren.ignore(pos, expected))
|
||||
return false;
|
||||
if (!attach)
|
||||
{
|
||||
if (!s_left_paren.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
if (!attributes_p.parse(pos, attributes, expected))
|
||||
return false;
|
||||
if (!attributes_p.parse(pos, attributes, expected))
|
||||
return false;
|
||||
|
||||
if (!s_right_paren.ignore(pos, expected))
|
||||
return false;
|
||||
if (!s_right_paren.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
if (!dictionary_p.parse(pos, dictionary, expected))
|
||||
return false;
|
||||
if (!dictionary_p.parse(pos, dictionary, expected))
|
||||
return false;
|
||||
}
|
||||
|
||||
auto query = std::make_shared<ASTCreateQuery>();
|
||||
node = query;
|
||||
query->is_dictionary = true;
|
||||
query->attach = attach;
|
||||
|
||||
if (database)
|
||||
query->database = typeid_cast<const ASTIdentifier &>(*database).name;
|
||||
|
@ -20,6 +20,7 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
ParserKeyword s_temporary("TEMPORARY");
|
||||
ParserKeyword s_tables("TABLES");
|
||||
ParserKeyword s_databases("DATABASES");
|
||||
ParserKeyword s_dictionaries("DICTIONARIES");
|
||||
ParserKeyword s_from("FROM");
|
||||
ParserKeyword s_not("NOT");
|
||||
ParserKeyword s_like("LIKE");
|
||||
@ -45,33 +46,36 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
if (s_temporary.ignore(pos))
|
||||
query->temporary = true;
|
||||
|
||||
if (s_tables.ignore(pos, expected))
|
||||
if (!s_tables.ignore(pos, expected))
|
||||
{
|
||||
if (s_from.ignore(pos, expected))
|
||||
{
|
||||
if (!name_p.parse(pos, database, expected))
|
||||
return false;
|
||||
}
|
||||
|
||||
if (s_not.ignore(pos, expected))
|
||||
query->not_like = true;
|
||||
|
||||
if (s_like.ignore(pos, expected))
|
||||
{
|
||||
if (!like_p.parse(pos, like, expected))
|
||||
return false;
|
||||
}
|
||||
else if (query->not_like)
|
||||
if (s_dictionaries.ignore(pos, expected))
|
||||
query->dictionaries = true;
|
||||
else
|
||||
return false;
|
||||
|
||||
if (s_limit.ignore(pos, expected))
|
||||
{
|
||||
if (!limit_p.parse(pos, query->limit_length, expected))
|
||||
return false;
|
||||
}
|
||||
}
|
||||
else
|
||||
|
||||
if (s_from.ignore(pos, expected))
|
||||
{
|
||||
if (!name_p.parse(pos, database, expected))
|
||||
return false;
|
||||
}
|
||||
|
||||
if (s_not.ignore(pos, expected))
|
||||
query->not_like = true;
|
||||
|
||||
if (s_like.ignore(pos, expected))
|
||||
{
|
||||
if (!like_p.parse(pos, like, expected))
|
||||
return false;
|
||||
}
|
||||
else if (query->not_like)
|
||||
return false;
|
||||
|
||||
if (s_limit.ignore(pos, expected))
|
||||
{
|
||||
if (!limit_p.parse(pos, query->limit_length, expected))
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
tryGetIdentifierNameInto(database, query->from);
|
||||
|
@ -291,7 +291,7 @@ TEST(ParserDictionaryDDL, Formatting)
|
||||
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
|
||||
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
|
||||
auto str = serializeAST(*create, true);
|
||||
EXPECT_EQ(str, "CREATE DICTIONARY test.dict5 (`key_column1` UInt64 DEFAULT 1 HIERARCHICAL INJECTIVE, `key_column2` String DEFAULT '', `second_column` UInt8 EXPRESSION intDiv(50, rand() % 1000), `third_column` UInt8) PRIMARY KEY key_column1, key_column2 MYSQL(HOST 'localhost' PORT 9000 USER 'default' REPLICA (HOST '127.0.0.1' PRIORITY 1) PASSWORD '') LIFETIME(MIN 1, MAX 10) LAYOUT(CACHE(SIZE_IN_CELLS 50)) RANGE(MIN second_column, MAX third_column)");
|
||||
EXPECT_EQ(str, "CREATE DICTIONARY test.dict5 (`key_column1` UInt64 DEFAULT 1 HIERARCHICAL INJECTIVE, `key_column2` String DEFAULT '', `second_column` UInt8 EXPRESSION intDiv(50, rand() % 1000), `third_column` UInt8) PRIMARY KEY key_column1, key_column2 SOURCE(MYSQL(HOST 'localhost' PORT 9000 USER 'default' REPLICA (HOST '127.0.0.1' PRIORITY 1) PASSWORD '')) LIFETIME(MIN 1 MAX 10) LAYOUT(CACHE(SIZE_IN_CELLS 50)) RANGE(MIN second_column MAX third_column)");
|
||||
}
|
||||
|
||||
TEST(ParserDictionaryDDL, ParseDropQuery)
|
||||
|
@ -143,7 +143,7 @@ QueryProcessingStage::Enum StorageMerge::getQueryProcessingStage(const Context &
|
||||
{
|
||||
auto stage_in_source_tables = QueryProcessingStage::FetchColumns;
|
||||
|
||||
DatabaseIteratorPtr iterator = getDatabaseIterator(context);
|
||||
DatabaseTablesIteratorPtr iterator = getDatabaseIterator(context);
|
||||
|
||||
size_t selected_table_size = 0;
|
||||
|
||||
@ -353,7 +353,7 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const String
|
||||
StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const ASTPtr & query, bool has_virtual_column, bool get_lock, const String & query_id) const
|
||||
{
|
||||
StorageListWithLocks selected_tables;
|
||||
DatabaseIteratorPtr iterator = getDatabaseIterator(global_context);
|
||||
DatabaseTablesIteratorPtr iterator = getDatabaseIterator(global_context);
|
||||
|
||||
auto virtual_column = ColumnString::create();
|
||||
|
||||
@ -387,12 +387,12 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const ASTPtr
|
||||
}
|
||||
|
||||
|
||||
DatabaseIteratorPtr StorageMerge::getDatabaseIterator(const Context & context) const
|
||||
DatabaseTablesIteratorPtr StorageMerge::getDatabaseIterator(const Context & context) const
|
||||
{
|
||||
checkStackSize();
|
||||
auto database = context.getDatabase(source_database);
|
||||
auto table_name_match = [this](const String & table_name_) { return table_name_regexp.match(table_name_); };
|
||||
return database->getIterator(global_context, table_name_match);
|
||||
return database->getTablesIterator(global_context, table_name_match);
|
||||
}
|
||||
|
||||
|
||||
|
@ -71,7 +71,7 @@ private:
|
||||
template <typename F>
|
||||
StoragePtr getFirstTable(F && predicate) const;
|
||||
|
||||
DatabaseIteratorPtr getDatabaseIterator(const Context & context) const;
|
||||
DatabaseTablesIteratorPtr getDatabaseIterator(const Context & context) const;
|
||||
|
||||
protected:
|
||||
StorageMerge(
|
||||
|
@ -292,7 +292,7 @@ BlockInputStreams StorageSystemColumns::read(
|
||||
const DatabasePtr database = databases.at(database_name);
|
||||
offsets[i] = i ? offsets[i - 1] : 0;
|
||||
|
||||
for (auto iterator = database->getIterator(context); iterator->isValid(); iterator->next())
|
||||
for (auto iterator = database->getTablesWithDictionaryTablesIterator(context); iterator->isValid(); iterator->next())
|
||||
{
|
||||
const String & table_name = iterator->name();
|
||||
storages.emplace(std::piecewise_construct,
|
||||
|
@ -9,6 +9,9 @@
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/ExternalDictionariesLoader.h>
|
||||
#include <Storages/System/StorageSystemDictionaries.h>
|
||||
#include <Storages/VirtualColumnUtils.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Core/Names.h>
|
||||
|
||||
#include <ext/map.h>
|
||||
#include <mutex>
|
||||
@ -19,34 +22,40 @@ namespace DB
|
||||
NamesAndTypesList StorageSystemDictionaries::getNamesAndTypes()
|
||||
{
|
||||
return {
|
||||
{ "name", std::make_shared<DataTypeString>() },
|
||||
{ "status", std::make_shared<DataTypeEnum8>(ExternalLoader::getStatusEnumAllPossibleValues()) },
|
||||
{ "origin", std::make_shared<DataTypeString>() },
|
||||
{ "type", std::make_shared<DataTypeString>() },
|
||||
{ "key", std::make_shared<DataTypeString>() },
|
||||
{ "attribute.names", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()) },
|
||||
{ "attribute.types", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()) },
|
||||
{ "bytes_allocated", std::make_shared<DataTypeUInt64>() },
|
||||
{ "query_count", std::make_shared<DataTypeUInt64>() },
|
||||
{ "hit_rate", std::make_shared<DataTypeFloat64>() },
|
||||
{ "element_count", std::make_shared<DataTypeUInt64>() },
|
||||
{ "load_factor", std::make_shared<DataTypeFloat64>() },
|
||||
{ "source", std::make_shared<DataTypeString>() },
|
||||
{ "loading_start_time", std::make_shared<DataTypeDateTime>() },
|
||||
{ "loading_duration", std::make_shared<DataTypeFloat32>() },
|
||||
{"database", std::make_shared<DataTypeString>()},
|
||||
{"name", std::make_shared<DataTypeString>()},
|
||||
{"status", std::make_shared<DataTypeEnum8>(ExternalLoader::getStatusEnumAllPossibleValues())},
|
||||
{"origin", std::make_shared<DataTypeString>()},
|
||||
{"type", std::make_shared<DataTypeString>()},
|
||||
{"key", std::make_shared<DataTypeString>()},
|
||||
{"attribute.names", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
|
||||
{"attribute.types", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
|
||||
{"bytes_allocated", std::make_shared<DataTypeUInt64>()},
|
||||
{"query_count", std::make_shared<DataTypeUInt64>()},
|
||||
{"hit_rate", std::make_shared<DataTypeFloat64>()},
|
||||
{"element_count", std::make_shared<DataTypeUInt64>()},
|
||||
{"load_factor", std::make_shared<DataTypeFloat64>()},
|
||||
{"source", std::make_shared<DataTypeString>()},
|
||||
{"loading_start_time", std::make_shared<DataTypeDateTime>()},
|
||||
{"loading_duration", std::make_shared<DataTypeFloat32>()},
|
||||
//{ "creation_time", std::make_shared<DataTypeDateTime>() },
|
||||
{ "last_exception", std::make_shared<DataTypeString>() },
|
||||
{"last_exception", std::make_shared<DataTypeString>()},
|
||||
};
|
||||
}
|
||||
|
||||
void StorageSystemDictionaries::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const
|
||||
void StorageSystemDictionaries::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & /*query_info*/) const
|
||||
{
|
||||
const auto & external_dictionaries = context.getExternalDictionariesLoader();
|
||||
for (const auto & [dict_name, load_result] : external_dictionaries.getCurrentLoadResults())
|
||||
{
|
||||
size_t i = 0;
|
||||
|
||||
res_columns[i++]->insert(dict_name);
|
||||
res_columns[i++]->insert(load_result.repository_name);
|
||||
if (!load_result.repository_name.empty())
|
||||
res_columns[i++]->insert(dict_name.substr(load_result.repository_name.length() + 1));
|
||||
else
|
||||
res_columns[i++]->insert(dict_name);
|
||||
|
||||
res_columns[i++]->insert(static_cast<Int8>(load_result.status));
|
||||
res_columns[i++]->insert(load_result.origin);
|
||||
|
||||
|
@ -36,7 +36,7 @@ StorageSystemGraphite::Configs StorageSystemGraphite::getConfigs(const Context &
|
||||
if (db.second->getEngineName() == "Lazy")
|
||||
continue;
|
||||
|
||||
for (auto iterator = db.second->getIterator(context); iterator->isValid(); iterator->next())
|
||||
for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next())
|
||||
{
|
||||
auto & table = iterator->table();
|
||||
|
||||
|
@ -43,7 +43,7 @@ void StorageSystemMutations::fillData(MutableColumns & res_columns, const Contex
|
||||
if (db.second->getEngineName() == "Lazy")
|
||||
continue;
|
||||
if (context.hasDatabaseAccessRights(db.first))
|
||||
for (auto iterator = db.second->getIterator(context); iterator->isValid(); iterator->next())
|
||||
for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next())
|
||||
if (dynamic_cast<const MergeTreeData *>(iterator->table().get()))
|
||||
merge_tree_tables[db.first][iterator->name()] = iterator->table();
|
||||
}
|
||||
|
@ -104,7 +104,7 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, const
|
||||
const DatabasePtr database = databases.at(database_name);
|
||||
|
||||
offsets[i] = i ? offsets[i - 1] : 0;
|
||||
for (auto iterator = database->getIterator(context); iterator->isValid(); iterator->next())
|
||||
for (auto iterator = database->getTablesIterator(context); iterator->isValid(); iterator->next())
|
||||
{
|
||||
String table_name = iterator->name();
|
||||
StoragePtr storage = iterator->table();
|
||||
|
@ -71,7 +71,7 @@ BlockInputStreams StorageSystemReplicas::read(
|
||||
continue;
|
||||
if (context.hasDatabaseAccessRights(db.first))
|
||||
{
|
||||
for (auto iterator = db.second->getIterator(context); iterator->isValid(); iterator->next())
|
||||
for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next())
|
||||
if (dynamic_cast<const StorageReplicatedMergeTree *>(iterator->table().get()))
|
||||
replicated_tables[db.first][iterator->name()] = iterator->table();
|
||||
}
|
||||
|
@ -57,7 +57,7 @@ void StorageSystemReplicationQueue::fillData(MutableColumns & res_columns, const
|
||||
|
||||
if (context.hasDatabaseAccessRights(db.first))
|
||||
{
|
||||
for (auto iterator = db.second->getIterator(context); iterator->isValid(); iterator->next())
|
||||
for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next())
|
||||
if (dynamic_cast<const StorageReplicatedMergeTree *>(iterator->table().get()))
|
||||
replicated_tables[db.first][iterator->name()] = iterator->table();
|
||||
}
|
||||
|
@ -86,7 +86,11 @@ public:
|
||||
UInt64 max_block_size_,
|
||||
ColumnPtr databases_,
|
||||
const Context & context_)
|
||||
: columns_mask(std::move(columns_mask_)), header(std::move(header_)), max_block_size(max_block_size_), databases(std::move(databases_)), context(context_) {}
|
||||
: columns_mask(std::move(columns_mask_))
|
||||
, header(std::move(header_))
|
||||
, max_block_size(max_block_size_)
|
||||
, databases(std::move(databases_))
|
||||
, context(context_) {}
|
||||
|
||||
String getName() const override { return "Tables"; }
|
||||
Block getHeader() const override { return header; }
|
||||
@ -189,7 +193,7 @@ protected:
|
||||
}
|
||||
|
||||
if (!tables_it || !tables_it->isValid())
|
||||
tables_it = database->getIterator(context);
|
||||
tables_it = database->getTablesWithDictionaryTablesIterator(context);
|
||||
|
||||
const bool need_lock_structure = needLockStructure(database, header);
|
||||
|
||||
@ -251,10 +255,10 @@ protected:
|
||||
}
|
||||
|
||||
if (columns_mask[src_index++])
|
||||
res_columns[res_index++]->insert(database->getTableMetadataPath(table_name));
|
||||
res_columns[res_index++]->insert(database->getObjectMetadataPath(table_name));
|
||||
|
||||
if (columns_mask[src_index++])
|
||||
res_columns[res_index++]->insert(static_cast<UInt64>(database->getTableMetadataModificationTime(context, table_name)));
|
||||
res_columns[res_index++]->insert(static_cast<UInt64>(database->getObjectMetadataModificationTime(context, table_name)));
|
||||
|
||||
{
|
||||
Array dependencies_table_name_array;
|
||||
@ -372,7 +376,7 @@ private:
|
||||
UInt64 max_block_size;
|
||||
ColumnPtr databases;
|
||||
size_t database_idx = 0;
|
||||
DatabaseIteratorPtr tables_it;
|
||||
DatabaseTablesIteratorPtr tables_it;
|
||||
const Context context;
|
||||
bool done = false;
|
||||
DatabasePtr database;
|
||||
|
@ -33,7 +33,7 @@ static NamesAndTypesList chooseColumns(const String & source_database, const Str
|
||||
|
||||
{
|
||||
auto database = context.getDatabase(source_database);
|
||||
auto iterator = database->getIterator(context, table_name_match);
|
||||
auto iterator = database->getTablesIterator(context, table_name_match);
|
||||
|
||||
if (iterator->isValid())
|
||||
any_table = iterator->table();
|
||||
|
@ -0,0 +1,19 @@
|
||||
<yandex>
|
||||
<logger>
|
||||
<level>trace</level>
|
||||
<log>/var/log/clickhouse-server/clickhouse-server.log</log>
|
||||
<errorlog>/var/log/clickhouse-server/clickhouse-server.err.log</errorlog>
|
||||
<size>1000M</size>
|
||||
<count>10</count>
|
||||
<stderr>/var/log/clickhouse-server/stderr.log</stderr>
|
||||
<stdout>/var/log/clickhouse-server/stdout.log</stdout>
|
||||
</logger>
|
||||
|
||||
<tcp_port>9000</tcp_port>
|
||||
<listen_host>127.0.0.1</listen_host>
|
||||
<max_concurrent_queries>500</max_concurrent_queries>
|
||||
<mark_cache_size>5368709120</mark_cache_size>
|
||||
<path>./clickhouse/</path>
|
||||
<users_config>users.xml</users_config>
|
||||
<dictionaries_config>/etc/clickhouse-server/config.d/*.xml</dictionaries_config>
|
||||
</yandex>
|
@ -0,0 +1,41 @@
|
||||
<yandex>
|
||||
<dictionary>
|
||||
<name>test.conflicting_dictionary</name>
|
||||
<source>
|
||||
<clickhouse>
|
||||
<host>localhost</host>
|
||||
<port>9000</port>
|
||||
<user>default</user>
|
||||
<password></password>
|
||||
<db>test</db>
|
||||
<table>xml_dictionary_table</table>
|
||||
</clickhouse>
|
||||
</source>
|
||||
|
||||
<lifetime>
|
||||
<min>0</min>
|
||||
<max>0</max>
|
||||
</lifetime>
|
||||
|
||||
<layout>
|
||||
<cache><size_in_cells>128</size_in_cells></cache>
|
||||
</layout>
|
||||
|
||||
<structure>
|
||||
<id>
|
||||
<name>id</name>
|
||||
</id>
|
||||
<attribute>
|
||||
<name>SomeValue1</name>
|
||||
<type>UInt8</type>
|
||||
<null_value>1</null_value>
|
||||
</attribute>
|
||||
|
||||
<attribute>
|
||||
<name>SomeValue2</name>
|
||||
<type>String</type>
|
||||
<null_value>''</null_value>
|
||||
</attribute>
|
||||
</structure>
|
||||
</dictionary>
|
||||
</yandex>
|
@ -0,0 +1,4 @@
|
||||
<yandex>
|
||||
<dictionaries_lazy_load>false</dictionaries_lazy_load>
|
||||
</yandex>
|
||||
|
@ -0,0 +1,41 @@
|
||||
<yandex>
|
||||
<dictionary>
|
||||
<name>xml_dictionary</name>
|
||||
<source>
|
||||
<clickhouse>
|
||||
<host>localhost</host>
|
||||
<port>9000</port>
|
||||
<user>default</user>
|
||||
<password></password>
|
||||
<db>test</db>
|
||||
<table>xml_dictionary_table</table>
|
||||
</clickhouse>
|
||||
</source>
|
||||
|
||||
<lifetime>
|
||||
<min>0</min>
|
||||
<max>0</max>
|
||||
</lifetime>
|
||||
|
||||
<layout>
|
||||
<cache><size_in_cells>128</size_in_cells></cache>
|
||||
</layout>
|
||||
|
||||
<structure>
|
||||
<id>
|
||||
<name>id</name>
|
||||
</id>
|
||||
<attribute>
|
||||
<name>SomeValue1</name>
|
||||
<type>UInt8</type>
|
||||
<null_value>1</null_value>
|
||||
</attribute>
|
||||
|
||||
<attribute>
|
||||
<name>SomeValue2</name>
|
||||
<type>String</type>
|
||||
<null_value>''</null_value>
|
||||
</attribute>
|
||||
</structure>
|
||||
</dictionary>
|
||||
</yandex>
|
@ -0,0 +1,36 @@
|
||||
<?xml version="1.0"?>
|
||||
<yandex>
|
||||
<profiles>
|
||||
<default>
|
||||
</default>
|
||||
</profiles>
|
||||
|
||||
<users>
|
||||
<default>
|
||||
<password></password>
|
||||
<networks incl="networks" replace="replace">
|
||||
<ip>::/0</ip>
|
||||
</networks>
|
||||
<profile>default</profile>
|
||||
<quota>default</quota>
|
||||
<allow_databases>
|
||||
<database>default</database>
|
||||
<database>test</database>
|
||||
</allow_databases>
|
||||
</default>
|
||||
|
||||
<admin>
|
||||
<password></password>
|
||||
<networks incl="networks" replace="replace">
|
||||
<ip>::/0</ip>
|
||||
</networks>
|
||||
<profile>default</profile>
|
||||
<quota>default</quota>
|
||||
</admin>
|
||||
</users>
|
||||
|
||||
<quotas>
|
||||
<default>
|
||||
</default>
|
||||
</quotas>
|
||||
</yandex>
|
184
dbms/tests/integration/test_dictionaries_ddl/test.py
Normal file
184
dbms/tests/integration/test_dictionaries_ddl/test.py
Normal file
@ -0,0 +1,184 @@
|
||||
import pytest
|
||||
import os
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from helpers.test_tools import TSV, assert_eq_with_retry
|
||||
from helpers.client import QueryRuntimeException
|
||||
import pymysql
|
||||
import warnings
|
||||
import time
|
||||
|
||||
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
|
||||
|
||||
cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs'))
|
||||
node1 = cluster.add_instance('node1', with_mysql=True, main_configs=['configs/dictionaries/simple_dictionary.xml'])
|
||||
node2 = cluster.add_instance('node2', with_mysql=True, main_configs=['configs/dictionaries/simple_dictionary.xml', 'configs/dictionaries/lazy_load.xml'])
|
||||
node3 = cluster.add_instance('node3', main_configs=['configs/dictionaries/dictionary_with_conflict_name.xml'])
|
||||
|
||||
|
||||
def create_mysql_conn(user, password, hostname, port):
|
||||
return pymysql.connect(
|
||||
user=user,
|
||||
password=password,
|
||||
host=hostname,
|
||||
port=port)
|
||||
|
||||
def execute_mysql_query(connection, query):
|
||||
with warnings.catch_warnings():
|
||||
warnings.simplefilter("ignore")
|
||||
with connection.cursor() as cursor:
|
||||
cursor.execute(query)
|
||||
connection.commit()
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def started_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
for clickhouse in [node1, node2, node3]:
|
||||
clickhouse.query("CREATE DATABASE test", user="admin")
|
||||
clickhouse.query("CREATE TABLE test.xml_dictionary_table (id UInt64, SomeValue1 UInt8, SomeValue2 String) ENGINE = MergeTree() ORDER BY id", user="admin")
|
||||
clickhouse.query("INSERT INTO test.xml_dictionary_table SELECT number, number % 23, hex(number) from numbers(1000)", user="admin")
|
||||
yield cluster
|
||||
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
|
||||
@pytest.mark.parametrize("clickhouse,name,layout", [
|
||||
(node1, 'complex_node1_hashed', 'LAYOUT(COMPLEX_KEY_HASHED())'),
|
||||
(node1, 'complex_node1_cache', 'LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 10))'),
|
||||
(node2, 'complex_node2_hashed', 'LAYOUT(COMPLEX_KEY_HASHED())'),
|
||||
(node2, 'complex_node2_cache', 'LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 10))'),
|
||||
])
|
||||
def test_crete_and_select_mysql(started_cluster, clickhouse, name, layout):
|
||||
mysql_conn = create_mysql_conn("root", "clickhouse", "localhost", 3308)
|
||||
execute_mysql_query(mysql_conn, "CREATE DATABASE IF NOT EXISTS clickhouse")
|
||||
execute_mysql_query(mysql_conn, "CREATE TABLE clickhouse.{} (key_field1 int, key_field2 bigint, value1 text, value2 float, PRIMARY KEY (key_field1, key_field2))".format(name))
|
||||
values = []
|
||||
for i in range(1000):
|
||||
values.append('(' + ','.join([str(i), str(i * i), str(i) * 5, str(i * 3.14)]) + ')')
|
||||
execute_mysql_query(mysql_conn, "INSERT INTO clickhouse.{} VALUES ".format(name) + ','.join(values))
|
||||
|
||||
clickhouse.query("""
|
||||
CREATE DICTIONARY default.{} (
|
||||
key_field1 Int32,
|
||||
key_field2 Int64,
|
||||
value1 String DEFAULT 'xxx',
|
||||
value2 Float32 DEFAULT 'yyy'
|
||||
)
|
||||
PRIMARY KEY key_field1, key_field2
|
||||
SOURCE(MYSQL(
|
||||
USER 'root'
|
||||
PASSWORD 'clickhouse'
|
||||
DB 'clickhouse'
|
||||
TABLE '{}'
|
||||
REPLICA(PRIORITY 1 HOST '127.0.0.1' PORT 3333)
|
||||
REPLICA(PRIORITY 2 HOST 'mysql1' PORT 3306)
|
||||
))
|
||||
{}
|
||||
LIFETIME(MIN 1 MAX 3)
|
||||
""".format(name, name, layout))
|
||||
|
||||
for i in range(172, 200):
|
||||
assert clickhouse.query("SELECT dictGetString('default.{}', 'value1', tuple(toInt32({}), toInt64({})))".format(name, i, i * i)) == str(i) * 5 + '\n'
|
||||
stroka = clickhouse.query("SELECT dictGetFloat32('default.{}', 'value2', tuple(toInt32({}), toInt64({})))".format(name, i, i * i)).strip()
|
||||
value = float(stroka)
|
||||
assert int(value) == int(i * 3.14)
|
||||
|
||||
|
||||
for i in range(1000):
|
||||
values.append('(' + ','.join([str(i), str(i * i), str(i) * 3, str(i * 2.718)]) + ')')
|
||||
execute_mysql_query(mysql_conn, "REPLACE INTO clickhouse.{} VALUES ".format(name) + ','.join(values))
|
||||
|
||||
clickhouse.query("SYSTEM RELOAD DICTIONARY 'default.{}'".format(name))
|
||||
|
||||
for i in range(172, 200):
|
||||
assert clickhouse.query("SELECT dictGetString('default.{}', 'value1', tuple(toInt32({}), toInt64({})))".format(name, i, i * i)) == str(i) * 3 + '\n'
|
||||
stroka = clickhouse.query("SELECT dictGetFloat32('default.{}', 'value2', tuple(toInt32({}), toInt64({})))".format(name, i, i * i)).strip()
|
||||
value = float(stroka)
|
||||
assert int(value) == int(i * 2.718)
|
||||
|
||||
clickhouse.query("select dictGetUInt8('xml_dictionary', 'SomeValue1', toUInt64(17))") == "17\n"
|
||||
clickhouse.query("select dictGetString('xml_dictionary', 'SomeValue2', toUInt64(977))") == str(hex(977))[2:] + '\n'
|
||||
|
||||
|
||||
def test_restricted_database(started_cluster):
|
||||
for node in [node1, node2]:
|
||||
node.query("CREATE DATABASE IF NOT EXISTS restricted_db", user="admin")
|
||||
node.query("CREATE TABLE restricted_db.table_in_restricted_db AS test.xml_dictionary_table", user="admin")
|
||||
|
||||
with pytest.raises(QueryRuntimeException):
|
||||
node1.query("""
|
||||
CREATE DICTIONARY restricted_db.some_dict(
|
||||
id UInt64,
|
||||
SomeValue1 UInt8,
|
||||
SomeValue2 String
|
||||
)
|
||||
PRIMARY KEY id
|
||||
LAYOUT(FLAT())
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_in_restricted_db' DB 'restricted_db'))
|
||||
LIFETIME(MIN 1 MAX 10)
|
||||
""")
|
||||
|
||||
with pytest.raises(QueryRuntimeException):
|
||||
node1.query("""
|
||||
CREATE DICTIONARY default.some_dict(
|
||||
id UInt64,
|
||||
SomeValue1 UInt8,
|
||||
SomeValue2 String
|
||||
)
|
||||
PRIMARY KEY id
|
||||
LAYOUT(FLAT())
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_in_restricted_db' DB 'restricted_db'))
|
||||
LIFETIME(MIN 1 MAX 10)
|
||||
""")
|
||||
|
||||
node1.query("SELECT dictGetUInt8('default.some_dict', 'SomeValue1', toUInt64(17))") == "17\n"
|
||||
|
||||
# with lazy load we don't need query to get exception
|
||||
with pytest.raises(QueryRuntimeException):
|
||||
node2.query("""
|
||||
CREATE DICTIONARY restricted_db.some_dict(
|
||||
id UInt64,
|
||||
SomeValue1 UInt8,
|
||||
SomeValue2 String
|
||||
)
|
||||
PRIMARY KEY id
|
||||
LAYOUT(FLAT())
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_in_restricted_db' DB 'restricted_db'))
|
||||
LIFETIME(MIN 1 MAX 10)
|
||||
""")
|
||||
|
||||
with pytest.raises(QueryRuntimeException):
|
||||
node2.query("""
|
||||
CREATE DICTIONARY default.some_dict(
|
||||
id UInt64,
|
||||
SomeValue1 UInt8,
|
||||
SomeValue2 String
|
||||
)
|
||||
PRIMARY KEY id
|
||||
LAYOUT(FLAT())
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_in_restricted_db' DB 'restricted_db'))
|
||||
LIFETIME(MIN 1 MAX 10)
|
||||
""")
|
||||
|
||||
|
||||
def test_conflicting_name(started_cluster):
|
||||
assert node3.query("select dictGetUInt8('test.conflicting_dictionary', 'SomeValue1', toUInt64(17))") == '17\n'
|
||||
|
||||
with pytest.raises(QueryRuntimeException):
|
||||
node3.query("""
|
||||
CREATE DICTIONARY test.conflicting_dictionary(
|
||||
id UInt64,
|
||||
SomeValue1 UInt8,
|
||||
SomeValue2 String
|
||||
)
|
||||
PRIMARY KEY id
|
||||
LAYOUT(FLAT())
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'xml_dictionary_table' DB 'test'))
|
||||
LIFETIME(MIN 1 MAX 10)
|
||||
""")
|
||||
|
||||
# old version still works
|
||||
node3.query("select dictGetUInt8('test.conflicting_dictionary', 'SomeValue1', toUInt64(17))") == '17\n'
|
@ -0,0 +1,7 @@
|
||||
2
|
||||
2
|
||||
2
|
||||
2
|
||||
2
|
||||
2
|
||||
2
|
120
dbms/tests/queries/0_stateless/01018_ddl_dictionaries_bad_queries.sh
Executable file
120
dbms/tests/queries/0_stateless/01018_ddl_dictionaries_bad_queries.sh
Executable file
@ -0,0 +1,120 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. $CURDIR/../shell_config.sh
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "DROP DICTIONARY IF EXISTS dict1"
|
||||
|
||||
# Simple layout, but with two keys
|
||||
$CLICKHOUSE_CLIENT -q "
|
||||
CREATE DICTIONARY dict1
|
||||
(
|
||||
key1 UInt64,
|
||||
key2 UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY key1, key2
|
||||
LAYOUT(HASHED())
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict1' DB '$CLICKHOUSE_DATABASE'))
|
||||
LIFETIME(MIN 1 MAX 10)
|
||||
" 2>&1 | grep -c 'Primary key for simple dictionary must contain exactly one element'
|
||||
|
||||
|
||||
# Simple layout, but with non existing key
|
||||
$CLICKHOUSE_CLIENT -q "
|
||||
CREATE DICTIONARY dict1
|
||||
(
|
||||
key1 UInt64,
|
||||
key2 UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY non_existing_column
|
||||
LAYOUT(HASHED())
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict1' DB '$CLICKHOUSE_DATABASE'))
|
||||
LIFETIME(MIN 1 MAX 10)
|
||||
" 2>&1 | grep -c "Unknown key attribute 'non_existing_column'"
|
||||
|
||||
# Complex layout, with non existing key
|
||||
$CLICKHOUSE_CLIENT -q "
|
||||
CREATE DICTIONARY dict1
|
||||
(
|
||||
key1 UInt64,
|
||||
key2 UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY non_existing_column, key1
|
||||
LAYOUT(COMPLEX_KEY_HASHED())
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict1' DB '$CLICKHOUSE_DATABASE'))
|
||||
LIFETIME(MIN 1 MAX 10)
|
||||
" 2>&1 | grep -c "Unknown key attribute 'non_existing_column'"
|
||||
|
||||
# No layout
|
||||
$CLICKHOUSE_CLIENT -q "
|
||||
CREATE DICTIONARY dict1
|
||||
(
|
||||
key1 UInt64,
|
||||
key2 UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY key2, key1
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict1' DB '$CLICKHOUSE_DATABASE'))
|
||||
LIFETIME(MIN 1 MAX 10)
|
||||
" 2>&1 | grep -c "Cannot create dictionary with empty layout"
|
||||
|
||||
# No PK
|
||||
$CLICKHOUSE_CLIENT -q "
|
||||
CREATE DICTIONARY dict1
|
||||
(
|
||||
key1 UInt64,
|
||||
key2 UInt64,
|
||||
value String
|
||||
)
|
||||
LAYOUT(COMPLEX_KEY_HASHED())
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict1' DB '$CLICKHOUSE_DATABASE'))
|
||||
LIFETIME(MIN 1 MAX 10)
|
||||
" 2>&1 | grep -c "Cannot create dictionary without primary key"
|
||||
|
||||
# No lifetime
|
||||
$CLICKHOUSE_CLIENT -q "
|
||||
CREATE DICTIONARY dict1
|
||||
(
|
||||
key1 UInt64,
|
||||
key2 UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY key2, key1
|
||||
LAYOUT(COMPLEX_KEY_HASHED())
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict1' DB '$CLICKHOUSE_DATABASE'))
|
||||
" 2>&1 | grep -c "Cannot create dictionary with empty lifetime"
|
||||
|
||||
# No source
|
||||
$CLICKHOUSE_CLIENT -q "
|
||||
CREATE DICTIONARY dict1
|
||||
(
|
||||
key1 UInt64,
|
||||
key2 UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY non_existing_column, key1
|
||||
LAYOUT(COMPLEX_KEY_HASHED())
|
||||
LIFETIME(MIN 1 MAX 10)
|
||||
" 2>&1 | grep -c "Cannot create dictionary with empty source"
|
||||
|
||||
|
||||
# Complex layout, but with one key
|
||||
$CLICKHOUSE_CLIENT -q "
|
||||
CREATE DICTIONARY dict1
|
||||
(
|
||||
key1 UInt64,
|
||||
key2 UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY key1
|
||||
LAYOUT(COMPLEX_KEY_HASHED())
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict1' DB '$CLICKHOUSE_DATABASE'))
|
||||
LIFETIME(MIN 1 MAX 10)
|
||||
" || exit 1
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "DROP DICTIONARY IF EXISTS dict1"
|
@ -0,0 +1 @@
|
||||
Still alive
|
122
dbms/tests/queries/0_stateless/01018_ddl_dictionaries_concurrent_requrests.sh
Executable file
122
dbms/tests/queries/0_stateless/01018_ddl_dictionaries_concurrent_requrests.sh
Executable file
@ -0,0 +1,122 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. $CURDIR/../shell_config.sh
|
||||
|
||||
set -e
|
||||
|
||||
$CLICKHOUSE_CLIENT -n -q "
|
||||
DROP DATABASE IF EXISTS database_for_dict;
|
||||
DROP TABLE IF EXISTS table_for_dict1;
|
||||
DROP TABLE IF EXISTS table_for_dict2;
|
||||
|
||||
CREATE TABLE table_for_dict1 (key_column UInt64, value_column String) ENGINE = MergeTree ORDER BY key_column;
|
||||
CREATE TABLE table_for_dict2 (key_column UInt64, value_column String) ENGINE = MergeTree ORDER BY key_column;
|
||||
|
||||
INSERT INTO table_for_dict1 SELECT number, toString(number) from numbers(1000);
|
||||
INSERT INTO table_for_dict2 SELECT number, toString(number) from numbers(1000, 1000);
|
||||
|
||||
CREATE DATABASE database_for_dict;
|
||||
|
||||
CREATE DICTIONARY database_for_dict.dict1 (key_column UInt64, value_column String) PRIMARY KEY key_column SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict1' PASSWORD '' DB '$CLICKHOUSE_DATABASE')) LIFETIME(MIN 1 MAX 5) LAYOUT(FLAT());
|
||||
|
||||
CREATE DICTIONARY database_for_dict.dict2 (key_column UInt64, value_column String) PRIMARY KEY key_column SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict2' PASSWORD '' DB '$CLICKHOUSE_DATABASE')) LIFETIME(MIN 1 MAX 5) LAYOUT(CACHE(SIZE_IN_CELLS 150));
|
||||
"
|
||||
|
||||
|
||||
function thread1()
|
||||
{
|
||||
while true; do $CLICKHOUSE_CLIENT --query "SELECT * FROM system.dictionaries FORMAT Null"; done
|
||||
}
|
||||
|
||||
function thread2()
|
||||
{
|
||||
while true; do CLICKHOUSE_CLIENT --query "ATTACH DICTIONARY database_for_dict.dict1" ||: ; done
|
||||
}
|
||||
|
||||
function thread3()
|
||||
{
|
||||
while true; do CLICKHOUSE_CLIENT --query "ATTACH DICTIONARY database_for_dict.dict2" ||:; done
|
||||
}
|
||||
|
||||
|
||||
function thread4()
|
||||
{
|
||||
while true; do $CLICKHOUSE_CLIENT -n -q "
|
||||
SELECT * FROM database_for_dict.dict1 FORMAT Null;
|
||||
SELECT * FROM database_for_dict.dict2 FORMAT Null;
|
||||
" ||: ; done
|
||||
}
|
||||
|
||||
function thread5()
|
||||
{
|
||||
while true; do $CLICKHOUSE_CLIENT -n -q "
|
||||
SELECT dictGetString('database_for_dict.dict1', 'value_column', toUInt64(number)) from numbers(1000) FROM FORMAT Null;
|
||||
SELECT dictGetString('database_for_dict.dict2', 'value_column', toUInt64(number)) from numbers(1000) FROM FORMAT Null;
|
||||
" ||: ; done
|
||||
}
|
||||
|
||||
function thread6()
|
||||
{
|
||||
while true; do $CLICKHOUSE_CLIENT -q "DETACH DICTIONARY database_for_dict.dict1"; done
|
||||
}
|
||||
|
||||
function thread7()
|
||||
{
|
||||
while true; do $CLICKHOUSE_CLIENT -q "DETACH DICTIONARY database_for_dict.dict2"; done
|
||||
}
|
||||
|
||||
|
||||
export -f thread1;
|
||||
export -f thread2;
|
||||
export -f thread3;
|
||||
export -f thread4;
|
||||
export -f thread5;
|
||||
export -f thread6;
|
||||
export -f thread7;
|
||||
|
||||
TIMEOUT=10
|
||||
|
||||
timeout $TIMEOUT bash -c thread1 2> /dev/null &
|
||||
timeout $TIMEOUT bash -c thread2 2> /dev/null &
|
||||
timeout $TIMEOUT bash -c thread3 2> /dev/null &
|
||||
timeout $TIMEOUT bash -c thread4 2> /dev/null &
|
||||
timeout $TIMEOUT bash -c thread5 2> /dev/null &
|
||||
timeout $TIMEOUT bash -c thread6 2> /dev/null &
|
||||
timeout $TIMEOUT bash -c thread7 2> /dev/null &
|
||||
|
||||
timeout $TIMEOUT bash -c thread1 2> /dev/null &
|
||||
timeout $TIMEOUT bash -c thread2 2> /dev/null &
|
||||
timeout $TIMEOUT bash -c thread3 2> /dev/null &
|
||||
timeout $TIMEOUT bash -c thread4 2> /dev/null &
|
||||
timeout $TIMEOUT bash -c thread5 2> /dev/null &
|
||||
timeout $TIMEOUT bash -c thread6 2> /dev/null &
|
||||
timeout $TIMEOUT bash -c thread7 2> /dev/null &
|
||||
|
||||
timeout $TIMEOUT bash -c thread1 2> /dev/null &
|
||||
timeout $TIMEOUT bash -c thread2 2> /dev/null &
|
||||
timeout $TIMEOUT bash -c thread3 2> /dev/null &
|
||||
timeout $TIMEOUT bash -c thread4 2> /dev/null &
|
||||
timeout $TIMEOUT bash -c thread5 2> /dev/null &
|
||||
timeout $TIMEOUT bash -c thread6 2> /dev/null &
|
||||
timeout $TIMEOUT bash -c thread7 2> /dev/null &
|
||||
|
||||
timeout $TIMEOUT bash -c thread1 2> /dev/null &
|
||||
timeout $TIMEOUT bash -c thread2 2> /dev/null &
|
||||
timeout $TIMEOUT bash -c thread3 2> /dev/null &
|
||||
timeout $TIMEOUT bash -c thread4 2> /dev/null &
|
||||
timeout $TIMEOUT bash -c thread5 2> /dev/null &
|
||||
timeout $TIMEOUT bash -c thread6 2> /dev/null &
|
||||
timeout $TIMEOUT bash -c thread7 2> /dev/null &
|
||||
|
||||
wait
|
||||
$CLICKHOUSE_CLIENT -q "SELECT 'Still alive'"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "ATTACH DICTIONARY database_for_dict.dict1"
|
||||
$CLICKHOUSE_CLIENT -q "ATTACH DICTIONARY database_for_dict.dict2"
|
||||
|
||||
$CLICKHOUSE_CLIENT -n -q "
|
||||
DROP TABLE table_for_dict1;
|
||||
DROP TABLE table_for_dict2;
|
||||
DROP DATABASE database_for_dict;
|
||||
"
|
@ -0,0 +1,19 @@
|
||||
=DICTIONARY in Ordinary DB
|
||||
CREATE DICTIONARY ordinary_db.dict1 (`key_column` UInt64 DEFAULT 0, `second_column` UInt8 DEFAULT 1, `third_column` String DEFAULT \'qqq\') PRIMARY KEY key_column SOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'table_for_dict\' PASSWORD \'\' DB \'database_for_dict\')) LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT())
|
||||
dict1
|
||||
1
|
||||
ordinary_db dict1
|
||||
==DETACH DICTIONARY
|
||||
0
|
||||
==ATTACH DICTIONARY
|
||||
dict1
|
||||
1
|
||||
ordinary_db dict1
|
||||
==DROP DICTIONARY
|
||||
0
|
||||
=DICTIONARY in Memory DB
|
||||
0
|
||||
=DICTIONARY in Lazy DB
|
||||
=DROP DATABASE WITH DICTIONARY
|
||||
dict4
|
||||
dict4
|
165
dbms/tests/queries/0_stateless/01018_ddl_dictionaries_create.sql
Normal file
165
dbms/tests/queries/0_stateless/01018_ddl_dictionaries_create.sql
Normal file
@ -0,0 +1,165 @@
|
||||
SET send_logs_level = 'none';
|
||||
|
||||
DROP DATABASE IF EXISTS database_for_dict;
|
||||
|
||||
CREATE DATABASE database_for_dict Engine = Ordinary;
|
||||
|
||||
DROP TABLE IF EXISTS database_for_dict.table_for_dict;
|
||||
|
||||
CREATE TABLE database_for_dict.table_for_dict
|
||||
(
|
||||
key_column UInt64,
|
||||
second_column UInt8,
|
||||
third_column String
|
||||
)
|
||||
ENGINE = MergeTree()
|
||||
ORDER BY key_column;
|
||||
|
||||
INSERT INTO database_for_dict.table_for_dict VALUES (1, 100, 'Hello world');
|
||||
|
||||
DROP DATABASE IF EXISTS ordinary_db;
|
||||
|
||||
CREATE DATABASE ordinary_db ENGINE = Ordinary;
|
||||
|
||||
SELECT '=DICTIONARY in Ordinary DB';
|
||||
|
||||
DROP DICTIONARY IF EXISTS ordinary_db.dict1;
|
||||
|
||||
CREATE DICTIONARY ordinary_db.dict1
|
||||
(
|
||||
key_column UInt64 DEFAULT 0,
|
||||
second_column UInt8 DEFAULT 1,
|
||||
third_column String DEFAULT 'qqq'
|
||||
)
|
||||
PRIMARY KEY key_column
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict'))
|
||||
LIFETIME(MIN 1 MAX 10)
|
||||
LAYOUT(FLAT());
|
||||
|
||||
SHOW CREATE DICTIONARY ordinary_db.dict1;
|
||||
|
||||
SHOW DICTIONARIES FROM ordinary_db LIKE 'dict1';
|
||||
|
||||
EXISTS DICTIONARY ordinary_db.dict1;
|
||||
|
||||
SELECT database, name FROM system.dictionaries WHERE name LIKE 'dict1';
|
||||
|
||||
SELECT '==DETACH DICTIONARY';
|
||||
DETACH DICTIONARY ordinary_db.dict1;
|
||||
|
||||
SHOW DICTIONARIES FROM ordinary_db LIKE 'dict1';
|
||||
|
||||
EXISTS DICTIONARY ordinary_db.dict1;
|
||||
|
||||
SELECT database, name FROM system.dictionaries WHERE name LIKE 'dict1';
|
||||
|
||||
SELECT '==ATTACH DICTIONARY';
|
||||
ATTACH DICTIONARY ordinary_db.dict1;
|
||||
|
||||
SHOW DICTIONARIES FROM ordinary_db LIKE 'dict1';
|
||||
|
||||
EXISTS DICTIONARY ordinary_db.dict1;
|
||||
|
||||
SELECT database, name FROM system.dictionaries WHERE name LIKE 'dict1';
|
||||
|
||||
SELECT '==DROP DICTIONARY';
|
||||
|
||||
DROP DICTIONARY IF EXISTS ordinary_db.dict1;
|
||||
|
||||
SYSTEM RELOAD DICTIONARY 'ordinary_db.dict1'; -- due to lazy_load at can persist for some time
|
||||
|
||||
SHOW DICTIONARIES FROM ordinary_db LIKE 'dict1';
|
||||
|
||||
EXISTS DICTIONARY ordinary_db.dict1;
|
||||
|
||||
SELECT database, name FROM system.dictionaries WHERE name LIKE 'dict1';
|
||||
|
||||
DROP DATABASE IF EXISTS ordinary_db;
|
||||
|
||||
DROP DATABASE IF EXISTS memory_db;
|
||||
|
||||
CREATE DATABASE memory_db ENGINE = Memory;
|
||||
|
||||
SELECT '=DICTIONARY in Memory DB';
|
||||
|
||||
CREATE DICTIONARY memory_db.dict2
|
||||
(
|
||||
key_column UInt64 DEFAULT 0 INJECTIVE,
|
||||
second_column UInt8 DEFAULT 1 EXPRESSION rand() % 222,
|
||||
third_column String DEFAULT 'qqq'
|
||||
)
|
||||
PRIMARY KEY key_column
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict'))
|
||||
LIFETIME(MIN 1 MAX 10)
|
||||
LAYOUT(FLAT()); -- {serverError 1}
|
||||
|
||||
SHOW CREATE DICTIONARY memory_db.dict2; -- {serverError 487}
|
||||
|
||||
SHOW DICTIONARIES FROM memory_db LIKE 'dict2';
|
||||
|
||||
EXISTS DICTIONARY memory_db.dict2;
|
||||
|
||||
SELECT database, name FROM system.dictionaries WHERE name LIKE 'dict2';
|
||||
|
||||
SELECT '=DICTIONARY in Lazy DB';
|
||||
|
||||
DROP DATABASE IF EXISTS lazy_db;
|
||||
|
||||
CREATE DATABASE lazy_db ENGINE = Lazy(1);
|
||||
|
||||
CREATE DICTIONARY lazy_db.dict3
|
||||
(
|
||||
key_column UInt64 DEFAULT 0 INJECTIVE,
|
||||
second_column UInt8 DEFAULT 1 EXPRESSION rand() % 222,
|
||||
third_column String DEFAULT 'qqq'
|
||||
)
|
||||
PRIMARY KEY key_column, second_column
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict'))
|
||||
LIFETIME(MIN 1 MAX 10)
|
||||
LAYOUT(COMPLEX_KEY_HASHED()); -- {serverError 1}
|
||||
|
||||
DROP DATABASE IF EXISTS lazy_db;
|
||||
|
||||
SELECT '=DROP DATABASE WITH DICTIONARY';
|
||||
|
||||
DROP DATABASE IF EXISTS ordinary_db;
|
||||
|
||||
CREATE DATABASE ordinary_db ENGINE = Ordinary;
|
||||
|
||||
CREATE DICTIONARY ordinary_db.dict4
|
||||
(
|
||||
key_column UInt64 DEFAULT 0,
|
||||
second_column UInt8 DEFAULT 1,
|
||||
third_column String DEFAULT 'qqq'
|
||||
)
|
||||
PRIMARY KEY key_column
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict'))
|
||||
LIFETIME(MIN 1 MAX 10)
|
||||
LAYOUT(FLAT());
|
||||
|
||||
SHOW DICTIONARIES FROM ordinary_db;
|
||||
|
||||
DROP DATABASE IF EXISTS ordinary_db;
|
||||
|
||||
CREATE DATABASE ordinary_db ENGINE = Ordinary;
|
||||
|
||||
SHOW DICTIONARIES FROM ordinary_db;
|
||||
|
||||
CREATE DICTIONARY ordinary_db.dict4
|
||||
(
|
||||
key_column UInt64 DEFAULT 0,
|
||||
second_column UInt8 DEFAULT 1,
|
||||
third_column String DEFAULT 'qqq'
|
||||
)
|
||||
PRIMARY KEY key_column
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict'))
|
||||
LIFETIME(MIN 1 MAX 10)
|
||||
LAYOUT(FLAT());
|
||||
|
||||
SHOW DICTIONARIES FROM ordinary_db;
|
||||
|
||||
DROP DATABASE IF EXISTS ordinary_db;
|
||||
|
||||
DROP TABLE IF EXISTS database_for_dict.table_for_dict;
|
||||
|
||||
DROP DATABASE IF EXISTS database_for_dict;
|
@ -0,0 +1,19 @@
|
||||
11
|
||||
11
|
||||
144
|
||||
144
|
||||
7
|
||||
7
|
||||
17
|
||||
11
|
||||
11
|
||||
7
|
||||
11
|
||||
6
|
||||
dict1 Dictionary
|
||||
dict2 Dictionary
|
||||
table_for_dict MergeTree
|
||||
database_for_dict dict1 ComplexKeyCache
|
||||
database_for_dict dict2 Hashed
|
||||
6
|
||||
6
|
118
dbms/tests/queries/0_stateless/01018_ddl_dictionaries_select.sql
Normal file
118
dbms/tests/queries/0_stateless/01018_ddl_dictionaries_select.sql
Normal file
@ -0,0 +1,118 @@
|
||||
SET send_logs_level = 'none';
|
||||
|
||||
DROP DATABASE IF EXISTS database_for_dict;
|
||||
|
||||
CREATE DATABASE database_for_dict Engine = Ordinary;
|
||||
|
||||
CREATE TABLE database_for_dict.table_for_dict
|
||||
(
|
||||
key_column UInt64,
|
||||
second_column UInt8,
|
||||
third_column String,
|
||||
fourth_column Float64
|
||||
)
|
||||
ENGINE = MergeTree()
|
||||
ORDER BY key_column;
|
||||
|
||||
INSERT INTO database_for_dict.table_for_dict SELECT number, number % 17, toString(number * number), number / 2.0 from numbers(100);
|
||||
|
||||
DROP DICTIONARY IF EXISTS database_for_dict.dict1;
|
||||
|
||||
CREATE DICTIONARY database_for_dict.dict1
|
||||
(
|
||||
key_column UInt64 DEFAULT 0,
|
||||
second_column UInt8 DEFAULT 1,
|
||||
third_column String DEFAULT 'qqq',
|
||||
fourth_column Float64 DEFAULT 42.0
|
||||
)
|
||||
PRIMARY KEY key_column
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict'))
|
||||
LIFETIME(MIN 1 MAX 10)
|
||||
LAYOUT(FLAT());
|
||||
|
||||
SELECT dictGetUInt8('database_for_dict.dict1', 'second_column', toUInt64(11));
|
||||
SELECT second_column FROM database_for_dict.dict1 WHERE key_column = 11;
|
||||
SELECT dictGetString('database_for_dict.dict1', 'third_column', toUInt64(12));
|
||||
SELECT third_column FROM database_for_dict.dict1 WHERE key_column = 12;
|
||||
SELECT dictGetFloat64('database_for_dict.dict1', 'fourth_column', toUInt64(14));
|
||||
SELECT fourth_column FROM database_for_dict.dict1 WHERE key_column = 14;
|
||||
|
||||
SELECT count(distinct(dictGetUInt8('database_for_dict.dict1', 'second_column', toUInt64(number)))) from numbers(100);
|
||||
|
||||
DETACH DICTIONARY database_for_dict.dict1;
|
||||
|
||||
SYSTEM RELOAD DICTIONARY 'database_for_dict.dict1';
|
||||
|
||||
SELECT dictGetUInt8('database_for_dict.dict1', 'second_column', toUInt64(11)); -- {serverError 36}
|
||||
|
||||
ATTACH DICTIONARY database_for_dict.dict1;
|
||||
|
||||
SELECT dictGetUInt8('database_for_dict.dict1', 'second_column', toUInt64(11));
|
||||
|
||||
DROP DICTIONARY database_for_dict.dict1;
|
||||
|
||||
SYSTEM RELOAD DICTIONARY 'database_for_dict.dict1';
|
||||
|
||||
SELECT dictGetUInt8('database_for_dict.dict1', 'second_column', toUInt64(11)); -- {serverError 36}
|
||||
|
||||
CREATE DICTIONARY database_for_dict.dict1
|
||||
(
|
||||
key_column UInt64 DEFAULT 0,
|
||||
second_column UInt8 DEFAULT 1,
|
||||
third_column String DEFAULT 'qqq',
|
||||
fourth_column Float64 DEFAULT 42.0
|
||||
)
|
||||
PRIMARY KEY key_column, third_column
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' DB 'database_for_dict'))
|
||||
LIFETIME(MIN 1 MAX 10)
|
||||
LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 1));
|
||||
|
||||
SELECT dictGetUInt8('database_for_dict.dict1', 'second_column', tuple(toUInt64(11), '121'));
|
||||
SELECT dictGetFloat64('database_for_dict.dict1', 'fourth_column', tuple(toUInt64(14), '196'));
|
||||
|
||||
DETACH DICTIONARY database_for_dict.dict1;
|
||||
|
||||
SELECT dictGetUInt8('database_for_dict.dict1', 'second_column', tuple(toUInt64(11), '121')); -- {serverError 36}
|
||||
|
||||
ATTACH DICTIONARY database_for_dict.dict1;
|
||||
|
||||
SELECT dictGetUInt8('database_for_dict.dict1', 'second_column', tuple(toUInt64(11), '121'));
|
||||
|
||||
CREATE DICTIONARY database_for_dict.dict2
|
||||
(
|
||||
key_column UInt64 DEFAULT 0,
|
||||
some_column String EXPRESSION toString(fourth_column),
|
||||
fourth_column Float64 DEFAULT 42.0
|
||||
)
|
||||
PRIMARY KEY key_column
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' DB 'database_for_dict'))
|
||||
LIFETIME(MIN 1 MAX 10)
|
||||
LAYOUT(HASHED());
|
||||
|
||||
SELECT dictGetString('database_for_dict.dict2', 'some_column', toUInt64(12));
|
||||
|
||||
SELECT name, engine FROM system.tables WHERE database = 'database_for_dict' ORDER BY name;
|
||||
|
||||
SELECT database, name, type FROM system.dictionaries WHERE database = 'database_for_dict' ORDER BY name;
|
||||
|
||||
-- check dictionary will not update
|
||||
CREATE DICTIONARY database_for_dict.dict3
|
||||
(
|
||||
key_column UInt64 DEFAULT 0,
|
||||
some_column String EXPRESSION toString(fourth_column),
|
||||
fourth_column Float64 DEFAULT 42.0
|
||||
)
|
||||
PRIMARY KEY key_column
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' DB 'database_for_dict'))
|
||||
LIFETIME(0)
|
||||
LAYOUT(HASHED());
|
||||
|
||||
SELECT dictGetString('database_for_dict.dict3', 'some_column', toUInt64(12));
|
||||
|
||||
DROP TABLE database_for_dict.table_for_dict;
|
||||
|
||||
SYSTEM RELOAD DICTIONARIES;
|
||||
|
||||
SELECT dictGetString('database_for_dict.dict3', 'some_column', toUInt64(12));
|
||||
|
||||
DROP DATABASE IF EXISTS database_for_dict;
|
@ -0,0 +1,11 @@
|
||||
0.33
|
||||
0.42
|
||||
0.46
|
||||
0
|
||||
17501
|
||||
NP
|
||||
Moscow
|
||||
[3,2,1,10000]
|
||||
1
|
||||
1
|
||||
0
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user