Skip to content

Commit

Permalink
Merge pull request #50492 from ClickHouse/50422-add-systemuser_proces…
Browse files Browse the repository at this point in the history
…ses-system-table

Add `system.user_processes` table
  • Loading branch information
serxa committed Jun 5, 2023
2 parents 5d0522c + c3d6e4c commit a1b961e
Show file tree
Hide file tree
Showing 8 changed files with 144 additions and 5 deletions.
10 changes: 5 additions & 5 deletions docs/en/operations/system-tables/processes.md
Expand Up @@ -10,14 +10,14 @@ Columns:
- `user` (String) – The user who made the query. Keep in mind that for distributed processing, queries are sent to remote servers under the `default` user. The field contains the username for a specific query, not for a query that this query initiated.
- `address` (String) – The IP address the request was made from. The same for distributed processing. To track where a distributed query was originally made from, look at `system.processes` on the query requestor server.
- `elapsed` (Float64) – The time in seconds since request execution started.
- `rows_read` (UInt64) – The number of rows read from the table. For distributed processing, on the requestor server, this is the total for all remote servers.
- `bytes_read` (UInt64) – The number of uncompressed bytes read from the table. For distributed processing, on the requestor server, this is the total for all remote servers.
- `read_rows` (UInt64) – The number of rows read from the table. For distributed processing, on the requestor server, this is the total for all remote servers.
- `read_bytes` (UInt64) – The number of uncompressed bytes read from the table. For distributed processing, on the requestor server, this is the total for all remote servers.
- `total_rows_approx` (UInt64) – The approximation of the total number of rows that should be read. For distributed processing, on the requestor server, this is the total for all remote servers. It can be updated during request processing, when new sources to process become known.
- `memory_usage` (UInt64) – Amount of RAM the request uses. It might not include some types of dedicated memory. See the [max_memory_usage](../../operations/settings/query-complexity.md#settings_max_memory_usage) setting.
- `memory_usage` (Int64) – Amount of RAM the request uses. It might not include some types of dedicated memory. See the [max_memory_usage](../../operations/settings/query-complexity.md#settings_max_memory_usage) setting.
- `query` (String) – The query text. For `INSERT`, it does not include the data to insert.
- `query_id` (String) – Query ID, if defined.
- `is_cancelled` (Int8) – Was query cancelled.
- `is_all_data_sent` (Int8) – Was all data sent to the client (in other words query had been finished on the server).
- `is_cancelled` (UInt8) – Was query cancelled.
- `is_all_data_sent` (UInt8) – Was all data sent to the client (in other words query had been finished on the server).

```sql
SELECT * FROM system.processes LIMIT 10 FORMAT Vertical;
Expand Down
28 changes: 28 additions & 0 deletions docs/en/operations/system-tables/user_processes.md
@@ -0,0 +1,28 @@
---
slug: /en/operations/system-tables/user_processes
---
# user_processes

This system table can be used to get overview of memory usage and ProfileEvents of users.

Columns:

- `user` ([String](../../sql-reference/data-types/string.md)) — User name.
- `memory_usage` ([Int64](../../sql-reference/data-types/int-uint#int-ranges)) – Sum of RAM used by all processes of the user. It might not include some types of dedicated memory. See the [max_memory_usage](../../operations/settings/query-complexity.md#settings_max_memory_usage) setting.
- `peak_memory_usage` ([Int64](../../sql-reference/data-types/int-uint#int-ranges)) — The peak of memory usage of the user. It can be reset when no queries are run for the user.
- `ProfileEvents` ([Map(String, UInt64)](../../sql-reference/data-types/map)) – Summary of ProfileEvents that measure different metrics for the user. The description of them could be found in the table [system.events](../../operations/system-tables/events.md#system_tables-events)

```sql
SELECT * FROM system.user_processes LIMIT 10 FORMAT Vertical;
```

```response
Row 1:
──────
user: default
memory_usage: 9832
peak_memory_usage: 9832
ProfileEvents: {'Query':5,'SelectQuery':5,'QueriesWithSubqueries':38,'SelectQueriesWithSubqueries':38,'QueryTimeMicroseconds':842048,'SelectQueryTimeMicroseconds':842048,'ReadBufferFromFileDescriptorRead':6,'ReadBufferFromFileDescriptorReadBytes':234,'IOBufferAllocs':3,'IOBufferAllocBytes':98493,'ArenaAllocChunks':283,'ArenaAllocBytes':1482752,'FunctionExecute':670,'TableFunctionExecute':16,'DiskReadElapsedMicroseconds':19,'NetworkSendElapsedMicroseconds':684,'NetworkSendBytes':139498,'SelectedRows':6076,'SelectedBytes':685802,'ContextLock':1140,'RWLockAcquiredReadLocks':193,'RWLockReadersWaitMilliseconds':4,'RealTimeMicroseconds':1585163,'UserTimeMicroseconds':889767,'SystemTimeMicroseconds':13630,'SoftPageFaults':1947,'OSCPUWaitMicroseconds':6,'OSCPUVirtualTimeMicroseconds':903251,'OSReadChars':28631,'OSWriteChars':28888,'QueryProfilerRuns':3,'LogTrace':79,'LogDebug':24}
1 row in set. Elapsed: 0.010 sec.
```
57 changes: 57 additions & 0 deletions src/Storages/System/StorageSystemUserProcesses.cpp
@@ -0,0 +1,57 @@
#include <Columns/ColumnArray.h>
#include <Columns/ColumnsNumber.h>
#include <Core/Settings.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeMap.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <Interpreters/Context.h>
#include <Interpreters/ProcessList.h>
#include <Interpreters/ProfileEventsExt.h>
#include <Storages/System/StorageSystemUserProcesses.h>


namespace DB
{

NamesAndTypesList StorageSystemUserProcesses::getNamesAndTypes()
{
return {
{"user", std::make_shared<DataTypeString>()},
{"memory_usage", std::make_shared<DataTypeInt64>()},
{"peak_memory_usage", std::make_shared<DataTypeInt64>()},
{"ProfileEvents", std::make_shared<DataTypeMap>(std::make_shared<DataTypeString>(), std::make_shared<DataTypeUInt64>())},
};
}

NamesAndAliases StorageSystemUserProcesses::getNamesAndAliases()
{
return {
{"ProfileEvents.Names", {std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())}, "mapKeys(ProfileEvents)"},
{"ProfileEvents.Values", {std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>())}, "mapValues(ProfileEvents)"}};
}

void StorageSystemUserProcesses::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
const auto user_info = context->getProcessList().getUserInfo(true);

for (const auto & [user, info] : user_info)
{
size_t i = 0;

res_columns[i++]->insert(user);
res_columns[i++]->insert(info.memory_usage);
res_columns[i++]->insert(info.peak_memory_usage);
{
IColumn * column = res_columns[i++].get();

if (info.profile_counters)
ProfileEvents::dumpToMapColumn(*info.profile_counters, column, true);
else
{
column->insertDefault();
}
}
}
}
}
29 changes: 29 additions & 0 deletions src/Storages/System/StorageSystemUserProcesses.h
@@ -0,0 +1,29 @@
#pragma once

#include <Storages/System/IStorageSystemOneBlock.h>


namespace DB
{

class Context;


/** Implements `processes` system table, which allows you to get information about the queries that are currently executing.
*/
class StorageSystemUserProcesses final : public IStorageSystemOneBlock<StorageSystemUserProcesses>
{
public:
std::string getName() const override { return "SystemUserProcesses"; }

static NamesAndTypesList getNamesAndTypes();

static NamesAndAliases getNamesAndAliases();

protected:
using IStorageSystemOneBlock::IStorageSystemOneBlock;

void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override;
};

}
2 changes: 2 additions & 0 deletions src/Storages/System/attachSystemTables.cpp
Expand Up @@ -36,6 +36,7 @@
#include <Storages/System/StorageSystemPartsColumns.h>
#include <Storages/System/StorageSystemProjectionPartsColumns.h>
#include <Storages/System/StorageSystemProcesses.h>
#include <Storages/System/StorageSystemUserProcesses.h>
#include <Storages/System/StorageSystemReplicas.h>
#include <Storages/System/StorageSystemReplicationQueue.h>
#include <Storages/System/StorageSystemDistributionQueue.h>
Expand Down Expand Up @@ -185,6 +186,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b
attach<StorageSystemRemoteDataPaths>(context, system_database, "remote_data_paths");
attach<StorageSystemCertificates>(context, system_database, "certificates");
attach<StorageSystemNamedCollections>(context, system_database, "named_collections");
attach<StorageSystemUserProcesses>(context, system_database, "user_processes");

if (has_zookeeper)
{
Expand Down
Expand Up @@ -6,6 +6,7 @@ Code: 516
1
Code: 516
processes
processes
Code: 81
[1]
Code: 73
@@ -0,0 +1,4 @@
0
0
default true true
2 2
18 changes: 18 additions & 0 deletions tests/queries/0_stateless/02771_system_user_processes.sh
@@ -0,0 +1,18 @@
#!/usr/bin/env bash

CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh

USER_POSTFIX=`random_str 10`
USER="test_user_02771_$USER_POSTFIX"

$CLICKHOUSE_CLIENT -q "DROP USER IF EXISTS $USER"
$CLICKHOUSE_CLIENT -q "CREATE USER $USER"
$CLICKHOUSE_CLIENT -q "GRANT SELECT ON system.* TO $USER"
$CLICKHOUSE_CLIENT -u "$USER" -q "SELECT * FROM system.numbers LIMIT 1"
$CLICKHOUSE_CLIENT -u "$USER" -q "SELECT * FROM system.numbers LIMIT 1"
$CLICKHOUSE_CLIENT -q "SELECT user, toBool(ProfileEvents['SelectQuery'] > 0), toBool(ProfileEvents['Query'] > 0) FROM system.user_processes WHERE user='default'"
$CLICKHOUSE_CLIENT -q "SELECT ProfileEvents['SelectQuery'], ProfileEvents['Query'] FROM system.user_processes WHERE user='$USER'"
$CLICKHOUSE_CLIENT -q "DROP USER $USER"

0 comments on commit a1b961e

Please sign in to comment.