support parts, tables, databases

This commit is contained in:
凌涛 2023-11-30 10:56:31 +08:00
parent 98a03ac36c
commit d9f4b4d2c0
13 changed files with 136 additions and 6 deletions

View File

@ -1280,6 +1280,9 @@ try
global_context->setMaxTableSizeToDrop(server_settings_.max_table_size_to_drop);
global_context->setMaxPartitionSizeToDrop(server_settings_.max_partition_size_to_drop);
global_context->setMaxTableNumToWarn(server_settings_.max_table_num_to_warn);
global_context->setMaxDatabaseNumToWarn(server_settings_.max_database_num_to_warn);
global_context->setMaxPartNumToWarn(server_settings_.max_part_num_to_warn);
ConcurrencyControl::SlotCount concurrent_threads_soft_limit = ConcurrencyControl::Unlimited;
if (server_settings_.concurrent_threads_soft_limit_num > 0 && server_settings_.concurrent_threads_soft_limit_num < concurrent_threads_soft_limit)

View File

@ -79,6 +79,9 @@ namespace DB
\
M(UInt64, max_table_size_to_drop, 50000000000lu, "If size of a table is greater than this value (in bytes) than table could not be dropped with any DROP query.", 0) \
M(UInt64, max_partition_size_to_drop, 50000000000lu, "Same as max_table_size_to_drop, but for the partitions.", 0) \
M(UInt64, max_table_num_to_warn, 5000lu, "If number of tables is greater than this value, server will create a warning that will displayed to user.", 0) \
M(UInt64, max_database_num_to_warn, 1000lu, "If number of databases is greater than this value, server will create a warning that will displayed to user.", 0) \
M(UInt64, max_part_num_to_warn, 100000lu, "If number of databases is greater than this value, server will create a warning that will displayed to user.", 0) \
M(UInt64, concurrent_threads_soft_limit_num, 0, "Sets how many concurrent thread can be allocated before applying CPU pressure. Zero means unlimited.", 0) \
M(UInt64, concurrent_threads_soft_limit_ratio_to_cores, 0, "Same as concurrent_threads_soft_limit_num, but with ratio to cores.", 0) \
\

View File

@ -18,6 +18,10 @@
namespace fs = std::filesystem;
namespace CurrentMetrics {
extern const Metric AttachedTable;
}
namespace DB
{
@ -64,7 +68,6 @@ void DatabaseLazy::createTable(
SCOPE_EXIT_MEMORY_SAFE({ clearExpiredTables(); });
if (!endsWith(table->getName(), "Log"))
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Lazy engine can be used only with *Log tables.");
std::cout<<"======= Flag 1"<<std::endl;
DatabaseOnDisk::createTable(local_context, table_name, table, query);
/// DatabaseOnDisk::createTable renames file, so we need to get new metadata_modification_time.
@ -175,6 +178,7 @@ void DatabaseLazy::attachTable(ContextPtr /* context_ */, const String & table_n
throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {}.{} already exists.", backQuote(database_name), backQuote(table_name));
it->second.expiration_iterator = cache_expiration_queue.emplace(cache_expiration_queue.end(), current_time, table_name);
CurrentMetrics::add(CurrentMetrics::AttachedTable, 1);
}
StoragePtr DatabaseLazy::detachTable(ContextPtr /* context */, const String & table_name)
@ -190,6 +194,7 @@ StoragePtr DatabaseLazy::detachTable(ContextPtr /* context */, const String & ta
if (it->second.expiration_iterator != cache_expiration_queue.end())
cache_expiration_queue.erase(it->second.expiration_iterator);
tables_cache.erase(it);
CurrentMetrics::sub(CurrentMetrics::AttachedTable, 1);
}
return res;
}

View File

@ -263,7 +263,6 @@ void DatabaseOnDisk::commitCreateTable(const ASTCreateQuery & query, const Stora
try
{
/// Add a table to the map of known tables.
std::cout<<"===== flag 3"<<std::endl;
attachTable(query_context, query.getTable(), table, getTableDataPath(query));
/// If it was ATTACH query and file with table metadata already exist

View File

@ -263,14 +263,12 @@ StoragePtr DatabaseWithOwnTablesBase::detachTableUnlocked(const String & table_n
void DatabaseWithOwnTablesBase::attachTable(ContextPtr /* context_ */, const String & table_name, const StoragePtr & table, const String &)
{
std::cout<<"========= Flag 5"<<std::endl;
std::lock_guard lock(mutex);
attachTableUnlocked(table_name, table);
}
void DatabaseWithOwnTablesBase::attachTableUnlocked(const String & table_name, const StoragePtr & table)
{
std::cout<<"========= Flag 6"<<std::endl;
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 `{}`",

View File

@ -5,8 +5,14 @@
#include <Common/quoteString.h>
#include <Interpreters/DatabaseCatalog.h>
#include <Common/NamePrompter.h>
#include <Common/CurrentMetrics.h>
namespace CurrentMetrics
{
extern const Metric AttachedDatabase;
}
namespace DB
{
@ -29,6 +35,15 @@ StoragePtr IDatabase::getTable(const String & name, ContextPtr context) const
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} does not exist. Maybe you meant {}?", backQuoteIfNeed(getDatabaseName()), backQuoteIfNeed(name), backQuoteIfNeed(names[0]));
}
IDatabase::IDatabase(String database_name_) : database_name(std::move(database_name_)) {
CurrentMetrics::add(CurrentMetrics::AttachedDatabase, 1);
}
IDatabase::~IDatabase()
{
CurrentMetrics::sub(CurrentMetrics::AttachedDatabase, 1);
}
std::vector<std::pair<ASTPtr, StoragePtr>> IDatabase::getTablesForBackup(const FilterByNameFunction &, const ContextPtr &) const
{
/// Cannot backup any table because IDatabase doesn't own any tables.

View File

@ -122,7 +122,7 @@ class IDatabase : public std::enable_shared_from_this<IDatabase>
{
public:
IDatabase() = delete;
explicit IDatabase(String database_name_) : database_name(std::move(database_name_)) {}
explicit IDatabase(String database_name_);
/// Get name of database engine.
virtual String getEngineName() const = 0;
@ -357,7 +357,7 @@ public:
/// Creates a table restored from backup.
virtual void createTableRestoredFromBackup(const ASTPtr & create_table_query, ContextMutablePtr context, std::shared_ptr<IRestoreCoordination> restore_coordination, UInt64 timeout_ms);
virtual ~IDatabase() = default;
virtual ~IDatabase();
protected:
virtual ASTPtr getCreateTableQueryImpl(const String & /*name*/, ContextPtr /*context*/, bool throw_on_error) const

View File

@ -142,6 +142,9 @@ namespace CurrentMetrics
extern const Metric IOWriterThreads;
extern const Metric IOWriterThreadsActive;
extern const Metric IOWriterThreadsScheduled;
extern const Metric AttachedTable;
extern const Metric AttachedDatabase;
extern const Metric PartsActive;
}
@ -323,6 +326,10 @@ struct ContextSharedPart : boost::noncopyable
std::optional<MergeTreeSettings> replicated_merge_tree_settings TSA_GUARDED_BY(mutex); /// Settings of ReplicatedMergeTree* engines.
std::atomic_size_t max_table_size_to_drop = 50000000000lu; /// Protects MergeTree tables from accidental DROP (50GB by default)
std::atomic_size_t max_partition_size_to_drop = 50000000000lu; /// Protects MergeTree partitions from accidental DROP (50GB by default)
/// std::atomic_size_t max_table_size_to_warn;
std::atomic_size_t max_database_num_to_warn = 1000lu;
std::atomic_size_t max_table_num_to_warn = 5000lu;
std::atomic_size_t max_part_num_to_warn = 100000lu;
/// No lock required for format_schema_path modified only during initialization
String format_schema_path; /// Path to a directory that contains schema files used by input formats.
mutable OnceFlag action_locks_manager_initialized;
@ -829,6 +836,15 @@ Strings Context::getWarnings() const
{
SharedLockGuard lock(shared->mutex);
common_warnings = shared->warnings;
if (CurrentMetrics::get(CurrentMetrics::AttachedTable) > static_cast<DB::Int64>(shared->max_table_num_to_warn))
common_warnings.emplace_back(fmt::format("Attached tables is more than {}", shared->max_table_num_to_warn));
if (CurrentMetrics::get(CurrentMetrics::AttachedDatabase) > static_cast<DB::Int64>(shared->max_database_num_to_warn))
common_warnings.emplace_back(fmt::format("Attached databases is more than {}", shared->max_table_num_to_warn));
if (CurrentMetrics::get(CurrentMetrics::PartsActive) > static_cast<DB::Int64>(shared->max_part_num_to_warn))
common_warnings.emplace_back(fmt::format("Active parts is more than {}", shared->max_part_num_to_warn));
}
/// Make setting's name ordered
std::set<String> obsolete_settings;
@ -3322,6 +3338,24 @@ UInt16 Context::getServerPort(const String & port_name) const
return it->second;
}
void Context::setMaxPartNumToWarn(size_t max_part_to_warn)
{
SharedLockGuard lock(shared->mutex);
shared->max_part_num_to_warn = max_part_to_warn;
}
void Context::setMaxTableNumToWarn(size_t max_table_to_warn)
{
SharedLockGuard lock(shared->mutex);
shared->max_table_num_to_warn= max_table_to_warn;
}
void Context::setMaxDatabaseNumToWarn(size_t max_database_to_warn)
{
SharedLockGuard lock(shared->mutex);
shared->max_database_num_to_warn= max_database_to_warn;
}
std::shared_ptr<Cluster> Context::getCluster(const std::string & cluster_name) const
{
if (auto res = tryGetCluster(cluster_name))

View File

@ -838,6 +838,9 @@ public:
void setHTTPHeaderFilter(const Poco::Util::AbstractConfiguration & config);
const HTTPHeaderFilter & getHTTPHeaderFilter() const;
void setMaxTableNumToWarn(size_t max_table_to_warn);
void setMaxDatabaseNumToWarn(size_t max_database_to_warn);
void setMaxPartNumToWarn(size_t max_part_to_warn);
/// The port that the server listens for executing SQL queries.
UInt16 getTCPPort() const;

View File

@ -0,0 +1,5 @@
<clickhouse>
<max_table_num_to_warn>10</max_table_num_to_warn>
<max_database_num_to_warn>10</max_database_num_to_warn>
<max_part_num_to_warn>10</max_part_num_to_warn>
</clickhouse>

View File

@ -16,6 +16,7 @@ mkdir -p $DEST_SERVER_PATH/users.d/
mkdir -p $DEST_CLIENT_PATH
ln -sf $SRC_PATH/config.d/zookeeper_write.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/max_num_to_warn.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/listen.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/text_log.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/blob_storage_log.xml $DEST_SERVER_PATH/config.d/

View File

@ -0,0 +1,3 @@
Attached tables is more than 10
Attached databases is more than 10
Active parts is more than 10

View File

@ -0,0 +1,61 @@
CREATE TABLE test_max_num_to_warn_1 (id Int32, str String) Engine=Memory;
CREATE TABLE test_max_num_to_warn_2 (id Int32, str String) Engine=Memory;
CREATE TABLE test_max_num_to_warn_3 (id Int32, str String) Engine=Memory;
CREATE TABLE test_max_num_to_warn_4 (id Int32, str String) Engine=Memory;
CREATE TABLE test_max_num_to_warn_5 (id Int32, str String) Engine=Memory;
CREATE TABLE test_max_num_to_warn_6 (id Int32, str String) Engine=Memory;
CREATE TABLE test_max_num_to_warn_7 (id Int32, str String) Engine=Memory;
CREATE TABLE test_max_num_to_warn_8 (id Int32, str String) Engine=Memory;
CREATE TABLE test_max_num_to_warn_9 (id Int32, str String) Engine=Memory;
CREATE TABLE test_max_num_to_warn_10 (id Int32, str String) Engine=Memory;
CREATE TABLE test_max_num_to_warn_11 (id Int32, str String) Engine=Memory;
CREATE DATABASE test_max_num_to_warn_1;
CREATE DATABASE test_max_num_to_warn_2;
CREATE DATABASE test_max_num_to_warn_3;
CREATE DATABASE test_max_num_to_warn_4;
CREATE DATABASE test_max_num_to_warn_5;
CREATE DATABASE test_max_num_to_warn_6;
CREATE DATABASE test_max_num_to_warn_7;
CREATE DATABASE test_max_num_to_warn_8;
CREATE DATABASE test_max_num_to_warn_9;
CREATE DATABASE test_max_num_to_warn_10;
CREATE DATABASE test_max_num_to_warn_11;
INSERT INTO test_max_num_to_warn_1 VALUES (1, 'Hello');
INSERT INTO test_max_num_to_warn_2 VALUES (1, 'Hello');
INSERT INTO test_max_num_to_warn_3 VALUES (1, 'Hello');
INSERT INTO test_max_num_to_warn_4 VALUES (1, 'Hello');
INSERT INTO test_max_num_to_warn_5 VALUES (1, 'Hello');
INSERT INTO test_max_num_to_warn_6 VALUES (1, 'Hello');
INSERT INTO test_max_num_to_warn_7 VALUES (1, 'Hello');
INSERT INTO test_max_num_to_warn_8 VALUES (1, 'Hello');
INSERT INTO test_max_num_to_warn_9 VALUES (1, 'Hello');
INSERT INTO test_max_num_to_warn_10 VALUES (1, 'Hello');
INSERT INTO test_max_num_to_warn_11 VALUES (1, 'Hello');
SELECT * FROM system.warnings where message in ('Attached tables is more than 10', 'Attached databases is more than 10', 'Active parts is more than 10');
DROP TABLE test_max_num_to_warn_1;
DROP TABLE test_max_num_to_warn_2;
DROP TABLE test_max_num_to_warn_3;
DROP TABLE test_max_num_to_warn_4;
DROP TABLE test_max_num_to_warn_5;
DROP TABLE test_max_num_to_warn_6;
DROP TABLE test_max_num_to_warn_7;
DROP TABLE test_max_num_to_warn_8;
DROP TABLE test_max_num_to_warn_9;
DROP TABLE test_max_num_to_warn_10;
DROP TABLE test_max_num_to_warn_11;
DROP DATABASE test_max_num_to_warn_1;
DROP DATABASE test_max_num_to_warn_2;
DROP DATABASE test_max_num_to_warn_3;
DROP DATABASE test_max_num_to_warn_4;
DROP DATABASE test_max_num_to_warn_5;
DROP DATABASE test_max_num_to_warn_6;
DROP DATABASE test_max_num_to_warn_7;
DROP DATABASE test_max_num_to_warn_8;
DROP DATABASE test_max_num_to_warn_9;
DROP DATABASE test_max_num_to_warn_10;
DROP DATABASE test_max_num_to_warn_11;