Merge branch 'master' into stack-frams-size-limit

This commit is contained in:
Alexey Milovidov 2020-08-04 15:28:47 +03:00
commit d91471ea00
191 changed files with 3771 additions and 2099 deletions

View File

@ -86,9 +86,17 @@ ln -sf /usr/share/clickhouse-test/config/client_config.xml /etc/clickhouse-clien
clickhouse-server --config /etc/clickhouse-server/config.xml --daemon
counter=0
until clickhouse-client --query "SELECT 1"
do
sleep 0.1
if [ "$counter" -gt 1200 ]
then
break
fi
counter=$(($counter + 1))
done
TESTS_TO_SKIP=(
@ -178,9 +186,16 @@ if [[ ! -z "$FAILED_TESTS" ]]; then
clickhouse-server --config /etc/clickhouse-server/config.xml --daemon
counter=0
until clickhouse-client --query "SELECT 1"
do
sleep 0.1
if [ "$counter" -gt 1200 ]
then
break
fi
counter=$(($counter + 1))
done
echo "Going to run again: $FAILED_TESTS"

View File

@ -121,7 +121,7 @@ tickTime=2000
# The number of ticks that the initial
# synchronization phase can take
# This value is not quite motivated
initLimit=30000
initLimit=300
# The number of ticks that can pass between
# sending a request and getting an acknowledgement
syncLimit=10

View File

@ -232,7 +232,7 @@ private:
context.setQueryParameters(query_parameters);
/// settings and limits could be specified in config file, but passed settings has higher priority
for (auto setting : context.getSettingsRef().allUnchanged())
for (const auto & setting : context.getSettingsRef().allUnchanged())
{
const auto & name = setting.getName();
if (config().has(name))
@ -2252,7 +2252,7 @@ public:
/// Copy settings-related program options to config.
/// TODO: Is this code necessary?
for (auto setting : context.getSettingsRef().all())
for (const auto & setting : context.getSettingsRef().all())
{
const auto & name = setting.getName();
if (options.count(name))

View File

@ -223,7 +223,7 @@ void checkForUserSettingsAtTopLevel(const Poco::Util::AbstractConfiguration & co
return;
Settings settings;
for (auto setting : settings.all())
for (const auto & setting : settings.all())
{
const auto & name = setting.getName();
if (config.has(name))
@ -644,6 +644,9 @@ int Server::main(const std::vector<std::string> & /*args*/)
global_context->setFormatSchemaPath(format_schema_path.path());
format_schema_path.createDirectories();
/// Check sanity of MergeTreeSettings on server startup
global_context->getMergeTreeSettings().sanityCheck(settings);
/// Limit on total memory usage
size_t max_server_memory_usage = config().getUInt64("max_server_memory_usage", 0);

View File

@ -22,7 +22,11 @@ Macros::Macros(const Poco::Util::AbstractConfiguration & config, const String &
}
}
String Macros::expand(const String & s, size_t level, const String & database_name, const String & table_name) const
String Macros::expand(const String & s,
size_t level,
const String & database_name,
const String & table_name,
const UUID & uuid) const
{
if (s.find('{') == String::npos)
return s;
@ -64,10 +68,12 @@ String Macros::expand(const String & s, size_t level, const String & database_na
res += database_name;
else if (macro_name == "table" && !table_name.empty())
res += table_name;
else if (macro_name == "uuid" && uuid != UUIDHelpers::Nil)
res += toString(uuid);
else
throw Exception("No macro '" + macro_name +
"' in config while processing substitutions in '" + s + "' at "
+ toString(begin), ErrorCodes::SYNTAX_ERROR);
"' in config while processing substitutions in '" + s + "' at '"
+ toString(begin) + "' or macro is not supported here", ErrorCodes::SYNTAX_ERROR);
pos = end + 1;
}
@ -82,9 +88,9 @@ String Macros::getValue(const String & key) const
throw Exception("No macro " + key + " in config", ErrorCodes::SYNTAX_ERROR);
}
String Macros::expand(const String & s, const String & database_name, const String & table_name) const
String Macros::expand(const String & s, const StorageID & table_id, bool allow_uuid) const
{
return expand(s, 0, database_name, table_name);
return expand(s, 0, table_id.database_name, table_id.table_name, allow_uuid ? table_id.uuid : UUIDHelpers::Nil);
}
Names Macros::expand(const Names & source_names, size_t level) const

View File

@ -2,6 +2,7 @@
#include <Core/Types.h>
#include <Core/Names.h>
#include <Interpreters/StorageID.h>
#include <map>
@ -30,9 +31,13 @@ public:
* If {database} and {table} macros aren`t defined explicitly, expand them as database_name and table_name respectively.
* level - the level of recursion.
*/
String expand(const String & s, size_t level = 0, const String & database_name = "", const String & table_name = "") const;
String expand(const String & s,
size_t level = 0,
const String & database_name = "",
const String & table_name = "",
const UUID & uuid = UUIDHelpers::Nil) const;
String expand(const String & s, const String & database_name, const String & table_name) const;
String expand(const String & s, const StorageID & table_id, bool allow_uuid) const;
/** Apply expand for the list.

View File

@ -11,9 +11,6 @@
M(FailedQuery, "Number of failed queries.") \
M(FailedSelectQuery, "Same as FailedQuery, but only for SELECT queries.") \
M(FailedInsertQuery, "Same as FailedQuery, but only for INSERT queries.") \
M(QueryTimeMicroseconds, "Total time of all queries.") \
M(SelectQueryTimeMicroseconds, "Total time of SELECT queries.") \
M(InsertQueryTimeMicroseconds, "Total time of INSERT queries.") \
M(FileOpen, "Number of files opened.") \
M(Seek, "Number of times the 'lseek' function was called.") \
M(ReadBufferFromFileDescriptorRead, "Number of reads (read/pread) from a file descriptor. Does not include sockets.") \

View File

@ -104,13 +104,10 @@ public:
private:
friend class BaseSettings;
SettingFieldRef(const typename Traits::Data & data_, const typename Traits::Accessor & accessor_, size_t index_) : data(&data_), accessor(&accessor_), index(index_) {}
SettingFieldRef(const CustomSettingMap::mapped_type & custom_setting_);
const typename Traits::Data * data = nullptr;
const typename Traits::Accessor * accessor = nullptr;
size_t index = 0;
std::conditional_t<Traits::allow_custom_settings, const CustomSettingMap::mapped_type*, boost::blank> custom_setting = {};
const BaseSettings * settings;
const typename Traits::Accessor * accessor;
size_t index;
std::conditional_t<Traits::allow_custom_settings, const CustomSettingMap::mapped_type*, boost::blank> custom_setting;
};
enum SkipFlags
@ -128,7 +125,7 @@ public:
public:
Iterator & operator++();
Iterator operator++(int);
SettingFieldRef operator *() const;
const SettingFieldRef & operator *() const { return field_ref; }
bool operator ==(const Iterator & other) const;
bool operator !=(const Iterator & other) const { return !(*this == other); }
@ -137,10 +134,9 @@ public:
friend class BaseSettings;
Iterator(const BaseSettings & settings_, const typename Traits::Accessor & accessor_, SkipFlags skip_flags_);
void doSkip();
void setPointerToCustomSetting();
const BaseSettings * settings = nullptr;
const typename Traits::Accessor * accessor = nullptr;
size_t index;
SettingFieldRef field_ref;
std::conditional_t<Traits::allow_custom_settings, CustomSettingMap::const_iterator, boost::blank> custom_settings_iterator;
SkipFlags skip_flags;
};
@ -557,13 +553,20 @@ const SettingFieldCustom * BaseSettings<Traits_>::tryGetCustomSetting(const std:
template <typename Traits_>
BaseSettings<Traits_>::Iterator::Iterator(const BaseSettings & settings_, const typename Traits::Accessor & accessor_, SkipFlags skip_flags_)
: settings(&settings_), accessor(&accessor_), skip_flags(skip_flags_)
: skip_flags(skip_flags_)
{
field_ref.settings = &settings_;
field_ref.accessor = &accessor_;
auto & index = field_ref.index;
if (skip_flags == SKIP_ALL)
{
index = accessor->size();
index = accessor_.size();
if constexpr (Traits::allow_custom_settings)
custom_settings_iterator = settings->custom_settings_map.end();
{
custom_settings_iterator = settings_.custom_settings_map.end();
field_ref.custom_setting = nullptr;
}
return;
}
@ -575,25 +578,28 @@ BaseSettings<Traits_>::Iterator::Iterator(const BaseSettings & settings_, const
}
if (skip_flags & SKIP_BUILTIN)
index = accessor->size();
index = accessor_.size();
else
index = 0;
if constexpr (Traits::allow_custom_settings)
{
if (skip_flags & SKIP_CUSTOM)
custom_settings_iterator = settings->custom_settings_map.end();
custom_settings_iterator = settings_.custom_settings_map.end();
else
custom_settings_iterator = settings->custom_settings_map.begin();
custom_settings_iterator = settings_.custom_settings_map.begin();
}
doSkip();
setPointerToCustomSetting();
}
template <typename Traits_>
typename BaseSettings<Traits_>::Iterator & BaseSettings<Traits_>::Iterator::operator++()
{
if (index != accessor->size())
const auto & accessor = *field_ref.accessor;
auto & index = field_ref.index;
if (index != accessor.size())
++index;
else
{
@ -601,6 +607,7 @@ typename BaseSettings<Traits_>::Iterator & BaseSettings<Traits_>::Iterator::oper
++custom_settings_iterator;
}
doSkip();
setPointerToCustomSetting();
return *this;
}
@ -612,32 +619,39 @@ typename BaseSettings<Traits_>::Iterator BaseSettings<Traits_>::Iterator::operat
return res;
}
template <typename Traits_>
typename BaseSettings<Traits_>::SettingFieldRef BaseSettings<Traits_>::Iterator::operator*() const
{
if constexpr (Traits::allow_custom_settings)
{
if (index == accessor->size())
return {custom_settings_iterator->second};
}
return {*settings, *accessor, index};
}
template <typename Traits_>
void BaseSettings<Traits_>::Iterator::doSkip()
{
const auto & accessor = *field_ref.accessor;
const auto & settings = *field_ref.settings;
auto & index = field_ref.index;
if (skip_flags & SKIP_CHANGED)
{
while ((index != accessor->size()) && accessor->isValueChanged(*settings, index))
while ((index != accessor.size()) && accessor.isValueChanged(settings, index))
++index;
}
else if (skip_flags & SKIP_UNCHANGED)
{
while ((index != accessor->size()) && !accessor->isValueChanged(*settings, index))
while ((index != accessor.size()) && !accessor.isValueChanged(settings, index))
++index;
}
}
template <typename Traits_>
void BaseSettings<Traits_>::Iterator::setPointerToCustomSetting()
{
if constexpr (Traits::allow_custom_settings)
{
const auto & accessor = *field_ref.accessor;
const auto & settings = *field_ref.settings;
const auto & index = field_ref.index;
if ((index == accessor.size()) && (custom_settings_iterator != settings.custom_settings_map.end()))
field_ref.custom_setting = &custom_settings_iterator->second;
else
field_ref.custom_setting = nullptr;
}
}
template <typename Traits_>
bool BaseSettings<Traits_>::Iterator::operator ==(const typename BaseSettings<Traits_>::Iterator & other) const
{
@ -646,14 +660,7 @@ bool BaseSettings<Traits_>::Iterator::operator ==(const typename BaseSettings<Tr
if (custom_settings_iterator != other.custom_settings_iterator)
return false;
}
return ((index == other.index) && (settings == other.settings));
}
template <typename Traits_>
BaseSettings<Traits_>::SettingFieldRef::SettingFieldRef(const CustomSettingMap::mapped_type & custom_setting_)
{
if constexpr (Traits_::allow_custom_settings)
custom_setting = &custom_setting_;
return ((field_ref.index == other.field_ref.index) && (field_ref.settings == other.field_ref.settings));
}
template <typename Traits_>
@ -675,7 +682,7 @@ Field BaseSettings<Traits_>::SettingFieldRef::getValue() const
if (custom_setting)
return static_cast<Field>(custom_setting->second);
}
return accessor->getValue(*data, index);
return accessor->getValue(*settings, index);
}
template <typename Traits_>
@ -686,7 +693,7 @@ String BaseSettings<Traits_>::SettingFieldRef::getValueString() const
if (custom_setting)
return custom_setting->second.toString();
}
return accessor->getValueString(*data, index);
return accessor->getValueString(*settings, index);
}
template <typename Traits_>
@ -697,7 +704,7 @@ bool BaseSettings<Traits_>::SettingFieldRef::isValueChanged() const
if (custom_setting)
return true;
}
return accessor->isValueChanged(*data, index);
return accessor->isValueChanged(*settings, index);
}
template <typename Traits_>

View File

@ -65,7 +65,7 @@ void Settings::dumpToArrayColumns(IColumn * column_names_, IColumn * column_valu
size_t count = 0;
for (auto setting : all(changed_only ? SKIP_UNCHANGED : SKIP_NONE))
for (const auto & setting : all(changed_only ? SKIP_UNCHANGED : SKIP_NONE))
{
if (column_names)
{
@ -95,7 +95,7 @@ void Settings::dumpToArrayColumns(IColumn * column_names_, IColumn * column_valu
void Settings::addProgramOptions(boost::program_options::options_description & options)
{
for (auto field : all())
for (const auto & field : all())
{
const std::string_view name = field.getName();
auto on_program_option

View File

@ -347,8 +347,8 @@ class IColumn;
M(UInt64, min_free_disk_space_for_temporary_data, 0, "The minimum disk space to keep while writing temporary data used in external sorting and aggregation.", 0) \
\
M(DefaultDatabaseEngine, default_database_engine, DefaultDatabaseEngine::Ordinary, "Default database engine.", 0) \
M(Bool, allow_experimental_database_atomic, false, "Allow to create database with Engine=Atomic.", 0) \
M(Bool, show_table_uuid_in_table_create_query_if_not_nil, true, "For tables in databases with Engine=Atomic show UUID of the table in its CREATE query.", 0) \
M(Bool, allow_experimental_database_atomic, true, "Allow to create database with Engine=Atomic.", 0) \
M(Bool, show_table_uuid_in_table_create_query_if_not_nil, false, "For tables in databases with Engine=Atomic show UUID of the table in its CREATE query.", 0) \
M(Bool, enable_scalar_subquery_optimization, true, "If it is set to true, prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once.", 0) \
M(Bool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.", 0) \
M(UInt64, mutations_sync, 0, "Wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). 0 - execute asynchronously. 1 - wait current server. 2 - wait all replicas if they exist.", 0) \

View File

@ -8,6 +8,7 @@
#include <Common/renameat2.h>
#include <Storages/StorageMaterializedView.h>
#include <Interpreters/Context.h>
#include <Interpreters/ExternalDictionariesLoader.h>
#include <filesystem>
@ -16,10 +17,13 @@ namespace DB
namespace ErrorCodes
{
extern const int UNKNOWN_TABLE;
extern const int UNKNOWN_DATABASE;
extern const int TABLE_ALREADY_EXISTS;
extern const int CANNOT_ASSIGN_ALTER;
extern const int DATABASE_NOT_EMPTY;
extern const int NOT_IMPLEMENTED;
extern const int FILE_ALREADY_EXISTS;
extern const int INCORRECT_QUERY;
}
class AtomicDatabaseTablesSnapshotIterator final : public DatabaseTablesSnapshotIterator
@ -31,13 +35,15 @@ public:
};
DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, Context & context_)
DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, UUID uuid, Context & context_)
: DatabaseOrdinary(name_, std::move(metadata_path_), "store/", "DatabaseAtomic (" + name_ + ")", context_)
, path_to_table_symlinks(context_.getPath() + "data/" + escapeForFileName(name_) + "/")
, path_to_table_symlinks(global_context.getPath() + "data/" + escapeForFileName(name_) + "/")
, path_to_metadata_symlink(global_context.getPath() + "metadata/" + escapeForFileName(name_))
, db_uuid(uuid)
{
/// Symlinks in data/db_name/ directory are not used by ClickHouse,
/// it's needed only for convenient introspection.
assert(db_uuid != UUIDHelpers::Nil);
Poco::File(path_to_table_symlinks).createDirectories();
tryCreateMetadataSymlink();
}
String DatabaseAtomic::getTableDataPath(const String & table_name) const
@ -45,7 +51,7 @@ String DatabaseAtomic::getTableDataPath(const String & table_name) const
std::lock_guard lock(mutex);
auto it = table_name_to_path.find(table_name);
if (it == table_name_to_path.end())
throw Exception("Table " + table_name + " not found in database " + getDatabaseName(), ErrorCodes::UNKNOWN_TABLE);
throw Exception("Table " + table_name + " not found in database " + database_name, ErrorCodes::UNKNOWN_TABLE);
assert(it->second != data_path && !it->second.empty());
return it->second;
}
@ -59,7 +65,15 @@ String DatabaseAtomic::getTableDataPath(const ASTCreateQuery & query) const
void DatabaseAtomic::drop(const Context &)
{
Poco::File(path_to_table_symlinks).remove(true);
try
{
Poco::File(path_to_metadata_symlink).remove();
Poco::File(path_to_table_symlinks).remove(true);
}
catch (...)
{
LOG_WARNING(log, getCurrentExceptionMessage(true));
}
Poco::File(getMetadataPath()).remove(true);
}
@ -69,10 +83,10 @@ void DatabaseAtomic::attachTable(const String & name, const StoragePtr & table,
DetachedTables not_in_use;
std::unique_lock lock(mutex);
not_in_use = cleenupDetachedTables();
assertDetachedTableNotInUse(table->getStorageID().uuid);
auto table_id = table->getStorageID();
assertDetachedTableNotInUse(table_id.uuid);
DatabaseWithDictionaries::attachTableUnlocked(name, table, lock);
table_name_to_path.emplace(std::make_pair(name, relative_table_path));
tryCreateSymlink(name, relative_table_path);
}
StoragePtr DatabaseAtomic::detachTable(const String & name)
@ -83,7 +97,6 @@ StoragePtr DatabaseAtomic::detachTable(const String & name)
table_name_to_path.erase(name);
detached_tables.emplace(table->getStorageID().uuid, table);
not_in_use = cleenupDetachedTables();
tryRemoveSymlink(name);
return table;
}
@ -107,16 +120,20 @@ void DatabaseAtomic::dropTable(const Context &, const String & table_name, bool
}
void DatabaseAtomic::renameTable(const Context & context, const String & table_name, IDatabase & to_database,
const String & to_table_name, bool exchange)
const String & to_table_name, bool exchange, bool dictionary)
{
if (typeid(*this) != typeid(to_database))
{
if (!typeid_cast<DatabaseOrdinary *>(&to_database))
throw Exception("Moving tables between databases of different engines is not supported", ErrorCodes::NOT_IMPLEMENTED);
/// Allow moving tables between Atomic and Ordinary (with table lock)
DatabaseOnDisk::renameTable(context, table_name, to_database, to_table_name, exchange);
DatabaseOnDisk::renameTable(context, table_name, to_database, to_table_name, exchange, dictionary);
return;
}
if (exchange && dictionary)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot exchange dictionaries");
auto & other_db = dynamic_cast<DatabaseAtomic &>(to_database);
bool inside_database = this == &other_db;
@ -125,16 +142,24 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n
auto detach = [](DatabaseAtomic & db, const String & table_name_)
{
auto table_data_path_saved = db.table_name_to_path.find(table_name_)->second;
auto it = db.table_name_to_path.find(table_name_);
String table_data_path_saved;
/// Path can be not set for DDL dictionaries, but it does not matter for StorageDictionary.
if (it != db.table_name_to_path.end())
table_data_path_saved = it->second;
assert(!table_data_path_saved.empty() || db.dictionaries.find(table_name_) != db.dictionaries.end());
db.tables.erase(table_name_);
db.table_name_to_path.erase(table_name_);
db.tryRemoveSymlink(table_name_);
if (!table_data_path_saved.empty())
db.tryRemoveSymlink(table_name_);
return table_data_path_saved;
};
auto attach = [](DatabaseAtomic & db, const String & table_name_, const String & table_data_path_, const StoragePtr & table_)
{
db.tables.emplace(table_name_, table_);
if (table_data_path_.empty())
return;
db.table_name_to_path.emplace(table_name_, table_data_path_);
db.tryCreateSymlink(table_name_, table_data_path_);
};
@ -169,6 +194,17 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n
db_lock = std::unique_lock{mutex};
}
bool is_dictionary = dictionaries.find(table_name) != dictionaries.end();
if (exchange && other_db.dictionaries.find(to_table_name) != other_db.dictionaries.end())
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot exchange dictionaries");
if (dictionary != is_dictionary)
throw Exception(ErrorCodes::INCORRECT_QUERY,
"Use RENAME DICTIONARY for dictionaries and RENAME TABLE for tables.");
if (is_dictionary && !inside_database)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot move dictionary to other database");
StoragePtr table = getTableUnlocked(table_name, db_lock);
assert_can_move_mat_view(table);
StoragePtr other_table;
@ -189,13 +225,15 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n
if (exchange)
other_table_data_path = detach(other_db, to_table_name);
table->renameInMemory({other_db.getDatabaseName(), to_table_name, table->getStorageID().uuid});
auto old_table_id = table->getStorageID();
table->renameInMemory({other_db.database_name, to_table_name, old_table_id.uuid});
if (exchange)
other_table->renameInMemory({getDatabaseName(), table_name, other_table->getStorageID().uuid});
other_table->renameInMemory({database_name, table_name, other_table->getStorageID().uuid});
if (!inside_database)
{
DatabaseCatalog::instance().updateUUIDMapping(table->getStorageID().uuid, other_db.shared_from_this(), table);
DatabaseCatalog::instance().updateUUIDMapping(old_table_id.uuid, other_db.shared_from_this(), table);
if (exchange)
DatabaseCatalog::instance().updateUUIDMapping(other_table->getStorageID().uuid, shared_from_this(), other_table);
}
@ -203,6 +241,12 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n
attach(other_db, to_table_name, table_data_path, table);
if (exchange)
attach(*this, table_name, other_table_data_path, other_table);
if (is_dictionary)
{
auto new_table_id = StorageID(other_db.database_name, to_table_name, old_table_id.uuid);
renameDictionaryInMemoryUnlocked(old_table_id, new_table_id);
}
}
void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table,
@ -213,6 +257,9 @@ void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const Stora
try
{
std::unique_lock lock{mutex};
if (query.database != database_name)
throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database was renamed to `{}`, cannot create table in `{}`",
database_name, query.database);
not_in_use = cleenupDetachedTables();
assertDetachedTableNotInUse(query.uuid);
renameNoReplace(table_metadata_tmp_path, table_metadata_path);
@ -229,7 +276,8 @@ void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const Stora
void DatabaseAtomic::commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path)
{
SCOPE_EXIT({ std::error_code code; std::filesystem::remove(table_metadata_tmp_path, code); });
bool check_file_exists = supportsRenameat2();
SCOPE_EXIT({ std::error_code code; if (check_file_exists) std::filesystem::remove(table_metadata_tmp_path, code); });
std::unique_lock lock{mutex};
auto actual_table_id = getTableUnlocked(table_id.table_name, lock)->getStorageID();
@ -237,7 +285,10 @@ void DatabaseAtomic::commitAlterTable(const StorageID & table_id, const String &
if (table_id.uuid != actual_table_id.uuid)
throw Exception("Cannot alter table because it was renamed", ErrorCodes::CANNOT_ASSIGN_ALTER);
renameExchange(table_metadata_tmp_path, table_metadata_path);
if (check_file_exists)
renameExchange(table_metadata_tmp_path, table_metadata_path);
else
std::filesystem::rename(table_metadata_tmp_path, table_metadata_path);
}
void DatabaseAtomic::assertDetachedTableNotInUse(const UUID & uuid)
@ -330,7 +381,7 @@ void DatabaseAtomic::tryCreateSymlink(const String & table_name, const String &
}
catch (...)
{
tryLogCurrentException(log);
LOG_WARNING(log, getCurrentExceptionMessage(true));
}
}
@ -343,9 +394,108 @@ void DatabaseAtomic::tryRemoveSymlink(const String & table_name)
}
catch (...)
{
tryLogCurrentException(log);
LOG_WARNING(log, getCurrentExceptionMessage(true));
}
}
void DatabaseAtomic::tryCreateMetadataSymlink()
{
/// Symlinks in data/db_name/ directory and metadata/db_name/ are not used by ClickHouse,
/// it's needed only for convenient introspection.
assert(path_to_metadata_symlink != metadata_path);
Poco::File metadata_symlink(path_to_metadata_symlink);
if (metadata_symlink.exists())
{
if (!metadata_symlink.isLink())
throw Exception(ErrorCodes::FILE_ALREADY_EXISTS, "Directory {} exists", path_to_metadata_symlink);
}
else
{
try
{
Poco::File{metadata_path}.linkTo(path_to_metadata_symlink, Poco::File::LINK_SYMBOLIC);
}
catch (...)
{
tryLogCurrentException(log);
}
}
}
void DatabaseAtomic::renameDatabase(const String & new_name)
{
/// CREATE, ATTACH, DROP, DETACH and RENAME DATABASE must hold DDLGuard
try
{
Poco::File(path_to_metadata_symlink).remove();
}
catch (...)
{
LOG_WARNING(log, getCurrentExceptionMessage(true));
}
auto new_name_escaped = escapeForFileName(new_name);
auto old_database_metadata_path = global_context.getPath() + "metadata/" + escapeForFileName(getDatabaseName()) + ".sql";
auto new_database_metadata_path = global_context.getPath() + "metadata/" + new_name_escaped + ".sql";
renameNoReplace(old_database_metadata_path, new_database_metadata_path);
String old_path_to_table_symlinks;
{
std::lock_guard lock(mutex);
DatabaseCatalog::instance().updateDatabaseName(database_name, new_name);
database_name = new_name;
for (auto & table : tables)
{
auto table_id = table.second->getStorageID();
table_id.database_name = database_name;
table.second->renameInMemory(table_id);
}
for (auto & dict : dictionaries)
{
auto old_name = StorageID(dict.second.create_query);
auto name = old_name;
name.database_name = database_name;
renameDictionaryInMemoryUnlocked(old_name, name);
}
path_to_metadata_symlink = global_context.getPath() + "metadata/" + new_name_escaped;
old_path_to_table_symlinks = path_to_table_symlinks;
path_to_table_symlinks = global_context.getPath() + "data/" + new_name_escaped + "/";
}
Poco::File(old_path_to_table_symlinks).renameTo(path_to_table_symlinks);
tryCreateMetadataSymlink();
}
void DatabaseAtomic::renameDictionaryInMemoryUnlocked(const StorageID & old_name, const StorageID & new_name)
{
auto it = dictionaries.find(old_name.table_name);
assert(it != dictionaries.end());
assert(it->second.config->getString("dictionary.uuid") == toString(old_name.uuid));
assert(old_name.uuid == new_name.uuid);
it->second.config->setString("dictionary.database", new_name.database_name);
it->second.config->setString("dictionary.name", new_name.table_name);
auto & create = it->second.create_query->as<ASTCreateQuery &>();
create.database = new_name.database_name;
create.table = new_name.table_name;
assert(create.uuid == new_name.uuid);
if (old_name.table_name != new_name.table_name)
{
auto attach_info = std::move(it->second);
dictionaries.erase(it);
dictionaries.emplace(new_name.table_name, std::move(attach_info));
}
auto result = external_loader.getLoadResult(toString(old_name.uuid));
if (!result.object)
return;
const auto & dict = dynamic_cast<const IDictionaryBase &>(*result.object);
dict.updateDictionaryName(new_name);
}
}

View File

@ -21,16 +21,20 @@ class DatabaseAtomic : public DatabaseOrdinary
{
public:
DatabaseAtomic(String name_, String metadata_path_, Context & context_);
DatabaseAtomic(String name_, String metadata_path_, UUID uuid, Context & context_);
String getEngineName() const override { return "Atomic"; }
UUID getUUID() const override { return db_uuid; }
void renameDatabase(const String & new_name) override;
void renameTable(
const Context & context,
const String & table_name,
IDatabase & to_database,
const String & to_table_name,
bool exchange) override;
bool exchange,
bool dictionary) override;
void dropTable(const Context & context, const String & table_name, bool no_delay) override;
@ -51,6 +55,9 @@ public:
UUID tryGetTableUUID(const String & table_name) const override;
void tryCreateSymlink(const String & table_name, const String & actual_data_path);
void tryRemoveSymlink(const String & table_name);
private:
void commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path) override;
void commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table,
@ -60,15 +67,18 @@ private:
typedef std::unordered_map<UUID, StoragePtr> DetachedTables;
[[nodiscard]] DetachedTables cleenupDetachedTables();
void tryCreateSymlink(const String & table_name, const String & actual_data_path);
void tryRemoveSymlink(const String & table_name);
void tryCreateMetadataSymlink();
void renameDictionaryInMemoryUnlocked(const StorageID & old_name, const StorageID & new_name);
//TODO store path in DatabaseWithOwnTables::tables
typedef std::unordered_map<String, String> NameToPathMap;
NameToPathMap table_name_to_path;
DetachedTables detached_tables;
const String path_to_table_symlinks;
String path_to_table_symlinks;
String path_to_metadata_symlink;
const UUID db_uuid;
};
}

View File

@ -55,9 +55,10 @@ Tables DatabaseDictionary::listTables(const FilterByNameFunction & filter_by_nam
{
Tables tables;
auto load_results = global_context.getExternalDictionariesLoader().getLoadResults(filter_by_name);
String db_name = getDatabaseName();
for (auto & load_result : load_results)
{
auto storage = createStorageDictionary(getDatabaseName(), load_result);
auto storage = createStorageDictionary(db_name, load_result);
if (storage)
tables.emplace(storage->getStorageID().table_name, storage);
}
@ -77,7 +78,7 @@ StoragePtr DatabaseDictionary::tryGetTable(const String & table_name, const Cont
DatabaseTablesIteratorPtr DatabaseDictionary::getTablesIterator(const Context &, const FilterByNameFunction & filter_by_table_name)
{
return std::make_unique<DatabaseTablesSnapshotIterator>(listTables(filter_by_table_name));
return std::make_unique<DatabaseTablesSnapshotIterator>(listTables(filter_by_table_name), getDatabaseName());
}
bool DatabaseDictionary::empty() const
@ -100,7 +101,7 @@ ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const String & table_name, co
}
auto names_and_types = StorageDictionary::getNamesAndTypes(ExternalDictionariesLoader::getDictionaryStructure(*load_result.config));
buffer << "CREATE TABLE " << backQuoteIfNeed(database_name) << '.' << backQuoteIfNeed(table_name) << " (";
buffer << "CREATE TABLE " << backQuoteIfNeed(getDatabaseName()) << '.' << backQuoteIfNeed(table_name) << " (";
buffer << StorageDictionary::generateNamesAndTypesDescription(names_and_types);
buffer << ") Engine = Dictionary(" << backQuoteIfNeed(table_name) << ")";
}
@ -123,7 +124,7 @@ ASTPtr DatabaseDictionary::getCreateDatabaseQuery() const
String query;
{
WriteBufferFromString buffer(query);
buffer << "CREATE DATABASE " << backQuoteIfNeed(database_name) << " ENGINE = Dictionary";
buffer << "CREATE DATABASE " << backQuoteIfNeed(getDatabaseName()) << " ENGINE = Dictionary";
}
auto settings = global_context.getSettingsRef();
ParserCreateQuery parser;

View File

@ -47,8 +47,6 @@ protected:
ASTPtr getCreateTableQueryImpl(const String & table_name, const Context & context, bool throw_on_error) const override;
private:
mutable std::mutex mutex;
Poco::Logger * log;
const Context & global_context;

View File

@ -12,6 +12,7 @@
#include <Common/parseAddress.h>
#include "DatabaseFactory.h"
#include <Poco/File.h>
#include <Poco/Path.h>
#if !defined(ARCADIA_BUILD)
# include "config_core.h"
@ -34,15 +35,19 @@ namespace ErrorCodes
extern const int CANNOT_CREATE_DATABASE;
}
DatabasePtr DatabaseFactory::get(
const String & database_name, const String & metadata_path, const ASTStorage * engine_define, Context & context)
DatabasePtr DatabaseFactory::get(const ASTCreateQuery & create, const String & metadata_path, Context & context)
{
bool created = false;
try
{
/// Creates store/xxx/ for Atomic
Poco::File(Poco::Path(metadata_path).makeParent()).createDirectories();
/// Before 20.7 it's possible that .sql metadata file does not exist for some old database.
/// In this case Ordinary database is created on server startup if the corresponding metadata directory exists.
/// So we should remove metadata directory if database creation failed.
created = Poco::File(metadata_path).createDirectory();
return getImpl(database_name, metadata_path, engine_define, context);
return getImpl(create, metadata_path, context);
}
catch (...)
{
@ -64,10 +69,12 @@ static inline ValueType safeGetLiteralValue(const ASTPtr &ast, const String &eng
return ast->as<ASTLiteral>()->value.safeGet<ValueType>();
}
DatabasePtr DatabaseFactory::getImpl(
const String & database_name, const String & metadata_path, const ASTStorage * engine_define, Context & context)
DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String & metadata_path, Context & context)
{
String engine_name = engine_define->engine->name;
const auto * engine_define = create.storage;
const String & database_name = create.database;
const String & engine_name = engine_define->engine->name;
const UUID & uuid = create.uuid;
if (engine_name != "MySQL" && engine_name != "Lazy" && engine_define->engine->arguments)
throw Exception("Database engine " + engine_name + " cannot have arguments", ErrorCodes::BAD_ARGUMENTS);
@ -80,7 +87,7 @@ DatabasePtr DatabaseFactory::getImpl(
if (engine_name == "Ordinary")
return std::make_shared<DatabaseOrdinary>(database_name, metadata_path, context);
else if (engine_name == "Atomic")
return std::make_shared<DatabaseAtomic>(database_name, metadata_path, context);
return std::make_shared<DatabaseAtomic>(database_name, metadata_path, uuid, context);
else if (engine_name == "Memory")
return std::make_shared<DatabaseMemory>(database_name, context);
else if (engine_name == "Dictionary")

View File

@ -6,14 +6,14 @@
namespace DB
{
class ASTStorage;
class ASTCreateQuery;
class DatabaseFactory
{
public:
static DatabasePtr get(const String & database_name, const String & metadata_path, const ASTStorage * engine_define, Context & context);
static DatabasePtr get(const ASTCreateQuery & create, const String & metadata_path, Context & context);
static DatabasePtr getImpl(const String & database_name, const String & metadata_path, const ASTStorage * engine_define, Context & context);
static DatabasePtr getImpl(const ASTCreateQuery & create, const String & metadata_path, Context & context);
};
}

View File

@ -78,10 +78,11 @@ void DatabaseLazy::renameTable(
const String & table_name,
IDatabase & to_database,
const String & to_table_name,
bool exchange)
bool exchange,
bool dictionary)
{
SCOPE_EXIT({ clearExpiredTables(); });
DatabaseOnDisk::renameTable(context, table_name, to_database, to_table_name, exchange);
DatabaseOnDisk::renameTable(context, table_name, to_database, to_table_name, exchange, dictionary);
}
@ -91,7 +92,7 @@ time_t DatabaseLazy::getObjectMetadataModificationTime(const String & table_name
auto it = tables_cache.find(table_name);
if (it != tables_cache.end())
return it->second.metadata_modification_time;
throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
throw Exception("Table " + backQuote(database_name) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
}
void DatabaseLazy::alterTable(
@ -160,7 +161,7 @@ void DatabaseLazy::attachTable(const String & table_name, const StoragePtr & tab
std::forward_as_tuple(table_name),
std::forward_as_tuple(table, current_time, DatabaseOnDisk::getObjectMetadataModificationTime(table_name)));
if (!inserted)
throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
throw Exception("Table " + backQuote(database_name) + "." + backQuote(table_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
it->second.expiration_iterator = cache_expiration_queue.emplace(cache_expiration_queue.end(), current_time, table_name);
}
@ -173,7 +174,7 @@ StoragePtr DatabaseLazy::detachTable(const String & table_name)
std::lock_guard lock(mutex);
auto it = tables_cache.find(table_name);
if (it == tables_cache.end())
throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
throw Exception("Table " + backQuote(database_name) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
res = it->second.table;
if (it->second.expiration_iterator != cache_expiration_queue.end())
cache_expiration_queue.erase(it->second.expiration_iterator);
@ -230,7 +231,7 @@ StoragePtr DatabaseLazy::loadTable(const String & table_name) const
{
const auto & ast_create = ast->as<const ASTCreateQuery &>();
String table_data_path_relative = getTableDataPath(ast_create);
table = createTableFromAST(ast_create, database_name, table_data_path_relative, context_copy, false).second;
table = createTableFromAST(ast_create, getDatabaseName(), table_data_path_relative, context_copy, false).second;
}
if (!ast || !endsWith(table->getName(), "Log"))
@ -239,7 +240,7 @@ StoragePtr DatabaseLazy::loadTable(const String & table_name) const
std::lock_guard lock(mutex);
auto it = tables_cache.find(table_name);
if (it == tables_cache.end())
throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
throw Exception("Table " + backQuote(database_name) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
if (it->second.expiration_iterator != cache_expiration_queue.end())
cache_expiration_queue.erase(it->second.expiration_iterator);
@ -299,6 +300,7 @@ DatabaseLazyIterator::DatabaseLazyIterator(DatabaseLazy & database_, Strings &&
, iterator(table_names.begin())
, current_storage(nullptr)
{
database_name = database.database_name;
}
void DatabaseLazyIterator::next()

View File

@ -42,7 +42,8 @@ public:
const String & table_name,
IDatabase & to_database,
const String & to_table_name,
bool exchange) override;
bool exchange,
bool dictionary) override;
void alterTable(
const Context & context,

View File

@ -58,7 +58,7 @@ void DatabaseMemory::dropTable(
ASTPtr DatabaseMemory::getCreateDatabaseQuery() const
{
auto create_query = std::make_shared<ASTCreateQuery>();
create_query->database = database_name;
create_query->database = getDatabaseName();
create_query->set(create_query->storage, std::make_shared<ASTStorage>());
create_query->storage->set(create_query->storage->engine, makeASTFunction(getEngineName()));
return create_query;

View File

@ -99,7 +99,7 @@ DatabaseTablesIteratorPtr DatabaseMySQL::getTablesIterator(const Context &, cons
if (!remove_or_detach_tables.count(table_name) && (!filter_by_table_name || filter_by_table_name(table_name)))
tables[table_name] = modify_time_and_storage.second;
return std::make_unique<DatabaseTablesSnapshotIterator>(tables);
return std::make_unique<DatabaseTablesSnapshotIterator>(tables, database_name);
}
bool DatabaseMySQL::isTableExist(const String & name, const Context &) const
@ -187,7 +187,7 @@ time_t DatabaseMySQL::getObjectMetadataModificationTime(const String & table_nam
ASTPtr DatabaseMySQL::getCreateDatabaseQuery() const
{
const auto & create_query = std::make_shared<ASTCreateQuery>();
create_query->database = database_name;
create_query->database = getDatabaseName();
create_query->set(create_query->storage, database_engine_define);
return create_query;
}
@ -378,11 +378,11 @@ void DatabaseMySQL::attachTable(const String & table_name, const StoragePtr & st
std::lock_guard<std::mutex> lock{mutex};
if (!local_tables_cache.count(table_name))
throw Exception("Cannot attach table " + backQuoteIfNeed(getDatabaseName()) + "." + backQuoteIfNeed(table_name) +
throw Exception("Cannot attach table " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(table_name) +
" because it does not exist.", ErrorCodes::UNKNOWN_TABLE);
if (!remove_or_detach_tables.count(table_name))
throw Exception("Cannot attach table " + backQuoteIfNeed(getDatabaseName()) + "." + backQuoteIfNeed(table_name) +
throw Exception("Cannot attach table " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(table_name) +
" because it already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
/// We use the new storage to replace the original storage, because the original storage may have been dropped
@ -401,11 +401,11 @@ StoragePtr DatabaseMySQL::detachTable(const String & table_name)
std::lock_guard<std::mutex> lock{mutex};
if (remove_or_detach_tables.count(table_name))
throw Exception("Table " + backQuoteIfNeed(getDatabaseName()) + "." + backQuoteIfNeed(table_name) + " is dropped",
throw Exception("Table " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(table_name) + " is dropped",
ErrorCodes::TABLE_IS_DROPPED);
if (!local_tables_cache.count(table_name))
throw Exception("Table " + backQuoteIfNeed(getDatabaseName()) + "." + backQuoteIfNeed(table_name) + " doesn't exist.",
throw Exception("Table " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(table_name) + " doesn't exist.",
ErrorCodes::UNKNOWN_TABLE);
remove_or_detach_tables.emplace(table_name);
@ -441,16 +441,16 @@ void DatabaseMySQL::dropTable(const Context &, const String & table_name, bool /
Poco::File remove_flag(getMetadataPath() + '/' + escapeForFileName(table_name) + suffix);
if (remove_or_detach_tables.count(table_name))
throw Exception("Table " + backQuoteIfNeed(getDatabaseName()) + "." + backQuoteIfNeed(table_name) + " is dropped",
throw Exception("Table " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(table_name) + " is dropped",
ErrorCodes::TABLE_IS_DROPPED);
if (remove_flag.exists())
throw Exception("The remove flag file already exists but the " + backQuoteIfNeed(getDatabaseName()) +
throw Exception("The remove flag file already exists but the " + backQuoteIfNeed(database_name) +
"." + backQuoteIfNeed(table_name) + " does not exists remove tables, it is bug.", ErrorCodes::LOGICAL_ERROR);
auto table_iter = local_tables_cache.find(table_name);
if (table_iter == local_tables_cache.end())
throw Exception("Table " + backQuoteIfNeed(getDatabaseName()) + "." + backQuoteIfNeed(table_name) + " doesn't exist.",
throw Exception("Table " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(table_name) + " doesn't exist.",
ErrorCodes::UNKNOWN_TABLE);
remove_or_detach_tables.emplace(table_name);

View File

@ -67,7 +67,6 @@ private:
ASTPtr database_engine_define;
String database_name_in_mysql;
mutable std::mutex mutex;
std::atomic<bool> quit{false};
std::condition_variable cond;

View File

@ -20,6 +20,7 @@
#include <Databases/DatabaseOrdinary.h>
#include <Databases/DatabaseAtomic.h>
#include <Common/assert_cast.h>
namespace DB
@ -145,7 +146,7 @@ void DatabaseOnDisk::createTable(
{
const auto & settings = context.getSettingsRef();
const auto & create = query->as<ASTCreateQuery &>();
assert(getDatabaseName() == create.database && table_name == create.table);
assert(table_name == create.table);
/// Create a file with metadata if necessary - if the query is not ATTACH.
/// Write the query of `ATTACH table` to it.
@ -216,7 +217,8 @@ void DatabaseOnDisk::dropTable(const Context & context, const String & table_nam
String table_metadata_path = getObjectMetadataPath(table_name);
String table_metadata_path_drop = table_metadata_path + drop_suffix;
String table_data_path_relative = getTableDataPath(table_name);
assert(!table_data_path_relative.empty());
if (table_data_path_relative.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Path is empty");
StoragePtr table = detachTable(table_name);
bool renamed = false;
@ -248,10 +250,13 @@ void DatabaseOnDisk::renameTable(
const String & table_name,
IDatabase & to_database,
const String & to_table_name,
bool exchange)
bool exchange,
bool dictionary)
{
if (exchange)
throw Exception("Tables can be exchanged only in Atomic databases", ErrorCodes::NOT_IMPLEMENTED);
if (dictionary)
throw Exception("Dictionaries can be renamed only in Atomic databases", ErrorCodes::NOT_IMPLEMENTED);
bool from_ordinary_to_atomic = false;
bool from_atomic_to_ordinary = false;
@ -305,6 +310,14 @@ void DatabaseOnDisk::renameTable(
to_database.createTable(context, to_table_name, table, attach_query);
Poco::File(table_metadata_path).remove();
/// Special case: usually no actions with symlinks are required when detaching/attaching table,
/// but not when moving from Atomic database to Ordinary
if (from_atomic_to_ordinary)
{
auto & atomic_db = assert_cast<DatabaseAtomic &>(*this);
atomic_db.tryRemoveSymlink(table_name);
}
}
ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(const String & table_name, const Context &, bool throw_on_error) const
@ -332,9 +345,14 @@ ASTPtr DatabaseOnDisk::getCreateDatabaseQuery() const
ASTPtr ast;
auto settings = global_context.getSettingsRef();
auto metadata_dir_path = getMetadataPath();
auto database_metadata_path = metadata_dir_path.substr(0, metadata_dir_path.size() - 1) + ".sql";
ast = getCreateQueryFromMetadata(database_metadata_path, true);
{
std::lock_guard lock(mutex);
auto database_metadata_path = global_context.getPath() + "metadata/" + escapeForFileName(database_name) + ".sql";
ast = parseQueryFromMetadata(log, global_context, database_metadata_path, true);
auto & ast_create_query = ast->as<ASTCreateQuery &>();
ast_create_query.attach = false;
ast_create_query.database = database_name;
}
if (!ast)
{
/// Handle databases (such as default) for which there are no database.sql files.
@ -462,7 +480,8 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata(Poco::Logger * logger, const Conte
*/
if (remove_empty && query.empty())
{
LOG_ERROR(logger, "File {} is empty. Removing.", metadata_file_path);
if (logger)
LOG_ERROR(logger, "File {} is empty. Removing.", metadata_file_path);
Poco::File(metadata_file_path).remove();
return nullptr;
}
@ -480,12 +499,12 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata(Poco::Logger * logger, const Conte
return nullptr;
auto & create = ast->as<ASTCreateQuery &>();
if (create.uuid != UUIDHelpers::Nil)
if (!create.table.empty() && create.uuid != UUIDHelpers::Nil)
{
String table_name = Poco::Path(metadata_file_path).makeFile().getBaseName();
table_name = unescapeForFileName(table_name);
if (create.table != TABLE_WITH_UUID_NAME_PLACEHOLDER)
if (create.table != TABLE_WITH_UUID_NAME_PLACEHOLDER && logger)
LOG_WARNING(logger, "File {} contains both UUID and table name. Will use name `{}` instead of `{}`", metadata_file_path, table_name, create.table);
create.table = table_name;
}
@ -501,7 +520,7 @@ ASTPtr DatabaseOnDisk::getCreateQueryFromMetadata(const String & database_metada
{
auto & ast_create_query = ast->as<ASTCreateQuery &>();
ast_create_query.attach = false;
ast_create_query.database = database_name;
ast_create_query.database = getDatabaseName();
}
return ast;

View File

@ -49,7 +49,8 @@ public:
const String & table_name,
IDatabase & to_database,
const String & to_table_name,
bool exchange) override;
bool exchange,
bool dictionary) override;
ASTPtr getCreateDatabaseQuery() const override;

View File

@ -129,6 +129,7 @@ void DatabaseOrdinary::loadStoredObjects(Context & context, bool has_force_resto
if (ast)
{
auto * create_query = ast->as<ASTCreateQuery>();
create_query->database = database_name;
std::lock_guard lock{file_names_mutex};
file_names[file_name] = ast;
total_dictionaries += create_query->is_dictionary;
@ -165,7 +166,7 @@ void DatabaseOrdinary::loadStoredObjects(Context & context, bool has_force_resto
context,
create_query,
*this,
getDatabaseName(),
database_name,
getMetadataPath() + name_with_query.first,
has_force_restore_data_flag);

View File

@ -34,21 +34,30 @@ namespace ErrorCodes
void DatabaseWithDictionaries::attachDictionary(const String & dictionary_name, const DictionaryAttachInfo & attach_info)
{
String full_name = getDatabaseName() + "." + dictionary_name;
auto dict_id = StorageID(attach_info.create_query);
String internal_name = dict_id.getInternalDictionaryName();
assert(attach_info.create_query->as<const ASTCreateQuery &>().table == dictionary_name);
assert(!dict_id.database_name.empty());
{
std::unique_lock lock(mutex);
auto [it, inserted] = dictionaries.emplace(dictionary_name, attach_info);
if (!inserted)
throw Exception("Dictionary " + full_name + " already exists.", ErrorCodes::DICTIONARY_ALREADY_EXISTS);
throw Exception(ErrorCodes::DICTIONARY_ALREADY_EXISTS,
"Dictionary {} already exists.", dict_id.getNameForLogs());
/// Attach the dictionary as table too.
try
{
/// TODO Make StorageDictionary an owner of IDictionaryBase objects.
/// All DDL operations with dictionaries will work with StorageDictionary table,
/// and StorageDictionary will be responsible for loading of DDL dictionaries.
/// ExternalLoaderDatabaseConfigRepository and other hacks related to ExternalLoader
/// will not be longer required.
attachTableUnlocked(
dictionary_name,
StorageDictionary::create(
StorageID(getDatabaseName(), dictionary_name),
full_name,
dict_id,
internal_name,
ExternalDictionariesLoader::getDictionaryStructure(*attach_info.config),
StorageDictionary::Location::SameDatabaseAndNameAsDictionary),
lock);
@ -60,11 +69,11 @@ void DatabaseWithDictionaries::attachDictionary(const String & dictionary_name,
}
}
CurrentStatusInfo::set(CurrentStatusInfo::DictionaryStatus, full_name, static_cast<Int8>(ExternalLoaderStatus::NOT_LOADED));
CurrentStatusInfo::set(CurrentStatusInfo::DictionaryStatus, internal_name, static_cast<Int8>(ExternalLoaderStatus::NOT_LOADED));
/// We want ExternalLoader::reloadConfig() to find out that the dictionary's config
/// has been added and in case `dictionaries_lazy_load == false` to load the dictionary.
reloadDictionaryConfig(full_name);
reloadDictionaryConfig(internal_name);
}
void DatabaseWithDictionaries::detachDictionary(const String & dictionary_name)
@ -75,20 +84,28 @@ void DatabaseWithDictionaries::detachDictionary(const String & dictionary_name)
void DatabaseWithDictionaries::detachDictionaryImpl(const String & dictionary_name, DictionaryAttachInfo & attach_info)
{
String full_name = getDatabaseName() + "." + dictionary_name;
auto dict_id = StorageID::createEmpty();
String internal_name;
{
std::unique_lock lock(mutex);
auto it = dictionaries.find(dictionary_name);
if (it == dictionaries.end())
throw Exception("Dictionary " + full_name + " doesn't exist.", ErrorCodes::UNKNOWN_DICTIONARY);
throw Exception(ErrorCodes::UNKNOWN_DICTIONARY,
"Dictionary {}.{} doesn't exist.", database_name, dictionary_name);
dict_id = StorageID(it->second.create_query);
internal_name = dict_id.getInternalDictionaryName();
assert(dict_id.table_name == dictionary_name);
assert(!dict_id.database_name.empty());
attach_info = std::move(it->second);
dictionaries.erase(it);
/// Detach the dictionary as table too.
try
{
detachTableUnlocked(dictionary_name, lock);
if (!dict_id.hasUUID())
detachTableUnlocked(dictionary_name, lock);
}
catch (...)
{
@ -97,11 +114,14 @@ void DatabaseWithDictionaries::detachDictionaryImpl(const String & dictionary_na
}
}
CurrentStatusInfo::unset(CurrentStatusInfo::DictionaryStatus, full_name);
CurrentStatusInfo::unset(CurrentStatusInfo::DictionaryStatus, internal_name);
/// We want ExternalLoader::reloadConfig() to find out that the dictionary's config
/// has been removed and to unload the dictionary.
reloadDictionaryConfig(full_name);
reloadDictionaryConfig(internal_name);
if (dict_id.hasUUID())
detachTable(dictionary_name);
}
void DatabaseWithDictionaries::createDictionary(const Context & context, const String & dictionary_name, const ASTPtr & query)
@ -116,20 +136,22 @@ void DatabaseWithDictionaries::createDictionary(const Context & context, const S
* - rename .sql.tmp to .sql.
*/
auto dict_id = StorageID(query);
assert(query->as<const ASTCreateQuery &>().table == dictionary_name);
assert(!dict_id.database_name.empty());
/// 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 (isDictionaryExist(dictionary_name))
throw Exception("Dictionary " + backQuote(getDatabaseName()) + "." + backQuote(dictionary_name) + " already exists.", ErrorCodes::DICTIONARY_ALREADY_EXISTS);
throw Exception(ErrorCodes::DICTIONARY_ALREADY_EXISTS, "Dictionary {} already exists.", dict_id.getFullTableName());
/// A dictionary with the same full name could be defined in *.xml config files.
String full_name = getDatabaseName() + "." + dictionary_name;
if (external_loader.getCurrentStatus(full_name) != ExternalLoader::Status::NOT_EXIST)
throw Exception(
"Dictionary " + backQuote(getDatabaseName()) + "." + backQuote(dictionary_name) + " already exists.",
ErrorCodes::DICTIONARY_ALREADY_EXISTS);
if (external_loader.getCurrentStatus(dict_id.getFullNameNotQuoted()) != ExternalLoader::Status::NOT_EXIST)
throw Exception(ErrorCodes::DICTIONARY_ALREADY_EXISTS,
"Dictionary {} already exists.", dict_id.getFullNameNotQuoted());
if (isTableExist(dictionary_name, global_context))
throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(dictionary_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {} already exists.", dict_id.getFullTableName());
String dictionary_metadata_path = getObjectMetadataPath(dictionary_name);
@ -162,7 +184,7 @@ void DatabaseWithDictionaries::createDictionary(const Context & context, const S
{
/// load() is called here to force loading the dictionary, wait until the loading is finished,
/// and throw an exception if the loading is failed.
external_loader.load(full_name);
external_loader.load(dict_id.getInternalDictionaryName());
}
auto config = getDictionaryConfigurationFromAST(query->as<const ASTCreateQuery &>());
@ -178,7 +200,7 @@ void DatabaseWithDictionaries::createDictionary(const Context & context, const S
/// ExternalDictionariesLoader doesn't know we renamed the metadata path.
/// That's why we have to call ExternalLoader::reloadConfig() here.
reloadDictionaryConfig(full_name);
reloadDictionaryConfig(dict_id.getInternalDictionaryName());
/// Everything's ok.
succeeded = true;
@ -193,7 +215,8 @@ void DatabaseWithDictionaries::removeDictionary(const Context &, const String &
{
String dictionary_metadata_path = getObjectMetadataPath(dictionary_name);
Poco::File(dictionary_metadata_path).remove();
CurrentStatusInfo::unset(CurrentStatusInfo::DictionaryStatus, getDatabaseName() + "." + dictionary_name);
CurrentStatusInfo::unset(CurrentStatusInfo::DictionaryStatus,
StorageID(attach_info.create_query).getInternalDictionaryName());
}
catch (...)
{
@ -206,14 +229,16 @@ void DatabaseWithDictionaries::removeDictionary(const Context &, const String &
DatabaseDictionariesIteratorPtr DatabaseWithDictionaries::getDictionariesIterator(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 | boost::adaptors::map_keys)
if (filter_by_dictionary_name(dictionary_name))
filtered_dictionaries.emplace_back(dictionary_name);
return std::make_unique<DatabaseDictionariesSnapshotIterator>(std::move(filtered_dictionaries));
DictionariesWithID filtered_dictionaries;
for (const auto & dictionary : dictionaries)
{
if (filter_by_dictionary_name && !filter_by_dictionary_name(dictionary.first))
continue;
filtered_dictionaries.emplace_back();
filtered_dictionaries.back().first = dictionary.first;
filtered_dictionaries.back().second = dictionary.second.create_query->as<const ASTCreateQuery &>().uuid;
}
return std::make_unique<DatabaseDictionariesSnapshotIterator>(std::move(filtered_dictionaries), database_name);
}
bool DatabaseWithDictionaries::isDictionaryExist(const String & dictionary_name) const
@ -235,7 +260,7 @@ ASTPtr DatabaseWithDictionaries::getCreateDictionaryQueryImpl(
ASTPtr ast = it->second.create_query->clone();
auto & create_query = ast->as<ASTCreateQuery &>();
create_query.attach = false;
create_query.database = getDatabaseName();
create_query.database = database_name;
return ast;
}
}
@ -296,8 +321,11 @@ void DatabaseWithDictionaries::reloadDictionaryConfig(const String & full_name)
{
/// Ensure that this database is attached to ExternalLoader as a config repository.
if (!database_as_config_repo_for_external_loader.load())
database_as_config_repo_for_external_loader = boost::make_shared<ext::scope_guard>(
external_loader.addConfigRepository(std::make_unique<ExternalLoaderDatabaseConfigRepository>(*this)));
{
auto repository = std::make_unique<ExternalLoaderDatabaseConfigRepository>(*this, global_context);
auto remove_repository_callback = external_loader.addConfigRepository(std::move(repository));
database_as_config_repo_for_external_loader = boost::make_shared<ext::scope_guard>(std::move(remove_repository_callback));
}
external_loader.reloadConfig(getDatabaseName(), full_name);
}

View File

@ -42,12 +42,12 @@ protected:
ASTPtr getCreateDictionaryQueryImpl(const String & dictionary_name, bool throw_on_error) const override;
std::unordered_map<String, DictionaryAttachInfo> dictionaries;
const ExternalDictionariesLoader & external_loader;
private:
void detachDictionaryImpl(const String & dictionary_name, DictionaryAttachInfo & attach_info);
void reloadDictionaryConfig(const String & full_name);
const ExternalDictionariesLoader & external_loader;
boost::atomic_shared_ptr<ext::scope_guard> database_as_config_repo_for_external_loader;
};

View File

@ -17,6 +17,7 @@ namespace ErrorCodes
{
extern const int TABLE_ALREADY_EXISTS;
extern const int UNKNOWN_TABLE;
extern const int UNKNOWN_DATABASE;
}
DatabaseWithOwnTablesBase::DatabaseWithOwnTablesBase(const String & name_, const String & logger, const Context & context)
@ -43,14 +44,14 @@ DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesIterator(const Con
{
std::lock_guard lock(mutex);
if (!filter_by_table_name)
return std::make_unique<DatabaseTablesSnapshotIterator>(tables);
return std::make_unique<DatabaseTablesSnapshotIterator>(tables, database_name);
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<DatabaseTablesSnapshotIterator>(std::move(filtered_tables));
return std::make_unique<DatabaseTablesSnapshotIterator>(std::move(filtered_tables), database_name);
}
bool DatabaseWithOwnTablesBase::empty() const
@ -71,14 +72,15 @@ StoragePtr DatabaseWithOwnTablesBase::detachTableUnlocked(const String & table_n
auto it = tables.find(table_name);
if (it == tables.end())
throw Exception("Table " + backQuote(database_name) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist.",
backQuote(database_name), backQuote(table_name));
res = it->second;
tables.erase(it);
auto table_id = res->getStorageID();
if (table_id.hasUUID())
{
assert(getDatabaseName() == DatabaseCatalog::TEMPORARY_DATABASE || getEngineName() == "Atomic");
assert(database_name == DatabaseCatalog::TEMPORARY_DATABASE || getEngineName() == "Atomic");
DatabaseCatalog::instance().removeUUIDMapping(table_id.uuid);
}
@ -93,12 +95,17 @@ void DatabaseWithOwnTablesBase::attachTable(const String & table_name, const Sto
void DatabaseWithOwnTablesBase::attachTableUnlocked(const String & table_name, const StoragePtr & table, std::unique_lock<std::mutex> &)
{
if (!tables.emplace(table_name, table).second)
throw Exception("Table " + database_name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
auto table_id = table->getStorageID();
if (table_id.database_name != database_name)
throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database was renamed to `{}`, cannot create table in `{}`",
database_name, table_id.database_name);
if (!tables.emplace(table_name, table).second)
throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {} already exists.", table_id.getFullTableName());
if (table_id.hasUUID())
{
assert(getDatabaseName() == DatabaseCatalog::TEMPORARY_DATABASE || getEngineName() == "Atomic");
assert(database_name == DatabaseCatalog::TEMPORARY_DATABASE || getEngineName() == "Atomic");
DatabaseCatalog::instance().addUUIDMapping(table_id.uuid, shared_from_this(), table);
}
}
@ -146,7 +153,8 @@ StoragePtr DatabaseWithOwnTablesBase::getTableUnlocked(const String & table_name
auto it = tables.find(table_name);
if (it != tables.end())
return it->second;
throw Exception("Table " + backQuote(database_name) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist.",
backQuote(database_name), backQuote(table_name));
}
}

View File

@ -36,7 +36,6 @@ public:
~DatabaseWithOwnTablesBase() override;
protected:
mutable std::mutex mutex;
Tables tables;
Poco::Logger * log;
const Context & global_context;

View File

@ -23,7 +23,7 @@ struct Settings;
struct ConstraintsDescription;
struct IndicesDescription;
class ASTCreateQuery;
using Dictionaries = std::vector<String>;
using DictionariesWithID = std::vector<std::pair<String, UUID>>;
namespace ErrorCodes
{
@ -48,6 +48,11 @@ public:
virtual ~IDatabaseTablesIterator() = default;
virtual UUID uuid() const { return UUIDHelpers::Nil; }
const String & databaseName() const { assert(!database_name.empty()); return database_name; }
protected:
String database_name;
};
/// Copies list of tables and iterates through such snapshot.
@ -65,12 +70,21 @@ protected:
other.it = other.tables.end();
it = tables.begin();
std::advance(it, idx);
database_name = std::move(other.database_name);
}
public:
DatabaseTablesSnapshotIterator(Tables & tables_) : tables(tables_), it(tables.begin()) {}
DatabaseTablesSnapshotIterator(const Tables & tables_, const String & database_name_)
: tables(tables_), it(tables.begin())
{
database_name = database_name_;
}
DatabaseTablesSnapshotIterator(Tables && tables_) : tables(tables_), it(tables.begin()) {}
DatabaseTablesSnapshotIterator(Tables && tables_, String && database_name_)
: tables(std::move(tables_)), it(tables.begin())
{
database_name = std::move(database_name_);
}
void next() override { ++it; }
@ -85,25 +99,30 @@ public:
class DatabaseDictionariesSnapshotIterator
{
private:
Dictionaries dictionaries;
Dictionaries::iterator it;
DictionariesWithID dictionaries;
DictionariesWithID::iterator it;
String database_name;
public:
DatabaseDictionariesSnapshotIterator() = default;
DatabaseDictionariesSnapshotIterator(Dictionaries & dictionaries_) : dictionaries(dictionaries_), it(dictionaries.begin()) {}
DatabaseDictionariesSnapshotIterator(Dictionaries && dictionaries_) : dictionaries(dictionaries_), it(dictionaries.begin()) {}
DatabaseDictionariesSnapshotIterator(const std::unordered_map<String, DictionaryAttachInfo> & dictionaries_)
DatabaseDictionariesSnapshotIterator(DictionariesWithID & dictionaries_, const String & database_name_)
: dictionaries(dictionaries_), it(dictionaries.begin()), database_name(database_name_)
{
}
DatabaseDictionariesSnapshotIterator(DictionariesWithID && dictionaries_, const String & database_name_)
: dictionaries(dictionaries_), it(dictionaries.begin()), database_name(database_name_)
{
boost::range::copy(dictionaries_ | boost::adaptors::map_keys, std::back_inserter(dictionaries));
it = dictionaries.begin();
}
void next() { ++it; }
bool isValid() const { return !dictionaries.empty() && it != dictionaries.end(); }
const String & name() const { return *it; }
const String & name() const { return it->first; }
const UUID & uuid() const { return it->second; }
const String & databaseName() const { assert(!database_name.empty()); return database_name; }
};
using DatabaseTablesIteratorPtr = std::unique_ptr<IDatabaseTablesIterator>;
@ -228,7 +247,8 @@ public:
const String & /*name*/,
IDatabase & /*to_database*/,
const String & /*to_name*/,
bool /*exchange*/)
bool /*exchange*/,
bool /*dictionary*/)
{
throw Exception(getEngineName() + ": renameTable() is not supported", ErrorCodes::NOT_IMPLEMENTED);
}
@ -282,7 +302,19 @@ public:
virtual ASTPtr getCreateDatabaseQuery() const = 0;
/// Get name of database.
String getDatabaseName() const { return database_name; }
String getDatabaseName() const
{
std::lock_guard lock{mutex};
return database_name;
}
/// Get UUID of database.
virtual UUID getUUID() const { return UUIDHelpers::Nil; }
virtual void renameDatabase(const String & /*new_name*/)
{
throw Exception(getEngineName() + ": RENAME DATABASE is not supported", ErrorCodes::NOT_IMPLEMENTED);
}
/// Returns path for persistent data storage if the database supports it, empty string otherwise
virtual String getDataPath() const { return {}; }
@ -321,6 +353,7 @@ protected:
return nullptr;
}
mutable std::mutex mutex;
String database_name;
};

View File

@ -60,8 +60,7 @@ inline size_t CacheDictionary::getCellIdx(const Key id) const
CacheDictionary::CacheDictionary(
const std::string & database_,
const std::string & name_,
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
DictionaryLifetime dict_lifetime_,
@ -72,9 +71,7 @@ CacheDictionary::CacheDictionary(
size_t update_queue_push_timeout_milliseconds_,
size_t query_wait_timeout_milliseconds_,
size_t max_threads_for_updates_)
: database(database_)
, name(name_)
, full_name{database_.empty() ? name_ : (database_ + "." + name_)}
: IDictionary(dict_id_)
, dict_struct(dict_struct_)
, source_ptr{std::move(source_ptr_)}
, dict_lifetime(dict_lifetime_)
@ -238,7 +235,7 @@ void CacheDictionary::isInConstantVector(const Key child_id, const PaddedPODArra
void CacheDictionary::getString(const std::string & attribute_name, const PaddedPODArray<Key> & ids, ColumnString * out) const
{
auto & attribute = getAttribute(attribute_name);
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString);
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString);
const auto null_value = StringRef{std::get<String>(attribute.null_values)};
@ -249,7 +246,7 @@ void CacheDictionary::getString(
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const ColumnString * const def, ColumnString * const out) const
{
auto & attribute = getAttribute(attribute_name);
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString);
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString);
getItemsString(attribute, ids, out, [&](const size_t row) { return def->getDataAt(row); });
}
@ -258,7 +255,7 @@ void CacheDictionary::getString(
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const String & def, ColumnString * const out) const
{
auto & attribute = getAttribute(attribute_name);
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString);
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString);
getItemsString(attribute, ids, out, [&](const size_t) { return StringRef{def}; });
}
@ -702,8 +699,7 @@ void registerDictionaryCache(DictionaryFactory & factory)
throw Exception{full_name + ": dictionary of layout 'cache' cannot have 'require_nonempty' attribute set",
ErrorCodes::BAD_ARGUMENTS};
const String database = config.getString(config_prefix + ".database", "");
const String name = config.getString(config_prefix + ".name");
const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix);
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};
const size_t strict_max_lifetime_seconds =
@ -712,7 +708,7 @@ void registerDictionaryCache(DictionaryFactory & factory)
const size_t max_update_queue_size =
config.getUInt64(layout_prefix + ".cache.max_update_queue_size", 100000);
if (max_update_queue_size == 0)
throw Exception{name + ": dictionary of layout 'cache' cannot have empty update queue of size 0",
throw Exception{full_name + ": dictionary of layout 'cache' cannot have empty update queue of size 0",
ErrorCodes::TOO_SMALL_BUFFER_SIZE};
const bool allow_read_expired_keys =
@ -721,7 +717,7 @@ void registerDictionaryCache(DictionaryFactory & factory)
const size_t update_queue_push_timeout_milliseconds =
config.getUInt64(layout_prefix + ".cache.update_queue_push_timeout_milliseconds", 10);
if (update_queue_push_timeout_milliseconds < 10)
throw Exception{name + ": dictionary of layout 'cache' have too little update_queue_push_timeout",
throw Exception{full_name + ": dictionary of layout 'cache' have too little update_queue_push_timeout",
ErrorCodes::BAD_ARGUMENTS};
const size_t query_wait_timeout_milliseconds =
@ -730,12 +726,11 @@ void registerDictionaryCache(DictionaryFactory & factory)
const size_t max_threads_for_updates =
config.getUInt64(layout_prefix + ".max_threads_for_updates", 4);
if (max_threads_for_updates == 0)
throw Exception{name + ": dictionary of layout 'cache' cannot have zero threads for updates.",
throw Exception{full_name + ": dictionary of layout 'cache' cannot have zero threads for updates.",
ErrorCodes::BAD_ARGUMENTS};
return std::make_unique<CacheDictionary>(
database,
name,
dict_id,
dict_struct,
std::move(source_ptr),
dict_lifetime,
@ -829,9 +824,9 @@ void CacheDictionary::waitForCurrentUpdateFinish(UpdateUnitPtr & update_unit_ptr
* intended to do a synchronous update. AsyncUpdate thread can touch deallocated memory and explode.
* */
update_unit_ptr->can_use_callback = false;
throw DB::Exception(
"Dictionary " + getName() + " source seems unavailable, because " +
toString(timeout_for_wait) + " timeout exceeded.", ErrorCodes::TIMEOUT_EXCEEDED);
throw DB::Exception(ErrorCodes::TIMEOUT_EXCEEDED,
"Dictionary {} source seems unavailable, because {} timeout exceeded.",
getDictionaryID().getNameForLogs(), toString(timeout_for_wait));
}
@ -842,10 +837,11 @@ void CacheDictionary::waitForCurrentUpdateFinish(UpdateUnitPtr & update_unit_ptr
void CacheDictionary::tryPushToUpdateQueueOrThrow(UpdateUnitPtr & update_unit_ptr) const
{
if (!update_queue.tryPush(update_unit_ptr, update_queue_push_timeout_milliseconds))
throw DB::Exception(
"Cannot push to internal update queue in dictionary " + getFullName() + ". Timelimit of " +
std::to_string(update_queue_push_timeout_milliseconds) + " ms. exceeded. Current queue size is " +
std::to_string(update_queue.size()), ErrorCodes::CACHE_DICTIONARY_UPDATE_FAIL);
throw DB::Exception(ErrorCodes::CACHE_DICTIONARY_UPDATE_FAIL,
"Cannot push to internal update queue in dictionary {}. "
"Timelimit of {} ms. exceeded. Current queue size is {}",
getDictionaryID().getNameForLogs(), std::to_string(update_queue_push_timeout_milliseconds),
std::to_string(update_queue.size()));
}
void CacheDictionary::update(BunchUpdateUnit & bunch_update_unit) const
@ -880,7 +876,8 @@ void CacheDictionary::update(BunchUpdateUnit & bunch_update_unit) const
const auto * id_column = typeid_cast<const ColumnUInt64 *>(block.safeGetByPosition(0).column.get());
if (!id_column)
throw Exception{name + ": id column has type different from UInt64.", ErrorCodes::TYPE_MISMATCH};
throw Exception{ErrorCodes::TYPE_MISMATCH,
"{}: id column has type different from UInt64.", getDictionaryID().getNameForLogs()};
const auto & ids = id_column->getData();
@ -945,8 +942,9 @@ void CacheDictionary::update(BunchUpdateUnit & bunch_update_unit) const
last_exception = std::current_exception();
backoff_end_time = now + std::chrono::seconds(calculateDurationWithBackoff(rnd_engine, error_count));
tryLogException(last_exception, log, "Could not update cache dictionary '" + getFullName() +
"', next update is scheduled at " + ext::to_string(backoff_end_time.load()));
tryLogException(last_exception, log,
"Could not update cache dictionary '" + getDictionaryID().getNameForLogs() +
"', next update is scheduled at " + ext::to_string(backoff_end_time.load()));
}
}

View File

@ -50,8 +50,7 @@ class CacheDictionary final : public IDictionary
{
public:
CacheDictionary(
const std::string & database_,
const std::string & name_,
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
DictionaryLifetime dict_lifetime_,
@ -65,10 +64,6 @@ public:
~CacheDictionary() override;
const std::string & getDatabase() const override { return database; }
const std::string & getName() const override { return name; }
const std::string & getFullName() const override { return full_name; }
std::string getTypeName() const override { return "Cache"; }
size_t getBytesAllocated() const override { return bytes_allocated + (string_arena ? string_arena->size() : 0); }
@ -89,8 +84,7 @@ public:
std::shared_ptr<const IExternalLoadable> clone() const override
{
return std::make_shared<CacheDictionary>(
database,
name,
getDictionaryID(),
dict_struct,
getSourceAndUpdateIfNeeded()->clone(),
dict_lifetime,
@ -321,9 +315,6 @@ private:
template <typename AncestorType>
void isInImpl(const PaddedPODArray<Key> & child_ids, const AncestorType & ancestor_ids, PaddedPODArray<UInt8> & out) const;
const std::string database;
const std::string name;
const std::string full_name;
const DictionaryStructure dict_struct;
/// Dictionary source should be used with mutex

View File

@ -8,7 +8,7 @@ namespace DB
const \
{ \
auto & attribute = getAttribute(attribute_name); \
checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
const auto null_value = std::get<TYPE>(attribute.null_values); \
getItemsNumberImpl<TYPE, TYPE>(attribute, ids, out, [&](const size_t) { return null_value; }); \
}

View File

@ -11,7 +11,7 @@ namespace DB
ResultArrayType<TYPE> & out) const \
{ \
auto & attribute = getAttribute(attribute_name); \
checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
getItemsNumberImpl<TYPE, TYPE>(attribute, ids, out, [&](const size_t row) { return def[row]; }); \
}

View File

@ -8,7 +8,7 @@ namespace DB
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const TYPE def, ResultArrayType<TYPE> & out) const \
{ \
auto & attribute = getAttribute(attribute_name); \
checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
getItemsNumberImpl<TYPE, TYPE>(attribute, ids, out, [&](const size_t) { return def; }); \
}

View File

@ -51,15 +51,12 @@ inline UInt64 ComplexKeyCacheDictionary::getCellIdx(const StringRef key) const
ComplexKeyCacheDictionary::ComplexKeyCacheDictionary(
const std::string & database_,
const std::string & name_,
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
const DictionaryLifetime dict_lifetime_,
const size_t size_)
: database(database_)
, name(name_)
, full_name{database_.empty() ? name_ : (database_ + "." + name_)}
: IDictionaryBase(dict_id_)
, dict_struct(dict_struct_)
, source_ptr{std::move(source_ptr_)}
, dict_lifetime(dict_lifetime_)
@ -80,7 +77,7 @@ void ComplexKeyCacheDictionary::getString(
dict_struct.validateKeyTypes(key_types);
auto & attribute = getAttribute(attribute_name);
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString);
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString);
const auto null_value = StringRef{std::get<String>(attribute.null_values)};
@ -97,7 +94,7 @@ void ComplexKeyCacheDictionary::getString(
dict_struct.validateKeyTypes(key_types);
auto & attribute = getAttribute(attribute_name);
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString);
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString);
getItemsString(attribute, key_columns, out, [&](const size_t row) { return def->getDataAt(row); });
}
@ -112,7 +109,7 @@ void ComplexKeyCacheDictionary::getString(
dict_struct.validateKeyTypes(key_types);
auto & attribute = getAttribute(attribute_name);
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString);
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString);
getItemsString(attribute, key_columns, out, [&](const size_t) { return StringRef{def}; });
}
@ -415,10 +412,9 @@ void registerDictionaryComplexKeyCache(DictionaryFactory & factory)
throw Exception{full_name + ": dictionary of layout 'cache' cannot have 'require_nonempty' attribute set",
ErrorCodes::BAD_ARGUMENTS};
const String database = config.getString(config_prefix + ".database", "");
const String name = config.getString(config_prefix + ".name");
const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix);
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};
return std::make_unique<ComplexKeyCacheDictionary>(database, name, dict_struct, std::move(source_ptr), dict_lifetime, size);
return std::make_unique<ComplexKeyCacheDictionary>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, size);
};
factory.registerLayout("complex_key_cache", create_layout, true);
}

View File

@ -42,8 +42,7 @@ class ComplexKeyCacheDictionary final : public IDictionaryBase
{
public:
ComplexKeyCacheDictionary(
const std::string & database_,
const std::string & name_,
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
const DictionaryLifetime dict_lifetime_,
@ -51,10 +50,6 @@ public:
std::string getKeyDescription() const { return key_description; }
const std::string & getDatabase() const override { return database; }
const std::string & getName() const override { return name; }
const std::string & getFullName() const override { return full_name; }
std::string getTypeName() const override { return "ComplexKeyCache"; }
size_t getBytesAllocated() const override
@ -78,7 +73,7 @@ public:
std::shared_ptr<const IExternalLoadable> clone() const override
{
return std::make_shared<ComplexKeyCacheDictionary>(database, name, dict_struct, source_ptr->clone(), dict_lifetime, size);
return std::make_shared<ComplexKeyCacheDictionary>(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, size);
}
const IDictionarySource * getSource() const override { return source_ptr.get(); }
@ -671,9 +666,6 @@ private:
bool isEmptyCell(const UInt64 idx) const;
const std::string database;
const std::string name;
const std::string full_name;
const DictionaryStructure dict_struct;
const DictionarySourcePtr source_ptr;
const DictionaryLifetime dict_lifetime;

View File

@ -8,7 +8,7 @@ namespace DB
{ \
dict_struct.validateKeyTypes(key_types); \
auto & attribute = getAttribute(attribute_name); \
checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
const auto null_value = std::get<TYPE>(attribute.null_values); \
getItemsNumberImpl<TYPE, TYPE>(attribute, key_columns, out, [&](const size_t) { return null_value; }); \
}

View File

@ -12,7 +12,7 @@ namespace DB
{ \
dict_struct.validateKeyTypes(key_types); \
auto & attribute = getAttribute(attribute_name); \
checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
getItemsNumberImpl<TYPE, TYPE>(attribute, key_columns, out, [&](const size_t row) { return def[row]; }); \
}

View File

@ -12,7 +12,7 @@ namespace DB
{ \
dict_struct.validateKeyTypes(key_types); \
auto & attribute = getAttribute(attribute_name); \
checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
getItemsNumberImpl<TYPE, TYPE>(attribute, key_columns, out, [&](const size_t) { return def; }); \
}

View File

@ -15,14 +15,11 @@ namespace ErrorCodes
ComplexKeyDirectDictionary::ComplexKeyDirectDictionary(
const std::string & database_,
const std::string & name_,
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
BlockPtr saved_block_)
: database(database_)
, name(name_)
, full_name{database_.empty() ? name_ : (database_ + "." + name_)}
: IDictionaryBase(dict_id_)
, dict_struct(dict_struct_)
, source_ptr{std::move(source_ptr_)}
, saved_block{std::move(saved_block_)}
@ -39,7 +36,7 @@ ComplexKeyDirectDictionary::ComplexKeyDirectDictionary(
{ \
dict_struct.validateKeyTypes(key_types); \
const auto & attribute = getAttribute(attribute_name); \
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
\
const auto null_value = std::get<TYPE>(attribute.null_values); \
\
@ -67,7 +64,7 @@ void ComplexKeyDirectDictionary::getString(
{
dict_struct.validateKeyTypes(key_types);
const auto & attribute = getAttribute(attribute_name);
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString);
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString);
const auto & null_value = std::get<StringRef>(attribute.null_values);
getItemsStringImpl<StringRef, StringRef>(
@ -87,7 +84,7 @@ void ComplexKeyDirectDictionary::getString(
{ \
dict_struct.validateKeyTypes(key_types); \
const auto & attribute = getAttribute(attribute_name); \
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
\
getItemsImpl<TYPE, TYPE>( \
attribute, key_columns, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t row) { return def[row]; }); \
@ -114,7 +111,7 @@ void ComplexKeyDirectDictionary::getString(
dict_struct.validateKeyTypes(key_types);
const auto & attribute = getAttribute(attribute_name);
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString);
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString);
getItemsStringImpl<StringRef, StringRef>(
attribute,
@ -129,7 +126,7 @@ void ComplexKeyDirectDictionary::getString(
{ \
dict_struct.validateKeyTypes(key_types); \
const auto & attribute = getAttribute(attribute_name); \
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
\
getItemsImpl<TYPE, TYPE>( \
attribute, key_columns, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t) { return def; }); \
@ -156,7 +153,7 @@ void ComplexKeyDirectDictionary::getString(
dict_struct.validateKeyTypes(key_types);
const auto & attribute = getAttribute(attribute_name);
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString);
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString);
ComplexKeyDirectDictionary::getItemsStringImpl<StringRef, StringRef>(
attribute,
@ -588,14 +585,13 @@ void registerDictionaryComplexKeyDirect(DictionaryFactory & factory)
"for a dictionary of layout 'range_hashed'",
ErrorCodes::BAD_ARGUMENTS};
const String database = config.getString(config_prefix + ".database", "");
const String name = config.getString(config_prefix + ".name");
const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix);
if (config.has(config_prefix + ".lifetime.min") || config.has(config_prefix + ".lifetime.max"))
throw Exception{"'lifetime' parameter is redundant for the dictionary' of layout 'direct'", ErrorCodes::BAD_ARGUMENTS};
return std::make_unique<ComplexKeyDirectDictionary>(database, name, dict_struct, std::move(source_ptr));
return std::make_unique<ComplexKeyDirectDictionary>(dict_id, dict_struct, std::move(source_ptr));
};
factory.registerLayout("complex_key_direct", create_layout, false);
}

View File

@ -25,16 +25,11 @@ class ComplexKeyDirectDictionary final : public IDictionaryBase
{
public:
ComplexKeyDirectDictionary(
const std::string & database_,
const std::string & name_,
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
BlockPtr saved_block_ = nullptr);
const std::string & getDatabase() const override { return database; }
const std::string & getName() const override { return name; }
const std::string & getFullName() const override { return full_name; }
std::string getTypeName() const override { return "ComplexKeyDirect"; }
size_t getBytesAllocated() const override { return 0; }
@ -51,7 +46,7 @@ public:
std::shared_ptr<const IExternalLoadable> clone() const override
{
return std::make_shared<ComplexKeyDirectDictionary>(database, name, dict_struct, source_ptr->clone(), saved_block);
return std::make_shared<ComplexKeyDirectDictionary>(getDictionaryID(), dict_struct, source_ptr->clone(), saved_block);
}
const IDictionarySource * getSource() const override { return source_ptr.get(); }
@ -205,9 +200,6 @@ private:
template <typename T>
void has(const Attribute & attribute, const Columns & key_columns, PaddedPODArray<UInt8> & out) const;
const std::string database;
const std::string name;
const std::string full_name;
const DictionaryStructure dict_struct;
const DictionarySourcePtr source_ptr;
const DictionaryLifetime dict_lifetime;

View File

@ -14,16 +14,13 @@ namespace ErrorCodes
}
ComplexKeyHashedDictionary::ComplexKeyHashedDictionary(
const std::string & database_,
const std::string & name_,
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
const DictionaryLifetime dict_lifetime_,
bool require_nonempty_,
BlockPtr saved_block_)
: database(database_)
, name(name_)
, full_name{database_.empty() ? name_ : (database_ + "." + name_)}
: IDictionaryBase(dict_id_)
, dict_struct(dict_struct_)
, source_ptr{std::move(source_ptr_)}
, dict_lifetime(dict_lifetime_)
@ -42,7 +39,7 @@ ComplexKeyHashedDictionary::ComplexKeyHashedDictionary(
dict_struct.validateKeyTypes(key_types); \
\
const auto & attribute = getAttribute(attribute_name); \
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
\
const auto null_value = std::get<TYPE>(attribute.null_values); \
\
@ -74,7 +71,7 @@ void ComplexKeyHashedDictionary::getString(
dict_struct.validateKeyTypes(key_types);
const auto & attribute = getAttribute(attribute_name);
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString);
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString);
const auto & null_value = StringRef{std::get<String>(attribute.null_values)};
@ -96,7 +93,7 @@ void ComplexKeyHashedDictionary::getString(
dict_struct.validateKeyTypes(key_types); \
\
const auto & attribute = getAttribute(attribute_name); \
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
\
getItemsImpl<TYPE, TYPE>( \
attribute, \
@ -130,7 +127,7 @@ void ComplexKeyHashedDictionary::getString(
dict_struct.validateKeyTypes(key_types);
const auto & attribute = getAttribute(attribute_name);
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString);
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString);
getItemsImpl<StringRef, StringRef>(
attribute,
@ -150,7 +147,7 @@ void ComplexKeyHashedDictionary::getString(
dict_struct.validateKeyTypes(key_types); \
\
const auto & attribute = getAttribute(attribute_name); \
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
\
getItemsImpl<TYPE, TYPE>( \
attribute, key_columns, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t) { return def; }); \
@ -181,7 +178,7 @@ void ComplexKeyHashedDictionary::getString(
dict_struct.validateKeyTypes(key_types);
const auto & attribute = getAttribute(attribute_name);
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString);
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString);
getItemsImpl<StringRef, StringRef>(
attribute,
@ -753,11 +750,10 @@ void registerDictionaryComplexKeyHashed(DictionaryFactory & factory)
if (!dict_struct.key)
throw Exception{"'key' is required for dictionary of layout 'complex_key_hashed'", ErrorCodes::BAD_ARGUMENTS};
const String database = config.getString(config_prefix + ".database", "");
const String name = config.getString(config_prefix + ".name");
const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix);
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};
const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false);
return std::make_unique<ComplexKeyHashedDictionary>(database, name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
return std::make_unique<ComplexKeyHashedDictionary>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
};
factory.registerLayout("complex_key_hashed", create_layout, true);
}

View File

@ -23,8 +23,7 @@ class ComplexKeyHashedDictionary final : public IDictionaryBase
{
public:
ComplexKeyHashedDictionary(
const std::string & database_,
const std::string & name_,
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
const DictionaryLifetime dict_lifetime_,
@ -33,10 +32,6 @@ public:
std::string getKeyDescription() const { return key_description; }
const std::string & getDatabase() const override { return database; }
const std::string & getName() const override { return name; }
const std::string & getFullName() const override { return full_name; }
std::string getTypeName() const override { return "ComplexKeyHashed"; }
size_t getBytesAllocated() const override { return bytes_allocated; }
@ -51,7 +46,7 @@ public:
std::shared_ptr<const IExternalLoadable> clone() const override
{
return std::make_shared<ComplexKeyHashedDictionary>(database, name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, saved_block);
return std::make_shared<ComplexKeyHashedDictionary>(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, saved_block);
}
const IDictionarySource * getSource() const override { return source_ptr.get(); }
@ -236,9 +231,6 @@ private:
template <typename T>
std::vector<StringRef> getKeys(const Attribute & attribute) const;
const std::string database;
const std::string name;
const std::string full_name;
const DictionaryStructure dict_struct;
const DictionarySourcePtr source_ptr;
const DictionaryLifetime dict_lifetime;

View File

@ -15,11 +15,6 @@
namespace DB
{
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
}
enum class AttributeUnderlyingType
{
utUInt8,
@ -44,15 +39,6 @@ AttributeUnderlyingType getAttributeUnderlyingType(const std::string & type);
std::string toString(const AttributeUnderlyingType type);
/// Implicit conversions in dictGet functions is disabled.
inline void checkAttributeType(const std::string & dict_name, const std::string & attribute_name,
AttributeUnderlyingType attribute_type, AttributeUnderlyingType to)
{
if (attribute_type != to)
throw Exception{dict_name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute_type)
+ ", expected " + toString(to), ErrorCodes::TYPE_MISMATCH};
}
/// Min and max lifetimes for a dictionary or it's entry
using DictionaryLifetime = ExternalLoadableLifetime;

View File

@ -16,14 +16,11 @@ namespace ErrorCodes
DirectDictionary::DirectDictionary(
const std::string & database_,
const std::string & name_,
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
BlockPtr saved_block_)
: database(database_)
, name(name_)
, full_name{database_.empty() ? name_ : (database_ + "." + name_)}
: IDictionary(dict_id_)
, dict_struct(dict_struct_)
, source_ptr{std::move(source_ptr_)}
, saved_block{std::move(saved_block_)}
@ -136,7 +133,7 @@ void DirectDictionary::isInConstantVector(const Key child_id, const PaddedPODArr
void DirectDictionary::get##TYPE(const std::string & attribute_name, const PaddedPODArray<Key> & ids, ResultArrayType<TYPE> & out) const \
{ \
const auto & attribute = getAttribute(attribute_name); \
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
\
const auto null_value = std::get<TYPE>(attribute.null_values); \
\
@ -162,7 +159,7 @@ DECLARE(Decimal128)
void DirectDictionary::getString(const std::string & attribute_name, const PaddedPODArray<Key> & ids, ColumnString * out) const
{
const auto & attribute = getAttribute(attribute_name);
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString);
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString);
const auto & null_value = std::get<StringRef>(attribute.null_values);
getItemsStringImpl<StringRef, StringRef>(
@ -180,7 +177,7 @@ void DirectDictionary::getString(const std::string & attribute_name, const Padde
ResultArrayType<TYPE> & out) const \
{ \
const auto & attribute = getAttribute(attribute_name); \
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
\
getItemsImpl<TYPE, TYPE>( \
attribute, ids, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t row) { return def[row]; }); \
@ -205,7 +202,7 @@ void DirectDictionary::getString(
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const ColumnString * const def, ColumnString * const out) const
{
const auto & attribute = getAttribute(attribute_name);
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString);
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString);
getItemsStringImpl<StringRef, StringRef>(
attribute,
@ -219,7 +216,7 @@ void DirectDictionary::getString(
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const TYPE def, ResultArrayType<TYPE> & out) const \
{ \
const auto & attribute = getAttribute(attribute_name); \
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
\
getItemsImpl<TYPE, TYPE>( \
attribute, ids, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t) { return def; }); \
@ -244,7 +241,7 @@ void DirectDictionary::getString(
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const String & def, ColumnString * const out) const
{
const auto & attribute = getAttribute(attribute_name);
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString);
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString);
DirectDictionary::getItemsStringImpl<StringRef, StringRef>(
attribute,
@ -577,14 +574,13 @@ void registerDictionaryDirect(DictionaryFactory & factory)
"for a dictionary of layout 'range_hashed'",
ErrorCodes::BAD_ARGUMENTS};
const String database = config.getString(config_prefix + ".database", "");
const String name = config.getString(config_prefix + ".name");
const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix);
if (config.has(config_prefix + ".lifetime.min") || config.has(config_prefix + ".lifetime.max"))
throw Exception{"'lifetime' parameter is redundant for the dictionary' of layout 'direct'", ErrorCodes::BAD_ARGUMENTS};
return std::make_unique<DirectDictionary>(database, name, dict_struct, std::move(source_ptr));
return std::make_unique<DirectDictionary>(dict_id, dict_struct, std::move(source_ptr));
};
factory.registerLayout("direct", create_layout, false);
}

View File

@ -23,16 +23,11 @@ class DirectDictionary final : public IDictionary
{
public:
DirectDictionary(
const std::string & database_,
const std::string & name_,
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
BlockPtr saved_block_ = nullptr);
const std::string & getDatabase() const override { return database; }
const std::string & getName() const override { return name; }
const std::string & getFullName() const override { return full_name; }
std::string getTypeName() const override { return "Direct"; }
size_t getBytesAllocated() const override { return 0; }
@ -47,7 +42,7 @@ public:
std::shared_ptr<const IExternalLoadable> clone() const override
{
return std::make_shared<DirectDictionary>(database, name, dict_struct, source_ptr->clone(), saved_block);
return std::make_shared<DirectDictionary>(getDictionaryID(), dict_struct, source_ptr->clone(), saved_block);
}
const IDictionarySource * getSource() const override { return source_ptr.get(); }
@ -206,9 +201,6 @@ private:
template <typename ChildType, typename AncestorType>
void isInImpl(const ChildType & child_ids, const AncestorType & ancestor_ids, PaddedPODArray<UInt8> & out) const;
const std::string database;
const std::string name;
const std::string full_name;
const DictionaryStructure dict_struct;
const DictionarySourcePtr source_ptr;
const DictionaryLifetime dict_lifetime;

View File

@ -21,16 +21,13 @@ static const auto max_array_size = 500000;
FlatDictionary::FlatDictionary(
const std::string & database_,
const std::string & name_,
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
const DictionaryLifetime dict_lifetime_,
bool require_nonempty_,
BlockPtr saved_block_)
: database(database_)
, name(name_)
, full_name{database_.empty() ? name_ : (database_ + "." + name_)}
: IDictionary(dict_id_)
, dict_struct(dict_struct_)
, source_ptr{std::move(source_ptr_)}
, dict_lifetime(dict_lifetime_)
@ -110,7 +107,7 @@ void FlatDictionary::isInConstantVector(const Key child_id, const PaddedPODArray
void FlatDictionary::get##TYPE(const std::string & attribute_name, const PaddedPODArray<Key> & ids, ResultArrayType<TYPE> & out) const \
{ \
const auto & attribute = getAttribute(attribute_name); \
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
\
const auto null_value = std::get<TYPE>(attribute.null_values); \
\
@ -136,7 +133,7 @@ DECLARE(Decimal128)
void FlatDictionary::getString(const std::string & attribute_name, const PaddedPODArray<Key> & ids, ColumnString * out) const
{
const auto & attribute = getAttribute(attribute_name);
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString);
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString);
const auto & null_value = std::get<StringRef>(attribute.null_values);
@ -155,7 +152,7 @@ void FlatDictionary::getString(const std::string & attribute_name, const PaddedP
ResultArrayType<TYPE> & out) const \
{ \
const auto & attribute = getAttribute(attribute_name); \
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
\
getItemsImpl<TYPE, TYPE>( \
attribute, ids, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t row) { return def[row]; }); \
@ -180,7 +177,7 @@ void FlatDictionary::getString(
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const ColumnString * const def, ColumnString * const out) const
{
const auto & attribute = getAttribute(attribute_name);
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString);
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString);
getItemsImpl<StringRef, StringRef>(
attribute,
@ -194,7 +191,7 @@ void FlatDictionary::getString(
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const TYPE def, ResultArrayType<TYPE> & out) const \
{ \
const auto & attribute = getAttribute(attribute_name); \
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
\
getItemsImpl<TYPE, TYPE>( \
attribute, ids, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t) { return def; }); \
@ -219,7 +216,7 @@ void FlatDictionary::getString(
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const String & def, ColumnString * const out) const
{
const auto & attribute = getAttribute(attribute_name);
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString);
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString);
FlatDictionary::getItemsImpl<StringRef, StringRef>(
attribute,
@ -724,11 +721,10 @@ void registerDictionaryFlat(DictionaryFactory & factory)
"for a dictionary of layout 'range_hashed'",
ErrorCodes::BAD_ARGUMENTS};
const String database = config.getString(config_prefix + ".database", "");
const String name = config.getString(config_prefix + ".name");
const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix);
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};
const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false);
return std::make_unique<FlatDictionary>(database, name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
return std::make_unique<FlatDictionary>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
};
factory.registerLayout("flat", create_layout, false);
}

View File

@ -22,18 +22,13 @@ class FlatDictionary final : public IDictionary
{
public:
FlatDictionary(
const std::string & database_,
const std::string & name_,
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
const DictionaryLifetime dict_lifetime_,
bool require_nonempty_,
BlockPtr saved_block_ = nullptr);
const std::string & getDatabase() const override { return database; }
const std::string & getName() const override { return name; }
const std::string & getFullName() const override { return full_name; }
std::string getTypeName() const override { return "Flat"; }
size_t getBytesAllocated() const override { return bytes_allocated; }
@ -48,7 +43,7 @@ public:
std::shared_ptr<const IExternalLoadable> clone() const override
{
return std::make_shared<FlatDictionary>(database, name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, saved_block);
return std::make_shared<FlatDictionary>(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, saved_block);
}
const IDictionarySource * getSource() const override { return source_ptr.get(); }
@ -225,9 +220,6 @@ private:
PaddedPODArray<Key> getIds() const;
const std::string database;
const std::string name;
const std::string full_name;
const DictionaryStructure dict_struct;
const DictionarySourcePtr source_ptr;
const DictionaryLifetime dict_lifetime;

View File

@ -32,17 +32,14 @@ namespace ErrorCodes
HashedDictionary::HashedDictionary(
const std::string & database_,
const std::string & name_,
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
const DictionaryLifetime dict_lifetime_,
bool require_nonempty_,
bool sparse_,
BlockPtr saved_block_)
: database(database_)
, name(name_)
, full_name{database_.empty() ? name_ : (database_ + "." + name_)}
: IDictionary(dict_id_)
, dict_struct(dict_struct_)
, source_ptr{std::move(source_ptr_)}
, dict_lifetime(dict_lifetime_)
@ -133,7 +130,7 @@ void HashedDictionary::isInConstantVector(const Key child_id, const PaddedPODArr
const \
{ \
const auto & attribute = getAttribute(attribute_name); \
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
\
const auto null_value = std::get<TYPE>(attribute.null_values); \
\
@ -159,7 +156,7 @@ DECLARE(Decimal128)
void HashedDictionary::getString(const std::string & attribute_name, const PaddedPODArray<Key> & ids, ColumnString * out) const
{
const auto & attribute = getAttribute(attribute_name);
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString);
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString);
const auto & null_value = StringRef{std::get<String>(attribute.null_values)};
@ -178,7 +175,7 @@ void HashedDictionary::getString(const std::string & attribute_name, const Padde
ResultArrayType<TYPE> & out) const \
{ \
const auto & attribute = getAttribute(attribute_name); \
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
\
getItemsImpl<TYPE, TYPE>( \
attribute, ids, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t row) { return def[row]; }); \
@ -203,7 +200,7 @@ void HashedDictionary::getString(
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const ColumnString * const def, ColumnString * const out) const
{
const auto & attribute = getAttribute(attribute_name);
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString);
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString);
getItemsImpl<StringRef, StringRef>(
attribute,
@ -217,7 +214,7 @@ void HashedDictionary::getString(
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const TYPE & def, ResultArrayType<TYPE> & out) const \
{ \
const auto & attribute = getAttribute(attribute_name); \
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
\
getItemsImpl<TYPE, TYPE>( \
attribute, ids, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t) { return def; }); \
@ -242,7 +239,7 @@ void HashedDictionary::getString(
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const String & def, ColumnString * const out) const
{
const auto & attribute = getAttribute(attribute_name);
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString);
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString);
getItemsImpl<StringRef, StringRef>(
attribute,
@ -788,11 +785,10 @@ void registerDictionaryHashed(DictionaryFactory & factory)
"for a dictionary of layout 'range_hashed'",
ErrorCodes::BAD_ARGUMENTS};
const String database = config.getString(config_prefix + ".database", "");
const String name = config.getString(config_prefix + ".name");
const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix);
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};
const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false);
return std::make_unique<HashedDictionary>(database, name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty, sparse);
return std::make_unique<HashedDictionary>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty, sparse);
};
using namespace std::placeholders;
factory.registerLayout("hashed",

View File

@ -26,8 +26,7 @@ class HashedDictionary final : public IDictionary
{
public:
HashedDictionary(
const std::string & database_,
const std::string & name_,
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
const DictionaryLifetime dict_lifetime_,
@ -35,10 +34,6 @@ public:
bool sparse_,
BlockPtr saved_block_ = nullptr);
const std::string & getDatabase() const override { return database; }
const std::string & getName() const override { return name; }
const std::string & getFullName() const override { return full_name; }
std::string getTypeName() const override { return sparse ? "SparseHashed" : "Hashed"; }
size_t getBytesAllocated() const override { return bytes_allocated; }
@ -53,7 +48,7 @@ public:
std::shared_ptr<const IExternalLoadable> clone() const override
{
return std::make_shared<HashedDictionary>(database, name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, sparse, saved_block);
return std::make_shared<HashedDictionary>(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, sparse, saved_block);
}
const IDictionarySource * getSource() const override { return source_ptr.get(); }
@ -271,9 +266,6 @@ private:
template <typename ChildType, typename AncestorType>
void isInImpl(const ChildType & child_ids, const AncestorType & ancestor_ids, PaddedPODArray<UInt8> & out) const;
const std::string database;
const std::string name;
const std::string full_name;
const DictionaryStructure dict_struct;
const DictionarySourcePtr source_ptr;
const DictionaryLifetime dict_lifetime;

View File

@ -4,19 +4,23 @@
#include <Core/Names.h>
#include <DataStreams/IBlockStream_fwd.h>
#include <Interpreters/IExternalLoadable.h>
#include <Interpreters/StorageID.h>
#include <Poco/Util/XMLConfiguration.h>
#include <Common/PODArray.h>
#include <common/StringRef.h>
#include "IDictionarySource.h"
#include <Dictionaries/DictionaryStructure.h>
#include <chrono>
#include <memory>
#include <mutex>
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
extern const int TYPE_MISMATCH;
}
struct IDictionaryBase;
@ -29,22 +33,37 @@ struct IDictionaryBase : public IExternalLoadable
{
using Key = UInt64;
virtual const std::string & getDatabase() const = 0;
virtual const std::string & getName() const = 0;
virtual const std::string & getFullName() const = 0;
IDictionaryBase(const StorageID & dict_id_)
: dict_id(dict_id_)
, full_name(dict_id.getInternalDictionaryName())
{
}
const std::string & getLoadableName() const override { return getFullName(); }
const std::string & getFullName() const{ return full_name; }
StorageID getDictionaryID() const
{
std::lock_guard lock{name_mutex};
return dict_id;
}
void updateDictionaryName(const StorageID & new_name) const
{
std::lock_guard lock{name_mutex};
assert(new_name.uuid == dict_id.uuid && dict_id.uuid != UUIDHelpers::Nil);
dict_id = new_name;
}
const std::string & getLoadableName() const override final { return getFullName(); }
/// Specifies that no database is used.
/// Sometimes we cannot simply use an empty string for that because an empty string is
/// usually replaced with the current database.
static constexpr char NO_DATABASE_TAG[] = "<no_database>";
std::string_view getDatabaseOrNoDatabaseTag() const
std::string getDatabaseOrNoDatabaseTag() const
{
const std::string & database = getDatabase();
if (!database.empty())
return database;
if (!dict_id.database_name.empty())
return dict_id.database_name;
return NO_DATABASE_TAG;
}
@ -87,11 +106,20 @@ struct IDictionaryBase : public IExternalLoadable
{
return std::static_pointer_cast<const IDictionaryBase>(IExternalLoadable::shared_from_this());
}
private:
mutable std::mutex name_mutex;
mutable StorageID dict_id;
protected:
const String full_name;
};
struct IDictionary : IDictionaryBase
{
IDictionary(const StorageID & dict_id_) : IDictionaryBase(dict_id_) {}
virtual bool hasHierarchy() const = 0;
virtual void toParent(const PaddedPODArray<Key> & ids, PaddedPODArray<Key> & out) const = 0;
@ -103,19 +131,22 @@ struct IDictionary : IDictionaryBase
virtual void isInVectorVector(
const PaddedPODArray<Key> & /*child_ids*/, const PaddedPODArray<Key> & /*ancestor_ids*/, PaddedPODArray<UInt8> & /*out*/) const
{
throw Exception("Hierarchy is not supported for " + getName() + " dictionary.", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
"Hierarchy is not supported for {} dictionary.", getDictionaryID().getNameForLogs());
}
virtual void
isInVectorConstant(const PaddedPODArray<Key> & /*child_ids*/, const Key /*ancestor_id*/, PaddedPODArray<UInt8> & /*out*/) const
{
throw Exception("Hierarchy is not supported for " + getName() + " dictionary.", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
"Hierarchy is not supported for {} dictionary.", getDictionaryID().getNameForLogs());
}
virtual void
isInConstantVector(const Key /*child_id*/, const PaddedPODArray<Key> & /*ancestor_ids*/, PaddedPODArray<UInt8> & /*out*/) const
{
throw Exception("Hierarchy is not supported for " + getName() + " dictionary.", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
"Hierarchy is not supported for {} dictionary.", getDictionaryID().getNameForLogs());
}
void isInConstantConstant(const Key child_id, const Key ancestor_id, UInt8 & out) const
@ -126,4 +157,14 @@ struct IDictionary : IDictionaryBase
}
};
/// Implicit conversions in dictGet functions is disabled.
inline void checkAttributeType(const IDictionaryBase * dictionary, const std::string & attribute_name,
AttributeUnderlyingType attribute_type, AttributeUnderlyingType to)
{
if (attribute_type != to)
throw Exception{ErrorCodes::TYPE_MISMATCH, "{}: type mismatch: attribute {} has type {}, expected {}",
dictionary->getDictionaryID().getNameForLogs(),
attribute_name, toString(attribute_type), toString(to)};
}
}

View File

@ -20,16 +20,13 @@ namespace ErrorCodes
IPolygonDictionary::IPolygonDictionary(
const std::string & database_,
const std::string & name_,
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
const DictionaryLifetime dict_lifetime_,
InputType input_type_,
PointType point_type_)
: database(database_)
, name(name_)
, full_name{database_.empty() ? name_ : (database_ + "." + name_)}
: IDictionaryBase(dict_id_)
, dict_struct(dict_struct_)
, source_ptr(std::move(source_ptr_))
, dict_lifetime(dict_lifetime_)
@ -40,21 +37,6 @@ IPolygonDictionary::IPolygonDictionary(
loadData();
}
const std::string & IPolygonDictionary::getDatabase() const
{
return database;
}
const std::string & IPolygonDictionary::getName() const
{
return name;
}
const std::string & IPolygonDictionary::getFullName() const
{
return full_name;
}
std::string IPolygonDictionary::getTypeName() const
{
return "Polygon";
@ -186,7 +168,9 @@ void IPolygonDictionary::createAttributes()
appendNullValue(attr.underlying_type, attr.null_value);
if (attr.hierarchical)
throw Exception{name + ": hierarchical attributes not supported for dictionary of polygonal type", ErrorCodes::TYPE_MISMATCH};
throw Exception{ErrorCodes::TYPE_MISMATCH,
"{}: hierarchical attributes not supported for dictionary of polygonal type",
getDictionaryID().getNameForLogs()};
}
}
@ -297,7 +281,7 @@ size_t IPolygonDictionary::getAttributeIndex(const std::string & attribute_name)
const std::string & attribute_name, const Columns & key_columns, const DataTypes &, ResultArrayType<TYPE> & out) const \
{ \
const auto ind = getAttributeIndex(attribute_name); \
checkAttributeType(name, attribute_name, dict_struct.attributes[ind].underlying_type, AttributeUnderlyingType::ut##TYPE); \
checkAttributeType(this, attribute_name, dict_struct.attributes[ind].underlying_type, AttributeUnderlyingType::ut##TYPE); \
\
const auto null_value = std::get<TYPE>(null_values[ind]); \
\
@ -327,7 +311,7 @@ void IPolygonDictionary::getString(
const std::string & attribute_name, const Columns & key_columns, const DataTypes &, ColumnString * out) const
{
const auto ind = getAttributeIndex(attribute_name);
checkAttributeType(name, attribute_name, dict_struct.attributes[ind].underlying_type, AttributeUnderlyingType::utString);
checkAttributeType(this, attribute_name, dict_struct.attributes[ind].underlying_type, AttributeUnderlyingType::utString);
const auto & null_value = StringRef{std::get<String>(null_values[ind])};
@ -347,7 +331,7 @@ void IPolygonDictionary::getString(
ResultArrayType<TYPE> & out) const \
{ \
const auto ind = getAttributeIndex(attribute_name); \
checkAttributeType(name, attribute_name, dict_struct.attributes[ind].underlying_type, AttributeUnderlyingType::ut##TYPE); \
checkAttributeType(this, attribute_name, dict_struct.attributes[ind].underlying_type, AttributeUnderlyingType::ut##TYPE); \
\
getItemsImpl<TYPE, TYPE>( \
ind, \
@ -379,7 +363,7 @@ void IPolygonDictionary::getString(
ColumnString * const out) const
{
const auto ind = getAttributeIndex(attribute_name);
checkAttributeType(name, attribute_name, dict_struct.attributes[ind].underlying_type, AttributeUnderlyingType::utString);
checkAttributeType(this, attribute_name, dict_struct.attributes[ind].underlying_type, AttributeUnderlyingType::utString);
getItemsImpl<String, StringRef>(
ind,
@ -397,7 +381,7 @@ void IPolygonDictionary::getString(
ResultArrayType<TYPE> & out) const \
{ \
const auto ind = getAttributeIndex(attribute_name); \
checkAttributeType(name, attribute_name, dict_struct.attributes[ind].underlying_type, AttributeUnderlyingType::ut##TYPE); \
checkAttributeType(this, attribute_name, dict_struct.attributes[ind].underlying_type, AttributeUnderlyingType::ut##TYPE); \
\
getItemsImpl<TYPE, TYPE>( \
ind, key_columns, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t) { return def; }); \
@ -426,7 +410,7 @@ void IPolygonDictionary::getString(
ColumnString * const out) const
{
const auto ind = getAttributeIndex(attribute_name);
checkAttributeType(name, attribute_name, dict_struct.attributes[ind].underlying_type, AttributeUnderlyingType::utString);
checkAttributeType(this, attribute_name, dict_struct.attributes[ind].underlying_type, AttributeUnderlyingType::utString);
getItemsImpl<String, StringRef>(
ind,

View File

@ -49,18 +49,13 @@ public:
Tuple,
};
IPolygonDictionary(
const std::string & database_,
const std::string & name_,
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
DictionaryLifetime dict_lifetime_,
InputType input_type_,
PointType point_type_);
const std::string & getDatabase() const override;
const std::string & getName() const override;
const std::string & getFullName() const override;
std::string getTypeName() const override;
std::string getKeyDescription() const;
@ -200,9 +195,6 @@ protected:
*/
std::vector<size_t> ids;
const std::string database;
const std::string name;
const std::string full_name;
const DictionaryStructure dict_struct;
const DictionarySourcePtr source_ptr;
const DictionaryLifetime dict_lifetime;

View File

@ -18,22 +18,20 @@ namespace ErrorCodes
}
PolygonDictionarySimple::PolygonDictionarySimple(
const std::string & database_,
const std::string & name_,
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
const DictionaryLifetime dict_lifetime_,
InputType input_type_,
PointType point_type_):
IPolygonDictionary(database_, name_, dict_struct_, std::move(source_ptr_), dict_lifetime_, input_type_, point_type_)
IPolygonDictionary(dict_id_, dict_struct_, std::move(source_ptr_), dict_lifetime_, input_type_, point_type_)
{
}
std::shared_ptr<const IExternalLoadable> PolygonDictionarySimple::clone() const
{
return std::make_shared<PolygonDictionarySimple>(
this->database,
this->name,
this->getDictionaryID(),
this->dict_struct,
this->source_ptr->clone(),
this->dict_lifetime,
@ -57,8 +55,7 @@ bool PolygonDictionarySimple::find(const Point & point, size_t & id) const
}
PolygonDictionaryIndexEach::PolygonDictionaryIndexEach(
const std::string & database_,
const std::string & name_,
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
const DictionaryLifetime dict_lifetime_,
@ -66,7 +63,7 @@ PolygonDictionaryIndexEach::PolygonDictionaryIndexEach(
PointType point_type_,
int min_intersections_,
int max_depth_)
: IPolygonDictionary(database_, name_, dict_struct_, std::move(source_ptr_), dict_lifetime_, input_type_, point_type_),
: IPolygonDictionary(dict_id_, dict_struct_, std::move(source_ptr_), dict_lifetime_, input_type_, point_type_),
grid(min_intersections_, max_depth_, polygons),
min_intersections(min_intersections_),
max_depth(max_depth_)
@ -83,8 +80,7 @@ PolygonDictionaryIndexEach::PolygonDictionaryIndexEach(
std::shared_ptr<const IExternalLoadable> PolygonDictionaryIndexEach::clone() const
{
return std::make_shared<PolygonDictionaryIndexEach>(
this->database,
this->name,
this->getDictionaryID(),
this->dict_struct,
this->source_ptr->clone(),
this->dict_lifetime,
@ -118,8 +114,7 @@ bool PolygonDictionaryIndexEach::find(const Point & point, size_t & id) const
}
PolygonDictionaryIndexCell::PolygonDictionaryIndexCell(
const std::string & database_,
const std::string & name_,
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
const DictionaryLifetime dict_lifetime_,
@ -127,7 +122,7 @@ PolygonDictionaryIndexCell::PolygonDictionaryIndexCell(
PointType point_type_,
size_t min_intersections_,
size_t max_depth_)
: IPolygonDictionary(database_, name_, dict_struct_, std::move(source_ptr_), dict_lifetime_, input_type_, point_type_),
: IPolygonDictionary(dict_id_, dict_struct_, std::move(source_ptr_), dict_lifetime_, input_type_, point_type_),
index(min_intersections_, max_depth_, polygons),
min_intersections(min_intersections_),
max_depth(max_depth_)
@ -137,8 +132,7 @@ PolygonDictionaryIndexCell::PolygonDictionaryIndexCell(
std::shared_ptr<const IExternalLoadable> PolygonDictionaryIndexCell::clone() const
{
return std::make_shared<PolygonDictionaryIndexCell>(
this->database,
this->name,
this->getDictionaryID(),
this->dict_struct,
this->source_ptr->clone(),
this->dict_lifetime,
@ -228,6 +222,8 @@ DictionaryPtr createLayout(const std::string & ,
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};
const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix);
if constexpr (std::is_same_v<PolygonDictionary, PolygonDictionaryIndexEach> || std::is_same_v<PolygonDictionary, PolygonDictionaryIndexCell>)
{
const auto & layout_prefix = config_prefix + ".layout";
@ -236,10 +232,10 @@ DictionaryPtr createLayout(const std::string & ,
const auto & dict_prefix = layout_prefix + "." + keys.front();
size_t max_depth = config.getUInt(dict_prefix + ".max_depth", PolygonDictionary::kMaxDepthDefault);
size_t min_intersections = config.getUInt(dict_prefix + ".min_intersections", PolygonDictionary::kMinIntersectionsDefault);
return std::make_unique<PolygonDictionary>(database, name, dict_struct, std::move(source_ptr), dict_lifetime, input_type, point_type, min_intersections, max_depth);
return std::make_unique<PolygonDictionary>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, input_type, point_type, min_intersections, max_depth);
}
else
return std::make_unique<PolygonDictionary>(database, name, dict_struct, std::move(source_ptr), dict_lifetime, input_type, point_type);
return std::make_unique<PolygonDictionary>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, input_type, point_type);
}
void registerDictionaryPolygon(DictionaryFactory & factory)

View File

@ -10,15 +10,14 @@ namespace DB
/** Simple implementation of the polygon dictionary. Doesn't generate anything during its construction.
* Iterates over all stored polygons for each query, checking each of them in linear time.
* Retrieves the polygon with the smallest area containing the given point.
* Retrieves the polygon with the smallest area containing the given point.
* If there is more than one any such polygon may be returned.
*/
class PolygonDictionarySimple : public IPolygonDictionary
{
public:
PolygonDictionarySimple(
const std::string & database_,
const std::string & name_,
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
DictionaryLifetime dict_lifetime_,
@ -32,17 +31,16 @@ private:
};
/** A polygon dictionary which generates a recursive grid in order to efficiently cut the number
* of polygons to be checked for a given point.
* of polygons to be checked for a given point.
* For more detail see the GridRoot and FinalCell classes.
* Separately, a slab index is built for each individual polygon. This allows to check the
* candidates more efficiently.
* Separately, a slab index is built for each individual polygon. This allows to check the
* candidates more efficiently.
*/
class PolygonDictionaryIndexEach : public IPolygonDictionary
{
public:
PolygonDictionaryIndexEach(
const std::string & database_,
const std::string & name_,
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
DictionaryLifetime dict_lifetime_,
@ -71,8 +69,7 @@ class PolygonDictionaryIndexCell : public IPolygonDictionary
{
public:
PolygonDictionaryIndexCell(
const std::string & database_,
const std::string & name_,
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
DictionaryLifetime dict_lifetime_,

View File

@ -69,15 +69,12 @@ static bool operator<(const RangeHashedDictionary::Range & left, const RangeHash
RangeHashedDictionary::RangeHashedDictionary(
const std::string & database_,
const std::string & name_,
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
const DictionaryLifetime dict_lifetime_,
bool require_nonempty_)
: database(database_)
, name(name_)
, full_name{database_.empty() ? name_ : (database_ + "." + name_)}
: IDictionaryBase(dict_id_)
, dict_struct(dict_struct_)
, source_ptr{std::move(source_ptr_)}
, dict_lifetime(dict_lifetime_)
@ -160,8 +157,8 @@ void RangeHashedDictionary::createAttributes()
attributes.push_back(createAttributeWithType(attribute.underlying_type, attribute.null_value));
if (attribute.hierarchical)
throw Exception{full_name + ": hierarchical attributes not supported by " + getName() + " dictionary.",
ErrorCodes::BAD_ARGUMENTS};
throw Exception{ErrorCodes::BAD_ARGUMENTS, "Hierarchical attributes not supported by {} dictionary.",
getDictionaryID().getNameForLogs()};
}
}
@ -689,11 +686,10 @@ void registerDictionaryRangeHashed(DictionaryFactory & factory)
throw Exception{full_name + ": dictionary of layout 'range_hashed' requires .structure.range_min and .structure.range_max",
ErrorCodes::BAD_ARGUMENTS};
const String database = config.getString(config_prefix + ".database", "");
const String name = config.getString(config_prefix + ".name");
const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix);
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};
const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false);
return std::make_unique<RangeHashedDictionary>(database, name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
return std::make_unique<RangeHashedDictionary>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
};
factory.registerLayout("range_hashed", create_layout, false);
}

View File

@ -18,17 +18,12 @@ class RangeHashedDictionary final : public IDictionaryBase
{
public:
RangeHashedDictionary(
const std::string & database_,
const std::string & name_,
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
const DictionaryLifetime dict_lifetime_,
bool require_nonempty_);
const std::string & getDatabase() const override { return database; }
const std::string & getName() const override { return name; }
const std::string & getFullName() const override { return full_name; }
std::string getTypeName() const override { return "RangeHashed"; }
size_t getBytesAllocated() const override { return bytes_allocated; }
@ -43,7 +38,7 @@ public:
std::shared_ptr<const IExternalLoadable> clone() const override
{
return std::make_shared<RangeHashedDictionary>(database, name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty);
return std::make_shared<RangeHashedDictionary>(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty);
}
const IDictionarySource * getSource() const override { return source_ptr.get(); }
@ -211,9 +206,6 @@ private:
friend struct RangeHashedDIctionaryCallGetBlockInputStreamImpl;
const std::string database;
const std::string name;
const std::string full_name;
const DictionaryStructure dict_struct;
const DictionarySourcePtr source_ptr;
const DictionaryLifetime dict_lifetime;

View File

@ -1276,7 +1276,7 @@ void SSDCacheStorage::collectGarbage()
}
SSDCacheDictionary::SSDCacheDictionary(
const std::string & name_,
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
const DictionaryLifetime dict_lifetime_,
@ -1287,7 +1287,7 @@ SSDCacheDictionary::SSDCacheDictionary(
const size_t read_buffer_size_,
const size_t write_buffer_size_,
const size_t max_stored_keys_)
: name(name_)
: IDictionary(dict_id_)
, dict_struct(dict_struct_)
, source_ptr(std::move(source_ptr_))
, dict_lifetime(dict_lifetime_)
@ -1314,7 +1314,7 @@ SSDCacheDictionary::SSDCacheDictionary(
const std::string & attribute_name, const PaddedPODArray<Key> & ids, ResultArrayType<TYPE> & out) const \
{ \
const auto index = getAttributeIndex(attribute_name); \
checkAttributeType(name, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::ut##TYPE); \
checkAttributeType(this, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::ut##TYPE); \
const auto null_value = std::get<TYPE>(null_values[index]); /* NOLINT */ \
getItemsNumberImpl<TYPE, TYPE>(index, ids, out, [&](const size_t) { return null_value; }); /* NOLINT */ \
}
@ -1343,7 +1343,7 @@ SSDCacheDictionary::SSDCacheDictionary(
ResultArrayType<TYPE> & out) const \
{ \
const auto index = getAttributeIndex(attribute_name); \
checkAttributeType(name, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::ut##TYPE); \
checkAttributeType(this, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::ut##TYPE); \
getItemsNumberImpl<TYPE, TYPE>( \
index, \
ids, \
@ -1374,7 +1374,7 @@ SSDCacheDictionary::SSDCacheDictionary(
ResultArrayType<TYPE> & out) const \
{ \
const auto index = getAttributeIndex(attribute_name); \
checkAttributeType(name, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::ut##TYPE); \
checkAttributeType(this, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::ut##TYPE); \
getItemsNumberImpl<TYPE, TYPE>( \
index, \
ids, \
@ -1430,7 +1430,7 @@ void SSDCacheDictionary::getItemsNumberImpl(
void SSDCacheDictionary::getString(const std::string & attribute_name, const PaddedPODArray<Key> & ids, ColumnString * out) const
{
const auto index = getAttributeIndex(attribute_name);
checkAttributeType(name, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::utString);
checkAttributeType(this, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::utString);
const auto null_value = StringRef{std::get<String>(null_values[index])};
@ -1441,7 +1441,7 @@ void SSDCacheDictionary::getString(
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const ColumnString * const def, ColumnString * const out) const
{
const auto index = getAttributeIndex(attribute_name);
checkAttributeType(name, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::utString);
checkAttributeType(this, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::utString);
getItemsStringImpl(index, ids, out, [&](const size_t row) { return def->getDataAt(row); });
}
@ -1450,7 +1450,7 @@ void SSDCacheDictionary::getString(
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const String & def, ColumnString * const out) const
{
const auto index = getAttributeIndex(attribute_name);
checkAttributeType(name, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::utString);
checkAttributeType(this, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::utString);
getItemsStringImpl(index, ids, out, [&](const size_t) { return StringRef{def}; });
}
@ -1640,6 +1640,8 @@ void registerDictionarySSDCache(DictionaryFactory & factory)
if (dict_struct.key)
throw Exception{"'key' is not supported for dictionary of layout 'cache'", ErrorCodes::UNSUPPORTED_METHOD};
const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix);
if (dict_struct.range_min || dict_struct.range_max)
throw Exception{name
+ ": elements .structure.range_min and .structure.range_max should be defined only "
@ -1686,7 +1688,7 @@ void registerDictionarySSDCache(DictionaryFactory & factory)
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};
return std::make_unique<SSDCacheDictionary>(
name, dict_struct, std::move(source_ptr), dict_lifetime, path,
dict_id, dict_struct, std::move(source_ptr), dict_lifetime, path,
max_partitions_count, file_size / block_size, block_size,
read_buffer_size / block_size, write_buffer_size / block_size,
max_stored_keys);

View File

@ -300,7 +300,7 @@ class SSDCacheDictionary final : public IDictionary
{
public:
SSDCacheDictionary(
const std::string & name_,
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
DictionaryLifetime dict_lifetime_,
@ -312,10 +312,6 @@ public:
size_t write_buffer_size_,
size_t max_stored_keys_);
const std::string & getDatabase() const override { return name; }
const std::string & getName() const override { return name; }
const std::string & getFullName() const override { return getName(); }
std::string getTypeName() const override { return "SSDCache"; }
size_t getBytesAllocated() const override { return storage.getBytesAllocated(); }
@ -335,8 +331,8 @@ public:
std::shared_ptr<const IExternalLoadable> clone() const override
{
return std::make_shared<SSDCacheDictionary>(name, dict_struct, source_ptr->clone(), dict_lifetime, path,
max_partitions_count, file_size, block_size, read_buffer_size, write_buffer_size, max_stored_keys);
return std::make_shared<SSDCacheDictionary>(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime,
path, max_partitions_count, file_size, block_size, read_buffer_size, write_buffer_size, max_stored_keys);
}
const IDictionarySource * getSource() const override { return source_ptr.get(); }

View File

@ -1323,7 +1323,7 @@ void SSDComplexKeyCacheStorage::collectGarbage()
}
SSDComplexKeyCacheDictionary::SSDComplexKeyCacheDictionary(
const std::string & name_,
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
const DictionaryLifetime dict_lifetime_,
@ -1334,7 +1334,7 @@ SSDComplexKeyCacheDictionary::SSDComplexKeyCacheDictionary(
const size_t read_buffer_size_,
const size_t write_buffer_size_,
const size_t max_stored_keys_)
: name(name_)
: IDictionaryBase(dict_id_)
, dict_struct(dict_struct_)
, source_ptr(std::move(source_ptr_))
, dict_lifetime(dict_lifetime_)
@ -1364,7 +1364,7 @@ SSDComplexKeyCacheDictionary::SSDComplexKeyCacheDictionary(
ResultArrayType<TYPE> & out) const \
{ \
const auto index = getAttributeIndex(attribute_name); \
checkAttributeType(name, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::ut##TYPE); \
checkAttributeType(this, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::ut##TYPE); \
const auto null_value = std::get<TYPE>(null_values[index]); /* NOLINT */ \
getItemsNumberImpl<TYPE, TYPE>(index, key_columns, key_types, out, [&](const size_t) { return null_value; }); /* NOLINT */ \
}
@ -1394,7 +1394,7 @@ SSDComplexKeyCacheDictionary::SSDComplexKeyCacheDictionary(
ResultArrayType<TYPE> & out) const \
{ \
const auto index = getAttributeIndex(attribute_name); \
checkAttributeType(name, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::ut##TYPE); \
checkAttributeType(this, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::ut##TYPE); \
getItemsNumberImpl<TYPE, TYPE>(index, key_columns, key_types, out, [&](const size_t row) { return def[row]; }); /* NOLINT */ \
}
DECLARE(UInt8)
@ -1422,7 +1422,7 @@ SSDComplexKeyCacheDictionary::SSDComplexKeyCacheDictionary(
ResultArrayType<TYPE> & out) const \
{ \
const auto index = getAttributeIndex(attribute_name); \
checkAttributeType(name, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::ut##TYPE); \
checkAttributeType(this, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::ut##TYPE); \
getItemsNumberImpl<TYPE, TYPE>(index, key_columns, key_types, out, [&](const size_t) { return def; }); /* NOLINT */ \
}
DECLARE(UInt8)
@ -1488,7 +1488,7 @@ void SSDComplexKeyCacheDictionary::getString(
const Columns & key_columns, const DataTypes & key_types, ColumnString * out) const
{
const auto index = getAttributeIndex(attribute_name);
checkAttributeType(name, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::utString);
checkAttributeType(this, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::utString);
const auto null_value = StringRef{std::get<String>(null_values[index])};
@ -1501,7 +1501,7 @@ void SSDComplexKeyCacheDictionary::getString(
const ColumnString * const def, ColumnString * const out) const
{
const auto index = getAttributeIndex(attribute_name);
checkAttributeType(name, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::utString);
checkAttributeType(this, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::utString);
getItemsStringImpl(index, key_columns, key_types, out, [&](const size_t row) { return def->getDataAt(row); });
}
@ -1514,7 +1514,7 @@ void SSDComplexKeyCacheDictionary::getString(
ColumnString * const out) const
{
const auto index = getAttributeIndex(attribute_name);
checkAttributeType(name, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::utString);
checkAttributeType(this, attribute_name, dict_struct.attributes[index].underlying_type, AttributeUnderlyingType::utString);
getItemsStringImpl(index, key_columns, key_types, out, [&](const size_t) { return StringRef{def}; });
}
@ -1736,6 +1736,8 @@ void registerDictionarySSDComplexKeyCache(DictionaryFactory & factory)
const std::string & config_prefix,
DictionarySourcePtr source_ptr) -> DictionaryPtr
{
const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix);
if (dict_struct.id)
throw Exception{"'id' is not supported for dictionary of layout 'complex_key_cache'", ErrorCodes::UNSUPPORTED_METHOD};
@ -1785,7 +1787,7 @@ void registerDictionarySSDComplexKeyCache(DictionaryFactory & factory)
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};
return std::make_unique<SSDComplexKeyCacheDictionary>(
name, dict_struct, std::move(source_ptr), dict_lifetime, path,
dict_id, dict_struct, std::move(source_ptr), dict_lifetime, path,
max_partitions_count, file_size / block_size, block_size,
read_buffer_size / block_size, write_buffer_size / block_size,
max_stored_keys);

View File

@ -522,7 +522,7 @@ class SSDComplexKeyCacheDictionary final : public IDictionaryBase
{
public:
SSDComplexKeyCacheDictionary(
const std::string & name_,
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
const DictionaryLifetime dict_lifetime_,
@ -534,10 +534,6 @@ public:
const size_t write_buffer_size_,
const size_t max_stored_keys_);
const std::string & getDatabase() const override { return name; }
const std::string & getName() const override { return name; }
const std::string & getFullName() const override { return getName(); }
std::string getKeyDescription() const { return dict_struct.getKeyDescription(); }
std::string getTypeName() const override { return "SSDComplexKeyCache"; }
@ -559,7 +555,7 @@ public:
std::shared_ptr<const IExternalLoadable> clone() const override
{
return std::make_shared<SSDComplexKeyCacheDictionary>(name, dict_struct, source_ptr->clone(), dict_lifetime, path,
return std::make_shared<SSDComplexKeyCacheDictionary>(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, path,
max_partitions_count, file_size, block_size, read_buffer_size, write_buffer_size, max_stored_keys);
}

View File

@ -47,15 +47,12 @@ static void validateKeyTypes(const DataTypes & key_types)
TrieDictionary::TrieDictionary(
const std::string & database_,
const std::string & name_,
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
const DictionaryLifetime dict_lifetime_,
bool require_nonempty_)
: database(database_)
, name(name_)
, full_name{database_.empty() ? name_ : (database_ + "." + name_)}
: IDictionaryBase(dict_id_)
, dict_struct(dict_struct_)
, source_ptr{std::move(source_ptr_)}
, dict_lifetime(dict_lifetime_)
@ -80,7 +77,7 @@ TrieDictionary::~TrieDictionary()
validateKeyTypes(key_types); \
\
const auto & attribute = getAttribute(attribute_name); \
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
\
const auto null_value = std::get<TYPE>(attribute.null_values); \
\
@ -112,7 +109,7 @@ void TrieDictionary::getString(
validateKeyTypes(key_types);
const auto & attribute = getAttribute(attribute_name);
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString);
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString);
const auto & null_value = StringRef{std::get<String>(attribute.null_values)};
@ -134,7 +131,7 @@ void TrieDictionary::getString(
validateKeyTypes(key_types); \
\
const auto & attribute = getAttribute(attribute_name); \
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
\
getItemsImpl<TYPE, TYPE>( \
attribute, \
@ -168,7 +165,7 @@ void TrieDictionary::getString(
validateKeyTypes(key_types);
const auto & attribute = getAttribute(attribute_name);
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString);
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString);
getItemsImpl<StringRef, StringRef>(
attribute,
@ -188,7 +185,7 @@ void TrieDictionary::getString(
validateKeyTypes(key_types); \
\
const auto & attribute = getAttribute(attribute_name); \
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::ut##TYPE); \
\
getItemsImpl<TYPE, TYPE>( \
attribute, key_columns, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t) { return def; }); \
@ -219,7 +216,7 @@ void TrieDictionary::getString(
validateKeyTypes(key_types);
const auto & attribute = getAttribute(attribute_name);
checkAttributeType(full_name, attribute_name, attribute.type, AttributeUnderlyingType::utString);
checkAttributeType(this, attribute_name, attribute.type, AttributeUnderlyingType::utString);
getItemsImpl<StringRef, StringRef>(
attribute,
@ -770,12 +767,11 @@ void registerDictionaryTrie(DictionaryFactory & factory)
if (!dict_struct.key)
throw Exception{"'key' is required for dictionary of layout 'ip_trie'", ErrorCodes::BAD_ARGUMENTS};
const String database = config.getString(config_prefix + ".database", "");
const String name = config.getString(config_prefix + ".name");
const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix);
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};
const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false);
// This is specialised trie for storing IPv4 and IPv6 prefixes.
return std::make_unique<TrieDictionary>(database, name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
return std::make_unique<TrieDictionary>(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
};
factory.registerLayout("ip_trie", create_layout, true);
}

View File

@ -23,8 +23,7 @@ class TrieDictionary final : public IDictionaryBase
{
public:
TrieDictionary(
const std::string & database_,
const std::string & name_,
const StorageID & dict_id_,
const DictionaryStructure & dict_struct_,
DictionarySourcePtr source_ptr_,
const DictionaryLifetime dict_lifetime_,
@ -34,10 +33,6 @@ public:
std::string getKeyDescription() const { return key_description; }
const std::string & getDatabase() const override { return database; }
const std::string & getName() const override { return name; }
const std::string & getFullName() const override { return full_name; }
std::string getTypeName() const override { return "Trie"; }
size_t getBytesAllocated() const override { return bytes_allocated; }
@ -52,7 +47,7 @@ public:
std::shared_ptr<const IExternalLoadable> clone() const override
{
return std::make_shared<TrieDictionary>(database, name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty);
return std::make_shared<TrieDictionary>(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty);
}
const IDictionarySource * getSource() const override { return source_ptr.get(); }
@ -230,9 +225,6 @@ private:
Columns getKeyColumns() const;
const std::string database;
const std::string name;
const std::string full_name;
const DictionaryStructure dict_struct;
const DictionarySourcePtr source_ptr;
const DictionaryLifetime dict_lifetime;

View File

@ -469,6 +469,14 @@ DictionaryConfigurationPtr getDictionaryConfigurationFromAST(const ASTCreateQuer
AutoPtr<Text> database(xml_document->createTextNode(!database_.empty() ? database_ : query.database));
database_element->appendChild(database);
if (query.uuid != UUIDHelpers::Nil)
{
AutoPtr<Poco::XML::Element> uuid_element(xml_document->createElement("uuid"));
current_dictionary->appendChild(uuid_element);
AutoPtr<Text> uuid(xml_document->createTextNode(toString(query.uuid)));
uuid_element->appendChild(uuid);
}
AutoPtr<Element> structure_element(xml_document->createElement("structure"));
current_dictionary->appendChild(structure_element);
Names pk_attrs = getPrimaryKeyColumns(query.dictionary->primary_key);

View File

@ -85,8 +85,9 @@ public:
auto dict = std::atomic_load(&dictionary);
if (dict)
return dict;
dict = external_loader.getDictionary(dictionary_name);
context.checkAccess(AccessType::dictGet, dict->getDatabaseOrNoDatabaseTag(), dict->getName());
String resolved_name = DatabaseCatalog::instance().resolveDictionaryName(dictionary_name);
dict = external_loader.getDictionary(resolved_name);
context.checkAccess(AccessType::dictGet, dict->getDatabaseOrNoDatabaseTag(), dict->getDictionaryID().getTableName());
std::atomic_store(&dictionary, dict);
return dict;
}

View File

@ -5,17 +5,26 @@
#include <Functions/FunctionFactory.h>
#include <Functions/IFunction.h>
#include <Common/typeid_cast.h>
#include <IO/WriteHelpers.h>
#include <ext/range.h>
#include <constants.h>
#include <h3api.h>
static constexpr size_t MAX_ARRAY_SIZE = 1 << 30;
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int TOO_LARGE_ARRAY_SIZE;
}
class FunctionH3ToChildren : public IFunction
{
public:
@ -63,7 +72,16 @@ public:
const UInt64 parent_hindex = col_hindex->getUInt(row);
const UInt8 child_resolution = col_resolution->getUInt(row);
const auto vec_size = maxH3ToChildrenSize(parent_hindex, child_resolution);
if (child_resolution > MAX_H3_RES)
throw Exception("The argument 'resolution' (" + toString(child_resolution) + ") of function " + getName()
+ " is out of bounds because the maximum resolution in H3 library is " + toString(MAX_H3_RES), ErrorCodes::ARGUMENT_OUT_OF_BOUND);
const size_t vec_size = maxH3ToChildrenSize(parent_hindex, child_resolution);
if (vec_size > MAX_ARRAY_SIZE)
throw Exception("The result of function" + getName()
+ " (array of " + toString(vec_size) + " elements) will be too large with resolution argument = "
+ toString(child_resolution), ErrorCodes::TOO_LARGE_ARRAY_SIZE);
hindex_vec.resize(vec_size);
h3ToChildren(parent_hindex, child_resolution, hindex_vec.data());

View File

@ -3,17 +3,22 @@
#include <Functions/FunctionFactory.h>
#include <Functions/IFunction.h>
#include <Common/typeid_cast.h>
#include <IO/WriteHelpers.h>
#include <ext/range.h>
#include <constants.h>
#include <h3api.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ARGUMENT_OUT_OF_BOUND;
}
class FunctionH3ToParent : public IFunction
{
public:
@ -57,6 +62,10 @@ public:
const UInt64 hindex = col_hindex->getUInt(row);
const UInt8 resolution = col_resolution->getUInt(row);
if (resolution > MAX_H3_RES)
throw Exception("The argument 'resolution' (" + toString(resolution) + ") of function " + getName()
+ " is out of bounds because the maximum resolution in H3 library is " + toString(MAX_H3_RES), ErrorCodes::ARGUMENT_OUT_OF_BOUND);
UInt64 res = h3ToParent(hindex, resolution);
dst_data[row] = res;

View File

@ -285,7 +285,7 @@ void SelectStreamFactory::createForShard(
}
};
res.emplace_back(createDelayedPipe(header, lazily_create_stream));
res.emplace_back(createDelayedPipe(header, lazily_create_stream, add_totals, add_extremes));
}
else
emplace_remote_stream();

View File

@ -12,6 +12,7 @@
#include <Parsers/formatAST.h>
#include <IO/ReadHelpers.h>
#include <Poco/DirectoryIterator.h>
#include <Common/renameat2.h>
#include <filesystem>
@ -144,9 +145,12 @@ void DatabaseCatalog::shutdownImpl()
for (auto & database : current_databases)
database.second->shutdown();
tables_marked_dropped.clear();
std::lock_guard lock(databases_mutex);
assert(std::find_if_not(uuid_map.begin(), uuid_map.end(), [](const auto & elem) { return elem.map.empty(); }) == uuid_map.end());
databases.clear();
db_uuid_map.clear();
view_dependencies.clear();
}
@ -215,6 +219,8 @@ DatabaseAndTable DatabaseCatalog::getTableImpl(
auto table = database->tryGetTable(table_id.table_name, context);
if (!table && exception)
exception->emplace("Table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
if (!table)
database = nullptr;
return {database, table};
}
@ -250,7 +256,11 @@ void DatabaseCatalog::attachDatabase(const String & database_name, const Databas
{
std::lock_guard lock{databases_mutex};
assertDatabaseDoesntExistUnlocked(database_name);
databases[database_name] = database;
databases.emplace(database_name, database);
UUID db_uuid = database->getUUID();
assert((db_uuid != UUIDHelpers::Nil) ^ (dynamic_cast<DatabaseAtomic *>(database.get()) == nullptr));
if (db_uuid != UUIDHelpers::Nil)
db_uuid_map.emplace(db_uuid, database);
}
@ -259,13 +269,18 @@ DatabasePtr DatabaseCatalog::detachDatabase(const String & database_name, bool d
if (database_name == TEMPORARY_DATABASE)
throw Exception("Cannot detach database with temporary tables.", ErrorCodes::DATABASE_ACCESS_DENIED);
std::shared_ptr<IDatabase> db;
DatabasePtr db;
{
std::lock_guard lock{databases_mutex};
assertDatabaseExistsUnlocked(database_name);
db = databases.find(database_name)->second;
db_uuid_map.erase(db->getUUID());
databases.erase(database_name);
}
if (check_empty)
if (check_empty)
{
try
{
if (!db->empty())
throw Exception("New table appeared in database being dropped or detached. Try again.",
@ -274,8 +289,11 @@ DatabasePtr DatabaseCatalog::detachDatabase(const String & database_name, bool d
if (!drop && database_atomic)
database_atomic->assertCanBeDetached(false);
}
databases.erase(database_name);
catch (...)
{
attachDatabase(database_name, db);
throw;
}
}
db->shutdown();
@ -295,6 +313,17 @@ DatabasePtr DatabaseCatalog::detachDatabase(const String & database_name, bool d
return db;
}
void DatabaseCatalog::updateDatabaseName(const String & old_name, const String & new_name)
{
std::lock_guard lock{databases_mutex};
assert(databases.find(new_name) == databases.end());
auto it = databases.find(old_name);
assert(it != databases.end());
auto db = it->second;
databases.erase(it);
databases.emplace(new_name, db);
}
DatabasePtr DatabaseCatalog::getDatabase(const String & database_name) const
{
std::lock_guard lock{databases_mutex};
@ -312,6 +341,25 @@ DatabasePtr DatabaseCatalog::tryGetDatabase(const String & database_name) const
return it->second;
}
DatabasePtr DatabaseCatalog::getDatabase(const UUID & uuid) const
{
std::lock_guard lock{databases_mutex};
auto it = db_uuid_map.find(uuid);
if (it == db_uuid_map.end())
throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database UUID {} does not exist", toString(uuid));
return it->second;
}
DatabasePtr DatabaseCatalog::tryGetDatabase(const UUID & uuid) const
{
assert(uuid != UUIDHelpers::Nil);
std::lock_guard lock{databases_mutex};
auto it = db_uuid_map.find(uuid);
if (it == db_uuid_map.end())
return {};
return it->second;
}
bool DatabaseCatalog::isDatabaseExist(const String & database_name) const
{
assert(!database_name.empty());
@ -717,6 +765,31 @@ String DatabaseCatalog::getPathForUUID(const UUID & uuid)
return toString(uuid).substr(0, uuid_prefix_len) + '/' + toString(uuid) + '/';
}
String DatabaseCatalog::resolveDictionaryName(const String & name) const
{
/// If it's dictionary from Atomic database, then we need to convert qualified name to UUID.
/// Try to split name and get id from associated StorageDictionary.
/// If something went wrong, return name as is.
/// TODO support dot in name for dictionaries in Atomic databases
auto pos = name.find('.');
if (pos == std::string::npos || name.find('.', pos + 1) != std::string::npos)
return name;
String maybe_database_name = name.substr(0, pos);
String maybe_table_name = name.substr(pos + 1);
auto db_and_table = tryGetDatabaseAndTable({maybe_database_name, maybe_table_name}, *global_context);
if (!db_and_table.first)
return name;
assert(db_and_table.second);
if (db_and_table.first->getUUID() == UUIDHelpers::Nil)
return name;
if (db_and_table.second->getName() != "Dictionary")
return name;
return toString(db_and_table.second->getStorageID().uuid);
}
DDLGuard::DDLGuard(Map & map_, std::unique_lock<std::mutex> guards_lock_, const String & elem)
: map(map_), guards_lock(std::move(guards_lock_))

View File

@ -123,10 +123,13 @@ public:
void attachDatabase(const String & database_name, const DatabasePtr & database);
DatabasePtr detachDatabase(const String & database_name, bool drop = false, bool check_empty = true);
void updateDatabaseName(const String & old_name, const String & new_name);
/// database_name must be not empty
DatabasePtr getDatabase(const String & database_name) const;
DatabasePtr tryGetDatabase(const String & database_name) const;
DatabasePtr getDatabase(const UUID & uuid) const;
DatabasePtr tryGetDatabase(const UUID & uuid) const;
bool isDatabaseExist(const String & database_name) const;
Databases getDatabases() const;
@ -168,6 +171,9 @@ public:
String getPathForDroppedMetadata(const StorageID & table_id) const;
void enqueueDroppedTableCleanup(StorageID table_id, StoragePtr table, String dropped_metadata_path, bool ignore_delay = false);
/// Try convert qualified dictionary name to persistent UUID
String resolveDictionaryName(const String & name) const;
private:
// The global instance of database catalog. unique_ptr is to allow
// deferred initialization. Thought I'd use std::optional, but I can't
@ -211,6 +217,8 @@ private:
static constexpr size_t reschedule_time_ms = 100;
private:
using UUIDToDatabaseMap = std::unordered_map<UUID, DatabasePtr>;
/// For some reason Context is required to get Storage from Database object
Context * global_context;
mutable std::mutex databases_mutex;
@ -218,6 +226,7 @@ private:
ViewDependencies view_dependencies;
Databases databases;
UUIDToDatabaseMap db_uuid_map;
UUIDToStorageMap uuid_map;
Poco::Logger * log;

View File

@ -18,7 +18,7 @@ ExternalDictionariesLoader::ExternalDictionariesLoader(Context & context_)
: ExternalLoader("external dictionary", &Poco::Logger::get("ExternalDictionariesLoader"))
, context(context_)
{
setConfigSettings({"dictionary", "name", "database"});
setConfigSettings({"dictionary", "name", "database", "uuid"});
enableAsyncLoading(true);
enablePeriodicUpdates(true);
}

View File

@ -290,18 +290,27 @@ private:
continue;
}
String object_name = file_contents.getString(key + "." + settings.external_name);
/// Use uuid as name if possible
String object_uuid = file_contents.getString(key + "." + settings.external_uuid, "");
String object_name;
if (object_uuid.empty())
object_name = file_contents.getString(key + "." + settings.external_name);
else
object_name = object_uuid;
if (object_name.empty())
{
LOG_WARNING(log, "{}: node '{}' defines {} with an empty name. It's not allowed", path, key, type_name);
continue;
}
String database;
if (!settings.external_database.empty())
database = file_contents.getString(key + "." + settings.external_database, "");
if (!database.empty())
object_name = database + "." + object_name;
if (object_uuid.empty())
{
String database;
if (!settings.external_database.empty())
database = file_contents.getString(key + "." + settings.external_database, "");
if (!database.empty())
object_name = database + "." + object_name;
}
objects.emplace(object_name, key);
}

View File

@ -27,6 +27,7 @@ struct ExternalLoaderConfigSettings
std::string external_config;
std::string external_name;
std::string external_database;
std::string external_uuid;
};
/** Interface for manage user-defined objects.

View File

@ -1,5 +1,9 @@
#include <Interpreters/ExternalLoaderDatabaseConfigRepository.h>
#include <Interpreters/Context.h>
#include <Storages/IStorage.h>
#include <Common/StringUtils/StringUtils.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
namespace DB
@ -12,11 +16,25 @@ namespace ErrorCodes
namespace
{
String trimDatabaseName(const std::string & loadable_definition_name, const String & database_name)
String trimDatabaseName(const std::string & loadable_definition_name, const String & database_name,
const IDatabase & database, const Context & global_context)
{
bool is_atomic_database = database.getUUID() != UUIDHelpers::Nil;
if (is_atomic_database)
{
/// We do not know actual database and dictionary names here
auto dict_id = StorageID::createEmpty();
dict_id.uuid = parseFromString<UUID>(loadable_definition_name);
assert(dict_id.uuid != UUIDHelpers::Nil);
/// Get associated StorageDictionary by UUID
auto table = DatabaseCatalog::instance().getTable(dict_id, global_context);
auto dict_id_with_names = table->getStorageID();
return dict_id_with_names.table_name;
}
if (!startsWith(loadable_definition_name, database_name))
throw Exception(
"Loadable '" + loadable_definition_name + "' is not from database '" + database_name, ErrorCodes::UNKNOWN_DICTIONARY);
throw Exception(ErrorCodes::UNKNOWN_DICTIONARY,
"Loadable '{}' is not from database '{}'", loadable_definition_name, database_name);
/// dbname.loadable_name
///--> remove <---
return loadable_definition_name.substr(database_name.length() + 1);
@ -24,34 +42,45 @@ namespace
}
ExternalLoaderDatabaseConfigRepository::ExternalLoaderDatabaseConfigRepository(IDatabase & database_)
: database_name(database_.getDatabaseName())
ExternalLoaderDatabaseConfigRepository::ExternalLoaderDatabaseConfigRepository(IDatabase & database_, const Context & global_context_)
: global_context(global_context_.getGlobalContext())
, database_name(database_.getDatabaseName())
, database(database_)
{
}
LoadablesConfigurationPtr ExternalLoaderDatabaseConfigRepository::load(const std::string & loadable_definition_name)
{
return database.getDictionaryConfiguration(trimDatabaseName(loadable_definition_name, database_name));
auto dict_name = trimDatabaseName(loadable_definition_name, database_name, database, global_context);
return database.getDictionaryConfiguration(dict_name);
}
bool ExternalLoaderDatabaseConfigRepository::exists(const std::string & loadable_definition_name)
{
return database.isDictionaryExist(trimDatabaseName(loadable_definition_name, database_name));
auto dict_name = trimDatabaseName(loadable_definition_name, database_name, database, global_context);
return database.isDictionaryExist(dict_name);
}
Poco::Timestamp ExternalLoaderDatabaseConfigRepository::getUpdateTime(const std::string & loadable_definition_name)
{
return database.getObjectMetadataModificationTime(trimDatabaseName(loadable_definition_name, database_name));
auto dict_name = trimDatabaseName(loadable_definition_name, database_name, database, global_context);
return database.getObjectMetadataModificationTime(dict_name);
}
std::set<std::string> ExternalLoaderDatabaseConfigRepository::getAllLoadablesDefinitionNames()
{
std::set<std::string> result;
auto itr = database.getDictionariesIterator();
bool is_atomic_database = database.getUUID() != UUIDHelpers::Nil;
while (itr && itr->isValid())
{
result.insert(database_name + "." + itr->name());
if (is_atomic_database)
{
assert(itr->uuid() != UUIDHelpers::Nil);
result.insert(toString(itr->uuid()));
}
else
result.insert(database_name + "." + itr->name());
itr->next();
}
return result;

View File

@ -12,7 +12,7 @@ namespace DB
class ExternalLoaderDatabaseConfigRepository : public IExternalLoaderConfigRepository
{
public:
ExternalLoaderDatabaseConfigRepository(IDatabase & database_);
ExternalLoaderDatabaseConfigRepository(IDatabase & database_, const Context & global_context_);
const std::string & getName() const override { return database_name; }
@ -25,6 +25,7 @@ public:
LoadablesConfigurationPtr load(const std::string & loadable_definition_name) override;
private:
const Context & global_context;
const String database_name;
IDatabase & database;
};

View File

@ -14,7 +14,7 @@ ExternalModelsLoader::ExternalModelsLoader(Context & context_)
: ExternalLoader("external model", &Poco::Logger::get("ExternalModelsLoader"))
, context(context_)
{
setConfigSettings({"model", "name", {}});
setConfigSettings({"model", "name", {}, {}});
enablePeriodicUpdates(true);
}

View File

@ -33,7 +33,6 @@ namespace DB
namespace ErrorCodes
{
extern const int BAD_TYPE_OF_FIELD;
extern const int NOT_IMPLEMENTED;
extern const int NO_SUCH_COLUMN_IN_TABLE;
extern const int INCOMPATIBLE_TYPE_OF_JOIN;
@ -158,25 +157,21 @@ HashJoin::HashJoin(std::shared_ptr<TableJoin> table_join_, const Block & right_s
}
else if (strictness == ASTTableJoin::Strictness::Asof)
{
if (kind != ASTTableJoin::Kind::Left and kind != ASTTableJoin::Kind::Inner)
throw Exception("ASOF only supports LEFT and INNER as base joins", ErrorCodes::NOT_IMPLEMENTED);
/// @note ASOF JOIN is not INNER. It's better avoid use of 'INNER ASOF' combination in messages.
/// In fact INNER means 'LEFT SEMI ASOF' while LEFT means 'LEFT OUTER ASOF'.
if (!isLeft(kind) && !isInner(kind))
throw Exception("Wrong ASOF JOIN type. Only ASOF and LEFT ASOF joins are supported", ErrorCodes::NOT_IMPLEMENTED);
if (key_columns.size() <= 1)
throw Exception("ASOF join needs at least one equi-join column", ErrorCodes::SYNTAX_ERROR);
if (right_table_keys.getByName(key_names_right.back()).type->isNullable())
throw Exception("ASOF join over right table Nullable column is not implemented", ErrorCodes::NOT_IMPLEMENTED);
const IColumn * asof_column = key_columns.back();
size_t asof_size;
asof_type = AsofRowRefs::getTypeSize(asof_column, asof_size);
if (!asof_type)
{
std::string msg = "ASOF join not supported for type: ";
msg += asof_column->getFamilyName();
throw Exception(msg, ErrorCodes::BAD_TYPE_OF_FIELD);
}
asof_type = AsofRowRefs::getTypeSize(*key_columns.back(), asof_size);
key_columns.pop_back();
if (key_columns.empty())
throw Exception("ASOF join cannot be done without a joining column", ErrorCodes::SYNTAX_ERROR);
/// this is going to set up the appropriate hash table for the direct lookup part of the join
/// However, this does not depend on the size of the asof join key (as that goes into the BST)
/// Therefore, add it back in such that it can be extracted appropriately from the full stored
@ -248,11 +243,6 @@ HashJoin::Type HashJoin::chooseMethod(const ColumnRawPtrs & key_columns, Sizes &
return Type::hashed;
}
static const IColumn * extractAsofColumn(const ColumnRawPtrs & key_columns)
{
return key_columns.back();
}
template<typename KeyGetter, bool is_asof_join>
static KeyGetter createKeyGetter(const ColumnRawPtrs & key_columns, const Sizes & key_sizes)
{
@ -428,14 +418,15 @@ namespace
}
static ALWAYS_INLINE void insertAsof(HashJoin & join, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool,
const IColumn * asof_column)
const IColumn & asof_column)
{
auto emplace_result = key_getter.emplaceKey(map, i, pool);
typename Map::mapped_type * time_series_map = &emplace_result.getMapped();
TypeIndex asof_type = *join.getAsofType();
if (emplace_result.isInserted())
time_series_map = new (time_series_map) typename Map::mapped_type(join.getAsofType());
time_series_map->insert(join.getAsofType(), asof_column, stored_block, i);
time_series_map = new (time_series_map) typename Map::mapped_type(asof_type);
time_series_map->insert(asof_type, asof_column, stored_block, i);
}
};
@ -451,7 +442,7 @@ namespace
const IColumn * asof_column [[maybe_unused]] = nullptr;
if constexpr (is_asof_join)
asof_column = extractAsofColumn(key_columns);
asof_column = key_columns.back();
auto key_getter = createKeyGetter<KeyGetter, is_asof_join>(key_columns, key_sizes);
@ -461,7 +452,7 @@ namespace
continue;
if constexpr (is_asof_join)
Inserter<Map, KeyGetter>::insertAsof(join, map, key_getter, stored_block, i, pool, asof_column);
Inserter<Map, KeyGetter>::insertAsof(join, map, key_getter, stored_block, i, pool, *asof_column);
else if constexpr (mapped_one)
Inserter<Map, KeyGetter>::insertOne(join, map, key_getter, stored_block, i, pool);
else
@ -614,21 +605,22 @@ class AddedColumns
public:
using TypeAndNames = std::vector<std::pair<decltype(ColumnWithTypeAndName::type), decltype(ColumnWithTypeAndName::name)>>;
AddedColumns(const Block & sample_block_with_columns_to_add,
const Block & block_with_columns_to_add,
AddedColumns(const Block & block_with_columns_to_add,
const Block & block,
const Block & saved_block_sample,
const ColumnsWithTypeAndName & extras,
const HashJoin & join_,
const HashJoin & join,
const ColumnRawPtrs & key_columns_,
const Sizes & key_sizes_)
: join(join_)
, key_columns(key_columns_)
const Sizes & key_sizes_,
bool is_asof_join)
: key_columns(key_columns_)
, key_sizes(key_sizes_)
, rows_to_add(block.rows())
, need_filter(false)
, asof_type(join.getAsofType())
, asof_inequality(join.getAsofInequality())
{
size_t num_columns_to_add = sample_block_with_columns_to_add.columns();
size_t num_columns_to_add = block_with_columns_to_add.columns();
if (is_asof_join)
++num_columns_to_add;
columns.reserve(num_columns_to_add);
type_name.reserve(num_columns_to_add);
@ -641,8 +633,12 @@ public:
addColumn(src_column);
}
for (const auto & extra : extras)
addColumn(extra);
if (is_asof_join)
{
const ColumnWithTypeAndName & right_asof_column = join.rightAsofKeyColumn();
addColumn(right_asof_column);
left_asof_key = key_columns.back();
}
for (auto & tn : type_name)
right_indexes.push_back(saved_block_sample.getPositionByName(tn.second));
@ -680,18 +676,25 @@ public:
}
}
const HashJoin & join;
TypeIndex asofType() const { return *asof_type; }
ASOF::Inequality asofInequality() const { return asof_inequality; }
const IColumn & leftAsofKey() const { return *left_asof_key; }
const ColumnRawPtrs & key_columns;
const Sizes & key_sizes;
size_t rows_to_add;
std::unique_ptr<IColumn::Offsets> offsets_to_replicate;
bool need_filter;
bool need_filter = false;
private:
TypeAndNames type_name;
MutableColumns columns;
std::vector<size_t> right_indexes;
size_t lazy_defaults_count = 0;
/// for ASOF
std::optional<TypeIndex> asof_type;
ASOF::Inequality asof_inequality;
const IColumn * left_asof_key = nullptr;
void addColumn(const ColumnWithTypeAndName & src_column)
{
@ -760,10 +763,6 @@ NO_INLINE IColumn::Filter joinRightColumns(const Map & map, AddedColumns & added
if constexpr (need_replication)
added_columns.offsets_to_replicate = std::make_unique<IColumn::Offsets>(rows);
const IColumn * asof_column [[maybe_unused]] = nullptr;
if constexpr (is_asof_join)
asof_column = extractAsofColumn(added_columns.key_columns);
auto key_getter = createKeyGetter<KeyGetter, is_asof_join>(added_columns.key_columns, added_columns.key_sizes);
IColumn::Offset current_offset = 0;
@ -790,8 +789,11 @@ NO_INLINE IColumn::Filter joinRightColumns(const Map & map, AddedColumns & added
if constexpr (is_asof_join)
{
const HashJoin & join = added_columns.join;
if (const RowRef * found = mapped.findAsof(join.getAsofType(), join.getAsofInequality(), asof_column, i))
TypeIndex asof_type = added_columns.asofType();
ASOF::Inequality asof_inequality = added_columns.asofInequality();
const IColumn & left_asof_key = added_columns.leftAsofKey();
if (const RowRef * found = mapped.findAsof(asof_type, asof_inequality, left_asof_key, i))
{
setUsed<need_filter>(filter, i);
mapped.setUsed();
@ -932,11 +934,11 @@ void HashJoin::joinBlockImpl(
/// Rare case, when keys are constant or low cardinality. To avoid code bloat, simply materialize them.
Columns materialized_keys = JoinCommon::materializeColumns(block, key_names_left);
ColumnRawPtrs key_columns = JoinCommon::getRawPointers(materialized_keys);
ColumnRawPtrs left_key_columns = JoinCommon::getRawPointers(materialized_keys);
/// Keys with NULL value in any column won't join to anything.
ConstNullMapPtr null_map{};
ColumnPtr null_map_holder = extractNestedColumnsAndNullMap(key_columns, null_map);
ColumnPtr null_map_holder = extractNestedColumnsAndNullMap(left_key_columns, null_map);
size_t existing_columns = block.columns();
@ -957,12 +959,8 @@ void HashJoin::joinBlockImpl(
* but they will not be used at this stage of joining (and will be in `AdderNonJoined`), and they need to be skipped.
* For ASOF, the last column is used as the ASOF column
*/
ColumnsWithTypeAndName extras;
if constexpr (is_asof_join)
extras.push_back(right_table_keys.getByName(key_names_right.back()));
AddedColumns added_columns(sample_block_with_columns_to_add, block_with_columns_to_add, block, savedBlockSample(),
extras, *this, key_columns, key_sizes);
AddedColumns added_columns(block_with_columns_to_add, block, savedBlockSample(), *this, left_key_columns, key_sizes, is_asof_join);
bool has_required_right_keys = (required_right_keys.columns() != 0);
added_columns.need_filter = need_filter || has_required_right_keys;

View File

@ -191,10 +191,16 @@ public:
ASTTableJoin::Kind getKind() const { return kind; }
ASTTableJoin::Strictness getStrictness() const { return strictness; }
TypeIndex getAsofType() const { return *asof_type; }
const std::optional<TypeIndex> & getAsofType() const { return asof_type; }
ASOF::Inequality getAsofInequality() const { return asof_inequality; }
bool anyTakeLastRow() const { return any_take_last_row; }
const ColumnWithTypeAndName & rightAsofKeyColumn() const
{
/// It should be nullable if nullable_right_side is true
return savedBlockSample().getByName(key_names_right.back());
}
/// Different types of keys for maps.
#define APPLY_FOR_JOIN_VARIANTS(M) \
M(key8) \

View File

@ -1,6 +1,6 @@
#include <memory>
#include <Poco/File.h>
#include <filesystem>
#include <Common/StringUtils/StringUtils.h>
#include <Common/escapeForFileName.h>
@ -47,6 +47,7 @@
#include <Databases/DatabaseFactory.h>
#include <Databases/IDatabase.h>
#include <Databases/DatabaseOnDisk.h>
#include <Dictionaries/getDictionaryConfigurationFromAST.h>
@ -76,6 +77,7 @@ namespace ErrorCodes
extern const int ILLEGAL_COLUMN;
}
namespace fs = std::filesystem;
InterpreterCreateQuery::InterpreterCreateQuery(const ASTPtr & query_ptr_, Context & context_)
: query_ptr(query_ptr_), context(context_)
@ -98,7 +100,27 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
throw Exception("Database " + database_name + " already exists.", ErrorCodes::DATABASE_ALREADY_EXISTS);
}
if (!create.storage)
/// Will write file with database metadata, if needed.
String database_name_escaped = escapeForFileName(database_name);
fs::path metadata_path = fs::canonical(context.getPath());
fs::path metadata_file_tmp_path = metadata_path / "metadata" / (database_name_escaped + ".sql.tmp");
fs::path metadata_file_path = metadata_path / "metadata" / (database_name_escaped + ".sql");
if (!create.storage && create.attach)
{
if (!fs::exists(metadata_file_path))
throw Exception("Database engine must be specified for ATTACH DATABASE query", ErrorCodes::UNKNOWN_DATABASE_ENGINE);
/// Short syntax: try read database definition from file
auto ast = DatabaseOnDisk::parseQueryFromMetadata(nullptr, context, metadata_file_path);
create = ast->as<ASTCreateQuery &>();
if (!create.table.empty() || !create.storage)
throw Exception(ErrorCodes::INCORRECT_QUERY, "Metadata file {} contains incorrect CREATE DATABASE query", metadata_file_path);
create.attach = true;
create.attach_short_syntax = true;
create.database = database_name;
}
else if (!create.storage)
{
/// For new-style databases engine is explicitly specified in .sql
/// When attaching old-style database during server startup, we must always use Ordinary engine
@ -119,20 +141,39 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
throw Exception("Unknown database engine: " + ostr.str(), ErrorCodes::UNKNOWN_DATABASE_ENGINE);
}
if (create.storage->engine->name == "Atomic" && !context.getSettingsRef().allow_experimental_database_atomic && !internal)
throw Exception("Atomic is an experimental database engine. Enable allow_experimental_database_atomic to use it.",
ErrorCodes::UNKNOWN_DATABASE_ENGINE);
if (create.storage->engine->name == "Atomic")
{
if (!context.getSettingsRef().allow_experimental_database_atomic && !internal)
throw Exception("Atomic is an experimental database engine. "
"Enable allow_experimental_database_atomic to use it.", ErrorCodes::UNKNOWN_DATABASE_ENGINE);
String database_name_escaped = escapeForFileName(database_name);
String path = context.getPath();
String metadata_path = path + "metadata/" + database_name_escaped + "/";
DatabasePtr database = DatabaseFactory::get(database_name, metadata_path, create.storage, context);
if (create.attach && create.uuid == UUIDHelpers::Nil)
throw Exception("UUID must be specified for ATTACH", ErrorCodes::INCORRECT_QUERY);
else if (create.uuid == UUIDHelpers::Nil)
create.uuid = UUIDHelpers::generateV4();
/// Will write file with database metadata, if needed.
String metadata_file_tmp_path = path + "metadata/" + database_name_escaped + ".sql.tmp";
String metadata_file_path = path + "metadata/" + database_name_escaped + ".sql";
metadata_path = metadata_path / "store" / DatabaseCatalog::getPathForUUID(create.uuid);
bool need_write_metadata = !create.attach;
if (!create.attach && fs::exists(metadata_path))
throw Exception(ErrorCodes::DATABASE_ALREADY_EXISTS, "Metadata directory {} already exists", metadata_path);
}
else
{
bool is_on_cluster = context.getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY;
if (create.uuid != UUIDHelpers::Nil && !is_on_cluster)
throw Exception("Ordinary database engine does not support UUID", ErrorCodes::INCORRECT_QUERY);
/// Ignore UUID if it's ON CLUSTER query
create.uuid = UUIDHelpers::Nil;
metadata_path = metadata_path / "metadata" / database_name_escaped;
}
DatabasePtr database = DatabaseFactory::get(create, metadata_path / "", context);
if (create.uuid != UUIDHelpers::Nil)
create.database = TABLE_WITH_UUID_NAME_PLACEHOLDER;
bool need_write_metadata = !create.attach || !fs::exists(metadata_file_path);
if (need_write_metadata)
{
@ -164,7 +205,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
if (need_write_metadata)
{
Poco::File(metadata_file_tmp_path).renameTo(metadata_file_path);
fs::rename(metadata_file_tmp_path, metadata_file_path);
renamed = true;
}
@ -173,7 +214,10 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
catch (...)
{
if (renamed)
Poco::File(metadata_file_tmp_path).remove();
{
[[maybe_unused]] bool removed = fs::remove(metadata_file_tmp_path);
assert(removed);
}
if (added)
DatabaseCatalog::instance().detachDatabase(database_name, false, false);
@ -561,6 +605,32 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const
}
}
void InterpreterCreateQuery::assertOrSetUUID(ASTCreateQuery & create, const DatabasePtr & database) const
{
const auto * kind = create.is_dictionary ? "Dictionary" : "Table";
const auto * kind_upper = create.is_dictionary ? "DICTIONARY" : "TABLE";
if (database->getEngineName() == "Atomic")
{
if (create.attach && create.uuid == UUIDHelpers::Nil)
throw Exception(ErrorCodes::INCORRECT_QUERY,
"UUID must be specified in ATTACH {} query for Atomic database engine",
kind_upper);
if (!create.attach && create.uuid == UUIDHelpers::Nil)
create.uuid = UUIDHelpers::generateV4();
}
else
{
bool is_on_cluster = context.getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY;
if (create.uuid != UUIDHelpers::Nil && !is_on_cluster)
throw Exception(ErrorCodes::INCORRECT_QUERY,
"{} UUID specified, but engine of database {} is not Atomic", kind, create.database);
/// Ignore UUID if it's ON CLUSTER query
create.uuid = UUIDHelpers::Nil;
}
}
BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
{
@ -621,19 +691,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create,
if (need_add_to_database)
{
database = DatabaseCatalog::instance().getDatabase(create.database);
if (database->getEngineName() == "Atomic")
{
/// TODO implement ATTACH FROM 'path/to/data': generate UUID and move table data to store/
if (create.attach && create.uuid == UUIDHelpers::Nil)
throw Exception("UUID must be specified in ATTACH TABLE query for Atomic database engine", ErrorCodes::INCORRECT_QUERY);
if (!create.attach && create.uuid == UUIDHelpers::Nil)
create.uuid = UUIDHelpers::generateV4();
}
else
{
if (create.uuid != UUIDHelpers::Nil)
throw Exception("Table UUID specified, but engine of database " + create.database + " is not Atomic", ErrorCodes::INCORRECT_QUERY);
}
assertOrSetUUID(create, database);
/** If the request specifies IF NOT EXISTS, we allow concurrent CREATE queries (which do nothing).
* If table doesnt exist, one thread is creating table, while others wait in DDLGuard.
@ -662,7 +720,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create,
}
data_path = database->getTableDataPath(create);
if (!create.attach && !data_path.empty() && Poco::File(context.getPath() + data_path).exists())
if (!create.attach && !data_path.empty() && fs::exists(fs::path{context.getPath()} / data_path))
throw Exception("Directory for table data " + data_path + " already exists", ErrorCodes::TABLE_ALREADY_EXISTS);
}
else
@ -760,6 +818,12 @@ BlockIO InterpreterCreateQuery::createDictionary(ASTCreateQuery & create)
auto query = DatabaseCatalog::instance().getDatabase(database_name)->getCreateDictionaryQuery(dictionary_name);
create = query->as<ASTCreateQuery &>();
create.attach = true;
}
assertOrSetUUID(create, database);
if (create.attach)
{
auto config = getDictionaryConfigurationFromAST(create);
auto modification_time = database->getObjectMetadataModificationTime(dictionary_name);
database->attachDictionary(dictionary_name, DictionaryAttachInfo{query_ptr, config, modification_time});
@ -775,8 +839,12 @@ BlockIO InterpreterCreateQuery::execute()
auto & create = query_ptr->as<ASTCreateQuery &>();
if (!create.cluster.empty())
{
/// NOTE: if it's CREATE query and create.database is DatabaseAtomic, different UUIDs will be generated on all servers.
/// However, it allows to use UUID as replica name.
/// Allows to execute ON CLUSTER queries during version upgrade
bool force_backward_compatibility = !context.getSettingsRef().show_table_uuid_in_table_create_query_if_not_nil;
/// For CREATE query generate UUID on initiator, so it will be the same on all hosts.
/// It will be ignored if database does not support UUIDs.
if (!force_backward_compatibility && !create.attach && create.uuid == UUIDHelpers::Nil)
create.uuid = UUIDHelpers::generateV4();
return executeDDLQueryOnCluster(query_ptr, context, getRequiredAccess());
}

View File

@ -73,6 +73,8 @@ private:
/// Inserts data in created table if it's CREATE ... SELECT
BlockIO fillTableIfNeeded(const ASTCreateQuery & create);
void assertOrSetUUID(ASTCreateQuery & create, const DatabasePtr & database) const;
ASTPtr query_ptr;
Context & context;

View File

@ -5,7 +5,6 @@
#include <Storages/IStorage.h>
#include <Interpreters/DDLWorker.h>
#include <Access/AccessRightsElement.h>
#include <Common/typeid_cast.h>
namespace DB
@ -18,23 +17,6 @@ InterpreterRenameQuery::InterpreterRenameQuery(const ASTPtr & query_ptr_, Contex
}
struct RenameDescription
{
RenameDescription(const ASTRenameQuery::Element & elem, const String & current_database) :
from_database_name(elem.from.database.empty() ? current_database : elem.from.database),
from_table_name(elem.from.table),
to_database_name(elem.to.database.empty() ? current_database : elem.to.database),
to_table_name(elem.to.table)
{}
String from_database_name;
String from_table_name;
String to_database_name;
String to_table_name;
};
BlockIO InterpreterRenameQuery::execute()
{
const auto & rename = query_ptr->as<const ASTRenameQuery &>();
@ -51,7 +33,7 @@ BlockIO InterpreterRenameQuery::execute()
* or we will be in inconsistent state. (It is worth to be fixed.)
*/
std::vector<RenameDescription> descriptions;
RenameDescriptions descriptions;
descriptions.reserve(rename.elements.size());
/// Don't allow to drop tables (that we are renaming); don't allow to create tables in places where tables will be renamed.
@ -75,22 +57,49 @@ BlockIO InterpreterRenameQuery::execute()
for (auto & table_guard : table_guards)
table_guard.second = database_catalog.getDDLGuard(table_guard.first.database_name, table_guard.first.table_name);
for (auto & elem : descriptions)
if (rename.database)
return executeToDatabase(rename, descriptions);
else
return executeToTables(rename, descriptions);
}
BlockIO InterpreterRenameQuery::executeToTables(const ASTRenameQuery & rename, const RenameDescriptions & descriptions)
{
auto & database_catalog = DatabaseCatalog::instance();
for (const auto & elem : descriptions)
{
if (!rename.exchange)
database_catalog.assertTableDoesntExist(StorageID(elem.to_database_name, elem.to_table_name), context);
database_catalog.getDatabase(elem.from_database_name)->renameTable(
context,
elem.from_table_name,
*database_catalog.getDatabase(elem.to_database_name),
elem.to_table_name,
rename.exchange);
context,
elem.from_table_name,
*database_catalog.getDatabase(elem.to_database_name),
elem.to_table_name,
rename.exchange,
rename.dictionary);
}
return {};
}
BlockIO InterpreterRenameQuery::executeToDatabase(const ASTRenameQuery &, const RenameDescriptions & descriptions)
{
assert(descriptions.size() == 1);
assert(descriptions.front().from_table_name.empty());
assert(descriptions.front().to_table_name.empty());
const auto & old_name = descriptions.front().from_database_name;
const auto & new_name = descriptions.back().to_database_name;
auto & catalog = DatabaseCatalog::instance();
auto db = catalog.getDatabase(old_name);
catalog.assertDatabaseDoesntExist(new_name);
db->renameDatabase(new_name);
return {};
}
AccessRightsElements InterpreterRenameQuery::getRequiredAccess() const
{
AccessRightsElements required_access;
@ -99,6 +108,11 @@ AccessRightsElements InterpreterRenameQuery::getRequiredAccess() const
{
required_access.emplace_back(AccessType::SELECT | AccessType::DROP_TABLE, elem.from.database, elem.from.table);
required_access.emplace_back(AccessType::CREATE_TABLE | AccessType::INSERT, elem.to.database, elem.to.table);
if (rename.exchange)
{
required_access.emplace_back(AccessType::CREATE_TABLE | AccessType::INSERT, elem.from.database, elem.from.table);
required_access.emplace_back(AccessType::SELECT | AccessType::DROP_TABLE, elem.to.database, elem.to.table);
}
}
return required_access;
}

View File

@ -1,7 +1,7 @@
#pragma once
#include <Interpreters/IInterpreter.h>
#include <Parsers/IAST_fwd.h>
#include <Parsers/ASTRenameQuery.h>
namespace DB
@ -25,6 +25,24 @@ struct UniqueTableName
}
};
struct RenameDescription
{
RenameDescription(const ASTRenameQuery::Element & elem, const String & current_database) :
from_database_name(elem.from.database.empty() ? current_database : elem.from.database),
from_table_name(elem.from.table),
to_database_name(elem.to.database.empty() ? current_database : elem.to.database),
to_table_name(elem.to.table)
{}
String from_database_name;
String from_table_name;
String to_database_name;
String to_table_name;
};
using RenameDescriptions = std::vector<RenameDescription>;
using TableGuards = std::map<UniqueTableName, std::unique_ptr<DDLGuard>>;
/** Rename one table
@ -37,6 +55,9 @@ public:
BlockIO execute() override;
private:
BlockIO executeToTables(const ASTRenameQuery & rename, const RenameDescriptions & descriptions);
static BlockIO executeToDatabase(const ASTRenameQuery & rename, const RenameDescriptions & descriptions);
AccessRightsElements getRequiredAccess() const;
ASTPtr query_ptr;

View File

@ -233,7 +233,8 @@ BlockIO InterpreterSystemQuery::execute()
#endif
case Type::RELOAD_DICTIONARY:
context.checkAccess(AccessType::SYSTEM_RELOAD_DICTIONARY);
system_context.getExternalDictionariesLoader().loadOrReload(query.target_dictionary);
system_context.getExternalDictionariesLoader().loadOrReload(
DatabaseCatalog::instance().resolveDictionaryName(query.target_dictionary));
ExternalDictionariesLoader::resetAll();
break;
case Type::RELOAD_DICTIONARIES:
@ -392,7 +393,7 @@ void InterpreterSystemQuery::restartReplicas(Context & system_context)
if (auto table = iterator->table())
{
if (dynamic_cast<const StorageReplicatedMergeTree *>(table.get()))
replica_names.emplace_back(StorageID{database->getDatabaseName(), iterator->name()});
replica_names.emplace_back(StorageID{iterator->databaseName(), iterator->name()});
}
}
}

View File

@ -12,6 +12,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_TYPE_OF_FIELD;
}
namespace
{
@ -56,7 +61,7 @@ AsofRowRefs::AsofRowRefs(TypeIndex type)
callWithType(type, call);
}
void AsofRowRefs::insert(TypeIndex type, const IColumn * asof_column, const Block * block, size_t row_num)
void AsofRowRefs::insert(TypeIndex type, const IColumn & asof_column, const Block * block, size_t row_num)
{
auto call = [&](const auto & t)
{
@ -64,9 +69,9 @@ void AsofRowRefs::insert(TypeIndex type, const IColumn * asof_column, const Bloc
using LookupPtr = typename Entry<T>::LookupPtr;
using ColumnType = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<T>>;
auto * column = typeid_cast<const ColumnType *>(asof_column);
const auto & column = typeid_cast<const ColumnType &>(asof_column);
T key = column->getElement(row_num);
T key = column.getElement(row_num);
auto entry = Entry<T>(key, RowRef(block, row_num));
std::get<LookupPtr>(lookups)->insert(entry);
};
@ -74,7 +79,7 @@ void AsofRowRefs::insert(TypeIndex type, const IColumn * asof_column, const Bloc
callWithType(type, call);
}
const RowRef * AsofRowRefs::findAsof(TypeIndex type, ASOF::Inequality inequality, const IColumn * asof_column, size_t row_num) const
const RowRef * AsofRowRefs::findAsof(TypeIndex type, ASOF::Inequality inequality, const IColumn & asof_column, size_t row_num) const
{
const RowRef * out = nullptr;
@ -88,8 +93,8 @@ const RowRef * AsofRowRefs::findAsof(TypeIndex type, ASOF::Inequality inequality
using LookupPtr = typename EntryType::LookupPtr;
using ColumnType = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<T>>;
auto * column = typeid_cast<const ColumnType *>(asof_column);
T key = column->getElement(row_num);
const auto & column = typeid_cast<const ColumnType &>(asof_column);
T key = column.getElement(row_num);
auto & typed_lookup = std::get<LookupPtr>(lookups);
if (is_strict)
@ -102,9 +107,9 @@ const RowRef * AsofRowRefs::findAsof(TypeIndex type, ASOF::Inequality inequality
return out;
}
std::optional<TypeIndex> AsofRowRefs::getTypeSize(const IColumn * asof_column, size_t & size)
std::optional<TypeIndex> AsofRowRefs::getTypeSize(const IColumn & asof_column, size_t & size)
{
TypeIndex idx = asof_column->getDataType();
TypeIndex idx = asof_column.getDataType();
switch (idx)
{
@ -152,8 +157,7 @@ std::optional<TypeIndex> AsofRowRefs::getTypeSize(const IColumn * asof_column, s
break;
}
size = 0;
return {};
throw Exception("ASOF join not supported for type: " + std::string(asof_column.getFamilyName()), ErrorCodes::BAD_TYPE_OF_FIELD);
}
}

View File

@ -233,13 +233,13 @@ public:
AsofRowRefs() {}
AsofRowRefs(TypeIndex t);
static std::optional<TypeIndex> getTypeSize(const IColumn * asof_column, size_t & type_size);
static std::optional<TypeIndex> getTypeSize(const IColumn & asof_column, size_t & type_size);
// This will be synchronized by the rwlock mutex in Join.h
void insert(TypeIndex type, const IColumn * asof_column, const Block * block, size_t row_num);
void insert(TypeIndex type, const IColumn & asof_column, const Block * block, size_t row_num);
// This will internally synchronize
const RowRef * findAsof(TypeIndex type, ASOF::Inequality inequality, const IColumn * asof_column, size_t row_num) const;
const RowRef * findAsof(TypeIndex type, ASOF::Inequality inequality, const IColumn & asof_column, size_t row_num) const;
private:
// Lookups can be stored in a HashTable because it is memmovable

View File

@ -3,7 +3,9 @@
#include <Parsers/ASTIdentifier.h>
#include <Common/quoteString.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Poco/Util/AbstractConfiguration.h>
namespace DB
{
@ -82,4 +84,31 @@ String StorageID::getFullTableName() const
return backQuoteIfNeed(getDatabaseName()) + "." + backQuoteIfNeed(table_name);
}
String StorageID::getFullNameNotQuoted() const
{
return getDatabaseName() + "." + table_name;
}
StorageID StorageID::fromDictionaryConfig(const Poco::Util::AbstractConfiguration & config,
const String & config_prefix)
{
StorageID res = StorageID::createEmpty();
res.database_name = config.getString(config_prefix + ".database", "");
res.table_name = config.getString(config_prefix + ".name");
const String uuid_str = config.getString(config_prefix + ".uuid", "");
if (!uuid_str.empty())
res.uuid = parseFromString<UUID>(uuid_str);
return res;
}
String StorageID::getInternalDictionaryName() const
{
assertNotEmpty();
if (hasUUID())
return toString(uuid);
if (database_name.empty())
return table_name;
return database_name + "." + table_name;
}
}

View File

@ -6,13 +6,20 @@
#include <Core/QualifiedTableName.h>
#include <Common/Exception.h>
namespace Poco
{
namespace Util
{
class AbstractConfiguration;
}
}
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int LOGICAL_ERROR;
extern const int UNKNOWN_TABLE;
}
static constexpr char const * TABLE_WITH_UUID_NAME_PLACEHOLDER = "_";
@ -42,6 +49,7 @@ struct StorageID
String getTableName() const;
String getFullTableName() const;
String getFullNameNotQuoted() const;
String getNameForLogs() const;
@ -66,15 +74,9 @@ struct StorageID
{
// Can be triggered by user input, e.g. SELECT joinGetOrNull('', 'num', 500)
if (empty())
throw Exception("Table name cannot be empty. Please specify a valid table name or UUID", ErrorCodes::BAD_ARGUMENTS);
// This can also be triggered by user input, but we haven't decided what
// to do about it: create table "_"(a int) engine Log;
if (table_name == TABLE_WITH_UUID_NAME_PLACEHOLDER && !hasUUID())
throw Exception("Table name was replaced with placeholder, but UUID is Nil", ErrorCodes::LOGICAL_ERROR);
throw Exception("Both table name and UUID are empty", ErrorCodes::UNKNOWN_TABLE);
if (table_name.empty() && !database_name.empty())
throw Exception("Table name is empty, but database name is not", ErrorCodes::LOGICAL_ERROR);
throw Exception("Table name is empty, but database name is not", ErrorCodes::UNKNOWN_TABLE);
}
/// Avoid implicit construction of empty StorageID. However, it's needed for deferred initialization.
@ -82,6 +84,13 @@ struct StorageID
QualifiedTableName getQualifiedName() const { return {database_name, getTableName()}; }
static StorageID fromDictionaryConfig(const Poco::Util::AbstractConfiguration & config,
const String & config_prefix);
/// If dictionary has UUID, then use it as dictionary name in ExternalLoader to allow dictionary renaming.
/// DatabaseCatalog::resolveDictionaryName(...) should be used to access such dictionaries by name.
String getInternalDictionaryName() const;
private:
StorageID() = default;
};

View File

@ -131,7 +131,8 @@ void optimizeGroupBy(ASTSelectQuery * select_query, const NameSet & source_colum
const auto & dict_name = dict_name_ast->value.safeGet<String>();
const auto & attr_name = attr_name_ast->value.safeGet<String>();
const auto & dict_ptr = context.getExternalDictionariesLoader().getDictionary(dict_name);
String resolved_name = DatabaseCatalog::instance().resolveDictionaryName(dict_name);
const auto & dict_ptr = context.getExternalDictionariesLoader().getDictionary(resolved_name);
if (!dict_ptr->isInjective(attr_name))
{
++i;

View File

@ -229,7 +229,6 @@ void executeScalarSubqueries(ASTPtr & query, const Context & context, size_t sub
ExecuteScalarSubqueriesVisitor(visitor_data, log.stream()).visit(query);
}
void getArrayJoinedColumns(ASTPtr & query, TreeRewriterResult & result, const ASTSelectQuery * select_query,
const NamesAndTypesList & source_columns, const NameSet & source_columns_set)
{

View File

@ -51,9 +51,6 @@ namespace ProfileEvents
extern const Event FailedQuery;
extern const Event FailedInsertQuery;
extern const Event FailedSelectQuery;
extern const Event QueryTimeMicroseconds;
extern const Event SelectQueryTimeMicroseconds;
extern const Event InsertQueryTimeMicroseconds;
}
namespace DB
@ -483,34 +480,8 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
query_log->add(elem);
}
/// Coomon code for finish and exception callbacks
auto status_info_to_query_log = [ast](QueryLogElement &element, const QueryStatusInfo &info) mutable
{
DB::UInt64 query_time = info.elapsed_seconds * 1000000;
ProfileEvents::increment(ProfileEvents::QueryTimeMicroseconds, query_time);
if (ast->as<ASTSelectQuery>() || ast->as<ASTSelectWithUnionQuery>())
{
ProfileEvents::increment(ProfileEvents::SelectQueryTimeMicroseconds, query_time);
}
else if (ast->as<ASTInsertQuery>())
{
ProfileEvents::increment(ProfileEvents::InsertQueryTimeMicroseconds, query_time);
}
element.query_duration_ms = info.elapsed_seconds * 1000;
element.read_rows = info.read_rows;
element.read_bytes = info.read_bytes;
element.memory_usage = info.peak_memory_usage > 0 ? info.peak_memory_usage : 0;
element.thread_ids = std::move(info.thread_ids);
element.profile_counters = std::move(info.profile_counters);
};
/// Also make possible for caller to log successful query finish and exception during execution.
auto finish_callback = [elem, &context, ast, log_queries, log_queries_min_type = settings.log_queries_min_type,
status_info_to_query_log]
auto finish_callback = [elem, &context, log_queries, log_queries_min_type = settings.log_queries_min_type]
(IBlockInputStream * stream_in, IBlockOutputStream * stream_out, QueryPipeline * query_pipeline) mutable
{
QueryStatus * process_list_elem = context.getProcessListElement();
@ -528,14 +499,21 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
elem.type = QueryLogElementType::QUERY_FINISH;
elem.event_time = time(nullptr);
elem.query_duration_ms = elapsed_seconds * 1000;
status_info_to_query_log(elem, info);
elem.read_rows = info.read_rows;
elem.read_bytes = info.read_bytes;
elem.written_rows = info.written_rows;
elem.written_bytes = info.written_bytes;
auto progress_callback = context.getProgressCallback();
if (progress_callback)
progress_callback(Progress(WriteProgress(info.written_rows, info.written_bytes)));
elem.memory_usage = info.peak_memory_usage > 0 ? info.peak_memory_usage : 0;
if (stream_in)
{
const BlockStreamProfileInfo & stream_in_info = stream_in->getProfileInfo();
@ -580,8 +558,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
}
};
auto exception_callback = [elem, &context, ast, log_queries, log_queries_min_type = settings.log_queries_min_type, quota(quota),
status_info_to_query_log] () mutable
auto exception_callback = [elem, &context, ast, log_queries, log_queries_min_type = settings.log_queries_min_type, quota(quota)] () mutable
{
if (quota)
quota->used(Quota::ERRORS, 1, /* check_exceeded = */ false);
@ -602,7 +579,16 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
if (process_list_elem)
{
QueryStatusInfo info = process_list_elem->getInfo(true, current_settings.log_profile_events, false);
status_info_to_query_log(elem, info);
elem.query_duration_ms = info.elapsed_seconds * 1000;
elem.read_rows = info.read_rows;
elem.read_bytes = info.read_bytes;
elem.memory_usage = info.peak_memory_usage > 0 ? info.peak_memory_usage : 0;
elem.thread_ids = std::move(info.thread_ids);
elem.profile_counters = std::move(info.profile_counters);
}
if (current_settings.calculate_text_stack_trace)

View File

@ -17,6 +17,7 @@
#include <Common/escapeForFileName.h>
#include <Common/typeid_cast.h>
#include <Common/StringUtils/StringUtils.h>
namespace DB
@ -94,9 +95,20 @@ void loadMetadata(Context & context, const String & default_database_name)
Poco::DirectoryIterator dir_end;
for (Poco::DirectoryIterator it(path); it != dir_end; ++it)
{
if (!it->isDirectory())
if (it->isLink())
continue;
if (!it->isDirectory())
{
if (endsWith(it.name(), ".sql"))
{
String db_name = it.name().substr(0, it.name().size() - 4);
if (db_name != SYSTEM_DATABASE)
databases.emplace(unescapeForFileName(db_name), path + "/" + db_name);
}
continue;
}
/// For '.svn', '.gitignore' directory and similar.
if (it.name().at(0) == '.')
continue;

View File

@ -205,6 +205,13 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
<< (if_not_exists ? "IF NOT EXISTS " : "")
<< (settings.hilite ? hilite_none : "")
<< backQuoteIfNeed(database);
if (uuid != UUIDHelpers::Nil)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " UUID " << (settings.hilite ? hilite_none : "")
<< quoteString(toString(uuid));
}
formatOnCluster(settings);
if (storage)
@ -247,6 +254,9 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
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 (uuid != UUIDHelpers::Nil)
settings.ostr << (settings.hilite ? hilite_keyword : "") << " UUID " << (settings.hilite ? hilite_none : "")
<< quoteString(toString(uuid));
formatOnCluster(settings);
}

View File

@ -30,6 +30,8 @@ public:
Elements elements;
bool exchange{false}; /// For EXCHANGE TABLES
bool database{false}; /// For RENAME DATABASE
bool dictionary{false}; /// For RENAME DICTIONARY
/** Get the text that identifies this element. */
String getID(char) const override { return "Rename"; }
@ -61,8 +63,24 @@ public:
protected:
void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override
{
settings.ostr << (settings.hilite ? hilite_keyword : "")
<< (exchange ? "EXCHANGE TABLES " : "RENAME TABLE ") << (settings.hilite ? hilite_none : "");
if (database)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "RENAME DATABASE " << (settings.hilite ? hilite_none : "");
settings.ostr << backQuoteIfNeed(elements.at(0).from.database);
settings.ostr << (settings.hilite ? hilite_keyword : "") << " TO " << (settings.hilite ? hilite_none : "");
settings.ostr << backQuoteIfNeed(elements.at(0).to.database);
formatOnCluster(settings);
return;
}
settings.ostr << (settings.hilite ? hilite_keyword : "");
if (exchange)
settings.ostr << "EXCHANGE TABLES ";
else if (dictionary)
settings.ostr << "RENAME DICTIONARY ";
else
settings.ostr << "RENAME TABLE ";
settings.ostr << (settings.hilite ? hilite_none : "");
for (auto it = elements.cbegin(); it != elements.cend(); ++it)
{

View File

@ -13,6 +13,7 @@
#include <Parsers/ASTConstraintDeclaration.h>
#include <Parsers/ParserDictionary.h>
#include <Parsers/ParserDictionaryAttributeDeclaration.h>
#include <IO/ReadHelpers.h>
namespace DB
@ -595,6 +596,7 @@ bool ParserCreateDatabaseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
ASTPtr database;
ASTPtr storage;
UUID uuid = UUIDHelpers::Nil;
String cluster_str;
bool attach = false;
@ -617,6 +619,15 @@ bool ParserCreateDatabaseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
if (!name_p.parse(pos, database, expected))
return false;
if (ParserKeyword("UUID").ignore(pos, expected))
{
ParserStringLiteral uuid_p;
ASTPtr ast_uuid;
if (!uuid_p.parse(pos, ast_uuid, expected))
return false;
uuid = parseFromString<UUID>(ast_uuid->as<ASTLiteral>()->value.get<String>());
}
if (ParserKeyword{"ON"}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
@ -633,6 +644,7 @@ bool ParserCreateDatabaseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
query->if_not_exists = if_not_exists;
tryGetIdentifierNameInto(database, query->database);
query->uuid = uuid;
query->cluster = cluster_str;
query->set(query->storage, storage);
@ -784,7 +796,7 @@ bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, E
ParserKeyword s_dictionary("DICTIONARY");
ParserKeyword s_if_not_exists("IF NOT EXISTS");
ParserKeyword s_on("ON");
ParserIdentifier name_p;
ParserCompoundIdentifier dict_name_p(true);
ParserToken s_left_paren(TokenType::OpeningRoundBracket);
ParserToken s_right_paren(TokenType::ClosingRoundBracket);
ParserToken s_dot(TokenType::Dot);
@ -793,7 +805,6 @@ bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, E
bool if_not_exists = false;
ASTPtr database;
ASTPtr name;
ASTPtr attributes;
ASTPtr dictionary;
@ -814,16 +825,9 @@ bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, E
if (s_if_not_exists.ignore(pos, expected))
if_not_exists = true;
if (!name_p.parse(pos, name, expected))
if (!dict_name_p.parse(pos, name, expected))
return false;
if (s_dot.ignore(pos))
{
database = name;
if (!name_p.parse(pos, name, expected))
return false;
}
if (s_on.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
@ -850,8 +854,10 @@ bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, E
query->is_dictionary = true;
query->attach = attach;
tryGetIdentifierNameInto(database, query->database);
tryGetIdentifierNameInto(name, query->table);
StorageID dict_id = getTableIdentifier(name);
query->database = dict_id.database_name;
query->table = dict_id.table_name;
query->uuid = dict_id.uuid;
query->if_not_exists = if_not_exists;
query->set(query->dictionary_attributes_list, attributes);

View File

@ -40,20 +40,52 @@ static bool parseDatabaseAndTable(
bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_rename_table("RENAME TABLE");
ParserKeyword s_to("TO");
ParserKeyword s_exchange_tables("EXCHANGE TABLES");
ParserKeyword s_rename_dictionary("RENAME DICTIONARY");
ParserKeyword s_rename_database("RENAME DATABASE");
ParserKeyword s_to("TO");
ParserKeyword s_and("AND");
ParserToken s_comma(TokenType::Comma);
bool exchange = false;
bool dictionary = false;
if (!s_rename_table.ignore(pos, expected))
if (s_rename_table.ignore(pos, expected))
;
else if (s_exchange_tables.ignore(pos, expected))
exchange = true;
else if (s_rename_dictionary.ignore(pos, expected))
dictionary = true;
else if (s_rename_database.ignore(pos, expected))
{
if (s_exchange_tables.ignore(pos, expected))
exchange = true;
else
ASTPtr from_db;
ASTPtr to_db;
ParserIdentifier db_name_p;
if (!db_name_p.parse(pos, from_db, expected))
return false;
if (!s_to.ignore(pos, expected))
return false;
if (!db_name_p.parse(pos, to_db, expected))
return false;
String cluster_str;
if (ParserKeyword{"ON"}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;
}
auto query = std::make_shared<ASTRenameQuery>();
query->database = true;
query->elements.emplace({});
tryGetIdentifierNameInto(from_db, query->elements.front().from.database);
tryGetIdentifierNameInto(to_db, query->elements.front().to.database);
query->cluster = cluster_str;
node = query;
return true;
}
else
return false;
ASTRenameQuery::Elements elements;
@ -88,6 +120,7 @@ bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
query->elements = elements;
query->exchange = exchange;
query->dictionary = dictionary;
return true;
}

View File

@ -1,13 +1,30 @@
#include <Processors/Sources/DelayedSource.h>
#include "NullSource.h"
#include <Processors/Sources/NullSource.h>
#include <Processors/NullSink.h>
namespace DB
{
DelayedSource::DelayedSource(const Block & header, Creator processors_creator)
: IProcessor({}, OutputPorts(3, header))
DelayedSource::DelayedSource(const Block & header, Creator processors_creator, bool add_totals_port, bool add_extremes_port)
: IProcessor({}, OutputPorts(1 + (add_totals_port ? 1 : 0) + (add_extremes_port ? 1 : 0), header))
, creator(std::move(processors_creator))
{
auto output = outputs.begin();
main = &*output;
++output;
if (add_totals_port)
{
totals = &*output;
++output;
}
if (add_extremes_port)
{
extremes = &*output;
++output;
}
}
IProcessor::Status DelayedSource::prepare()
@ -66,6 +83,31 @@ IProcessor::Status DelayedSource::prepare()
return Status::Finished;
}
/// Fix port from returned pipe. Create source_port if created or drop if source_port is null.
void synchronizePorts(OutputPort *& pipe_port, OutputPort * source_port, const Block & header, Processors & processors)
{
if (source_port)
{
/// Need port in DelayedSource. Create NullSource.
if (!pipe_port)
{
processors.emplace_back(std::make_shared<NullSource>(header));
pipe_port = &processors.back()->getOutputs().back();
}
}
else
{
/// Has port in pipe, but don't need it. Create NullSink.
if (pipe_port)
{
auto sink = std::make_shared<NullSink>(header);
connect(*pipe_port, sink->getPort());
processors.emplace_back(std::move(sink));
pipe_port = nullptr;
}
}
}
void DelayedSource::work()
{
auto pipe = creator();
@ -76,17 +118,8 @@ void DelayedSource::work()
processors = std::move(pipe).detachProcessors();
if (!totals_output)
{
processors.emplace_back(std::make_shared<NullSource>(main_output->getHeader()));
totals_output = &processors.back()->getOutputs().back();
}
if (!extremes_output)
{
processors.emplace_back(std::make_shared<NullSource>(main_output->getHeader()));
extremes_output = &processors.back()->getOutputs().back();
}
synchronizePorts(totals_output, totals, main->getHeader(), processors);
synchronizePorts(extremes_output, extremes, main->getHeader(), processors);
}
Processors DelayedSource::expandPipeline()
@ -94,6 +127,9 @@ Processors DelayedSource::expandPipeline()
/// Add new inputs. They must have the same header as output.
for (const auto & output : {main_output, totals_output, extremes_output})
{
if (!output)
continue;
inputs.emplace_back(outputs.front().getHeader(), this);
/// Connect checks that header is same for ports.
connect(*output, inputs.back());
@ -104,13 +140,13 @@ Processors DelayedSource::expandPipeline()
return std::move(processors);
}
Pipe createDelayedPipe(const Block & header, DelayedSource::Creator processors_creator)
Pipe createDelayedPipe(const Block & header, DelayedSource::Creator processors_creator, bool add_totals_port, bool add_extremes_port)
{
auto source = std::make_shared<DelayedSource>(header, std::move(processors_creator));
auto source = std::make_shared<DelayedSource>(header, std::move(processors_creator), add_totals_port, add_extremes_port);
Pipe pipe(&source->getPort(DelayedSource::Main));
pipe.setTotalsPort(&source->getPort(DelayedSource::Totals));
pipe.setExtremesPort(&source->getPort(DelayedSource::Extremes));
Pipe pipe(&source->getPort());
pipe.setTotalsPort(source->getTotalsPort());
pipe.setExtremesPort(source->getExtremesPort());
pipe.addProcessors({std::move(source)});
return pipe;

View File

@ -19,20 +19,26 @@ class DelayedSource : public IProcessor
public:
using Creator = std::function<Pipe()>;
DelayedSource(const Block & header, Creator processors_creator);
DelayedSource(const Block & header, Creator processors_creator, bool add_totals_port, bool add_extremes_port);
String getName() const override { return "Delayed"; }
Status prepare() override;
void work() override;
Processors expandPipeline() override;
enum PortKind { Main = 0, Totals = 1, Extremes = 2 };
OutputPort & getPort(PortKind kind) { return *std::next(outputs.begin(), kind); }
OutputPort & getPort() { return *main; }
OutputPort * getTotalsPort() { return totals; }
OutputPort * getExtremesPort() { return extremes; }
private:
Creator creator;
Processors processors;
/// Outputs for DelayedSource.
OutputPort * main = nullptr;
OutputPort * totals = nullptr;
OutputPort * extremes = nullptr;
/// Outputs from returned pipe.
OutputPort * main_output = nullptr;
OutputPort * totals_output = nullptr;
@ -40,6 +46,6 @@ private:
};
/// Creates pipe from DelayedSource.
Pipe createDelayedPipe(const Block & header, DelayedSource::Creator processors_creator);
Pipe createDelayedPipe(const Block & header, DelayedSource::Creator processors_creator, bool add_totals_port, bool add_extremes_port);
}

View File

@ -169,7 +169,7 @@ SettingsChanges StorageKafka::createSettingsAdjustments()
if (!schema_name.empty())
result.emplace_back("format_schema", schema_name);
for (auto setting : *kafka_settings)
for (const auto & setting : *kafka_settings)
{
const auto & name = setting.getName();
if (name.find("kafka_") == std::string::npos)

View File

@ -146,6 +146,10 @@ MergeTreeData::MergeTreeData(
if (relative_data_path.empty())
throw Exception("MergeTree storages require data path", ErrorCodes::INCORRECT_FILE_NAME);
/// Check sanity of MergeTreeSettings. Only when table is created.
if (!attach)
settings->sanityCheck(global_context.getSettingsRef());
MergeTreeDataFormatVersion min_format_version(0);
if (!date_column_name.empty())
{
@ -1608,6 +1612,7 @@ void MergeTreeData::changeSettings(
const auto & new_changes = new_settings->as<const ASTSetQuery &>().changes;
for (const auto & change : new_changes)
{
if (change.name == "storage_policy")
{
StoragePolicyPtr new_storage_policy = global_context.getStoragePolicy(change.value.safeGet<String>());
@ -1642,9 +1647,13 @@ void MergeTreeData::changeSettings(
has_storage_policy_changed = true;
}
}
}
MergeTreeSettings copy = *getSettings();
copy.applyChanges(new_changes);
copy.sanityCheck(global_context.getSettingsRef());
storage_settings.set(std::make_unique<const MergeTreeSettings>(copy));
StorageInMemoryMetadata new_metadata = getInMemoryMetadata();
new_metadata.setSettingsChanges(new_settings);

View File

@ -75,4 +75,31 @@ void MergeTreeSettings::loadFromQuery(ASTStorage & storage_def)
#undef ADD_IF_ABSENT
}
void MergeTreeSettings::sanityCheck(const Settings & query_settings) const
{
if (number_of_free_entries_in_pool_to_execute_mutation >= query_settings.background_pool_size)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "The value of 'number_of_free_entries_in_pool_to_execute_mutation' setting"
" ({}) (default values are defined in <merge_tree> section of config.xml"
" or the value can be specified per table in SETTINGS section of CREATE TABLE query)"
" is greater or equals to the value of 'background_pool_size'"
" ({}) (the value is defined in users.xml for default profile)."
" This indicates incorrect configuration because mutations cannot work with these settings.",
number_of_free_entries_in_pool_to_execute_mutation,
query_settings.background_pool_size);
}
if (number_of_free_entries_in_pool_to_lower_max_size_of_merge >= query_settings.background_pool_size)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "The value of 'number_of_free_entries_in_pool_to_lower_max_size_of_merge' setting"
" ({}) (default values are defined in <merge_tree> section of config.xml"
" or the value can be specified per table in SETTINGS section of CREATE TABLE query)"
" is greater or equals to the value of 'background_pool_size'"
" ({}) (the value is defined in users.xml for default profile)."
" This indicates incorrect configuration because the maximum size of merge will be always lowered.",
number_of_free_entries_in_pool_to_execute_mutation,
query_settings.background_pool_size);
}
}
}

Some files were not shown because too many files have changed in this diff Show More