diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 476cdd57e18..c2e76de5e14 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -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 diff --git a/contrib/azure b/contrib/azure index ad2d3d42356..b90fd3c6ef3 160000 --- a/contrib/azure +++ b/contrib/azure @@ -1 +1 @@ -Subproject commit ad2d3d423565b8a8e7b7ec863eae9318a8283878 +Subproject commit b90fd3c6ef3185f5be3408056567bca0854129b6 diff --git a/docs/en/sql-reference/functions/string-replace-functions.md b/docs/en/sql-reference/functions/string-replace-functions.md index 60fe286de25..0b761b62006 100644 --- a/docs/en/sql-reference/functions/string-replace-functions.md +++ b/docs/en/sql-reference/functions/string-replace-functions.md @@ -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** diff --git a/docs/en/sql-reference/statements/grant.md b/docs/en/sql-reference/statements/grant.md index a93db29e82c..ccef5066e89 100644 --- a/docs/en/sql-reference/statements/grant.md +++ b/docs/en/sql-reference/statements/grant.md @@ -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. diff --git a/programs/server/config.xml b/programs/server/config.xml index 5ca753b84be..b5e2a88a1a7 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -742,6 +742,10 @@ It also enables 'changeable_in_readonly' constraint type --> true + + true + 600 diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index d02af01126a..da047d1cb1d 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -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_); diff --git a/src/Access/AccessControl.h b/src/Access/AccessControl.h index 1af74e02fb7..d1537219a06 100644 --- a/src/Access/AccessControl.h +++ b/src/Access/AccessControl.h @@ -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 getContextAccess(const ContextAccessParams & params) const; std::shared_ptr 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 default_password_type = AuthenticationType::SHA256_PASSWORD; }; diff --git a/src/Access/Common/AccessFlags.cpp b/src/Access/Common/AccessFlags.cpp index d3b6d5a9818..9d846821e42 100644 --- a/src/Access/Common/AccessFlags.cpp +++ b/src/Access/Common/AccessFlags.cpp @@ -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 keyword_to_flags_map; std::vector access_type_to_flags_mapping; Flags all_flags; - Flags all_flags_for_target[static_cast(USER_NAME) + 1]; + Flags all_flags_for_target[static_cast(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::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(); } diff --git a/src/Access/Common/AccessFlags.h b/src/Access/Common/AccessFlags.h index a684731055c..e2c0611be52 100644 --- a/src/Access/Common/AccessFlags.h +++ b/src/Access/Common/AccessFlags.h @@ -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(); diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 63de202c060..570e36bf6db 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -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) \ diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 0943e797e3f..2736d13e751 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -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> 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; } diff --git a/src/Access/UsersConfigAccessStorage.cpp b/src/Access/UsersConfigAccessStorage.cpp index e3c45eb45ae..908ff780c62 100644 --- a/src/Access/UsersConfigAccessStorage.cpp +++ b/src/Access/UsersConfigAccessStorage.cpp @@ -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); diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 67f97a34c62..4d5c961fdfa 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -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; } diff --git a/src/Storages/System/StorageSystemPrivileges.cpp b/src/Storages/System/StorageSystemPrivileges.cpp index 58dcf62115e..15ab25ea820 100644 --- a/src/Storages/System/StorageSystemPrivileges.cpp +++ b/src/Storages/System/StorageSystemPrivileges.cpp @@ -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(COLUMN)); enum_values.emplace_back("NAMED_COLLECTION", static_cast(NAMED_COLLECTION)); enum_values.emplace_back("USER_NAME", static_cast(USER_NAME)); + enum_values.emplace_back("TABLE_ENGINE", static_cast(TABLE_ENGINE)); return enum_values; } } diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index 48640f15ac0..c1dd2910788 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -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) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index f11d62e9136..24d6d95bd0d 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -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, diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index 56728c3d3ba..2ee526bdd39 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -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 diff --git a/tests/ci/docker_server.py b/tests/ci/docker_server.py index 230f3e56110..151cc5a4c02 100644 --- a/tests/ci/docker_server.py +++ b/tests/ci/docker_server.py @@ -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: diff --git a/tests/ci/finish_check.py b/tests/ci/finish_check.py index 617f4c9d88c..79926b33dc0 100644 --- a/tests/ci/finish_check.py +++ b/tests/ci/finish_check.py @@ -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 diff --git a/tests/ci/jepsen_check.py b/tests/ci/jepsen_check.py index 011ecff635e..6ed411a11ef 100644 --- a/tests/ci/jepsen_check.py +++ b/tests/ci/jepsen_check.py @@ -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") diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 293004fc4f3..c61e62f334c 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -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: diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 435a5f726f2..262786d8228 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -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, diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 4580f007606..d49cd283e9f 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -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 diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 7b532bbcbfa..74137b9dc32 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -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 " diff --git a/tests/config/config.d/enable_access_control_improvements.xml b/tests/config/config.d/enable_access_control_improvements.xml index 564b656a0ad..350c6fc4f44 100644 --- a/tests/config/config.d/enable_access_control_improvements.xml +++ b/tests/config/config.d/enable_access_control_improvements.xml @@ -5,5 +5,6 @@ true true true + true diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index 4b6ccd24e4f..ef9e536976b 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -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" ) diff --git a/tests/integration/test_distributed_ddl/test.py b/tests/integration/test_distributed_ddl/test.py index 7cee60a7f35..f08c6265b82 100755 --- a/tests/integration/test_distributed_ddl/test.py +++ b/tests/integration/test_distributed_ddl/test.py @@ -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"} diff --git a/tests/integration/test_grant_and_revoke/configs/config.xml b/tests/integration/test_grant_and_revoke/configs/config.xml new file mode 100644 index 00000000000..fa009296dd3 --- /dev/null +++ b/tests/integration/test_grant_and_revoke/configs/config.xml @@ -0,0 +1,5 @@ + + + true + + diff --git a/tests/integration/test_grant_and_revoke/test.py b/tests/integration/test_grant_and_revoke/test.py index a7beb97f113..e533cced1e4 100644 --- a/tests/integration/test_grant_and_revoke/test.py +++ b/tests/integration/test_grant_and_revoke/test.py @@ -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") diff --git a/tests/integration/test_settings_profile/test.py b/tests/integration/test_settings_profile/test.py index 61237af08c5..e5c0a072ff9 100644 --- a/tests/integration/test_settings_profile/test.py +++ b/tests/integration/test_settings_profile/test.py @@ -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") diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index 48d78c04dbc..2a7ceab57ba 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -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 diff --git a/tests/queries/0_stateless/02232_allow_only_replicated_engine.sh b/tests/queries/0_stateless/02232_allow_only_replicated_engine.sh index 193d5fdb6d5..791102b9cbd 100755 --- a/tests/queries/0_stateless/02232_allow_only_replicated_engine.sh +++ b/tests/queries/0_stateless/02232_allow_only_replicated_engine.sh @@ -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" diff --git a/tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.sh b/tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.sh index ee51640488e..a050f7b00d7 100755 --- a/tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.sh +++ b/tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.sh @@ -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 }" diff --git a/tests/queries/0_stateless/02561_temporary_table_grants.sh b/tests/queries/0_stateless/02561_temporary_table_grants.sh index 33784f1d536..6bd6383d310 100755 --- a/tests/queries/0_stateless/02561_temporary_table_grants.sh +++ b/tests/queries/0_stateless/02561_temporary_table_grants.sh @@ -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"