Merge branch 'master' into suffix_size

This commit is contained in:
Eridanus 2023-05-04 17:39:27 +08:00 committed by GitHub
commit d428c461fc
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
25 changed files with 365 additions and 234 deletions

View File

@ -2,7 +2,7 @@
A technical comment, you are free to remove or leave it as it is when PR is created
The following categories are used in the next scripts, update them accordingly
utils/changelog/changelog.py
tests/ci/run_check.py
tests/ci/cancel_and_rerun_workflow_lambda/app.py
-->
### Changelog category (leave one):
- New Feature

View File

@ -2068,3 +2068,20 @@ Possible values:
- Positive integer.
Default value: `10000`.
## display_secrets_in_show_and_select {#display_secrets_in_show_and_select}
Enables or disables showing secrets in `SHOW` and `SELECT` queries for tables, databases,
table functions, and dictionaries.
User wishing to see secrets must also have
[`format_display_secrets_in_show_and_select` format setting](../settings/formats#format_display_secrets_in_show_and_select)
turned on and a
[`displaySecretsInShowAndSelect`](../../sql-reference/statements/grant#grant-display-secrets) privilege.
Possible values:
- 0 — Disabled.
- 1 — Enabled.
Default value: 0.

View File

@ -7,6 +7,23 @@ toc_max_heading_level: 2
# Format settings {#format-settings}
## format_display_secrets_in_show_and_select {#format_display_secrets_in_show_and_select}
Enables or disables showing secrets in `SHOW` and `SELECT` queries for tables, databases,
table functions, and dictionaries.
User wishing to see secrets must also have
[`display_secrets_in_show_and_select` server setting](../server-configuration-parameters/settings#display_secrets_in_show_and_select)
turned on and a
[`displaySecretsInShowAndSelect`](../../sql-reference/statements/grant#grant-display-secrets) privilege.
Possible values:
- 0 — Disabled.
- 1 — Enabled.
Default value: 0.
## input_format_skip_unknown_fields {#input_format_skip_unknown_fields}
Enables or disables skipping insertion of extra data.

View File

@ -200,6 +200,7 @@ Hierarchy of privileges:
- `HDFS`
- `S3`
- [dictGet](#grant-dictget)
- [displaySecretsInShowAndSelect](#grant-display-secrets)
Examples of how this hierarchy is treated:
@ -485,6 +486,15 @@ Privilege level: `DICTIONARY`.
- `GRANT dictGet ON mydb.mydictionary TO john`
- `GRANT dictGet ON mydictionary TO john`
### displaySecretsInShowAndSelect {#grant-display-secrets}
Allows a user to view secrets in `SHOW` and `SELECT` queries if both
[`display_secrets_in_show_and_select` server setting](../../operations/server-configuration-parameters/settings#display_secrets_in_show_and_select)
and
[`format_display_secrets_in_show_and_select` format setting](../../operations/settings/formats#format_display_secrets_in_show_and_select)
are turned on.
### ALL
Grants all the privileges on regulated entity to a user account or a role.

View File

@ -6,6 +6,13 @@ sidebar_label: SHOW
# SHOW Statements
N.B. `SHOW CREATE (TABLE|DATABASE|USER)` hides secrets unless
[`display_secrets_in_show_and_select` server setting](../../operations/server-configuration-parameters/settings#display_secrets_in_show_and_select)
is turned on,
[`format_display_secrets_in_show_and_select` format setting](../../operations/settings/formats#format_display_secrets_in_show_and_select)
is turned on and user has
[`displaySecretsInShowAndSelect`](grant.md#grant-display-secrets) privilege.
## SHOW CREATE TABLE | DICTIONARY | VIEW | DATABASE
``` sql
@ -293,8 +300,6 @@ If user is not specified, the query returns privileges for the current user.
Shows parameters that were used at a [user creation](../../sql-reference/statements/create/user.md).
`SHOW CREATE USER` does not output user passwords.
**Syntax**
``` sql

View File

@ -188,6 +188,7 @@ enum class AccessType
M(SYSTEM, "", GROUP, ALL) /* allows to execute SYSTEM {SHUTDOWN|RELOAD CONFIG|...} */ \
\
M(dictGet, "dictHas, dictGetHierarchy, dictIsIn", DICTIONARY, ALL) /* allows to execute functions dictGet(), dictHas(), dictGetHierarchy(), dictIsIn() */\
M(displaySecretsInShowAndSelect, "", GLOBAL, ALL) /* allows to show plaintext secrets in SELECT and SHOW queries. display_secrets_in_show_and_select format and server settings must be turned on */\
\
M(addressToLine, "", GLOBAL, INTROSPECTION) /* allows to execute function addressToLine() */\
M(addressToLineWithInlines, "", GLOBAL, INTROSPECTION) /* allows to execute function addressToLineWithInlines() */\

View File

@ -76,6 +76,7 @@ namespace DB
M(UInt64, background_schedule_pool_size, 128, "The maximum number of threads that will be used for constantly executing some lightweight periodic operations.", 0) \
M(UInt64, background_message_broker_schedule_pool_size, 16, "The maximum number of threads that will be used for executing background operations for message streaming.", 0) \
M(UInt64, background_distributed_schedule_pool_size, 16, "The maximum number of threads that will be used for executing distributed sends.", 0) \
M(Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0)
DECLARE_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS)

View File

@ -970,6 +970,7 @@ class IColumn;
M(Bool, input_format_bson_skip_fields_with_unsupported_types_in_schema_inference, false, "Skip fields with unsupported types while schema inference for format BSON.", 0) \
\
M(Bool, regexp_dict_allow_other_sources, false, "Allow regexp_tree dictionary to use sources other than yaml source.", 0) \
M(Bool, format_display_secrets_in_show_and_select, false, "Do not hide secrets in SHOW and SELECT queries.", IMPORTANT) \
M(Bool, regexp_dict_allow_hyperscan, true, "Allow regexp_tree dictionary using Hyperscan library.", 0) \
\
M(Bool, dictionary_use_async_executor, false, "Execute a pipeline for reading from a dictionary with several threads. It's supported only by DIRECT dictionary with CLICKHOUSE source.", 0) \

View File

@ -4,6 +4,7 @@
#include <Interpreters/Context.h>
#include <Interpreters/Access/InterpreterShowCreateAccessEntityQuery.h>
#include <Interpreters/Access/InterpreterShowGrantsQuery.h>
#include <Interpreters/formatWithPossiblyHidingSecrets.h>
#include <Columns/ColumnString.h>
#include <Processors/Sources/SourceFromSingleChunk.h>
#include <DataTypes/DataTypeString.h>
@ -33,7 +34,7 @@ QueryPipeline InterpreterShowAccessQuery::executeImpl() const
/// Build the result column.
MutableColumnPtr column = ColumnString::create();
for (const auto & query : queries)
column->insert(query->formatWithSecretsHidden());
column->insert(format({getContext(), *query}));
String desc = "ACCESS";
return QueryPipeline(std::make_shared<SourceFromSingleChunk>(Block{{std::move(column), std::make_shared<DataTypeString>(), desc}}));

View File

@ -1,4 +1,5 @@
#include <Interpreters/Access/InterpreterShowCreateAccessEntityQuery.h>
#include <Interpreters/formatWithPossiblyHidingSecrets.h>
#include <Parsers/Access/ASTShowCreateAccessEntityQuery.h>
#include <Parsers/Access/ASTCreateUserQuery.h>
#include <Parsers/Access/ASTCreateRoleQuery.h>
@ -254,7 +255,7 @@ QueryPipeline InterpreterShowCreateAccessEntityQuery::executeImpl()
/// Build the result column.
MutableColumnPtr column = ColumnString::create();
for (const auto & create_query : create_queries)
column->insert(create_query->formatWithSecretsHidden());
column->insert(format({getContext(), *create_query}));
/// Prepare description of the result column.
const auto & show_query = query_ptr->as<const ASTShowCreateAccessEntityQuery &>();

View File

@ -1612,26 +1612,42 @@ StoragePtr Context::getViewSource() const
return view_source;
}
bool Context::displaySecretsInShowAndSelect() const
{
return shared->server_settings.display_secrets_in_show_and_select;
}
Settings Context::getSettings() const
{
auto lock = getLock();
return settings;
}
void Context::setSettings(const Settings & settings_)
{
auto lock = getLock();
auto old_readonly = settings.readonly;
auto old_allow_ddl = settings.allow_ddl;
auto old_allow_introspection_functions = settings.allow_introspection_functions;
const auto old_readonly = settings.readonly;
const auto old_allow_ddl = settings.allow_ddl;
const auto old_allow_introspection_functions = settings.allow_introspection_functions;
const auto old_display_secrets = settings.format_display_secrets_in_show_and_select;
settings = settings_;
if ((settings.readonly != old_readonly) || (settings.allow_ddl != old_allow_ddl) || (settings.allow_introspection_functions != old_allow_introspection_functions))
if ((settings.readonly != old_readonly)
|| (settings.allow_ddl != old_allow_ddl)
|| (settings.allow_introspection_functions != old_allow_introspection_functions)
|| (settings.format_display_secrets_in_show_and_select != old_display_secrets))
calculateAccessRights();
}
void Context::recalculateAccessRightsIfNeeded(std::string_view name)
{
if (name == "readonly"
|| name == "allow_ddl"
|| name == "allow_introspection_functions"
|| name == "format_display_secrets_in_show_and_select")
calculateAccessRights();
}
void Context::setSetting(std::string_view name, const String & value)
{
@ -1642,12 +1658,9 @@ void Context::setSetting(std::string_view name, const String & value)
return;
}
settings.set(name, value);
if (name == "readonly" || name == "allow_ddl" || name == "allow_introspection_functions")
calculateAccessRights();
recalculateAccessRightsIfNeeded(name);
}
void Context::setSetting(std::string_view name, const Field & value)
{
auto lock = getLock();
@ -1657,12 +1670,9 @@ void Context::setSetting(std::string_view name, const Field & value)
return;
}
settings.set(name, value);
if (name == "readonly" || name == "allow_ddl" || name == "allow_introspection_functions")
calculateAccessRights();
recalculateAccessRightsIfNeeded(name);
}
void Context::applySettingChange(const SettingChange & change)
{
try

View File

@ -683,6 +683,7 @@ public:
MultiVersion<Macros>::Version getMacros() const;
void setMacros(std::unique_ptr<Macros> && macros);
bool displaySecretsInShowAndSelect() const;
Settings getSettings() const;
void setSettings(const Settings & settings_);
@ -1145,6 +1146,7 @@ private:
/// Compute and set actual user settings, client_info.current_user should be set
void calculateAccessRights();
void recalculateAccessRightsIfNeeded(std::string_view setting_name);
template <typename... Args>
void checkAccessImpl(const Args &... args) const;

View File

@ -9,6 +9,7 @@
#include <Common/typeid_cast.h>
#include <Access/Common/AccessFlags.h>
#include <Interpreters/Context.h>
#include <Interpreters/formatWithPossiblyHidingSecrets.h>
#include <Interpreters/InterpreterShowCreateQuery.h>
#include <Parsers/ASTCreateQuery.h>
@ -94,10 +95,8 @@ QueryPipeline InterpreterShowCreateQuery::executeImpl()
create.to_inner_uuid = UUIDHelpers::Nil;
}
String res = create_query->formatWithSecretsHidden(/* max_length= */ 0, /* one_line= */ false);
MutableColumnPtr column = ColumnString::create();
column->insert(res);
column->insert(format({.ctx = getContext(), .query = *create_query, .one_line = false}));
return QueryPipeline(std::make_shared<SourceFromSingleChunk>(Block{{
std::move(column),

View File

@ -0,0 +1,25 @@
#pragma once
#include "Access/ContextAccess.h"
#include "Interpreters/Context.h"
namespace DB
{
struct SecretHidingFormatSettings
{
// We can't store const Context& as there's a dangerous usage {.ctx = *getContext()}
// which is UB in case getContext()'s return ptr is the only one holding the object
const ContextPtr & ctx;
const IAST & query;
size_t max_length = 0;
bool one_line = true;
};
inline String format(const SecretHidingFormatSettings & settings)
{
const bool show_secrets = settings.ctx->displaySecretsInShowAndSelect()
&& settings.ctx->getSettingsRef().format_display_secrets_in_show_and_select
&& settings.ctx->getAccess()->isGranted(AccessType::displaySecretsInShowAndSelect);
return settings.query.formatWithPossiblyHidingSensitiveData(settings.max_length, settings.one_line, show_secrets);
}
}

View File

@ -167,14 +167,10 @@ size_t IAST::checkDepthImpl(size_t max_depth) const
return res;
}
String IAST::formatWithSecretsHidden(size_t max_length, bool one_line) const
String IAST::formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets) const
{
WriteBufferFromOwnString buf;
FormatSettings settings{buf, one_line};
settings.show_secrets = false;
format(settings);
format({buf, one_line, show_secrets});
return wipeSensitiveDataAndCutToLength(buf.str(), max_length);
}

View File

@ -200,8 +200,8 @@ public:
// Newline or whitespace.
char nl_or_ws;
FormatSettings(WriteBuffer & ostr_, bool one_line_)
: ostr(ostr_), one_line(one_line_)
FormatSettings(WriteBuffer & ostr_, bool one_line_, bool show_secrets_ = true)
: ostr(ostr_), one_line(one_line_), show_secrets(show_secrets_)
{
nl_or_ws = one_line ? ' ' : '\n';
}
@ -251,12 +251,26 @@ public:
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown element in AST: {}", getID());
}
// A simple way to add some user-readable context to an error message.
String formatWithSecretsHidden(size_t max_length = 0, bool one_line = true) const;
String formatForLogging(size_t max_length = 0) const { return formatWithSecretsHidden(max_length, true); }
String formatForErrorMessage() const { return formatWithSecretsHidden(0, true); }
// Secrets are displayed regarding show_secrets, then SensitiveDataMasker is applied.
// You can use Interpreters/formatWithPossiblyHidingSecrets.h for convenience.
String formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets) const;
/*
* formatForLogging and formatForErrorMessage always hide secrets. This inconsistent
* behaviour is due to the fact such functions are called from Client which knows nothing about
* access rights and settings. Moreover, the only use case for displaying secrets are backups,
* and backup tools use only direct input and ignore logs and error messages.
*/
String formatForLogging(size_t max_length = 0) const
{
return formatWithPossiblyHidingSensitiveData(max_length, true, false);
}
String formatForErrorMessage() const
{
return formatWithPossiblyHidingSensitiveData(0, true, false);
}
/// If an AST has secret parts then formatForLogging() will replace them with the placeholder '[HIDDEN]'.
virtual bool hasSecretParts() const { return childrenHaveSecretParts(); }
void cloneChildren();

View File

@ -244,7 +244,9 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
auto default_function = std::make_shared<ASTFunction>();
default_function->name = "defaultValueOfTypeName";
default_function->arguments = std::make_shared<ASTExpressionList>();
default_function->arguments->children.emplace_back(std::make_shared<ASTLiteral>(type->as<ASTFunction>()->formatWithSecretsHidden()));
// Ephemeral columns don't really have secrets but we need to format
// into a String, hence the strange call
default_function->arguments->children.emplace_back(std::make_shared<ASTLiteral>(type->as<ASTFunction>()->formatForLogging()));
default_expression = default_function;
}

View File

@ -2,6 +2,7 @@
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeUUID.h>
#include <Interpreters/Context.h>
#include <Interpreters/formatWithPossiblyHidingSecrets.h>
#include <Access/ContextAccess.h>
#include <Storages/System/StorageSystemDatabases.h>
#include <Parsers/ASTCreateQuery.h>
@ -31,7 +32,7 @@ NamesAndAliases StorageSystemDatabases::getNamesAndAliases()
};
}
static String getEngineFull(const DatabasePtr & database)
static String getEngineFull(const ContextPtr & ctx, const DatabasePtr & database)
{
DDLGuardPtr guard;
while (true)
@ -59,7 +60,7 @@ static String getEngineFull(const DatabasePtr & database)
if (!ast_create || !ast_create->storage)
return {};
String engine_full = ast_create->storage->formatWithSecretsHidden();
String engine_full = format({ctx, *ast_create->storage});
static const char * const extra_head = " ENGINE = ";
if (startsWith(engine_full, extra_head))
@ -87,7 +88,7 @@ void StorageSystemDatabases::fillData(MutableColumns & res_columns, ContextPtr c
res_columns[2]->insert(context->getPath() + database->getDataPath());
res_columns[3]->insert(database->getMetadataPath());
res_columns[4]->insert(database->getUUID());
res_columns[5]->insert(getEngineFull(database));
res_columns[5]->insert(getEngineFull(context, database));
res_columns[6]->insert(database->getDatabaseComment());
}
}

View File

@ -9,6 +9,7 @@
#include <Databases/IDatabase.h>
#include <Access/ContextAccess.h>
#include <Interpreters/Context.h>
#include <Interpreters/formatWithPossiblyHidingSecrets.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Common/typeid_cast.h>
@ -220,7 +221,7 @@ protected:
{
auto temp_db = DatabaseCatalog::instance().getDatabaseForTemporaryTables();
ASTPtr ast = temp_db ? temp_db->tryGetCreateTableQuery(table.second->getStorageID().getTableName(), context) : nullptr;
res_columns[res_index++]->insert(ast ? ast->formatWithSecretsHidden() : "");
res_columns[res_index++]->insert(ast ? format({context, *ast}) : "");
}
// engine_full
@ -366,7 +367,7 @@ protected:
}
if (columns_mask[src_index++])
res_columns[res_index++]->insert(ast ? ast->formatWithSecretsHidden() : "");
res_columns[res_index++]->insert(ast ? format({context, *ast}) : "");
if (columns_mask[src_index++])
{
@ -374,7 +375,7 @@ protected:
if (ast_create && ast_create->storage)
{
engine_full = ast_create->storage->formatWithSecretsHidden();
engine_full = format({context, *ast_create->storage});
static const char * const extra_head = " ENGINE = ";
if (startsWith(engine_full, extra_head))
@ -388,7 +389,7 @@ protected:
{
String as_select;
if (ast_create && ast_create->select)
as_select = ast_create->select->formatWithSecretsHidden();
as_select = format({context, *ast_create->select});
res_columns[res_index++]->insert(as_select);
}
}
@ -401,7 +402,7 @@ protected:
if (columns_mask[src_index++])
{
if (metadata_snapshot && (expression_ptr = metadata_snapshot->getPartitionKeyAST()))
res_columns[res_index++]->insert(expression_ptr->formatWithSecretsHidden());
res_columns[res_index++]->insert(format({context, *expression_ptr}));
else
res_columns[res_index++]->insertDefault();
}
@ -409,7 +410,7 @@ protected:
if (columns_mask[src_index++])
{
if (metadata_snapshot && (expression_ptr = metadata_snapshot->getSortingKey().expression_list_ast))
res_columns[res_index++]->insert(expression_ptr->formatWithSecretsHidden());
res_columns[res_index++]->insert(format({context, *expression_ptr}));
else
res_columns[res_index++]->insertDefault();
}
@ -417,7 +418,7 @@ protected:
if (columns_mask[src_index++])
{
if (metadata_snapshot && (expression_ptr = metadata_snapshot->getPrimaryKey().expression_list_ast))
res_columns[res_index++]->insert(expression_ptr->formatWithSecretsHidden());
res_columns[res_index++]->insert(format({context, *expression_ptr}));
else
res_columns[res_index++]->insertDefault();
}
@ -425,7 +426,7 @@ protected:
if (columns_mask[src_index++])
{
if (metadata_snapshot && (expression_ptr = metadata_snapshot->getSamplingKeyAST()))
res_columns[res_index++]->insert(expression_ptr->formatWithSecretsHidden());
res_columns[res_index++]->insert(format({context, *expression_ptr}));
else
res_columns[res_index++]->insertDefault();
}

View File

@ -2,11 +2,11 @@
from base64 import b64decode
from collections import namedtuple
from typing import Any, Dict, List
from typing import Any, Dict, List, Optional, Tuple
from threading import Thread
from queue import Queue
import json
import os
import re
import time
import jwt
@ -27,6 +27,123 @@ MAX_RETRY = 5
DEBUG_INFO = {} # type: Dict[str, Any]
# Descriptions are used in .github/PULL_REQUEST_TEMPLATE.md, keep comments there
# updated accordingly
# The following lists are append only, try to avoid editing them
# They still could be cleaned out after the decent time though.
LABELS = {
"pr-backward-incompatible": ["Backward Incompatible Change"],
"pr-bugfix": [
"Bug Fix",
"Bug Fix (user-visible misbehavior in an official stable release)",
"Bug Fix (user-visible misbehaviour in official stable or prestable release)",
"Bug Fix (user-visible misbehavior in official stable or prestable release)",
],
"pr-build": [
"Build/Testing/Packaging Improvement",
"Build Improvement",
"Build/Testing Improvement",
"Build",
"Packaging Improvement",
],
"pr-documentation": [
"Documentation (changelog entry is not required)",
"Documentation",
],
"pr-feature": ["New Feature"],
"pr-improvement": ["Improvement"],
"pr-not-for-changelog": [
"Not for changelog (changelog entry is not required)",
"Not for changelog",
],
"pr-performance": ["Performance Improvement"],
}
CATEGORY_TO_LABEL = {c: lb for lb, categories in LABELS.items() for c in categories}
def check_pr_description(pr_body: str) -> Tuple[str, str]:
"""The function checks the body to being properly formatted according to
.github/PULL_REQUEST_TEMPLATE.md, if the first returned string is not empty,
then there is an error."""
lines = list(map(lambda x: x.strip(), pr_body.split("\n") if pr_body else []))
lines = [re.sub(r"\s+", " ", line) for line in lines]
# Check if body contains "Reverts ClickHouse/ClickHouse#36337"
if [
True
for line in lines
if re.match(r"\AReverts {GITHUB_REPOSITORY}#[\d]+\Z", line)
]:
return "", LABELS["pr-not-for-changelog"][0]
category = ""
entry = ""
description_error = ""
i = 0
while i < len(lines):
if re.match(r"(?i)^[#>*_ ]*change\s*log\s*category", lines[i]):
i += 1
if i >= len(lines):
break
# Can have one empty line between header and the category
# itself. Filter it out.
if not lines[i]:
i += 1
if i >= len(lines):
break
category = re.sub(r"^[-*\s]*", "", lines[i])
i += 1
# Should not have more than one category. Require empty line
# after the first found category.
if i >= len(lines):
break
if lines[i]:
second_category = re.sub(r"^[-*\s]*", "", lines[i])
description_error = (
"More than one changelog category specified: "
f"'{category}', '{second_category}'"
)
return description_error, category
elif re.match(
r"(?i)^[#>*_ ]*(short\s*description|change\s*log\s*entry)", lines[i]
):
i += 1
# Can have one empty line between header and the entry itself.
# Filter it out.
if i < len(lines) and not lines[i]:
i += 1
# All following lines until empty one are the changelog entry.
entry_lines = []
while i < len(lines) and lines[i]:
entry_lines.append(lines[i])
i += 1
entry = " ".join(entry_lines)
# Don't accept changelog entries like '...'.
entry = re.sub(r"[#>*_.\- ]", "", entry)
# Don't accept changelog entries like 'Close #12345'.
entry = re.sub(r"^[\w\-\s]{0,10}#?\d{5,6}\.?$", "", entry)
else:
i += 1
if not category:
description_error = "Changelog category is empty"
# Filter out the PR categories that are not for changelog.
elif re.match(
r"(?i)doc|((non|in|not|un)[-\s]*significant)|(not[ ]*for[ ]*changelog)",
category,
):
pass # to not check the rest of the conditions
elif category not in CATEGORY_TO_LABEL:
description_error, category = f"Category '{category}' is not valid", ""
elif not entry:
description_error = f"Changelog entry required for category '{category}'"
return description_error, category
class Worker(Thread):
def __init__(
@ -268,11 +385,11 @@ def get_workflow_description(workflow_url: str, token: str) -> WorkflowDescripti
)
def _exec_post_with_retry(url, token):
def _exec_post_with_retry(url: str, token: str, json: Optional[Any] = None) -> Any:
headers = {"Authorization": f"token {token}"}
for i in range(MAX_RETRY):
try:
response = requests.post(url, headers=headers)
response = requests.post(url, headers=headers, json=json)
response.raise_for_status()
return response.json()
except Exception as ex:
@ -373,27 +490,20 @@ def main(event):
return
if action == "edited":
print("PR is edited, check if it needs to rerun")
workflow_descriptions = get_workflows_description_for_pull_request(
pull_request, token
)
workflow_descriptions = (
workflow_descriptions
or get_workflow_description_fallback(pull_request, token)
)
workflow_descriptions.sort(key=lambda x: x.run_id) # type: ignore
most_recent_workflow = workflow_descriptions[-1]
if (
most_recent_workflow.status == "completed"
and most_recent_workflow.name in NEED_RERUN_ON_EDITED
):
print("PR is edited, check if the body is correct")
error, category = check_pr_description(pull_request["body"])
if error:
print(
"The PR's body is changed and workflow is finished. "
"Rerun to check the description"
f"The PR's body is wrong, is going to comment it. The error is: {error}"
)
exec_workflow_url([most_recent_workflow.rerun_url], token)
print("Rerun finished, exiting")
return
post_json = {
"body": "This is an automatic comment. The PR descriptions does not "
f"match the [template]({pull_request['base']['repo']['html_url']}/"
"blob/master/.github/PULL_REQUEST_TEMPLATE.md?plain=1).\n\n"
f"Please, edit it accordingly.\n\nThe error is: {error}"
}
_exec_post_with_retry(pull_request["comments_url"], token, json=post_json)
return
if action == "synchronize":
print("PR is synchronized, going to stop old actions")

View File

@ -1,7 +1,6 @@
#!/usr/bin/env python3
import sys
import logging
import re
from typing import Tuple
from github import Github
@ -21,6 +20,8 @@ from docs_check import NAME as DOCS_NAME
from env_helper import GITHUB_REPOSITORY, GITHUB_SERVER_URL
from get_robot_token import get_best_robot_token
from pr_info import FORCE_TESTS_LABEL, PRInfo
from cancel_and_rerun_workflow_lambda.app import CATEGORY_TO_LABEL, check_pr_description
from workflow_approve_rerun_lambda.app import TRUSTED_CONTRIBUTORS
TRUSTED_ORG_IDS = {
@ -33,40 +34,6 @@ DO_NOT_TEST_LABEL = "do not test"
FEATURE_LABEL = "pr-feature"
SUBMODULE_CHANGED_LABEL = "submodule changed"
# They are used in .github/PULL_REQUEST_TEMPLATE.md, keep comments there
# updated accordingly
# The following lists are append only, try to avoid editing them
# They atill could be cleaned out after the decent time though.
LABELS = {
"pr-backward-incompatible": ["Backward Incompatible Change"],
"pr-bugfix": [
"Bug Fix",
"Bug Fix (user-visible misbehavior in an official stable release)",
"Bug Fix (user-visible misbehaviour in official stable or prestable release)",
"Bug Fix (user-visible misbehavior in official stable or prestable release)",
],
"pr-build": [
"Build/Testing/Packaging Improvement",
"Build Improvement",
"Build/Testing Improvement",
"Build",
"Packaging Improvement",
],
"pr-documentation": [
"Documentation (changelog entry is not required)",
"Documentation",
],
"pr-feature": ["New Feature"],
"pr-improvement": ["Improvement"],
"pr-not-for-changelog": [
"Not for changelog (changelog entry is not required)",
"Not for changelog",
],
"pr-performance": ["Performance Improvement"],
}
CATEGORY_TO_LABEL = {c: lb for lb, categories in LABELS.items() for c in categories}
def pr_is_by_trusted_user(pr_user_login, pr_user_orgs):
if pr_user_login.lower() in TRUSTED_CONTRIBUTORS:
@ -120,91 +87,6 @@ def should_run_ci_for_pr(pr_info: PRInfo) -> Tuple[bool, str, str]:
return True, "No special conditions apply", "pending"
def check_pr_description(pr_info: PRInfo) -> Tuple[str, str]:
lines = list(
map(lambda x: x.strip(), pr_info.body.split("\n") if pr_info.body else [])
)
lines = [re.sub(r"\s+", " ", line) for line in lines]
# Check if body contains "Reverts ClickHouse/ClickHouse#36337"
if [
True
for line in lines
if re.match(rf"\AReverts {GITHUB_REPOSITORY}#[\d]+\Z", line)
]:
return "", LABELS["pr-not-for-changelog"][0]
category = ""
entry = ""
description_error = ""
i = 0
while i < len(lines):
if re.match(r"(?i)^[#>*_ ]*change\s*log\s*category", lines[i]):
i += 1
if i >= len(lines):
break
# Can have one empty line between header and the category
# itself. Filter it out.
if not lines[i]:
i += 1
if i >= len(lines):
break
category = re.sub(r"^[-*\s]*", "", lines[i])
i += 1
# Should not have more than one category. Require empty line
# after the first found category.
if i >= len(lines):
break
if lines[i]:
second_category = re.sub(r"^[-*\s]*", "", lines[i])
result_status = (
"More than one changelog category specified: '"
+ category
+ "', '"
+ second_category
+ "'"
)
return result_status, category
elif re.match(
r"(?i)^[#>*_ ]*(short\s*description|change\s*log\s*entry)", lines[i]
):
i += 1
# Can have one empty line between header and the entry itself.
# Filter it out.
if i < len(lines) and not lines[i]:
i += 1
# All following lines until empty one are the changelog entry.
entry_lines = []
while i < len(lines) and lines[i]:
entry_lines.append(lines[i])
i += 1
entry = " ".join(entry_lines)
# Don't accept changelog entries like '...'.
entry = re.sub(r"[#>*_.\- ]", "", entry)
# Don't accept changelog entries like 'Close #12345'.
entry = re.sub(r"^[\w\-\s]{0,10}#?\d{5,6}\.?$", "", entry)
else:
i += 1
if not category:
description_error = "Changelog category is empty"
# Filter out the PR categories that are not for changelog.
elif re.match(
r"(?i)doc|((non|in|not|un)[-\s]*significant)|(not[ ]*for[ ]*changelog)",
category,
):
pass # to not check the rest of the conditions
elif category not in CATEGORY_TO_LABEL:
description_error, category = f"Category '{category}' is not valid", ""
elif not entry:
description_error = f"Changelog entry required for category '{category}'"
return description_error, category
def main():
logging.basicConfig(level=logging.INFO)
@ -224,7 +106,7 @@ def main():
gh = Github(get_best_robot_token(), per_page=100)
commit = get_commit(gh, pr_info.sha)
description_error, category = check_pr_description(pr_info)
description_error, category = check_pr_description(pr_info.body)
pr_labels_to_add = []
pr_labels_to_remove = []
if (
@ -262,7 +144,7 @@ def main():
f"expect adding docs for {FEATURE_LABEL}",
DOCS_NAME,
)
else:
elif not description_error:
set_mergeable_check(commit, "skipped")
if description_error:

View File

@ -1,4 +1,5 @@
<clickhouse>
<display_secrets_in_show_and_select>1</display_secrets_in_show_and_select>
<named_collections>
<named_collection_1/>
<named_collection_2/>

View File

@ -65,14 +65,15 @@ def system_query_log_contains_search_pattern(search_pattern):
)
# Generates a random string.
def new_password(len=16):
return "".join(
random.choice(string.ascii_uppercase + string.digits) for _ in range(len)
)
# Passwords in CREATE/ALTER queries must be hidden in logs.
show_secrets = "SETTINGS format_display_secrets_in_show_and_select"
def test_create_alter_user():
password = new_password()
@ -107,9 +108,25 @@ def test_create_alter_user():
],
)
assert "BY" in node.query(f"SHOW CREATE USER u1 {show_secrets}=1")
assert "BY" in node.query(f"SHOW CREATE USER u2 {show_secrets}=1")
node.query("DROP USER u1, u2")
def check_secrets_for_tables(tables, table_name_prefix, password):
for i, table in enumerate(tables):
table_name = table_name_prefix + str(i)
if password in table:
assert password in node.query(
f"SHOW CREATE TABLE {table_name} {show_secrets}=1"
)
assert password in node.query(
f"SELECT create_table_query, engine_full FROM system.tables WHERE name = '{table_name}' "
f"{show_secrets}=1"
)
def test_create_table():
password = new_password()
@ -133,21 +150,25 @@ def test_create_table():
for i, table_engine in enumerate(table_engines):
node.query(f"CREATE TABLE table{i} (x int) ENGINE = {table_engine}")
assert (
node.query("SHOW CREATE TABLE table0")
== "CREATE TABLE default.table0\\n(\\n `x` Int32\\n)\\nENGINE = MySQL(\\'mysql57:3306\\', \\'mysql_db\\', \\'mysql_table\\', \\'mysql_user\\', \\'[HIDDEN]\\')\n"
)
for toggle, secret in enumerate(["[HIDDEN]", password]):
assert (
node.query(f"SHOW CREATE TABLE table0 {show_secrets}={toggle}")
== "CREATE TABLE default.table0\\n(\\n `x` Int32\\n)\\n"
"ENGINE = MySQL(\\'mysql57:3306\\', \\'mysql_db\\', "
f"\\'mysql_table\\', \\'mysql_user\\', \\'{secret}\\')\n"
)
assert node.query(
"SELECT create_table_query, engine_full FROM system.tables WHERE name = 'table0'"
) == TSV(
[
assert node.query(
f"SELECT create_table_query, engine_full FROM system.tables WHERE name = 'table0' {show_secrets}={toggle}"
) == TSV(
[
"CREATE TABLE default.table0 (`x` Int32) ENGINE = MySQL(\\'mysql57:3306\\', \\'mysql_db\\', \\'mysql_table\\', \\'mysql_user\\', \\'[HIDDEN]\\')",
"MySQL(\\'mysql57:3306\\', \\'mysql_db\\', \\'mysql_table\\', \\'mysql_user\\', \\'[HIDDEN]\\')",
],
]
)
[
"CREATE TABLE default.table0 (`x` Int32) ENGINE = MySQL(\\'mysql57:3306\\', \\'mysql_db\\', "
f"\\'mysql_table\\', \\'mysql_user\\', \\'{secret}\\')",
f"MySQL(\\'mysql57:3306\\', \\'mysql_db\\', \\'mysql_table\\', \\'mysql_user\\', \\'{secret}\\')",
],
]
)
check_logs(
must_contain=[
@ -169,7 +190,9 @@ def test_create_table():
must_not_contain=[password],
)
for i in range(0, len(table_engines)):
check_secrets_for_tables(table_engines, "table", password)
for i in range(len(table_engines)):
node.query(f"DROP TABLE table{i}")
@ -198,7 +221,7 @@ def test_create_database():
must_not_contain=[password],
)
for i in range(0, len(database_engines)):
for i in range(len(database_engines)):
node.query(f"DROP DATABASE IF EXISTS database{i}")
@ -241,21 +264,26 @@ def test_table_functions():
for i, table_function in enumerate(table_functions):
node.query(f"CREATE TABLE tablefunc{i} (x int) AS {table_function}")
assert (
node.query("SHOW CREATE TABLE tablefunc0")
== "CREATE TABLE default.tablefunc0\\n(\\n `x` Int32\\n) AS mysql(\\'mysql57:3306\\', \\'mysql_db\\', \\'mysql_table\\', \\'mysql_user\\', \\'[HIDDEN]\\')\n"
)
for toggle, secret in enumerate(["[HIDDEN]", password]):
assert (
node.query(f"SHOW CREATE TABLE tablefunc0 {show_secrets}={toggle}")
== "CREATE TABLE default.tablefunc0\\n(\\n `x` Int32\\n) AS "
"mysql(\\'mysql57:3306\\', \\'mysql_db\\', \\'mysql_table\\', "
f"\\'mysql_user\\', \\'{secret}\\')\n"
)
assert node.query(
"SELECT create_table_query, engine_full FROM system.tables WHERE name = 'tablefunc0'"
) == TSV(
[
assert node.query(
"SELECT create_table_query, engine_full FROM system.tables WHERE name = 'tablefunc0' "
f"{show_secrets}={toggle}"
) == TSV(
[
"CREATE TABLE default.tablefunc0 (`x` Int32) AS mysql(\\'mysql57:3306\\', \\'mysql_db\\', \\'mysql_table\\', \\'mysql_user\\', \\'[HIDDEN]\\')",
"",
],
]
)
[
"CREATE TABLE default.tablefunc0 (`x` Int32) AS mysql(\\'mysql57:3306\\', "
f"\\'mysql_db\\', \\'mysql_table\\', \\'mysql_user\\', \\'{secret}\\')",
"",
],
]
)
check_logs(
must_contain=[
@ -293,7 +321,9 @@ def test_table_functions():
must_not_contain=[password],
)
for i in range(0, len(table_functions)):
check_secrets_for_tables(table_functions, "tablefunc", password)
for i in range(len(table_functions)):
node.query(f"DROP TABLE tablefunc{i}")
@ -369,15 +399,18 @@ def test_create_dictionary():
f"LIFETIME(MIN 0 MAX 10) LAYOUT(FLAT())"
)
assert (
node.query("SHOW CREATE TABLE dict1")
== "CREATE DICTIONARY default.dict1\\n(\\n `n` int DEFAULT 0,\\n `m` int DEFAULT 1\\n)\\nPRIMARY KEY n\\nSOURCE(CLICKHOUSE(HOST \\'localhost\\' PORT 9000 USER \\'user1\\' TABLE \\'test\\' PASSWORD \\'[HIDDEN]\\' DB \\'default\\'))\\nLIFETIME(MIN 0 MAX 10)\\nLAYOUT(FLAT())\n"
)
for toggle, secret in enumerate(["[HIDDEN]", password]):
assert (
node.query(f"SHOW CREATE TABLE dict1 {show_secrets}={toggle}")
== f"CREATE DICTIONARY default.dict1\\n(\\n `n` int DEFAULT 0,\\n `m` int DEFAULT 1\\n)\\nPRIMARY KEY n\\nSOURCE(CLICKHOUSE(HOST \\'localhost\\' PORT 9000 USER \\'user1\\' TABLE \\'test\\' PASSWORD \\'{secret}\\' DB \\'default\\'))\\nLIFETIME(MIN 0 MAX 10)\\nLAYOUT(FLAT())\n"
)
assert (
node.query("SELECT create_table_query FROM system.tables WHERE name = 'dict1'")
== "CREATE DICTIONARY default.dict1 (`n` int DEFAULT 0, `m` int DEFAULT 1) PRIMARY KEY n SOURCE(CLICKHOUSE(HOST \\'localhost\\' PORT 9000 USER \\'user1\\' TABLE \\'test\\' PASSWORD \\'[HIDDEN]\\' DB \\'default\\')) LIFETIME(MIN 0 MAX 10) LAYOUT(FLAT())\n"
)
assert (
node.query(
f"SELECT create_table_query FROM system.tables WHERE name = 'dict1' {show_secrets}={toggle}"
)
== f"CREATE DICTIONARY default.dict1 (`n` int DEFAULT 0, `m` int DEFAULT 1) PRIMARY KEY n SOURCE(CLICKHOUSE(HOST \\'localhost\\' PORT 9000 USER \\'user1\\' TABLE \\'test\\' PASSWORD \\'{secret}\\' DB \\'default\\')) LIFETIME(MIN 0 MAX 10) LAYOUT(FLAT())\n"
)
check_logs(
must_contain=[
@ -448,4 +481,4 @@ def test_on_cluster():
"%CREATE TABLE default.table_oncl UUID \\'%\\' (`x` Int32) ENGINE = MySQL(\\'mysql57:3307\\', \\'mysql_db\\', \\'mysql_table\\', \\'mysql_user\\', \\'[HIDDEN]\\')"
)
node.query(f"DROP TABLE table_oncl")
node.query("DROP TABLE table_oncl")

View File

@ -138,6 +138,7 @@ SYSTEM THREAD FUZZER ['SYSTEM START THREAD FUZZER','SYSTEM STOP THREAD FUZZER','
SYSTEM UNFREEZE ['SYSTEM UNFREEZE'] GLOBAL SYSTEM
SYSTEM [] \N ALL
dictGet ['dictHas','dictGetHierarchy','dictIsIn'] DICTIONARY ALL
displaySecretsInShowAndSelect [] GLOBAL ALL
addressToLine [] GLOBAL INTROSPECTION
addressToLineWithInlines [] GLOBAL INTROSPECTION
addressToSymbol [] GLOBAL INTROSPECTION

View File

@ -297,7 +297,7 @@ CREATE TABLE system.grants
(
`user_name` Nullable(String),
`role_name` Nullable(String),
`access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH' = 135, 'SYSTEM THREAD FUZZER' = 136, 'SYSTEM UNFREEZE' = 137, 'SYSTEM' = 138, 'dictGet' = 139, 'addressToLine' = 140, 'addressToLineWithInlines' = 141, 'addressToSymbol' = 142, 'demangle' = 143, 'INTROSPECTION' = 144, 'FILE' = 145, 'URL' = 146, 'REMOTE' = 147, 'MONGO' = 148, 'MEILISEARCH' = 149, 'MYSQL' = 150, 'POSTGRES' = 151, 'SQLITE' = 152, 'ODBC' = 153, 'JDBC' = 154, 'HDFS' = 155, 'S3' = 156, 'HIVE' = 157, 'SOURCES' = 158, 'CLUSTER' = 159, 'ALL' = 160, 'NONE' = 161),
`access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH' = 135, 'SYSTEM THREAD FUZZER' = 136, 'SYSTEM UNFREEZE' = 137, 'SYSTEM' = 138, 'dictGet' = 139, 'displaySecretsInShowAndSelect' = 140, 'addressToLine' = 141, 'addressToLineWithInlines' = 142, 'addressToSymbol' = 143, 'demangle' = 144, 'INTROSPECTION' = 145, 'FILE' = 146, 'URL' = 147, 'REMOTE' = 148, 'MONGO' = 149, 'MEILISEARCH' = 150, 'MYSQL' = 151, 'POSTGRES' = 152, 'SQLITE' = 153, 'ODBC' = 154, 'JDBC' = 155, 'HDFS' = 156, 'S3' = 157, 'HIVE' = 158, 'SOURCES' = 159, 'CLUSTER' = 160, 'ALL' = 161, 'NONE' = 162),
`database` Nullable(String),
`table` Nullable(String),
`column` Nullable(String),
@ -581,10 +581,10 @@ ENGINE = SystemPartsColumns
COMMENT 'SYSTEM TABLE is built on the fly.'
CREATE TABLE system.privileges
(
`privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH' = 135, 'SYSTEM THREAD FUZZER' = 136, 'SYSTEM UNFREEZE' = 137, 'SYSTEM' = 138, 'dictGet' = 139, 'addressToLine' = 140, 'addressToLineWithInlines' = 141, 'addressToSymbol' = 142, 'demangle' = 143, 'INTROSPECTION' = 144, 'FILE' = 145, 'URL' = 146, 'REMOTE' = 147, 'MONGO' = 148, 'MEILISEARCH' = 149, 'MYSQL' = 150, 'POSTGRES' = 151, 'SQLITE' = 152, 'ODBC' = 153, 'JDBC' = 154, 'HDFS' = 155, 'S3' = 156, 'HIVE' = 157, 'SOURCES' = 158, 'CLUSTER' = 159, 'ALL' = 160, 'NONE' = 161),
`privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH' = 135, 'SYSTEM THREAD FUZZER' = 136, 'SYSTEM UNFREEZE' = 137, 'SYSTEM' = 138, 'dictGet' = 139, 'displaySecretsInShowAndSelect' = 140, 'addressToLine' = 141, 'addressToLineWithInlines' = 142, 'addressToSymbol' = 143, 'demangle' = 144, 'INTROSPECTION' = 145, 'FILE' = 146, 'URL' = 147, 'REMOTE' = 148, 'MONGO' = 149, 'MEILISEARCH' = 150, 'MYSQL' = 151, 'POSTGRES' = 152, 'SQLITE' = 153, 'ODBC' = 154, 'JDBC' = 155, 'HDFS' = 156, 'S3' = 157, 'HIVE' = 158, 'SOURCES' = 159, 'CLUSTER' = 160, 'ALL' = 161, 'NONE' = 162),
`aliases` Array(String),
`level` Nullable(Enum8('GLOBAL' = 0, 'DATABASE' = 1, 'TABLE' = 2, 'DICTIONARY' = 3, 'VIEW' = 4, 'COLUMN' = 5, 'NAMED_COLLECTION' = 6)),
`parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH' = 135, 'SYSTEM THREAD FUZZER' = 136, 'SYSTEM UNFREEZE' = 137, 'SYSTEM' = 138, 'dictGet' = 139, 'addressToLine' = 140, 'addressToLineWithInlines' = 141, 'addressToSymbol' = 142, 'demangle' = 143, 'INTROSPECTION' = 144, 'FILE' = 145, 'URL' = 146, 'REMOTE' = 147, 'MONGO' = 148, 'MEILISEARCH' = 149, 'MYSQL' = 150, 'POSTGRES' = 151, 'SQLITE' = 152, 'ODBC' = 153, 'JDBC' = 154, 'HDFS' = 155, 'S3' = 156, 'HIVE' = 157, 'SOURCES' = 158, 'CLUSTER' = 159, 'ALL' = 160, 'NONE' = 161))
`parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH' = 135, 'SYSTEM THREAD FUZZER' = 136, 'SYSTEM UNFREEZE' = 137, 'SYSTEM' = 138, 'dictGet' = 139, 'displaySecretsInShowAndSelect' = 140, 'addressToLine' = 141, 'addressToLineWithInlines' = 142, 'addressToSymbol' = 143, 'demangle' = 144, 'INTROSPECTION' = 145, 'FILE' = 146, 'URL' = 147, 'REMOTE' = 148, 'MONGO' = 149, 'MEILISEARCH' = 150, 'MYSQL' = 151, 'POSTGRES' = 152, 'SQLITE' = 153, 'ODBC' = 154, 'JDBC' = 155, 'HDFS' = 156, 'S3' = 157, 'HIVE' = 158, 'SOURCES' = 159, 'CLUSTER' = 160, 'ALL' = 161, 'NONE' = 162))
)
ENGINE = SystemPrivileges
COMMENT 'SYSTEM TABLE is built on the fly.'