Distributed: More accurate distribution_queue counters

So now system.distribution_queue will show accurate statistics, so tests
does not requires sleep anymore.

But note that with too much distributed pending this will iterate over
all directories.
This commit is contained in:
Azat Khuzhin 2021-01-26 21:45:37 +03:00
parent b5a5778589
commit b43046ba06
6 changed files with 12 additions and 18 deletions

View File

@ -435,7 +435,7 @@ ConnectionPoolPtr StorageDistributedDirectoryMonitor::createPool(const std::stri
}
std::map<UInt64, std::string> StorageDistributedDirectoryMonitor::getFiles()
std::map<UInt64, std::string> StorageDistributedDirectoryMonitor::getFiles(bool lock_metrics) const
{
std::map<UInt64, std::string> files;
size_t new_bytes_count = 0;
@ -456,7 +456,9 @@ std::map<UInt64, std::string> StorageDistributedDirectoryMonitor::getFiles()
metric_pending_files.changeTo(files.size());
{
std::unique_lock metrics_lock(metrics_mutex);
std::unique_lock metrics_lock(metrics_mutex, std::defer_lock);
if (lock_metrics)
metrics_lock.lock();
files_count = files.size();
bytes_count = new_bytes_count;
}
@ -758,6 +760,9 @@ StorageDistributedDirectoryMonitor::Status StorageDistributedDirectoryMonitor::g
{
std::unique_lock metrics_lock(metrics_mutex);
/// Recalculate counters
getFiles(false /* metrics_lock already acquired */);
return Status{
path,
last_exception,

View File

@ -65,7 +65,7 @@ public:
private:
void run();
std::map<UInt64, std::string> getFiles();
std::map<UInt64, std::string> getFiles(bool lock_metrics = true) const;
bool processFiles(const std::map<UInt64, std::string> & files);
void processFile(const std::string & file_path);
void processFilesWithBatching(const std::map<UInt64, std::string> & files);
@ -93,8 +93,8 @@ private:
mutable std::mutex metrics_mutex;
size_t error_count = 0;
size_t files_count = 0;
size_t bytes_count = 0;
mutable size_t files_count = 0;
mutable size_t bytes_count = 0;
std::exception_ptr last_exception;
const std::chrono::milliseconds default_sleep_time;
@ -108,7 +108,7 @@ private:
BackgroundSchedulePoolTaskHolder task_handle;
CurrentMetrics::Increment metric_pending_files;
mutable CurrentMetrics::Increment metric_pending_files;
friend class DirectoryMonitorBlockInputStream;
};

View File

@ -982,7 +982,6 @@ void StorageDistributed::throwInsertIfNeeded() const
if (!distributed_settings.bytes_to_throw_insert)
return;
/// TODO: update the counters
UInt64 total_bytes = *totalBytes(global_context.getSettingsRef());
if (total_bytes > distributed_settings.bytes_to_throw_insert)
{

View File

@ -47,5 +47,5 @@ Check total_bytes/total_rows for Join
10240 100
Check total_bytes/total_rows for Distributed
0 \N
629 \N
581 \N
0 \N

View File

@ -128,16 +128,11 @@ SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tab
DROP TABLE check_system_tables;
SELECT 'Check total_bytes/total_rows for Distributed';
-- metrics updated only after distributed_directory_monitor_sleep_time_ms
SET distributed_directory_monitor_sleep_time_ms=10;
CREATE TABLE check_system_tables_null (key Int) Engine=Null();
CREATE TABLE check_system_tables AS check_system_tables_null Engine=Distributed(test_shard_localhost, currentDatabase(), check_system_tables_null);
SYSTEM STOP DISTRIBUTED SENDS check_system_tables;
SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables';
INSERT INTO check_system_tables SELECT * FROM numbers(1) SETTINGS prefer_localhost_replica=0;
-- 1 second should guarantee metrics update
-- XXX: but this is kind of quirk, way more better will be account this metrics without any delays.
SELECT sleep(1) FORMAT Null;
SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables';
SYSTEM FLUSH DISTRIBUTED check_system_tables;
SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables';

View File

@ -10,11 +10,6 @@ select * from system.distribution_queue;
select 'INSERT';
system stop distributed sends dist_01293;
insert into dist_01293 select * from numbers(10);
-- metrics updated only after distributed_directory_monitor_sleep_time_ms
set distributed_directory_monitor_sleep_time_ms=10;
-- 1 second should guarantee metrics update
-- XXX: but this is kind of quirk, way more better will be account this metrics without any delays.
select sleep(1) format Null;
select is_blocked, error_count, data_files, data_compressed_bytes>100 from system.distribution_queue;
system flush distributed dist_01293;