diff --git a/src/Access/UsersConfigAccessStorage.cpp b/src/Access/UsersConfigAccessStorage.cpp index df0e4584709..187258d0fcd 100644 --- a/src/Access/UsersConfigAccessStorage.cpp +++ b/src/Access/UsersConfigAccessStorage.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -13,6 +14,7 @@ #include #include #include +#include #include #include #include @@ -52,11 +54,64 @@ namespace UUID generateID(const IAccessEntity & entity) { return generateID(entity.getType(), entity.getName()); } + template + void parseGrant(T & entity, const String & string_query, const std::unordered_set & allowed_role_ids) + { + ParserGrantQuery parser; + parser.setParseWithoutGrantees(); + + String error_message; + const char * pos = string_query.data(); + auto ast = tryParseQuery(parser, pos, pos + string_query.size(), error_message, false, "", false, 0, 0); + + if (!ast) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Failed to parse grant query. Error: {}", error_message); + + auto & query = ast->as(); + + if (query.roles && query.is_revoke) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Roles can't be revoked in config file"); + + if (!query.cluster.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Can't grant on cluster using config file"); + + if (query.grantees) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "You can't specify grantees in query using config file"); + + for (auto & element : query.access_rights_elements) + { + if (query.is_revoke) + entity.access.revoke(element); + else + entity.access.grant(element); + } + + if (query.roles) + { + std::vector roles_to_grant; + roles_to_grant.reserve(query.roles->size()); + + for (const auto & role_name : query.roles->names) + { + auto role_id = generateID(AccessEntityType::ROLE, role_name); + if (!allowed_role_ids.contains(role_id)) + throw Exception(ErrorCodes::THERE_IS_NO_PROFILE, "Role {} was not found", role_name); + + roles_to_grant.push_back(role_id); + } + + if (query.admin_option) + entity.granted_roles.grantWithAdminOption(roles_to_grant); + else + entity.granted_roles.grant(roles_to_grant); + } + } UserPtr parseUser( const Poco::Util::AbstractConfiguration & config, const String & user_name, const std::unordered_set & allowed_profile_ids, + const std::unordered_set & allowed_role_ids, bool allow_no_password, bool allow_plaintext_password) { @@ -241,37 +296,8 @@ namespace if (grant_queries) { - ParserGrantQuery parser; - parser.parseWithoutGrantees(); - for (const auto & string_query : *grant_queries) - { - String error_message; - const char * pos = string_query.data(); - auto ast = tryParseQuery(parser, pos, pos + string_query.size(), error_message, false, "", false, 0, 0); - - if (!ast) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Failed to parse grant query. Error: {}", error_message); - - auto & query = ast->as(); - - if (query.roles) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Roles can't be granted in config file"); - - if (!query.cluster.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Can't grant on cluster using config file"); - - if (query.grantees) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "You can't specify grantees in query using config file"); - - for (auto & element : query.access_rights_elements) - { - if (query.is_revoke) - user->access.revoke(element); - else - user->access.grant(element); - } - } + parseGrant(*user, string_query, allowed_role_ids); } else { @@ -321,6 +347,7 @@ namespace std::vector parseUsers( const Poco::Util::AbstractConfiguration & config, const std::unordered_set & allowed_profile_ids, + const std::unordered_set & allowed_role_ids, bool allow_no_password, bool allow_plaintext_password) { @@ -333,7 +360,7 @@ namespace { try { - users.push_back(parseUser(config, user_name, allowed_profile_ids, allow_no_password, allow_plaintext_password)); + users.push_back(parseUser(config, user_name, allowed_profile_ids, allowed_role_ids, allow_no_password, allow_plaintext_password)); } catch (Exception & e) { @@ -345,6 +372,55 @@ namespace return users; } + RolePtr parseRole( + const Poco::Util::AbstractConfiguration & config, + const String & role_name, + const std::unordered_set & allowed_role_ids) + { + auto role = std::make_shared(); + role->setName(role_name); + String role_config = "roles." + role_name; + + const auto grants_config = role_config + ".grants"; + if (config.has(grants_config)) + { + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(grants_config, keys); + for (const auto & key : keys) + { + const auto query = config.getString(grants_config + "." + key); + parseGrant(*role, query, allowed_role_ids); + } + } + + return role; + } + + std::vector parseRoles( + const Poco::Util::AbstractConfiguration & config, + const std::unordered_set & allowed_role_ids) + { + Poco::Util::AbstractConfiguration::Keys role_names; + config.keys("roles", role_names); + + std::vector roles; + roles.reserve(role_names.size()); + for (const auto & role_name : role_names) + { + try + { + roles.push_back(parseRole(config, role_name, allowed_role_ids)); + } + catch (Exception & e) + { + e.addMessage(fmt::format("while parsing roles '{}' in users configuration file", role_name)); + throw; + } + } + + return roles; + } + QuotaPtr parseQuota(const Poco::Util::AbstractConfiguration & config, const String & quota_name, const std::vector & user_ids) { @@ -635,14 +711,16 @@ namespace return profiles; } - - std::unordered_set getAllowedSettingsProfileIDs(const Poco::Util::AbstractConfiguration & config) + std::unordered_set getAllowedIDs( + const Poco::Util::AbstractConfiguration & config, + const String & configuration_key, + const AccessEntityType type) { - Poco::Util::AbstractConfiguration::Keys profile_names; - config.keys("profiles", profile_names); + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(configuration_key, keys); std::unordered_set ids; - for (const auto & profile_name : profile_names) - ids.emplace(generateID(AccessEntityType::SETTINGS_PROFILE, profile_name)); + for (const auto & key : keys) + ids.emplace(generateID(type, key)); return ids; } } @@ -693,12 +771,13 @@ void UsersConfigAccessStorage::parseFromConfig(const Poco::Util::AbstractConfigu { try { - auto allowed_profile_ids = getAllowedSettingsProfileIDs(config); + auto allowed_profile_ids = getAllowedIDs(config, "profiles", AccessEntityType::SETTINGS_PROFILE); + auto allowed_role_ids = getAllowedIDs(config, "roles", AccessEntityType::ROLE); bool no_password_allowed = access_control.isNoPasswordAllowed(); bool plaintext_password_allowed = access_control.isPlaintextPasswordAllowed(); std::vector> all_entities; - for (const auto & entity : parseUsers(config, allowed_profile_ids, no_password_allowed, plaintext_password_allowed)) + for (const auto & entity : parseUsers(config, allowed_profile_ids, allowed_role_ids, no_password_allowed, plaintext_password_allowed)) all_entities.emplace_back(generateID(*entity), entity); for (const auto & entity : parseQuotas(config)) all_entities.emplace_back(generateID(*entity), entity); @@ -706,6 +785,8 @@ void UsersConfigAccessStorage::parseFromConfig(const Poco::Util::AbstractConfigu all_entities.emplace_back(generateID(*entity), entity); for (const auto & entity : parseSettingsProfiles(config, allowed_profile_ids, access_control)) all_entities.emplace_back(generateID(*entity), entity); + for (const auto & entity : parseRoles(config, allowed_role_ids)) + all_entities.emplace_back(generateID(*entity), entity); memory_storage.setAll(all_entities); } catch (Exception & e) diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index 4c10d76690a..51e3dac781d 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -202,15 +202,16 @@ bool QueryNode::isEqualImpl(const IQueryTreeNode & rhs) const return is_subquery == rhs_typed.is_subquery && is_cte == rhs_typed.is_cte && - cte_name == rhs_typed.cte_name && - projection_columns == rhs_typed.projection_columns && is_distinct == rhs_typed.is_distinct && is_limit_with_ties == rhs_typed.is_limit_with_ties && is_group_by_with_totals == rhs_typed.is_group_by_with_totals && is_group_by_with_rollup == rhs_typed.is_group_by_with_rollup && is_group_by_with_cube == rhs_typed.is_group_by_with_cube && is_group_by_with_grouping_sets == rhs_typed.is_group_by_with_grouping_sets && - is_group_by_all == rhs_typed.is_group_by_all; + is_group_by_all == rhs_typed.is_group_by_all && + cte_name == rhs_typed.cte_name && + projection_columns == rhs_typed.projection_columns && + settings_changes == rhs_typed.settings_changes; } void QueryNode::updateTreeHashImpl(HashState & state) const @@ -239,6 +240,18 @@ void QueryNode::updateTreeHashImpl(HashState & state) const state.update(is_group_by_with_cube); state.update(is_group_by_with_grouping_sets); state.update(is_group_by_all); + + state.update(settings_changes.size()); + + for (const auto & setting_change : settings_changes) + { + state.update(setting_change.name.size()); + state.update(setting_change.name); + + auto setting_change_value_dump = setting_change.value.dump(); + state.update(setting_change_value_dump.size()); + state.update(setting_change_value_dump); + } } QueryTreeNodePtr QueryNode::cloneImpl() const @@ -256,6 +269,7 @@ QueryTreeNodePtr QueryNode::cloneImpl() const result_query_node->is_group_by_all = is_group_by_all; result_query_node->cte_name = cte_name; result_query_node->projection_columns = projection_columns; + result_query_node->settings_changes = settings_changes; return result_query_node; } diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 622e18d4ff7..e1359a5a8aa 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -346,6 +346,7 @@ target_link_libraries(clickhouse_common_io PUBLIC boost::program_options boost::system + boost::context ch_contrib::cityhash ch_contrib::re2 ch_contrib::re2_st diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 9f4fc2d135b..fdee9902634 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -381,6 +381,9 @@ The server successfully detected this situation and will download merged part fr M(CachedWriteBufferCacheWriteBytes, "Bytes written from source (remote fs, etc) to filesystem cache") \ M(CachedWriteBufferCacheWriteMicroseconds, "Time spent writing data into filesystem cache") \ \ + M(FilesystemCacheEvictedBytes, "Number of bytes evicted from filesystem cache") \ + M(FilesystemCacheEvictedFileSegments, "Number of file segments evicted from filesystem cache") \ + \ M(RemoteFSSeeks, "Total number of seeks for async buffer") \ M(RemoteFSPrefetches, "Number of prefetches made with asynchronous reading from remote filesystem") \ M(RemoteFSCancelledPrefetches, "Number of cancelled prefecthes (because of seek)") \ @@ -429,10 +432,10 @@ The server successfully detected this situation and will download merged part fr M(AggregationPreallocatedElementsInHashTables, "How many elements were preallocated in hash tables for aggregation.") \ M(AggregationHashTablesInitializedAsTwoLevel, "How many hash tables were inited as two-level for aggregation.") \ \ - M(MergeTreeMetadataCacheGet, "Number of rocksdb reads(used for merge tree metadata cache)") \ - M(MergeTreeMetadataCachePut, "Number of rocksdb puts(used for merge tree metadata cache)") \ - M(MergeTreeMetadataCacheDelete, "Number of rocksdb deletes(used for merge tree metadata cache)") \ - M(MergeTreeMetadataCacheSeek, "Number of rocksdb seeks(used for merge tree metadata cache)") \ + M(MergeTreeMetadataCacheGet, "Number of rocksdb reads (used for merge tree metadata cache)") \ + M(MergeTreeMetadataCachePut, "Number of rocksdb puts (used for merge tree metadata cache)") \ + M(MergeTreeMetadataCacheDelete, "Number of rocksdb deletes (used for merge tree metadata cache)") \ + M(MergeTreeMetadataCacheSeek, "Number of rocksdb seeks (used for merge tree metadata cache)") \ M(MergeTreeMetadataCacheHit, "Number of times the read of meta file was done from MergeTree metadata cache") \ M(MergeTreeMetadataCacheMiss, "Number of times the read of meta file was not done from MergeTree metadata cache") \ \ diff --git a/src/Core/Settings.h b/src/Core/Settings.h index db65cd0f178..f40dd01a51b 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -510,7 +510,7 @@ class IColumn; M(Bool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.", 0) \ M(Bool, optimize_respect_aliases, true, "If it is set to true, it will respect aliases in WHERE/GROUP BY/ORDER BY, that will help with partition pruning/secondary indexes/optimize_aggregation_in_order/optimize_read_in_order/optimize_trivial_count", 0) \ M(UInt64, mutations_sync, 0, "Wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). 0 - execute asynchronously. 1 - wait current server. 2 - wait all replicas if they exist.", 0) \ - M(Bool, enable_lightweight_delete, true, "Enable lightweight DELETE mutations for mergetree tables.", 0) \ + M(Bool, enable_lightweight_delete, true, "Enable lightweight DELETE mutations for mergetree tables.", 0) ALIAS(allow_experimental_lightweight_delete) \ M(Bool, optimize_move_functions_out_of_any, false, "Move functions out of aggregate functions 'any', 'anyLast'.", 0) \ M(Bool, optimize_normalize_count_variants, true, "Rewrite aggregate functions that semantically equals to count() as count().", 0) \ M(Bool, optimize_injective_functions_inside_uniq, true, "Delete injective functions of one argument inside uniq*() functions.", 0) \ @@ -560,7 +560,7 @@ class IColumn; M(Bool, asterisk_include_alias_columns, false, "Include ALIAS columns for wildcard query", 0) \ M(Bool, optimize_skip_merged_partitions, false, "Skip partitions with one part with level > 0 in optimize final", 0) \ M(Bool, optimize_on_insert, true, "Do the same transformation for inserted block of data as if merge was done on this block.", 0) \ - M(Bool, optimize_use_projections, true, "Automatically choose projections to perform SELECT query", 0) \ + M(Bool, optimize_use_projections, true, "Automatically choose projections to perform SELECT query", 0) ALIAS(allow_experimental_projection_optimization) \ M(Bool, force_optimize_projection, false, "If projection optimization is enabled, SELECT queries need to use projection", 0) \ M(Bool, async_socket_for_remote, true, "Asynchronously read from socket executing remote query", 0) \ M(Bool, async_query_sending_for_remote, true, "Asynchronously create connections and send query to shards in remote query", 0) \ @@ -770,7 +770,6 @@ class IColumn; MAKE_OBSOLETE(M, Bool, allow_experimental_database_atomic, true) \ MAKE_OBSOLETE(M, Bool, allow_experimental_bigint_types, true) \ MAKE_OBSOLETE(M, Bool, allow_experimental_window_functions, true) \ - MAKE_OBSOLETE(M, Bool, allow_experimental_lightweight_delete, true) \ MAKE_OBSOLETE(M, Bool, allow_experimental_geo_types, true) \ \ MAKE_OBSOLETE(M, Milliseconds, async_insert_stale_timeout_ms, 0) \ @@ -783,7 +782,6 @@ class IColumn; MAKE_OBSOLETE(M, UInt64, merge_tree_clear_old_parts_interval_seconds, 1) \ MAKE_OBSOLETE(M, UInt64, partial_merge_join_optimizations, 0) \ MAKE_OBSOLETE(M, MaxThreads, max_alter_threads, 0) \ - MAKE_OBSOLETE(M, Bool, allow_experimental_projection_optimization, true) \ MAKE_OBSOLETE(M, Bool, allow_experimental_query_cache, true) \ /* moved to config.xml: see also src/Core/ServerSettings.h */ \ MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, background_buffer_flush_schedule_pool_size, 16) \ diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index c4dd0161c70..0358b4e915a 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -67,6 +67,51 @@ bool AzureObjectStorage::exists(const StoredObject & object) const return false; } +void AzureObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const +{ + auto client_ptr = client.get(); + + /// What a shame, no Exists method... + Azure::Storage::Blobs::ListBlobsOptions options; + options.Prefix = path; + if (max_keys) + options.PageSizeHint = max_keys; + else + options.PageSizeHint = settings.get()->list_object_keys_size; + Azure::Storage::Blobs::ListBlobsPagedResponse blob_list_response; + + while (true) + { + blob_list_response = client_ptr->ListBlobs(options); + auto blobs_list = blob_list_response.Blobs; + + for (const auto & blob : blobs_list) + { + children.emplace_back( + blob.Name, + ObjectMetadata{ + static_cast(blob.BlobSize), + Poco::Timestamp::fromEpochTime( + std::chrono::duration_cast( + blob.Details.LastModified.time_since_epoch()).count()), + {}}); + } + + if (max_keys) + { + int keys_left = max_keys - static_cast(children.size()); + if (keys_left <= 0) + break; + options.PageSizeHint = keys_left; + } + + if (blob_list_response.HasPage()) + options.ContinuationToken = blob_list_response.NextPageToken; + else + break; + } +} + std::unique_ptr AzureObjectStorage::readObject( /// NOLINT const StoredObject & object, const ReadSettings & read_settings, @@ -146,33 +191,6 @@ std::unique_ptr AzureObjectStorage::writeObject( /// NO patchSettings(write_settings)); } -void AzureObjectStorage::findAllFiles(const std::string & path, RelativePathsWithSize & children, int max_keys) const -{ - auto client_ptr = client.get(); - - Azure::Storage::Blobs::ListBlobsOptions blobs_list_options; - blobs_list_options.Prefix = path; - if (max_keys) - blobs_list_options.PageSizeHint = max_keys; - else - blobs_list_options.PageSizeHint = settings.get()->list_object_keys_size; - - auto blobs_list_response = client_ptr->ListBlobs(blobs_list_options); - for (;;) - { - auto blobs_list = blobs_list_response.Blobs; - - for (const auto & blob : blobs_list) - children.emplace_back(blob.Name, blob.BlobSize); - - if (max_keys && children.size() >= static_cast(max_keys)) - break; - if (!blobs_list_response.HasPage()) - break; - blobs_list_response.MoveToNextPage(); - } -} - /// Remove file. Throws exception if file doesn't exists or it's a directory. void AzureObjectStorage::removeObject(const StoredObject & object) { diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 0c2aecd5c62..a36a03bcda4 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -58,6 +58,8 @@ public: AzureClientPtr && client_, SettingsPtr && settings_); + void listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const override; + DataSourceDescription getDataSourceDescription() const override { return data_source_description; } std::string getName() const override { return "AzureObjectStorage"; } @@ -84,8 +86,6 @@ public: size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, const WriteSettings & write_settings = {}) override; - void findAllFiles(const std::string & path, RelativePathsWithSize & children, int max_keys) const override; - /// Remove file. Throws exception if file doesn't exists or it's a directory. void removeObject(const StoredObject & object) override; diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp index acf9430e85c..1d24d9d5411 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp @@ -201,9 +201,9 @@ std::unique_ptr CachedObjectStorage::cloneObjectStorage( return object_storage->cloneObjectStorage(new_namespace, config, config_prefix, context); } -void CachedObjectStorage::findAllFiles(const std::string & path, RelativePathsWithSize & children, int max_keys) const +void CachedObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const { - object_storage->findAllFiles(path, children, max_keys); + object_storage->listObjects(path, children, max_keys); } ObjectMetadata CachedObjectStorage::getObjectMetadata(const std::string & path) const diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h index f8e346e1aed..b5186d39c32 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h @@ -71,7 +71,7 @@ public: const std::string & config_prefix, ContextPtr context) override; - void findAllFiles(const std::string & path, RelativePathsWithSize & children, int max_keys) const override; + void listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const override; ObjectMetadata getObjectMetadata(const std::string & path) const override; diff --git a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp index c3284b635da..19d5a8e3567 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp @@ -54,7 +54,7 @@ void DiskObjectStorageMetadata::deserialize(ReadBuffer & buf) assertChar('\n', buf); storage_objects[i].relative_path = object_relative_path; - storage_objects[i].bytes_size = object_size; + storage_objects[i].metadata.size_bytes = object_size; } readIntText(ref_count, buf); @@ -93,9 +93,9 @@ void DiskObjectStorageMetadata::serialize(WriteBuffer & buf, bool sync) const writeIntText(total_size, buf); writeChar('\n', buf); - for (const auto & [object_relative_path, object_size] : storage_objects) + for (const auto & [object_relative_path, object_metadata] : storage_objects) { - writeIntText(object_size, buf); + writeIntText(object_metadata.size_bytes, buf); writeChar('\t', buf); writeEscapedString(object_relative_path, buf); writeChar('\n', buf); @@ -139,7 +139,7 @@ DiskObjectStorageMetadata::DiskObjectStorageMetadata( void DiskObjectStorageMetadata::addObject(const String & path, size_t size) { total_size += size; - storage_objects.emplace_back(path, size); + storage_objects.emplace_back(path, ObjectMetadata{size, {}, {}}); } diff --git a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h index a2d0653e4aa..6dced85d0b1 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h +++ b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h @@ -21,7 +21,7 @@ private: const std::string & common_metadata_path; /// Relative paths of blobs. - RelativePathsWithSize storage_objects; + RelativePathsWithMetadata storage_objects; const std::string object_storage_root_path; @@ -63,7 +63,7 @@ public: return object_storage_root_path; } - RelativePathsWithSize getBlobsRelativePaths() const + RelativePathsWithMetadata getBlobsRelativePaths() const { return storage_objects; } diff --git a/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp b/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp index 4cca89b9a4f..74d1698bf01 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp @@ -356,7 +356,7 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::restoreFiles(IObjectStorage * LOG_INFO(disk->log, "Starting restore files for disk {}", disk->name); std::vector> results; - auto restore_files = [this, &source_object_storage, &restore_information, &results](const RelativePathsWithSize & objects) + auto restore_files = [this, &source_object_storage, &restore_information, &results](const RelativePathsWithMetadata & objects) { std::vector keys_names; for (const auto & object : objects) @@ -389,8 +389,8 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::restoreFiles(IObjectStorage * return true; }; - RelativePathsWithSize children; - source_object_storage->findAllFiles(restore_information.source_path, children, /* max_keys= */ 0); + RelativePathsWithMetadata children; + source_object_storage->listObjects(restore_information.source_path, children, /* max_keys= */ 0); restore_files(children); @@ -472,7 +472,7 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::restoreFileOperations(IObject || disk->object_storage_root_path != restore_information.source_path; std::set renames; - auto restore_file_operations = [this, &source_object_storage, &restore_information, &renames, &send_metadata](const RelativePathsWithSize & objects) + auto restore_file_operations = [this, &source_object_storage, &restore_information, &renames, &send_metadata](const RelativePathsWithMetadata & objects) { const String rename = "rename"; const String hardlink = "hardlink"; @@ -539,8 +539,8 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::restoreFileOperations(IObject return true; }; - RelativePathsWithSize children; - source_object_storage->findAllFiles(restore_information.source_path + "operations/", children, /* max_keys= */ 0); + RelativePathsWithMetadata children; + source_object_storage->listObjects(restore_information.source_path + "operations/", children, /* max_keys= */ 0); restore_file_operations(children); if (restore_information.detached) diff --git a/src/Disks/ObjectStorages/IObjectStorage.cpp b/src/Disks/ObjectStorages/IObjectStorage.cpp index a810db0cdf8..a5903f9d429 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.cpp +++ b/src/Disks/ObjectStorages/IObjectStorage.cpp @@ -16,15 +16,29 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -void IObjectStorage::findAllFiles(const std::string &, RelativePathsWithSize &, int) const +bool IObjectStorage::existsOrHasAnyChild(const std::string & path) const { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "findAllFiles() is not supported"); + RelativePathsWithMetadata files; + listObjects(path, files, 1); + return !files.empty(); } -void IObjectStorage::getDirectoryContents(const std::string &, - RelativePathsWithSize &, - std::vector &) const + +void IObjectStorage::listObjects(const std::string &, RelativePathsWithMetadata &, int) const { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "getDirectoryContents() is not supported"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "listObjects() is not supported"); +} + + +std::optional IObjectStorage::tryGetObjectMetadata(const std::string & path) const +{ + try + { + return getObjectMetadata(path); + } + catch (...) + { + return {}; + } } ThreadPool & IObjectStorage::getThreadPoolWriter() diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index 8babb2fbf1a..28de80a88cd 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -30,19 +30,6 @@ class WriteBufferFromFileBase; using ObjectAttributes = std::map; -struct RelativePathWithSize -{ - String relative_path; - size_t bytes_size; - - RelativePathWithSize() = default; - - RelativePathWithSize(const String & relative_path_, size_t bytes_size_) - : relative_path(relative_path_), bytes_size(bytes_size_) {} -}; -using RelativePathsWithSize = std::vector; - - struct ObjectMetadata { uint64_t size_bytes; @@ -50,6 +37,21 @@ struct ObjectMetadata std::optional attributes; }; +struct RelativePathWithMetadata +{ + String relative_path; + ObjectMetadata metadata{}; + + RelativePathWithMetadata() = default; + + RelativePathWithMetadata(const String & relative_path_, const ObjectMetadata & metadata_) + : relative_path(relative_path_), metadata(metadata_) + {} +}; + +using RelativePathsWithMetadata = std::vector; + + /// Base class for all object storages which implement some subset of ordinary filesystem operations. /// /// Examples of object storages are S3, Azure Blob Storage, HDFS. @@ -65,36 +67,17 @@ public: /// Object exists or not virtual bool exists(const StoredObject & object) const = 0; - /// List all objects with specific prefix. - /// - /// For example if you do this over filesystem, you should skip folders and - /// return files only, so something like on local filesystem: - /// - /// find . -type f - /// - /// @param children - out files (relative paths) with their sizes. - /// @param max_keys - return not more then max_keys children - /// NOTE: max_keys is not the same as list_object_keys_size (disk property) - /// - if max_keys is set not more then max_keys keys should be returned - /// - however list_object_keys_size determine the size of the batch and should return all keys - /// - /// NOTE: It makes sense only for real object storages (S3, Azure), since - /// it is used only for one of the following: - /// - send_metadata (to restore metadata) - /// - see DiskObjectStorage::restoreMetadataIfNeeded() - /// - MetadataStorageFromPlainObjectStorage - only for s3_plain disk - virtual void findAllFiles(const std::string & path, RelativePathsWithSize & children, int max_keys) const; + /// Object exists or any child on the specified path exists. + /// We have this method because object storages are flat for example + /// /a/b/c/d may exist but /a/b/c may not. So this method will return true for + /// /, /a, /a/b, /a/b/c, /a/b/c/d while exists will return true only for /a/b/c/d + virtual bool existsOrHasAnyChild(const std::string & path) const; - /// Analog of directory content for object storage (object storage does not - /// have "directory" definition, but it can be emulated with usage of - /// "delimiter"), so this is analog of: - /// - /// find . -maxdepth 1 $path - /// - /// Return files in @files and directories in @directories - virtual void getDirectoryContents(const std::string & path, - RelativePathsWithSize & files, - std::vector & directories) const; + virtual void listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const; + + /// Get object metadata if supported. It should be possible to receive + /// at least size of object + virtual std::optional tryGetObjectMetadata(const std::string & path) const; /// Get object metadata if supported. It should be possible to receive /// at least size of object diff --git a/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp b/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp index 6adf24b5bda..9461a82845f 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp @@ -142,10 +142,10 @@ StoredObjects MetadataStorageFromDisk::getStorageObjects(const std::string & pat object_storage_paths.reserve(object_storage_relative_paths.size()); /// Relative paths -> absolute. - for (auto & [object_relative_path, size] : object_storage_relative_paths) + for (auto & [object_relative_path, object_meta] : object_storage_relative_paths) { auto object_path = fs::path(metadata->getBlobsCommonPrefix()) / object_relative_path; - StoredObject object{ object_path, size, path }; + StoredObject object{ object_path, object_meta.size_bytes, path }; object_storage_paths.push_back(object); } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index a680a344746..c119e9f3adc 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -10,11 +10,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - MetadataStorageFromPlainObjectStorage::MetadataStorageFromPlainObjectStorage( ObjectStoragePtr object_storage_, const std::string & object_storage_root_path_) @@ -39,11 +34,10 @@ std::filesystem::path MetadataStorageFromPlainObjectStorage::getAbsolutePath(con bool MetadataStorageFromPlainObjectStorage::exists(const std::string & path) const { - RelativePathsWithSize children; /// NOTE: exists() cannot be used here since it works only for existing /// key, and does not work for some intermediate path. - object_storage->findAllFiles(getAbsolutePath(path), children, 1); - return !children.empty(); + std::string abs_path = getAbsolutePath(path); + return object_storage->existsOrHasAnyChild(abs_path); } bool MetadataStorageFromPlainObjectStorage::isFile(const std::string & path) const @@ -55,45 +49,50 @@ bool MetadataStorageFromPlainObjectStorage::isFile(const std::string & path) con bool MetadataStorageFromPlainObjectStorage::isDirectory(const std::string & path) const { std::string directory = getAbsolutePath(path); - trimRight(directory); - directory += "/"; + if (!directory.ends_with('/')) + directory += '/'; - /// NOTE: This check is far from ideal, since it work only if the directory - /// really has files, and has excessive API calls - RelativePathsWithSize files; - std::vector directories; - object_storage->getDirectoryContents(directory, files, directories); - return !files.empty() || !directories.empty(); + RelativePathsWithMetadata files; + object_storage->listObjects(directory, files, 1); + return !files.empty(); } uint64_t MetadataStorageFromPlainObjectStorage::getFileSize(const String & path) const { - RelativePathsWithSize children; - object_storage->findAllFiles(getAbsolutePath(path), children, 1); - if (children.empty()) - return 0; - if (children.size() != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "findAllFiles() return multiple paths ({}) for {}", children.size(), path); - return children.front().bytes_size; + RelativePathsWithMetadata children; + auto metadata = object_storage->tryGetObjectMetadata(getAbsolutePath(path)); + if (metadata) + return metadata->size_bytes; + return 0; } std::vector MetadataStorageFromPlainObjectStorage::listDirectory(const std::string & path) const { - RelativePathsWithSize files; - std::vector directories; - object_storage->getDirectoryContents(getAbsolutePath(path), files, directories); + RelativePathsWithMetadata files; + std::string abs_path = getAbsolutePath(path); + if (!abs_path.ends_with('/')) + abs_path += '/'; + + object_storage->listObjects(abs_path, files, 0); std::vector result; for (const auto & path_size : files) + { result.push_back(path_size.relative_path); - for (const auto & directory : directories) - result.push_back(directory); + } + + std::unordered_set duplicates_filter; for (auto & row : result) { - chassert(row.starts_with(object_storage_root_path)); - row.erase(0, object_storage_root_path.size()); + chassert(row.starts_with(abs_path)); + row.erase(0, abs_path.size()); + auto slash_pos = row.find_first_of('/'); + if (slash_pos != std::string::npos) + row.erase(slash_pos, row.size() - slash_pos); + duplicates_filter.insert(row); } - return result; + + return std::vector(duplicates_filter.begin(), duplicates_filter.end()); } DirectoryIteratorPtr MetadataStorageFromPlainObjectStorage::iterateDirectory(const std::string & path) const diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 79e9e1141bb..6e63efcc1e3 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -183,7 +183,7 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN disk_write_settings); } -void S3ObjectStorage::findAllFiles(const std::string & path, RelativePathsWithSize & children, int max_keys) const +void S3ObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const { auto settings_ptr = s3_settings.get(); auto client_ptr = client.get(); @@ -211,7 +211,7 @@ void S3ObjectStorage::findAllFiles(const std::string & path, RelativePathsWithSi break; for (const auto & object : objects) - children.emplace_back(object.GetKey(), object.GetSize()); + children.emplace_back(object.GetKey(), ObjectMetadata{static_cast(object.GetSize()), Poco::Timestamp::fromEpochTime(object.GetLastModified().Seconds()), {}}); if (max_keys) { @@ -225,54 +225,6 @@ void S3ObjectStorage::findAllFiles(const std::string & path, RelativePathsWithSi } while (outcome.GetResult().GetIsTruncated()); } -void S3ObjectStorage::getDirectoryContents(const std::string & path, - RelativePathsWithSize & files, - std::vector & directories) const -{ - auto settings_ptr = s3_settings.get(); - auto client_ptr = client.get(); - - S3::ListObjectsV2Request request; - request.SetBucket(bucket); - /// NOTE: if you do "ls /foo" instead of "ls /foo/" over S3 with this API - /// it will return only "/foo" itself without any underlying nodes. - if (path.ends_with("/")) - request.SetPrefix(path); - else - request.SetPrefix(path + "/"); - request.SetMaxKeys(settings_ptr->list_object_keys_size); - request.SetDelimiter("/"); - - Aws::S3::Model::ListObjectsV2Outcome outcome; - do - { - ProfileEvents::increment(ProfileEvents::S3ListObjects); - ProfileEvents::increment(ProfileEvents::DiskS3ListObjects); - outcome = client_ptr->ListObjectsV2(request); - throwIfError(outcome); - - auto result = outcome.GetResult(); - auto result_objects = result.GetContents(); - auto result_common_prefixes = result.GetCommonPrefixes(); - - if (result_objects.empty() && result_common_prefixes.empty()) - break; - - for (const auto & object : result_objects) - files.emplace_back(object.GetKey(), object.GetSize()); - - for (const auto & common_prefix : result_common_prefixes) - { - std::string directory = common_prefix.GetPrefix(); - /// Make it compatible with std::filesystem::path::filename() - trimRight(directory, '/'); - directories.emplace_back(directory); - } - - request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); - } while (outcome.GetResult().GetIsTruncated()); -} - void S3ObjectStorage::removeObjectImpl(const StoredObject & object, bool if_exists) { auto client_ptr = client.get(); @@ -359,6 +311,22 @@ void S3ObjectStorage::removeObjectsIfExist(const StoredObjects & objects) removeObjectsImpl(objects, true); } +std::optional S3ObjectStorage::tryGetObjectMetadata(const std::string & path) const +{ + auto settings_ptr = s3_settings.get(); + auto object_info = S3::getObjectInfo(*client.get(), bucket, path, {}, settings_ptr->request_settings, /* with_metadata= */ true, /* for_disk_s3= */ true, /* throw_on_error= */ false); + + if (object_info.size == 0 && object_info.last_modification_time == 0 && object_info.metadata.empty()) + return {}; + + ObjectMetadata result; + result.size_bytes = object_info.size; + result.last_modified = object_info.last_modification_time; + result.attributes = object_info.metadata; + + return result; +} + ObjectMetadata S3ObjectStorage::getObjectMetadata(const std::string & path) const { auto settings_ptr = s3_settings.get(); diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index 70ed899586e..b0eb01aec0d 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -100,10 +100,7 @@ public: size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, const WriteSettings & write_settings = {}) override; - void findAllFiles(const std::string & path, RelativePathsWithSize & children, int max_keys) const override; - void getDirectoryContents(const std::string & path, - RelativePathsWithSize & files, - std::vector & directories) const override; + void listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const override; /// Uses `DeleteObjectRequest`. void removeObject(const StoredObject & object) override; @@ -121,6 +118,8 @@ public: ObjectMetadata getObjectMetadata(const std::string & path) const override; + std::optional tryGetObjectMetadata(const std::string & path) const override; + void copyObject( /// NOLINT const StoredObject & object_from, const StoredObject & object_to, diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index d9e32ca1494..8a24a4fe5ee 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -17,6 +17,12 @@ namespace fs = std::filesystem; +namespace ProfileEvents +{ + extern const Event FilesystemCacheEvictedBytes; + extern const Event FilesystemCacheEvictedFileSegments; +} + namespace { @@ -643,7 +649,9 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size) return PriorityIterationResult::CONTINUE; } - /// TODO: we can resize if partially downloaded instead. + ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedFileSegments); + ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedBytes, segment->range().size()); + locked_key.removeFileSegment(segment->offset(), segment->lock()); return PriorityIterationResult::REMOVE_AND_CONTINUE; } @@ -721,6 +729,10 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size) chassert(candidate->releasable()); const auto * segment = candidate->file_segment.get(); + + ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedFileSegments); + ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedBytes, segment->range().size()); + locked_key->removeFileSegment(segment->offset(), segment->lock()); segment->getQueueIterator()->remove(cache_lock); diff --git a/src/Parsers/Access/ParserGrantQuery.h b/src/Parsers/Access/ParserGrantQuery.h index 58c2be433d5..0ecfef916f5 100644 --- a/src/Parsers/Access/ParserGrantQuery.h +++ b/src/Parsers/Access/ParserGrantQuery.h @@ -14,7 +14,7 @@ class ParserGrantQuery : public IParserBase public: ParserGrantQuery & useAttachMode(bool attach_mode_ = true) { attach_mode = attach_mode_; return *this; } - ParserGrantQuery & parseWithoutGrantees(bool allow_no_grantees_ = true) { allow_no_grantees = allow_no_grantees_; return *this; } + ParserGrantQuery & setParseWithoutGrantees(bool allow_no_grantees_ = true) { allow_no_grantees = allow_no_grantees_; return *this; } protected: const char * getName() const override { return "GRANT or REVOKE query"; } diff --git a/tests/integration/test_user_grants_from_config/configs/another_user.xml b/tests/integration/test_user_grants_from_config/configs/another_user.xml index 16c026e81d0..0b0b2473142 100644 --- a/tests/integration/test_user_grants_from_config/configs/another_user.xml +++ b/tests/integration/test_user_grants_from_config/configs/another_user.xml @@ -14,5 +14,30 @@ REVOKE CREATE ON system.* + + + + ::/0 + + default + default + + GRANT admin_role + + + + + + GRANT SHOW ON *.* + REVOKE SHOW ON system.* + GRANT CREATE ON *.* WITH GRANT OPTION + + + + + GRANT ALL ON *.* WITH GRANT OPTION + + + diff --git a/tests/integration/test_user_grants_from_config/test.py b/tests/integration/test_user_grants_from_config/test.py index a4d5c0c904b..d2bd0b0facd 100644 --- a/tests/integration/test_user_grants_from_config/test.py +++ b/tests/integration/test_user_grants_from_config/test.py @@ -42,7 +42,7 @@ def test_allow_read_from_system_tables(): ) -def test_grants_from_config(): +def test_user_grants_from_config(): assert node.query("SHOW GRANTS FOR another") == TSV( [ "GRANT SHOW ON *.* TO another", @@ -51,3 +51,19 @@ def test_grants_from_config(): "REVOKE CREATE DATABASE, CREATE TABLE, CREATE VIEW, CREATE DICTIONARY ON system.* FROM another", ] ) + + assert node.query("SHOW GRANTS FOR admin_user") == TSV( + [ + "GRANT admin_role TO admin_user", + ] + ) + + +def test_role_grants_from_config(): + assert node.query("SHOW GRANTS FOR test_role") == TSV( + [ + "GRANT SHOW ON *.* TO test_role", + "GRANT CREATE ON *.* TO test_role WITH GRANT OPTION", + "REVOKE SHOW ON system.* FROM test_role", + ] + ) diff --git a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql index 9413c664293..050b8e37722 100644 --- a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql +++ b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql @@ -110,7 +110,7 @@ DROP TABLE t_proj; CREATE TABLE merge_table_standard_delete(id Int32, name String) ENGINE = MergeTree order by id settings min_bytes_for_wide_part=0; SET allow_experimental_lightweight_delete = false; -DELETE FROM merge_table_standard_delete WHERE id = 10; -- allow_experimental_lightweight_delete=false is now ignored +DELETE FROM merge_table_standard_delete WHERE id = 10; -- { serverError SUPPORT_IS_DISABLED } SET enable_lightweight_delete = false; DELETE FROM merge_table_standard_delete WHERE id = 10; -- { serverError SUPPORT_IS_DISABLED } DROP TABLE merge_table_standard_delete;