Merge branch 'master' into pamarcos/system-unload-primary-key

This commit is contained in:
Pablo Marcos 2024-04-18 17:53:11 +02:00
commit c28af20e2a
34 changed files with 237 additions and 61 deletions

View File

@ -97,7 +97,7 @@ jobs:
# for main CI chain
#
Builds_1:
needs: [RunConfig, FastTest]
needs: [RunConfig, StyleCheck, FastTest]
if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).stages_data.stages_to_do, 'Builds_1') }}
# using callable wf (reusable_stage.yml) allows to group all nested jobs under a tab
uses: ./.github/workflows/reusable_build_stage.yml

2
contrib/azure vendored

@ -1 +1 @@
Subproject commit ad2d3d423565b8a8e7b7ec863eae9318a8283878
Subproject commit b90fd3c6ef3185f5be3408056567bca0854129b6

View File

@ -208,7 +208,7 @@ translateUTF8(s, from, to)
**Returned value**
- `s`: A string type [String](/docs/en/sql-reference/data-types/string.md).
- A [String](/docs/en/sql-reference/data-types/string.md) data type value.
**Examples**

View File

@ -209,6 +209,7 @@ Hierarchy of privileges:
- `SHOW NAMED COLLECTIONS`
- `SHOW NAMED COLLECTIONS SECRETS`
- `NAMED COLLECTION`
- [TABLE ENGINE](#grant-table-engine)
Examples of how this hierarchy is treated:
@ -505,6 +506,7 @@ and
[`format_display_secrets_in_show_and_select` format setting](../../operations/settings/formats#format_display_secrets_in_show_and_select)
are turned on.
### NAMED COLLECTION ADMIN
Allows a certain operation on a specified named collection. Before version 23.7 it was called NAMED COLLECTION CONTROL, and after 23.7 NAMED COLLECTION ADMIN was added and NAMED COLLECTION CONTROL is preserved as an alias.
@ -524,6 +526,17 @@ Unlike all other grants (CREATE, DROP, ALTER, SHOW) grant NAMED COLLECTION was a
Assuming a named collection is called abc, we grant privilege CREATE NAMED COLLECTION to user john.
- `GRANT CREATE NAMED COLLECTION ON abc TO john`
### TABLE ENGINE
Allows using a specified table engine when creating a table. Applies to [table engines](../../engines/table-engines/index.md).
**Examples**
- `GRANT TABLE ENGINE ON * TO john`
- `GRANT TABLE ENGINE ON TinyLog TO john`
### ALL
Grants all the privileges on regulated entity to a user account or a role.

View File

@ -742,6 +742,10 @@
It also enables 'changeable_in_readonly' constraint type -->
<settings_constraints_replace_previous>true</settings_constraints_replace_previous>
<!-- By default, for backward compatibility create table with a specific table engine ignores grant,
however you can change this behaviour by setting this to true -->
<table_engines_require_grant>true</table_engines_require_grant>
<!-- Number of seconds since last access a role is stored in the Role Cache -->
<role_cache_expiration_time_seconds>600</role_cache_expiration_time_seconds>
</access_control_improvements>

View File

@ -285,6 +285,7 @@ void AccessControl::setUpFromMainConfig(const Poco::Util::AbstractConfiguration
setSelectFromSystemDatabaseRequiresGrant(config_.getBool("access_control_improvements.select_from_system_db_requires_grant", false));
setSelectFromInformationSchemaRequiresGrant(config_.getBool("access_control_improvements.select_from_information_schema_requires_grant", false));
setSettingsConstraintsReplacePrevious(config_.getBool("access_control_improvements.settings_constraints_replace_previous", false));
setTableEnginesRequireGrant(config_.getBool("access_control_improvements.table_engines_require_grant", false));
addStoragesFromMainConfig(config_, config_path_, get_zookeeper_function_);

View File

@ -182,6 +182,9 @@ public:
void setSettingsConstraintsReplacePrevious(bool enable) { settings_constraints_replace_previous = enable; }
bool doesSettingsConstraintsReplacePrevious() const { return settings_constraints_replace_previous; }
void setTableEnginesRequireGrant(bool enable) { table_engines_require_grant = enable; }
bool doesTableEnginesRequireGrant() const { return table_engines_require_grant; }
std::shared_ptr<const ContextAccess> getContextAccess(const ContextAccessParams & params) const;
std::shared_ptr<const EnabledRoles> getEnabledRoles(
@ -258,6 +261,7 @@ private:
std::atomic_bool select_from_system_db_requires_grant = false;
std::atomic_bool select_from_information_schema_requires_grant = false;
std::atomic_bool settings_constraints_replace_previous = false;
std::atomic_bool table_engines_require_grant = false;
std::atomic_int bcrypt_workfactor = 12;
std::atomic<AuthenticationType> default_password_type = AuthenticationType::SHA256_PASSWORD;
};

View File

@ -101,8 +101,9 @@ namespace
const Flags & getTableFlags() const { return all_flags_for_target[TABLE]; }
const Flags & getColumnFlags() const { return all_flags_for_target[COLUMN]; }
const Flags & getDictionaryFlags() const { return all_flags_for_target[DICTIONARY]; }
const Flags & getNamedCollectionFlags() const { return all_flags_for_target[NAMED_COLLECTION]; }
const Flags & getTableEngineFlags() const { return all_flags_for_target[TABLE_ENGINE]; }
const Flags & getUserNameFlags() const { return all_flags_for_target[USER_NAME]; }
const Flags & getNamedCollectionFlags() const { return all_flags_for_target[NAMED_COLLECTION]; }
const Flags & getAllFlagsGrantableOnGlobalLevel() const { return getAllFlags(); }
const Flags & getAllFlagsGrantableOnGlobalWithParameterLevel() const { return getGlobalWithParameterFlags(); }
const Flags & getAllFlagsGrantableOnDatabaseLevel() const { return all_flags_grantable_on_database_level; }
@ -122,6 +123,7 @@ namespace
DICTIONARY,
NAMED_COLLECTION,
USER_NAME,
TABLE_ENGINE,
};
struct Node;
@ -301,7 +303,7 @@ namespace
collectAllFlags(child.get());
all_flags_grantable_on_table_level = all_flags_for_target[TABLE] | all_flags_for_target[DICTIONARY] | all_flags_for_target[COLUMN];
all_flags_grantable_on_global_with_parameter_level = all_flags_for_target[NAMED_COLLECTION] | all_flags_for_target[USER_NAME];
all_flags_grantable_on_global_with_parameter_level = all_flags_for_target[NAMED_COLLECTION] | all_flags_for_target[USER_NAME] | all_flags_for_target[TABLE_ENGINE];
all_flags_grantable_on_database_level = all_flags_for_target[DATABASE] | all_flags_grantable_on_table_level;
}
@ -352,7 +354,7 @@ namespace
std::unordered_map<std::string_view, Flags> keyword_to_flags_map;
std::vector<Flags> access_type_to_flags_mapping;
Flags all_flags;
Flags all_flags_for_target[static_cast<size_t>(USER_NAME) + 1];
Flags all_flags_for_target[static_cast<size_t>(TABLE_ENGINE) + 1];
Flags all_flags_grantable_on_database_level;
Flags all_flags_grantable_on_table_level;
Flags all_flags_grantable_on_global_with_parameter_level;
@ -376,7 +378,11 @@ std::unordered_map<AccessFlags::ParameterType, AccessFlags> AccessFlags::splitIn
if (user_flags)
result.emplace(ParameterType::USER_NAME, user_flags);
auto other_flags = (~named_collection_flags & ~user_flags) & *this;
auto table_engine_flags = AccessFlags::allTableEngineFlags() & *this;
if (table_engine_flags)
result.emplace(ParameterType::TABLE_ENGINE, table_engine_flags);
auto other_flags = (~named_collection_flags & ~user_flags & ~table_engine_flags) & *this;
if (other_flags)
result.emplace(ParameterType::NONE, other_flags);
@ -395,6 +401,10 @@ AccessFlags::ParameterType AccessFlags::getParameterType() const
if (AccessFlags::allUserNameFlags().contains(*this))
return AccessFlags::USER_NAME;
/// All flags refer to TABLE ENGINE access type.
if (AccessFlags::allTableEngineFlags().contains(*this))
return AccessFlags::TABLE_ENGINE;
throw Exception(ErrorCodes::MIXED_ACCESS_PARAMETER_TYPES, "Having mixed parameter types: {}", toString());
}
@ -414,6 +424,7 @@ AccessFlags AccessFlags::allColumnFlags() { return Helper::instance().getColumnF
AccessFlags AccessFlags::allDictionaryFlags() { return Helper::instance().getDictionaryFlags(); }
AccessFlags AccessFlags::allNamedCollectionFlags() { return Helper::instance().getNamedCollectionFlags(); }
AccessFlags AccessFlags::allUserNameFlags() { return Helper::instance().getUserNameFlags(); }
AccessFlags AccessFlags::allTableEngineFlags() { return Helper::instance().getTableEngineFlags(); }
AccessFlags AccessFlags::allFlagsGrantableOnGlobalLevel() { return Helper::instance().getAllFlagsGrantableOnGlobalLevel(); }
AccessFlags AccessFlags::allFlagsGrantableOnGlobalWithParameterLevel() { return Helper::instance().getAllFlagsGrantableOnGlobalWithParameterLevel(); }
AccessFlags AccessFlags::allFlagsGrantableOnDatabaseLevel() { return Helper::instance().getAllFlagsGrantableOnDatabaseLevel(); }

View File

@ -56,6 +56,7 @@ public:
enum ParameterType
{
NONE,
TABLE_ENGINE,
NAMED_COLLECTION,
USER_NAME,
};
@ -107,6 +108,9 @@ public:
/// Returns all the flags related to a user.
static AccessFlags allUserNameFlags();
/// Returns all the flags related to a table engine.
static AccessFlags allTableEngineFlags();
/// Returns all the flags which could be granted on the global level.
/// The same as allFlags().
static AccessFlags allFlagsGrantableOnGlobalLevel();

View File

@ -12,7 +12,7 @@ enum class AccessType
/// Macro M should be defined as M(name, aliases, node_type, parent_group_name)
/// where name is identifier with underscores (instead of spaces);
/// aliases is a string containing comma-separated list;
/// node_type either specifies access type's level (GLOBAL/NAMED_COLLECTION/USER_NAME/DATABASE/TABLE/DICTIONARY/VIEW/COLUMNS),
/// node_type either specifies access type's level (GLOBAL/NAMED_COLLECTION/USER_NAME/TABLE_ENGINE/DATABASE/TABLE/DICTIONARY/VIEW/COLUMNS),
/// or specifies that the access type is a GROUP of other access types;
/// parent_group_name is the name of the group containing this access type (or NONE if there is no such group).
/// NOTE A parent group must be declared AFTER all its children.
@ -153,6 +153,8 @@ enum class AccessType
M(NAMED_COLLECTION_ADMIN, "NAMED COLLECTION CONTROL", NAMED_COLLECTION, ALL) \
M(SET_DEFINER, "", USER_NAME, ALL) \
\
M(TABLE_ENGINE, "TABLE ENGINE", TABLE_ENGINE, ALL) \
\
M(SYSTEM_SHUTDOWN, "SYSTEM KILL, SHUTDOWN", GLOBAL, SYSTEM) \
M(SYSTEM_DROP_DNS_CACHE, "SYSTEM DROP DNS, DROP DNS CACHE, DROP DNS", GLOBAL, SYSTEM_DROP_CACHE) \
M(SYSTEM_DROP_CONNECTIONS_CACHE, "SYSTEM DROP CONNECTIONS CACHE, DROP CONNECTIONS CACHE", GLOBAL, SYSTEM_DROP_CACHE) \

View File

@ -204,6 +204,52 @@ namespace
res.grant(AccessType::SELECT, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE);
}
/// There is overlap between AccessType sources and table engines, so the following code avoids user granting twice.
static const std::vector<std::tuple<AccessFlags, std::string>> source_and_table_engines = {
{AccessType::FILE, "File"},
{AccessType::URL, "URL"},
{AccessType::REMOTE, "Distributed"},
{AccessType::MONGO, "MongoDB"},
{AccessType::REDIS, "Redis"},
{AccessType::MYSQL, "MySQL"},
{AccessType::POSTGRES, "PostgreSQL"},
{AccessType::SQLITE, "SQLite"},
{AccessType::ODBC, "ODBC"},
{AccessType::JDBC, "JDBC"},
{AccessType::HDFS, "HDFS"},
{AccessType::S3, "S3"},
{AccessType::HIVE, "Hive"},
{AccessType::AZURE, "AzureBlobStorage"}
};
/// Sync SOURCE and TABLE_ENGINE, so only need to check TABLE_ENGINE later.
if (access_control.doesTableEnginesRequireGrant())
{
for (const auto & source_and_table_engine : source_and_table_engines)
{
const auto & source = std::get<0>(source_and_table_engine);
if (res.isGranted(source))
{
const auto & table_engine = std::get<1>(source_and_table_engine);
res.grant(AccessType::TABLE_ENGINE, table_engine);
}
}
}
else
{
/// Add TABLE_ENGINE on * and then remove TABLE_ENGINE on particular engines.
res.grant(AccessType::TABLE_ENGINE);
for (const auto & source_and_table_engine : source_and_table_engines)
{
const auto & source = std::get<0>(source_and_table_engine);
if (!res.isGranted(source))
{
const auto & table_engine = std::get<1>(source_and_table_engine);
res.revoke(AccessType::TABLE_ENGINE, table_engine);
}
}
}
return res;
}

View File

@ -371,6 +371,7 @@ namespace
if (databases)
{
user->access.revoke(AccessFlags::allFlags() - AccessFlags::allGlobalFlags());
user->access.grantWithGrantOption(AccessType::TABLE_ENGINE);
user->access.grantWithGrantOption(AccessFlags::allDictionaryFlags(), IDictionary::NO_DATABASE_TAG);
for (const String & database : *databases)
user->access.grantWithGrantOption(AccessFlags::allFlags(), database);

View File

@ -732,11 +732,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti
/// We have to check access rights again (in case engine was changed).
if (create.storage && create.storage->engine)
{
auto source_access_type = StorageFactory::instance().getSourceAccessType(create.storage->engine->name);
if (source_access_type != AccessType::NONE)
getContext()->checkAccess(source_access_type);
}
getContext()->checkAccess(AccessType::TABLE_ENGINE, create.storage->engine->name);
TableProperties properties;
TableLockHolder as_storage_lock;
@ -1842,11 +1838,7 @@ AccessRightsElements InterpreterCreateQuery::getRequiredAccess() const
required_access.emplace_back(AccessType::SELECT | AccessType::INSERT, create.to_table_id.database_name, create.to_table_id.table_name);
if (create.storage && create.storage->engine)
{
auto source_access_type = StorageFactory::instance().getSourceAccessType(create.storage->engine->name);
if (source_access_type != AccessType::NONE)
required_access.emplace_back(source_access_type);
}
required_access.emplace_back(AccessType::TABLE_ENGINE, create.storage->engine->name);
return required_access;
}

View File

@ -30,6 +30,7 @@ namespace
COLUMN,
NAMED_COLLECTION,
USER_NAME,
TABLE_ENGINE,
};
DataTypeEnum8::Values getLevelEnumValues()
@ -43,6 +44,7 @@ namespace
enum_values.emplace_back("COLUMN", static_cast<Int8>(COLUMN));
enum_values.emplace_back("NAMED_COLLECTION", static_cast<Int8>(NAMED_COLLECTION));
enum_values.emplace_back("USER_NAME", static_cast<Int8>(USER_NAME));
enum_values.emplace_back("TABLE_ENGINE", static_cast<Int8>(TABLE_ENGINE));
return enum_values;
}
}

View File

@ -50,7 +50,7 @@ def main():
builds_for_check = CI_CONFIG.get_builds_for_report(
build_check_name,
release=pr_info.is_release(),
release=pr_info.is_release,
backport=pr_info.head_ref.startswith("backport/"),
)
required_builds = len(builds_for_check)

View File

@ -17,7 +17,7 @@ from typing import Any, Dict, List, Optional, Sequence, Set, Tuple, Union
import docker_images_helper
import upload_result_helper
from build_check import get_release_or_pr
from ci_config import CI_CONFIG, Build, CIStages, Labels, JobNames
from ci_config import CI_CONFIG, Build, CIStages, JobNames, Labels
from ci_utils import GHActions, is_hex, normalize_string
from clickhouse_helper import (
CiLogsCredentials,
@ -49,7 +49,7 @@ from env_helper import (
from get_robot_token import get_best_robot_token
from git_helper import GIT_PREFIX, Git
from git_helper import Runner as GitRunner
from github import Github
from github_helper import GitHub
from pr_info import PRInfo
from report import ERROR, SUCCESS, BuildResult, JobReport
from s3_helper import S3Helper
@ -770,7 +770,7 @@ class CiOptions:
res = CiOptions()
pr_info = PRInfo()
if (
not pr_info.is_pr() and not debug_message
not pr_info.is_pr and not debug_message
): # if commit_message is provided it's test/debug scenario - do not return
# CI options can be configured in PRs only
# if debug_message is provided - it's a test
@ -1218,19 +1218,19 @@ def _mark_success_action(
if job_config.run_always or job_config.run_by_label:
print(f"Job [{job}] runs always or by label in CI - do not cache")
else:
if pr_info.is_master():
if pr_info.is_master:
pass
# delete method is disabled for ci_cache. need it?
# pending enabled for master branch jobs only
# ci_cache.delete_pending(job, batch, num_batches, release_branch=True)
if job_status and job_status.is_ok():
ci_cache.push_successful(
job, batch, num_batches, job_status, pr_info.is_release_branch()
job, batch, num_batches, job_status, pr_info.is_release_branch
)
print(f"Job [{job}] is ok")
elif job_status and not job_status.is_ok():
ci_cache.push_failed(
job, batch, num_batches, job_status, pr_info.is_release_branch()
job, batch, num_batches, job_status, pr_info.is_release_branch
)
print(f"Job [{job}] is failed with status [{job_status.status}]")
else:
@ -1238,7 +1238,7 @@ def _mark_success_action(
description="dummy description", status=ERROR, report_url="dummy url"
)
ci_cache.push_failed(
job, batch, num_batches, job_status, pr_info.is_release_branch()
job, batch, num_batches, job_status, pr_info.is_release_branch
)
print(f"No CommitStatusData for [{job}], push dummy failure to ci_cache")
@ -1354,9 +1354,9 @@ def _configure_jobs(
batches_to_do: List[int] = []
add_to_skip = False
if job_config.pr_only and pr_info.is_release_branch():
if job_config.pr_only and pr_info.is_release_branch:
continue
if job_config.release_only and not pr_info.is_release_branch():
if job_config.release_only and not pr_info.is_release_branch:
continue
# fill job randomization buckets (for jobs with configured @random_bucket property))
@ -1379,7 +1379,7 @@ def _configure_jobs(
job,
batch,
num_batches,
release_branch=pr_info.is_release_branch()
release_branch=pr_info.is_release_branch
and job_config.required_on_release_branch,
):
# ci cache is enabled and job is not in the cache - add
@ -1390,7 +1390,7 @@ def _configure_jobs(
job,
batch,
num_batches,
release_branch=pr_info.is_release_branch()
release_branch=pr_info.is_release_branch
and job_config.required_on_release_branch,
):
if job in jobs_to_wait:
@ -1413,7 +1413,7 @@ def _configure_jobs(
# treat job as being skipped only if it's controlled by digest
jobs_to_skip.append(job)
if not pr_info.is_release_branch():
if not pr_info.is_release_branch:
# randomization bucket filtering (pick one random job from each bucket, for jobs with configured random_bucket property)
for _, jobs in randomization_buckets.items():
jobs_to_remove_randomization = set()
@ -1435,7 +1435,7 @@ def _configure_jobs(
jobs_to_do, jobs_to_skip, jobs_params
)
if pr_info.is_merge_queue():
if pr_info.is_merge_queue:
# FIXME: Quick support for MQ workflow which is only StyleCheck for now
jobs_to_do = [JobNames.STYLE_CHECK]
jobs_to_skip = []
@ -1504,7 +1504,7 @@ def _update_gh_statuses_action(indata: Dict, s3: S3Helper) -> None:
# create GH status
pr_info = PRInfo()
commit = get_commit(Github(get_best_robot_token(), per_page=100), pr_info.sha)
commit = get_commit(GitHub(get_best_robot_token(), per_page=100), pr_info.sha)
def _concurrent_create_status(job: str, batch: int, num_batches: int) -> None:
job_status = ci_cache.get_successful(job, batch, num_batches)
@ -1551,7 +1551,7 @@ def _fetch_commit_tokens(message: str, pr_info: PRInfo) -> List[str]:
]
print(f"CI modifyers from commit message: [{res}]")
res_2 = []
if pr_info.is_pr():
if pr_info.is_pr:
matches = [match[-1] for match in re.findall(pattern, pr_info.body)]
res_2 = [
match
@ -1626,7 +1626,7 @@ def _upload_build_artifacts(
# Upload head master binaries
static_bin_name = CI_CONFIG.build_config[build_name].static_binary_name
if pr_info.is_master() and static_bin_name:
if pr_info.is_master and static_bin_name:
# Full binary with debug info:
s3_path_full = "/".join((pr_info.base_ref, static_bin_name, "clickhouse-full"))
binary_full = Path(job_report.build_dir_for_upload) / "clickhouse"
@ -1908,11 +1908,11 @@ def main() -> int:
if not args.skip_jobs:
ci_cache = CiCache(s3, jobs_data["digests"])
if pr_info.is_master():
if pr_info.is_master:
# wait for pending jobs to be finished, await_jobs is a long blocking call
# wait pending jobs (for now only on release/master branches)
ready_jobs_batches_dict = ci_cache.await_jobs(
jobs_data.get("jobs_to_wait", {}), pr_info.is_release_branch()
jobs_data.get("jobs_to_wait", {}), pr_info.is_release_branch
)
jobs_to_do = jobs_data["jobs_to_do"]
jobs_to_skip = jobs_data["jobs_to_skip"]
@ -1929,7 +1929,7 @@ def main() -> int:
del jobs_params[job]
# set planned jobs as pending in the CI cache if on the master
if pr_info.is_master():
if pr_info.is_master:
for job in jobs_data["jobs_to_do"]:
config = CI_CONFIG.get_job_config(job)
if config.run_always or config.run_by_label:
@ -1939,7 +1939,7 @@ def main() -> int:
job,
job_params["batches"],
config.num_batches,
release_branch=pr_info.is_release_branch(),
release_branch=pr_info.is_release_branch,
)
if "jobs_to_wait" in jobs_data:
@ -1994,7 +1994,7 @@ def main() -> int:
else:
# this is a test job - check if GH commit status or cache record is present
commit = get_commit(
Github(get_best_robot_token(), per_page=100), pr_info.sha
GitHub(get_best_robot_token(), per_page=100), pr_info.sha
)
# rerun helper check
@ -2110,7 +2110,7 @@ def main() -> int:
additional_urls=additional_urls or None,
)
commit = get_commit(
Github(get_best_robot_token(), per_page=100), pr_info.sha
GitHub(get_best_robot_token(), per_page=100), pr_info.sha
)
post_commit_status(
commit,
@ -2121,7 +2121,7 @@ def main() -> int:
pr_info,
dump_to_file=True,
)
if not pr_info.is_merge_queue():
if not pr_info.is_merge_queue:
# in the merge queue mergeable status must be set only in FinishCheck (last job in wf)
update_mergeable_check(
commit,

View File

@ -149,7 +149,7 @@ def set_status_comment(commit: Commit, pr_info: PRInfo) -> None:
one, so the method does nothing for simple pushes and pull requests with
`release`/`release-lts` labels"""
if pr_info.is_merge_queue():
if pr_info.is_merge_queue:
# skip report creation for the MQ
return
@ -448,7 +448,7 @@ def update_mergeable_check(commit: Commit, pr_info: PRInfo, check_name: str) ->
)
# FIXME: For now, always set mergeable check in the Merge Queue. It's required to pass MQ
if not_run and not pr_info.is_merge_queue():
if not_run and not pr_info.is_merge_queue:
# Let's avoid unnecessary work
return

View File

@ -362,7 +362,7 @@ def main():
del args.image_repo
del args.push
if pr_info.is_master():
if pr_info.is_master:
push = True
image = DockerImageData(image_path, image_repo, False)
@ -374,9 +374,10 @@ def main():
for arch, build_name in zip(ARCH, ("package_release", "package_aarch64")):
if not args.bucket_prefix:
repo_urls[
arch
] = f"{S3_DOWNLOAD}/{S3_BUILDS_BUCKET}/{release_or_pr}/{pr_info.sha}/{build_name}"
repo_urls[arch] = (
f"{S3_DOWNLOAD}/{S3_BUILDS_BUCKET}/"
f"{release_or_pr}/{pr_info.sha}/{build_name}"
)
else:
repo_urls[arch] = f"{args.bucket_prefix}/{build_name}"
if args.allow_build_reuse:

View File

@ -28,7 +28,7 @@ def main():
statuses = get_commit_filtered_statuses(commit)
trigger_mergeable_check(commit, statuses)
if not pr_info.is_merge_queue():
if not pr_info.is_merge_queue:
statuses = [s for s in statuses if s.context == CI_STATUS_NAME]
if not statuses:
return

View File

@ -200,7 +200,7 @@ def main():
# always use latest
docker_image = KEEPER_IMAGE_NAME if args.program == "keeper" else SERVER_IMAGE_NAME
if pr_info.is_scheduled() or pr_info.is_dispatched():
if pr_info.is_scheduled or pr_info.is_dispatched:
# get latest clcikhouse by the static link for latest master buit - get its version and provide permanent url for this version to the jepsen
build_url = f"{S3_URL}/{S3_BUILDS_BUCKET}/master/amd64/clickhouse"
download_build_with_progress(build_url, Path(TEMP_PATH) / "clickhouse")

View File

@ -310,27 +310,34 @@ class PRInfo:
if need_changed_files:
self.fetch_changed_files()
@property
def is_master(self) -> bool:
return self.number == 0 and self.head_ref == "master"
@property
def is_release(self) -> bool:
return self.number == 0 and bool(
re.match(r"^2[1-9]\.[1-9][0-9]*$", self.head_ref)
)
@property
def is_release_branch(self) -> bool:
return self.number == 0
@property
def is_pr(self):
return self.event_type == EventType.PULL_REQUEST
def is_scheduled(self):
@property
def is_scheduled(self) -> bool:
return self.event_type == EventType.SCHEDULE
def is_merge_queue(self):
@property
def is_merge_queue(self) -> bool:
return self.event_type == EventType.MERGE_QUEUE
def is_dispatched(self):
@property
def is_dispatched(self) -> bool:
return self.event_type == EventType.DISPATCH
def compare_pr_url(self, pr_object: dict) -> str:

View File

@ -8,6 +8,7 @@ from github import Github
# isort: on
from cherry_pick import Labels
from commit_status_helper import (
CI_STATUS_NAME,
create_ci_report,
@ -26,7 +27,6 @@ from lambda_shared_package.lambda_shared.pr import (
)
from pr_info import PRInfo
from report import FAILURE, PENDING, SUCCESS
from cherry_pick import Labels
TRUSTED_ORG_IDS = {
54801242, # clickhouse
@ -202,7 +202,7 @@ def main():
ci_report_url = create_ci_report(pr_info, [])
print("::notice ::Can run")
if not pr_info.is_merge_queue():
if not pr_info.is_merge_queue:
# we need clean CI status for MQ to merge (no pending statuses)
post_commit_status(
commit,

View File

@ -132,7 +132,7 @@ def main():
pr_info = PRInfo()
if pr_info.is_merge_queue() and args.push:
if pr_info.is_merge_queue and args.push:
print("Auto style fix will be disabled for Merge Queue workflow")
args.push = False

View File

@ -3368,6 +3368,7 @@ if __name__ == "__main__":
os.environ["CLICKHOUSE_CLIENT_OPT"] += " "
else:
os.environ["CLICKHOUSE_CLIENT_OPT"] = ""
os.environ["CLICKHOUSE_CLIENT_OPT"] += get_additional_client_options(args)
if args.secure:
os.environ["CLICKHOUSE_CLIENT_OPT"] += " --secure "

View File

@ -5,5 +5,6 @@
<select_from_system_db_requires_grant>true</select_from_system_db_requires_grant>
<select_from_information_schema_requires_grant>true</select_from_information_schema_requires_grant>
<settings_constraints_replace_previous>true</settings_constraints_replace_previous>
<table_engines_require_grant>true</table_engines_require_grant>
</access_control_improvements>
</clickhouse>

View File

@ -1036,6 +1036,7 @@ def test_required_privileges():
)
instance.query("GRANT INSERT, CREATE ON test.table2 TO u1")
instance.query("GRANT TABLE ENGINE ON MergeTree TO u1")
instance.query(
f"RESTORE TABLE test.table AS test.table2 FROM {backup_name}", user="u1"
)

View File

@ -300,11 +300,13 @@ def test_allowed_databases(test_cluster):
"CREATE TABLE db2.t2 ON CLUSTER cluster (i Int8) ENGINE = Memory",
settings={"user": "restricted_user"},
)
with pytest.raises(Exception):
instance.query(
"CREATE TABLE t3 ON CLUSTER cluster (i Int8) ENGINE = Memory",
settings={"user": "restricted_user"},
)
with pytest.raises(Exception):
instance.query(
"DROP DATABASE db2 ON CLUSTER cluster", settings={"user": "restricted_user"}

View File

@ -0,0 +1,5 @@
<clickhouse>
<access_control_improvements>
<table_engines_require_grant>true</table_engines_require_grant>
</access_control_improvements>
</clickhouse>

View File

@ -5,9 +5,8 @@ from helpers.test_tools import TSV
cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance(
"instance",
user_configs=[
"configs/users.d/users.xml",
],
main_configs=["configs/config.xml"],
user_configs=["configs/users.d/users.xml"],
)
@ -370,6 +369,7 @@ def test_implicit_create_temporary_table_grant():
)
instance.query("GRANT CREATE TABLE ON test.* TO A")
instance.query("GRANT TABLE ENGINE ON Memory TO A")
instance.query("CREATE TEMPORARY TABLE tmp(name String)", user="A")
instance.query("REVOKE CREATE TABLE ON *.* FROM A")
@ -718,3 +718,74 @@ def test_current_grants_override():
"REVOKE SELECT ON test.* FROM B",
]
)
def test_table_engine_grant_and_revoke():
instance.query("DROP USER IF EXISTS A")
instance.query("CREATE USER A")
instance.query("GRANT CREATE TABLE ON test.table1 TO A")
assert "Not enough privileges" in instance.query_and_get_error(
"CREATE TABLE test.table1(a Integer) engine=TinyLog", user="A"
)
instance.query("GRANT TABLE ENGINE ON TinyLog TO A")
instance.query("CREATE TABLE test.table1(a Integer) engine=TinyLog", user="A")
assert instance.query("SHOW GRANTS FOR A") == TSV(
[
"GRANT TABLE ENGINE ON TinyLog TO A",
"GRANT CREATE TABLE ON test.table1 TO A",
]
)
instance.query("REVOKE TABLE ENGINE ON TinyLog FROM A")
assert "Not enough privileges" in instance.query_and_get_error(
"CREATE TABLE test.table1(a Integer) engine=TinyLog", user="A"
)
instance.query("REVOKE CREATE TABLE ON test.table1 FROM A")
instance.query("DROP TABLE test.table1")
assert instance.query("SHOW GRANTS FOR A") == TSV([])
def test_table_engine_and_source_grant():
instance.query("DROP USER IF EXISTS A")
instance.query("CREATE USER A")
instance.query("GRANT CREATE TABLE ON test.table1 TO A")
instance.query("GRANT TABLE ENGINE ON PostgreSQL TO A")
instance.query(
"""
CREATE TABLE test.table1(a Integer)
engine=PostgreSQL('localhost:5432', 'dummy', 'dummy', 'dummy', 'dummy');
""",
user="A",
)
instance.query("DROP TABLE test.table1")
instance.query("REVOKE TABLE ENGINE ON PostgreSQL FROM A")
assert "Not enough privileges" in instance.query_and_get_error(
"""
CREATE TABLE test.table1(a Integer)
engine=PostgreSQL('localhost:5432', 'dummy', 'dummy', 'dummy', 'dummy');
""",
user="A",
)
instance.query("GRANT SOURCES ON *.* TO A")
instance.query(
"""
CREATE TABLE test.table1(a Integer)
engine=PostgreSQL('localhost:5432', 'dummy', 'dummy', 'dummy', 'dummy');
""",
user="A",
)
instance.query("DROP TABLE test.table1")

View File

@ -621,6 +621,7 @@ def test_allow_ddl():
)
instance.query("GRANT CREATE ON tbl TO robin")
instance.query("GRANT TABLE ENGINE ON Log TO robin")
instance.query("CREATE TABLE tbl(a Int32) ENGINE=Log", user="robin")
instance.query("DROP TABLE tbl")

View File

@ -103,6 +103,7 @@ SHOW NAMED COLLECTIONS SECRETS ['SHOW NAMED COLLECTIONS SECRETS'] NAMED_COLLECTI
NAMED COLLECTION ['NAMED COLLECTION USAGE','USE NAMED COLLECTION'] NAMED_COLLECTION NAMED COLLECTION ADMIN
NAMED COLLECTION ADMIN ['NAMED COLLECTION CONTROL'] NAMED_COLLECTION ALL
SET DEFINER [] USER_NAME ALL
TABLE ENGINE ['TABLE ENGINE'] TABLE_ENGINE ALL
SYSTEM SHUTDOWN ['SYSTEM KILL','SHUTDOWN'] GLOBAL SYSTEM
SYSTEM DROP DNS CACHE ['SYSTEM DROP DNS','DROP DNS CACHE','DROP DNS'] GLOBAL SYSTEM DROP CACHE
SYSTEM DROP CONNECTIONS CACHE ['SYSTEM DROP CONNECTIONS CACHE','DROP CONNECTIONS CACHE'] GLOBAL SYSTEM DROP CACHE

View File

@ -9,6 +9,7 @@ ${CLICKHOUSE_CLIENT} -q "create table mute_stylecheck (x UInt32) engine = Replic
${CLICKHOUSE_CLIENT} -q "CREATE USER user_${CLICKHOUSE_DATABASE} settings database_replicated_allow_only_replicated_engine=1"
${CLICKHOUSE_CLIENT} -q "GRANT CREATE TABLE ON ${CLICKHOUSE_DATABASE}_db.* TO user_${CLICKHOUSE_DATABASE}"
${CLICKHOUSE_CLIENT} -q "GRANT TABLE ENGINE ON Memory, TABLE ENGINE ON MergeTree, TABLE ENGINE ON ReplicatedMergeTree TO user_${CLICKHOUSE_DATABASE}"
${CLICKHOUSE_CLIENT} --allow_experimental_database_replicated=1 --query "CREATE DATABASE ${CLICKHOUSE_DATABASE}_db engine = Replicated('/clickhouse/databases/${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}/${CLICKHOUSE_DATABASE}_db', '{shard}', '{replica}')"
${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_memory (x UInt32) engine = Memory;"
${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" -n --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_mt (x UInt32) engine = MergeTree order by x;" 2>&1 | grep -o "Only tables with a Replicated engine"

View File

@ -12,6 +12,7 @@ ${CLICKHOUSE_CLIENT} -q "create table mute_stylecheck (x UInt32) engine = Replic
${CLICKHOUSE_CLIENT} -q "CREATE USER user_${CLICKHOUSE_DATABASE} settings database_replicated_allow_replicated_engine_arguments=0"
${CLICKHOUSE_CLIENT} -q "GRANT CREATE TABLE ON ${CLICKHOUSE_DATABASE}_db.* TO user_${CLICKHOUSE_DATABASE}"
${CLICKHOUSE_CLIENT} -q "GRANT TABLE ENGINE ON ReplicatedMergeTree TO user_${CLICKHOUSE_DATABASE}"
${CLICKHOUSE_CLIENT} --allow_experimental_database_replicated=1 --query "CREATE DATABASE ${CLICKHOUSE_DATABASE}_db engine = Replicated('/clickhouse/databases/${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}/${CLICKHOUSE_DATABASE}_db', '{shard}', '{replica}')"
${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" -n --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_rmt_ok (x UInt32) engine = ReplicatedMergeTree order by x;"
${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" -n --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_rmt_fail (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/root/{shard}', '{replica}') order by x; -- { serverError 80 }"

View File

@ -13,21 +13,24 @@ $CLICKHOUSE_CLIENT --query "CREATE USER $user IDENTIFIED WITH PLAINTEXT_PASSWORD
$CLICKHOUSE_CLIENT --user $user --password hello --query "CREATE TEMPORARY TABLE table_memory_02561(name String)" 2>&1 | grep -F "Not enough privileges. To execute this query, it's necessary to have the grant CREATE TEMPORARY TABLE" > /dev/null && echo "OK"
$CLICKHOUSE_CLIENT --query "GRANT CREATE TEMPORARY TABLE ON *.* TO $user"
$CLICKHOUSE_CLIENT --query "GRANT TABLE ENGINE ON Memory TO $user"
$CLICKHOUSE_CLIENT --user $user --password hello --query "CREATE TEMPORARY TABLE table_memory_02561(name String)"
$CLICKHOUSE_CLIENT --user $user --password hello --query "CREATE TEMPORARY TABLE table_merge_tree_02561(name String) ENGINE = MergeTree() ORDER BY name" 2>&1 | grep -F "Not enough privileges. To execute this query, it's necessary to have the grant CREATE ARBITRARY TEMPORARY TABLE" > /dev/null && echo "OK"
$CLICKHOUSE_CLIENT --query "GRANT CREATE ARBITRARY TEMPORARY TABLE ON *.* TO $user"
$CLICKHOUSE_CLIENT --query "GRANT TABLE ENGINE ON MergeTree TO $user"
$CLICKHOUSE_CLIENT --user $user --password hello --query "CREATE TEMPORARY TABLE table_merge_tree_02561(name String) ENGINE = MergeTree() ORDER BY name"
$CLICKHOUSE_CLIENT --user $user --password hello --query "CREATE TEMPORARY TABLE table_file_02561(name String) ENGINE = File(TabSeparated)" 2>&1 | grep -F "Not enough privileges. To execute this query, it's necessary to have the grant FILE" > /dev/null && echo "OK"
$CLICKHOUSE_CLIENT --user $user --password hello --query "CREATE TEMPORARY TABLE table_file_02561(name String) ENGINE = File(TabSeparated)" 2>&1 | grep -F "Not enough privileges. To execute this query, it's necessary to have the grant TABLE ENGINE ON File" > /dev/null && echo "OK"
$CLICKHOUSE_CLIENT --query "GRANT FILE ON *.* TO $user"
$CLICKHOUSE_CLIENT --user $user --password hello --query "CREATE TEMPORARY TABLE table_file_02561(name String) ENGINE = File(TabSeparated)"
$CLICKHOUSE_CLIENT --user $user --password hello --query "CREATE TEMPORARY TABLE table_url_02561(name String) ENGINE = URL('http://127.0.0.1:8123?query=select+12', 'RawBLOB')" 2>&1 | grep -F "Not enough privileges. To execute this query, it's necessary to have the grant URL" > /dev/null && echo "OK"
$CLICKHOUSE_CLIENT --user $user --password hello --query "CREATE TEMPORARY TABLE table_url_02561(name String) ENGINE = URL('http://127.0.0.1:8123?query=select+12', 'RawBLOB')" 2>&1 | grep -F "Not enough privileges. To execute this query, it's necessary to have the grant TABLE ENGINE ON URL" > /dev/null && echo "OK"
$CLICKHOUSE_CLIENT --query "GRANT URL ON *.* TO $user"