mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 07:01:59 +00:00
Add system.distribution_queue
system.distribution_queue contains the following columns: - database - table - data_path - is_blocked - error_count - data_files - data_compressed_bytes
This commit is contained in:
parent
60d10f1bac
commit
389f78ceee
@ -108,11 +108,19 @@ StorageDistributedDirectoryMonitor::~StorageDistributedDirectoryMonitor()
|
|||||||
|
|
||||||
void StorageDistributedDirectoryMonitor::flushAllData()
|
void StorageDistributedDirectoryMonitor::flushAllData()
|
||||||
{
|
{
|
||||||
if (!quit)
|
if (quit)
|
||||||
|
return;
|
||||||
|
|
||||||
|
CurrentMetrics::Increment metric_pending_files{CurrentMetrics::DistributedFilesToInsert, 0};
|
||||||
|
std::unique_lock lock{mutex};
|
||||||
|
|
||||||
|
const auto & files = getFiles(metric_pending_files);
|
||||||
|
if (!files.empty())
|
||||||
{
|
{
|
||||||
CurrentMetrics::Increment metric_pending_files{CurrentMetrics::DistributedFilesToInsert, 0};
|
processFiles(files, metric_pending_files);
|
||||||
std::unique_lock lock{mutex};
|
|
||||||
processFiles(metric_pending_files);
|
/// Update counters
|
||||||
|
getFiles(metric_pending_files);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -139,11 +147,16 @@ void StorageDistributedDirectoryMonitor::run()
|
|||||||
while (!quit)
|
while (!quit)
|
||||||
{
|
{
|
||||||
do_sleep = true;
|
do_sleep = true;
|
||||||
|
|
||||||
|
const auto & files = getFiles(metric_pending_files);
|
||||||
|
if (files.empty())
|
||||||
|
break;
|
||||||
|
|
||||||
if (!monitor_blocker.isCancelled())
|
if (!monitor_blocker.isCancelled())
|
||||||
{
|
{
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
do_sleep = !processFiles(metric_pending_files);
|
do_sleep = !processFiles(files, metric_pending_files);
|
||||||
}
|
}
|
||||||
catch (...)
|
catch (...)
|
||||||
{
|
{
|
||||||
@ -171,6 +184,9 @@ void StorageDistributedDirectoryMonitor::run()
|
|||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Update counters
|
||||||
|
getFiles(metric_pending_files);
|
||||||
|
|
||||||
if (!quit && do_sleep)
|
if (!quit && do_sleep)
|
||||||
task_handle->scheduleAfter(sleep_time.count());
|
task_handle->scheduleAfter(sleep_time.count());
|
||||||
}
|
}
|
||||||
@ -226,9 +242,10 @@ ConnectionPoolPtr StorageDistributedDirectoryMonitor::createPool(const std::stri
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
bool StorageDistributedDirectoryMonitor::processFiles(CurrentMetrics::Increment & metric_pending_files)
|
std::map<UInt64, std::string> StorageDistributedDirectoryMonitor::getFiles(CurrentMetrics::Increment & metric_pending_files)
|
||||||
{
|
{
|
||||||
std::map<UInt64, std::string> files;
|
std::map<UInt64, std::string> files;
|
||||||
|
size_t new_bytes_count = 0;
|
||||||
|
|
||||||
Poco::DirectoryIterator end;
|
Poco::DirectoryIterator end;
|
||||||
for (Poco::DirectoryIterator it{path}; it != end; ++it)
|
for (Poco::DirectoryIterator it{path}; it != end; ++it)
|
||||||
@ -237,16 +254,23 @@ bool StorageDistributedDirectoryMonitor::processFiles(CurrentMetrics::Increment
|
|||||||
Poco::Path file_path{file_path_str};
|
Poco::Path file_path{file_path_str};
|
||||||
|
|
||||||
if (!it->isDirectory() && startsWith(file_path.getExtension(), "bin"))
|
if (!it->isDirectory() && startsWith(file_path.getExtension(), "bin"))
|
||||||
|
{
|
||||||
files[parse<UInt64>(file_path.getBaseName())] = file_path_str;
|
files[parse<UInt64>(file_path.getBaseName())] = file_path_str;
|
||||||
|
new_bytes_count += Poco::File(file_path).getSize();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
files_count = files.size();
|
||||||
|
bytes_count = new_bytes_count;
|
||||||
|
|
||||||
/// Note: the value of this metric will be kept if this function will throw an exception.
|
/// Note: the value of this metric will be kept if this function will throw an exception.
|
||||||
/// This is needed, because in case of exception, files still pending.
|
/// This is needed, because in case of exception, files still pending.
|
||||||
metric_pending_files.changeTo(files.size());
|
metric_pending_files.changeTo(files.size());
|
||||||
|
|
||||||
if (files.empty())
|
return files;
|
||||||
return false;
|
}
|
||||||
|
bool StorageDistributedDirectoryMonitor::processFiles(const std::map<UInt64, std::string> & files, CurrentMetrics::Increment & metric_pending_files)
|
||||||
|
{
|
||||||
if (should_batch_inserts)
|
if (should_batch_inserts)
|
||||||
{
|
{
|
||||||
processFilesWithBatching(files, metric_pending_files);
|
processFilesWithBatching(files, metric_pending_files);
|
||||||
|
@ -37,9 +37,20 @@ public:
|
|||||||
|
|
||||||
/// For scheduling via DistributedBlockOutputStream
|
/// For scheduling via DistributedBlockOutputStream
|
||||||
bool scheduleAfter(size_t ms);
|
bool scheduleAfter(size_t ms);
|
||||||
|
|
||||||
|
/// system.distribution_queue interface
|
||||||
|
std::string getPath() const { return path; }
|
||||||
|
/// Racy but ok
|
||||||
|
size_t getErrorCount() const { return error_count; }
|
||||||
|
size_t getFilesCount() const { return files_count; }
|
||||||
|
size_t getBytesCount() const { return bytes_count; }
|
||||||
|
size_t isBlocked() const { return monitor_blocker.isCancelled(); }
|
||||||
|
|
||||||
private:
|
private:
|
||||||
void run();
|
void run();
|
||||||
bool processFiles(CurrentMetrics::Increment & metric_pending_files);
|
|
||||||
|
std::map<UInt64, std::string> getFiles(CurrentMetrics::Increment & metric_pending_files);
|
||||||
|
bool processFiles(const std::map<UInt64, std::string> & files, CurrentMetrics::Increment & metric_pending_files);
|
||||||
void processFile(const std::string & file_path, CurrentMetrics::Increment & metric_pending_files);
|
void processFile(const std::string & file_path, CurrentMetrics::Increment & metric_pending_files);
|
||||||
void processFilesWithBatching(const std::map<UInt64, std::string> & files, CurrentMetrics::Increment & metric_pending_files);
|
void processFilesWithBatching(const std::map<UInt64, std::string> & files, CurrentMetrics::Increment & metric_pending_files);
|
||||||
|
|
||||||
@ -61,7 +72,10 @@ private:
|
|||||||
struct BatchHeader;
|
struct BatchHeader;
|
||||||
struct Batch;
|
struct Batch;
|
||||||
|
|
||||||
size_t error_count{};
|
size_t error_count = 0;
|
||||||
|
size_t files_count = 0;
|
||||||
|
size_t bytes_count = 0;
|
||||||
|
|
||||||
const std::chrono::milliseconds default_sleep_time;
|
const std::chrono::milliseconds default_sleep_time;
|
||||||
std::chrono::milliseconds sleep_time;
|
std::chrono::milliseconds sleep_time;
|
||||||
const std::chrono::milliseconds max_sleep_time;
|
const std::chrono::milliseconds max_sleep_time;
|
||||||
|
@ -656,6 +656,17 @@ StorageDistributedDirectoryMonitor& StorageDistributed::requireDirectoryMonitor(
|
|||||||
return *node_data.directory_monitor;
|
return *node_data.directory_monitor;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
std::vector<StorageDistributedDirectoryMonitor *> StorageDistributed::getAllDirectoryMonitors()
|
||||||
|
{
|
||||||
|
std::vector<StorageDistributedDirectoryMonitor *> monitors;
|
||||||
|
{
|
||||||
|
std::lock_guard lock(cluster_nodes_mutex);
|
||||||
|
for (auto & node : cluster_nodes_data)
|
||||||
|
monitors.push_back(node.second.directory_monitor.get());
|
||||||
|
}
|
||||||
|
return monitors;
|
||||||
|
}
|
||||||
|
|
||||||
size_t StorageDistributed::getShardCount() const
|
size_t StorageDistributed::getShardCount() const
|
||||||
{
|
{
|
||||||
return getCluster()->getShardCount();
|
return getCluster()->getShardCount();
|
||||||
|
@ -94,6 +94,7 @@ public:
|
|||||||
void shutdown() override;
|
void shutdown() override;
|
||||||
|
|
||||||
Strings getDataPaths() const override;
|
Strings getDataPaths() const override;
|
||||||
|
size_t getInsertQueueSize() const { return 0; }
|
||||||
|
|
||||||
const ExpressionActionsPtr & getShardingKeyExpr() const { return sharding_key_expr; }
|
const ExpressionActionsPtr & getShardingKeyExpr() const { return sharding_key_expr; }
|
||||||
const String & getShardingKeyColumnName() const { return sharding_key_column_name; }
|
const String & getShardingKeyColumnName() const { return sharding_key_column_name; }
|
||||||
@ -107,6 +108,8 @@ public:
|
|||||||
void createDirectoryMonitors(const std::string & disk);
|
void createDirectoryMonitors(const std::string & disk);
|
||||||
/// ensure directory monitor thread and connectoin pool creation by disk and subdirectory name
|
/// ensure directory monitor thread and connectoin pool creation by disk and subdirectory name
|
||||||
StorageDistributedDirectoryMonitor & requireDirectoryMonitor(const std::string & disk, const std::string & name);
|
StorageDistributedDirectoryMonitor & requireDirectoryMonitor(const std::string & disk, const std::string & name);
|
||||||
|
/// Return list of all monitors lazy (because there are no monitors until at least one INSERT executed)
|
||||||
|
std::vector<StorageDistributedDirectoryMonitor *> getAllDirectoryMonitors();
|
||||||
|
|
||||||
void flushClusterNodesAllData();
|
void flushClusterNodesAllData();
|
||||||
|
|
||||||
|
110
src/Storages/System/StorageSystemDistributionQueue.cpp
Normal file
110
src/Storages/System/StorageSystemDistributionQueue.cpp
Normal file
@ -0,0 +1,110 @@
|
|||||||
|
#include <Columns/ColumnString.h>
|
||||||
|
#include <Columns/ColumnsNumber.h>
|
||||||
|
#include <DataTypes/DataTypeString.h>
|
||||||
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
|
#include <Storages/System/StorageSystemDistributionQueue.h>
|
||||||
|
#include <Storages/Distributed/DirectoryMonitor.h>
|
||||||
|
#include <Storages/StorageDistributed.h>
|
||||||
|
#include <Storages/VirtualColumnUtils.h>
|
||||||
|
#include <Access/ContextAccess.h>
|
||||||
|
#include <Common/typeid_cast.h>
|
||||||
|
#include <Databases/IDatabase.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
|
||||||
|
NamesAndTypesList StorageSystemDistributionQueue::getNamesAndTypes()
|
||||||
|
{
|
||||||
|
return {
|
||||||
|
{ "database", std::make_shared<DataTypeString>() },
|
||||||
|
{ "table", std::make_shared<DataTypeString>() },
|
||||||
|
{ "data_path", std::make_shared<DataTypeString>() },
|
||||||
|
{ "is_blocked", std::make_shared<DataTypeUInt8>() },
|
||||||
|
{ "error_count", std::make_shared<DataTypeUInt64>() },
|
||||||
|
{ "data_files", std::make_shared<DataTypeUInt64>() },
|
||||||
|
{ "data_compressed_bytes", std::make_shared<DataTypeUInt64>() },
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void StorageSystemDistributionQueue::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const
|
||||||
|
{
|
||||||
|
const auto access = context.getAccess();
|
||||||
|
const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES);
|
||||||
|
|
||||||
|
std::map<String, std::map<String, StoragePtr>> tables;
|
||||||
|
for (const auto & db : DatabaseCatalog::instance().getDatabases())
|
||||||
|
{
|
||||||
|
/// Lazy database can not contain distributed tables
|
||||||
|
if (db.second->getEngineName() == "Lazy")
|
||||||
|
continue;
|
||||||
|
|
||||||
|
const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, db.first);
|
||||||
|
|
||||||
|
for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next())
|
||||||
|
{
|
||||||
|
if (!dynamic_cast<const StorageDistributed *>(iterator->table().get()))
|
||||||
|
continue;
|
||||||
|
if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, db.first, iterator->name()))
|
||||||
|
continue;
|
||||||
|
tables[db.first][iterator->name()] = iterator->table();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
MutableColumnPtr col_database_mut = ColumnString::create();
|
||||||
|
MutableColumnPtr col_table_mut = ColumnString::create();
|
||||||
|
|
||||||
|
for (auto & db : tables)
|
||||||
|
{
|
||||||
|
for (auto & table : db.second)
|
||||||
|
{
|
||||||
|
col_database_mut->insert(db.first);
|
||||||
|
col_table_mut->insert(table.first);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
ColumnPtr col_database_to_filter = std::move(col_database_mut);
|
||||||
|
ColumnPtr col_table_to_filter = std::move(col_table_mut);
|
||||||
|
|
||||||
|
/// Determine what tables are needed by the conditions in the query.
|
||||||
|
{
|
||||||
|
Block filtered_block
|
||||||
|
{
|
||||||
|
{ col_database_to_filter, std::make_shared<DataTypeString>(), "database" },
|
||||||
|
{ col_table_to_filter, std::make_shared<DataTypeString>(), "table" },
|
||||||
|
};
|
||||||
|
|
||||||
|
VirtualColumnUtils::filterBlockWithQuery(query_info.query, filtered_block, context);
|
||||||
|
|
||||||
|
if (!filtered_block.rows())
|
||||||
|
return;
|
||||||
|
|
||||||
|
col_database_to_filter = filtered_block.getByName("database").column;
|
||||||
|
col_table_to_filter = filtered_block.getByName("table").column;
|
||||||
|
}
|
||||||
|
|
||||||
|
for (size_t i = 0, tables_size = col_database_to_filter->size(); i < tables_size; ++i)
|
||||||
|
{
|
||||||
|
String database = (*col_database_to_filter)[i].safeGet<const String &>();
|
||||||
|
String table = (*col_table_to_filter)[i].safeGet<const String &>();
|
||||||
|
|
||||||
|
auto & distributed_table = dynamic_cast<StorageDistributed &>(*tables[database][table]);
|
||||||
|
|
||||||
|
for (auto * monitor : distributed_table.getAllDirectoryMonitors())
|
||||||
|
{
|
||||||
|
size_t col_num = 0;
|
||||||
|
res_columns[col_num++]->insert(database);
|
||||||
|
res_columns[col_num++]->insert(table);
|
||||||
|
res_columns[col_num++]->insert(monitor->getPath());
|
||||||
|
res_columns[col_num++]->insert(monitor->isBlocked());
|
||||||
|
res_columns[col_num++]->insert(monitor->getErrorCount());
|
||||||
|
res_columns[col_num++]->insert(monitor->getFilesCount());
|
||||||
|
res_columns[col_num++]->insert(monitor->getBytesCount());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
29
src/Storages/System/StorageSystemDistributionQueue.h
Normal file
29
src/Storages/System/StorageSystemDistributionQueue.h
Normal file
@ -0,0 +1,29 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <ext/shared_ptr_helper.h>
|
||||||
|
#include <Storages/System/IStorageSystemOneBlock.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
class Context;
|
||||||
|
|
||||||
|
|
||||||
|
/** Implements the `distribution_queue` system table, which allows you to view the INSERT queues for the Distributed tables.
|
||||||
|
*/
|
||||||
|
class StorageSystemDistributionQueue final : public ext::shared_ptr_helper<StorageSystemDistributionQueue>, public IStorageSystemOneBlock<StorageSystemDistributionQueue>
|
||||||
|
{
|
||||||
|
friend struct ext::shared_ptr_helper<StorageSystemDistributionQueue>;
|
||||||
|
public:
|
||||||
|
std::string getName() const override { return "SystemDistributionQueue"; }
|
||||||
|
|
||||||
|
static NamesAndTypesList getNamesAndTypes();
|
||||||
|
|
||||||
|
protected:
|
||||||
|
using IStorageSystemOneBlock::IStorageSystemOneBlock;
|
||||||
|
|
||||||
|
void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override;
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
@ -28,6 +28,7 @@
|
|||||||
#include <Storages/System/StorageSystemProcesses.h>
|
#include <Storages/System/StorageSystemProcesses.h>
|
||||||
#include <Storages/System/StorageSystemReplicas.h>
|
#include <Storages/System/StorageSystemReplicas.h>
|
||||||
#include <Storages/System/StorageSystemReplicationQueue.h>
|
#include <Storages/System/StorageSystemReplicationQueue.h>
|
||||||
|
#include <Storages/System/StorageSystemDistributionQueue.h>
|
||||||
#include <Storages/System/StorageSystemSettings.h>
|
#include <Storages/System/StorageSystemSettings.h>
|
||||||
#include <Storages/System/StorageSystemMergeTreeSettings.h>
|
#include <Storages/System/StorageSystemMergeTreeSettings.h>
|
||||||
#include <Storages/System/StorageSystemTableEngines.h>
|
#include <Storages/System/StorageSystemTableEngines.h>
|
||||||
@ -124,6 +125,7 @@ void attachSystemTablesServer(IDatabase & system_database, bool has_zookeeper)
|
|||||||
system_database.attachTable("mutations", StorageSystemMutations::create("mutations"));
|
system_database.attachTable("mutations", StorageSystemMutations::create("mutations"));
|
||||||
system_database.attachTable("replicas", StorageSystemReplicas::create("replicas"));
|
system_database.attachTable("replicas", StorageSystemReplicas::create("replicas"));
|
||||||
system_database.attachTable("replication_queue", StorageSystemReplicationQueue::create("replication_queue"));
|
system_database.attachTable("replication_queue", StorageSystemReplicationQueue::create("replication_queue"));
|
||||||
|
system_database.attachTable("distribution_queue", StorageSystemDistributionQueue::create("distribution_queue"));
|
||||||
system_database.attachTable("dictionaries", StorageSystemDictionaries::create("dictionaries"));
|
system_database.attachTable("dictionaries", StorageSystemDictionaries::create("dictionaries"));
|
||||||
system_database.attachTable("models", StorageSystemModels::create("models"));
|
system_database.attachTable("models", StorageSystemModels::create("models"));
|
||||||
system_database.attachTable("clusters", StorageSystemClusters::create("clusters"));
|
system_database.attachTable("clusters", StorageSystemClusters::create("clusters"));
|
||||||
|
@ -121,6 +121,7 @@ SRCS(
|
|||||||
System/StorageSystemQuotasUsage.cpp
|
System/StorageSystemQuotasUsage.cpp
|
||||||
System/StorageSystemReplicas.cpp
|
System/StorageSystemReplicas.cpp
|
||||||
System/StorageSystemReplicationQueue.cpp
|
System/StorageSystemReplicationQueue.cpp
|
||||||
|
System/StorageSystemDistributionQueue.cpp
|
||||||
System/StorageSystemRoleGrants.cpp
|
System/StorageSystemRoleGrants.cpp
|
||||||
System/StorageSystemRoles.cpp
|
System/StorageSystemRoles.cpp
|
||||||
System/StorageSystemRowPolicies.cpp
|
System/StorageSystemRowPolicies.cpp
|
||||||
|
@ -0,0 +1,6 @@
|
|||||||
|
INSERT
|
||||||
|
1 0 1 1
|
||||||
|
FLUSH
|
||||||
|
1 0 0 0
|
||||||
|
UNBLOCK
|
||||||
|
0 0 0 0
|
@ -0,0 +1,21 @@
|
|||||||
|
drop table if exists null_01293;
|
||||||
|
drop table if exists dist_01293;
|
||||||
|
|
||||||
|
create table null_01293 (key Int) engine=Null();
|
||||||
|
create table dist_01293 as null_01293 engine=Distributed(test_cluster_two_shards, currentDatabase(), null_01293, key);
|
||||||
|
|
||||||
|
-- no rows, since no active monitor
|
||||||
|
select * from system.distribution_queue;
|
||||||
|
|
||||||
|
select 'INSERT';
|
||||||
|
system stop distributed sends dist_01293;
|
||||||
|
insert into dist_01293 select * from numbers(10);
|
||||||
|
select is_blocked, error_count, data_files, data_compressed_bytes>100 from system.distribution_queue;
|
||||||
|
system flush distributed dist_01293;
|
||||||
|
|
||||||
|
select 'FLUSH';
|
||||||
|
select is_blocked, error_count, data_files, data_compressed_bytes from system.distribution_queue;
|
||||||
|
|
||||||
|
select 'UNBLOCK';
|
||||||
|
system start distributed sends dist_01293;
|
||||||
|
select is_blocked, error_count, data_files, data_compressed_bytes from system.distribution_queue;
|
Loading…
Reference in New Issue
Block a user