Skip to content
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
62 commits
Select commit Hold shift + click to select a range
fd271f7
Advertise host from config for replicated databases
tilman-aiven Dec 2, 2025
d2853b9
Enable internal replication for DatabaseReplicated clusters
tilman-aiven Dec 4, 2025
a2d31af
Enable ALTER DATABASE MODIFY SETTING for Replicated databases
tilman-aiven Dec 4, 2025
609861c
Replace MergeTree with ReplicatedMergeTree in Replicated databases
tilman-aiven Dec 5, 2025
9aa01b8
Tolerate ZooKeeper restart with increased retries and exponential bac…
tilman-aiven Dec 6, 2025
68cc3fe
Fix ClickHouse restart with replicated tables containing {shard} macro
tilman-aiven Dec 7, 2025
7ce073c
Add missing settings to recoverLostReplica
tilman-aiven Dec 7, 2025
392b9c7
Fix unbounded replication queue growth
tilman-aiven Dec 7, 2025
18f6623
Replicate ALTER TABLE MOVE PARTITION queries through DatabaseReplicated
tilman-aiven Dec 7, 2025
4f53d4e
Change default logs_to_keep from 1000 to 300 for DatabaseReplicated
tilman-aiven Dec 8, 2025
b8d3358
Restrict SHOW CREATE DATABASE access
tilman-aiven Dec 9, 2025
83f8c09
Allow custom CA certificate path for S3 connections
tilman-aiven Dec 9, 2025
cbf5f9a
Allow custom CA certificate path for Azure Blob Storage connections
tilman-aiven Dec 9, 2025
220d4dd
Fix default profile escape vulnerability
tilman-aiven Dec 10, 2025
8dae4eb
Allow delegating S3 signature to a separate process
tilman-aiven Dec 12, 2025
862eaf0
Allow delegating Azure signature to a separate process
tilman-aiven Dec 13, 2025
6c912b5
Enforce SSL in the MySQL handler
tilman-aiven Dec 13, 2025
ca38ca1
Enforce HTTPS for URL storage and HTTPDictionarySource
tilman-aiven Dec 13, 2025
c29b119
Allow avnadmin creating database using sql
tilman-aiven Dec 13, 2025
c47bfa7
Add CHECK TABLE to default privileges
tilman-aiven Dec 13, 2025
c2e2cac
Add SSL/TLS configuration support for PostgreSQL and MySQL connections
tilman-aiven Dec 15, 2025
7696e2b
Fix: stop wiping secrets from remote calls
tilman-aiven Dec 15, 2025
85db23f
Added support for protected users
tilman-aiven Dec 15, 2025
a129038
Fix IPv6 S3 object storage host
tilman-aiven Dec 15, 2025
8266d31
Fix IPv6 Azure object storage host
tilman-aiven Dec 16, 2025
41a6661
Add support for Azure object storage path prefix
tilman-aiven Dec 16, 2025
5d8e850
Add Backup disk type
tilman-aiven Dec 16, 2025
e1af696
Fix uncaught exception if S3 storage fails
tilman-aiven Dec 18, 2025
3ebadf5
Skip attempt to create a container in azure blob storage
tilman-aiven Dec 18, 2025
c6d94aa
Add Kafka configuration support for SASL and SSL settings
tilman-aiven Dec 22, 2025
8d057af
Allow decreasing number of Kafka consumers to zero
tilman-aiven Dec 22, 2025
691d5cc
Support per-table schema registry with authentication
tilman-aiven Dec 22, 2025
7fc1bd1
Add kafka_auto_offset_reset and kafka_date_time_input_format settings
tilman-aiven Dec 23, 2025
5d2e539
Add extra settings to Kafka Table Engine
tilman-aiven Dec 23, 2025
0fb01ff
Unlock PostgreSQL database
tilman-aiven Jan 2, 2026
e6fe108
Add support for integration metadata to named collections validation
tilman-aiven Jan 2, 2026
cc5c316
Multiple changes in PostgreSQL dictionary
tilman-aiven Jan 2, 2026
e31322f
Ignore unreadable sensors
tilman-aiven Jan 5, 2026
35940d9
Fix tcp_port_secure from ZK
tilman-aiven Jan 5, 2026
9264b1c
Disable thread fuzzer
tilman-aiven Jan 5, 2026
feb7bcc
Disable replicas_status endpoint
tilman-aiven Jan 5, 2026
84f8217
Fix swap drift
tilman-aiven Jan 5, 2026
5921a74
Fix ZK node leak after create delete table
tilman-aiven Jan 5, 2026
4c929b9
Remove SSBS check from `NO_ARMV81_OR_HIGHER`
tilman-aiven Jan 5, 2026
29abd88
Enable curl ipv6
tilman-aiven Jan 5, 2026
e5439d9
Allow disabling of individual dictionary sources
tilman-aiven Jan 6, 2026
679386a
Add early fetch pool
tilman-aiven Jan 6, 2026
c6b53b9
Add per-server override for max bytes to merge/mutate
tilman-aiven Jan 6, 2026
41b77b1
Zero copy fixes
tilman-aiven Jan 7, 2026
e2d4912
Fix refreshable materialized views where there is a shard macro in th…
tilman-aiven Jan 7, 2026
01b5941
Allow refreshable materialized views when using ZooKeeper (rather tha…
tilman-aiven Jan 8, 2026
13e36e3
Added support for disabling various table engines and table functions
tilman-aiven Jan 8, 2026
31933fb
additional compiler flags for newer engines/function
tilman-aiven Jan 8, 2026
3f05475
Check MergeTree settings constraints before enqueuing DDL queries
tilman-aiven Jan 9, 2026
a99d191
Add read-only setting to KeeperMap storage
tilman-aiven Jan 9, 2026
f1f1c95
Add named_collection column to system.tables
tilman-aiven Jan 9, 2026
a64854f
Include metadata_version.txt when freezing
tilman-aiven Jan 9, 2026
04e81a2
remove all cloud-specific settings
tilman-aiven Jan 12, 2026
e257028
Disallow replication parameters customization
tilman-aiven Jan 13, 2026
58f10db
Fix ClickHouse trying to read non-existent certificate files
tilman-aiven Jan 13, 2026
1ef824e
Fix alter order by
tilman-aiven Jan 13, 2026
8530fbf
Changes for ClickHouse dictionary source, both remote and local.
tilman-aiven Jan 21, 2026
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
9 changes: 6 additions & 3 deletions .gitmodules
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,8 @@
url = https://github.com/google/re2
[submodule "contrib/mariadb-connector-c"]
path = contrib/mariadb-connector-c
url = https://github.com/ClickHouse/mariadb-connector-c
url = https://github.com/aiven/mariadb-connector-c
branch = aiven/clickhouse-v25.8.12.129
[submodule "contrib/jemalloc"]
path = contrib/jemalloc
url = https://github.com/jemalloc/jemalloc
Expand Down Expand Up @@ -87,7 +88,8 @@
url = https://github.com/ClickHouse/grpc
[submodule "contrib/aws"]
path = contrib/aws
url = https://github.com/ClickHouse/aws-sdk-cpp
url = https://github.com/aiven/aws-sdk-cpp
branch = aiven/clickhouse-v25.8.12.129
[submodule "contrib/aws-c-event-stream"]
path = contrib/aws-c-event-stream
url = https://github.com/awslabs/aws-c-event-stream
Expand Down Expand Up @@ -214,7 +216,8 @@
url = https://github.com/ClickHouse/hive-metastore
[submodule "contrib/azure"]
path = contrib/azure
url = https://github.com/ClickHouse/azure-sdk-for-cpp
url = https://github.com/aiven/azure-sdk-for-cpp
branch = aiven/clickhouse-v25.8.12.129
[submodule "contrib/minizip-ng"]
path = contrib/minizip-ng
url = https://github.com/zlib-ng/minizip-ng
Expand Down
2 changes: 1 addition & 1 deletion cmake/cpu_features.cmake
Original file line number Diff line number Diff line change
Expand Up @@ -104,7 +104,7 @@ elseif (ARCH_AARCH64)
# jscvt fcma lrcpc dcpop sha3 sm3 sm4 asimddp sha512 sve asimdfhm dit uscat ilrcpc flagm ssbs paca pacg dcpodp svei8mm svebf16 i8mm
# bf16 dgh rng")
execute_process(
COMMAND grep -P "^(?=.*atomic)(?=.*ssbs)" /proc/cpuinfo
COMMAND grep -P "^(?=.*atomic)" /proc/cpuinfo
OUTPUT_VARIABLE FLAGS)
if (NOT FLAGS)
MESSAGE(FATAL_ERROR "The build machine does not satisfy the minimum CPU requirements, try to run cmake with -DNO_ARMV81_OR_HIGHER=1")
Expand Down
2 changes: 2 additions & 0 deletions contrib/curl-cmake/CMakeLists.txt
Original file line number Diff line number Diff line change
@@ -1,4 +1,5 @@
option (ENABLE_CURL "Enable curl" ${ENABLE_LIBRARIES})
option (ENABLE_IPV6 "Enable IPv6" 1)

if (NOT ENABLE_CURL)
message(STATUS "Not using curl")
Expand Down Expand Up @@ -192,6 +193,7 @@ target_compile_definitions (_curl PRIVATE
BUILDING_LIBCURL
CURL_HIDDEN_SYMBOLS
libcurl_EXPORTS
ENABLE_IPV6
OPENSSL_NO_ENGINE
CURL_OS="${CMAKE_SYSTEM_NAME}"
)
Expand Down
2 changes: 1 addition & 1 deletion contrib/mariadb-connector-c
4 changes: 4 additions & 0 deletions docs/en/operations/system-tables/metrics.md
Original file line number Diff line number Diff line change
Expand Up @@ -628,6 +628,10 @@ Number of data parts checking for consistency

Number of data parts being fetched from replica

### ReplicatedQueuesTotalSize

Total number of items contained in all nodes' replicated queues.

### ReplicatedSend {#replicatedsend}

Number of data parts being sent to replicas
Expand Down
7 changes: 6 additions & 1 deletion programs/server/Server.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -331,6 +331,8 @@ namespace ServerSetting
extern const ServerSettingsUInt64 page_cache_max_size;
extern const ServerSettingsDouble page_cache_free_memory_ratio;
extern const ServerSettingsUInt64 page_cache_shards;
extern const ServerSettingsUInt64 max_bytes_to_merge_override;
extern const ServerSettingsUInt64 max_bytes_to_mutate_override;
extern const ServerSettingsUInt64 os_cpu_busy_time_threshold;
extern const ServerSettingsFloat min_os_cpu_wait_time_ratio_to_drop_connection;
extern const ServerSettingsFloat max_os_cpu_wait_time_ratio_to_drop_connection;
Expand Down Expand Up @@ -2011,6 +2013,9 @@ try
global_context->setMaxPendingMutationsExecutionTimeToWarn(new_server_settings[ServerSetting::max_pending_mutations_execution_time_to_warn]);
global_context->getAccessControl().setAllowTierSettings(new_server_settings[ServerSetting::allow_feature_tier]);

global_context->setMaxBytesToMergeOverride(new_server_settings[ServerSetting::max_bytes_to_merge_override]);
global_context->setMaxBytesToMutateOverride(new_server_settings[ServerSetting::max_bytes_to_mutate_override]);

global_context->setS3QueueDisableStreaming(new_server_settings[ServerSetting::s3queue_disable_streaming]);

global_context->setOSCPUOverloadSettings(new_server_settings[ServerSetting::min_os_cpu_wait_time_ratio_to_drop_connection], new_server_settings[ServerSetting::max_os_cpu_wait_time_ratio_to_drop_connection]);
Expand Down Expand Up @@ -2178,7 +2183,7 @@ try

CompressionCodecEncrypted::Configuration::instance().tryLoad(*config, "encryption_codecs");
#if USE_SSL
CertificateReloader::instance().tryReloadAll(*config);
CertificateReloader::instance().tryLoad(*config);
#endif
NamedCollectionFactory::instance().reloadFromConfig(*config);
FileCacheFactory::instance().updateSettingsFromConfig(*config);
Expand Down
3 changes: 2 additions & 1 deletion src/Access/AccessBackup.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -561,7 +561,8 @@ void restoreAccessEntitiesFromBackup(
LOG_TRACE(log, "{}: Adding with UUID {}", AccessEntityTypeInfo::get(type).formatEntityNameWithType(name), id);

UUID existing_id;
if (destination_access_storage.insert(id, entity, replace_if_exists, throw_if_exists, &existing_id))
auto check_func = [](const AccessEntityPtr &){}; // No check needed during restore
if (destination_access_storage.insert(id, entity, check_func, replace_if_exists, throw_if_exists, &existing_id))
{
LOG_TRACE(log, "{}: Added successfully", AccessEntityTypeInfo::get(type).formatEntityNameWithType(name));
restored_ids.emplace(id);
Expand Down
25 changes: 21 additions & 4 deletions src/Access/AccessControl.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -551,19 +551,19 @@ scope_guard AccessControl::subscribeForChanges(const std::vector<UUID> & ids, co
return changes_notifier->subscribeForChanges(ids, handler);
}

bool AccessControl::insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id)
bool AccessControl::insertImpl(const UUID & id, const AccessEntityPtr & entity, const CheckFunc & check_func, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id)
{
if (MultipleAccessStorage::insertImpl(id, entity, replace_if_exists, throw_if_exists, conflicting_id))
if (MultipleAccessStorage::insertImpl(id, entity, check_func, replace_if_exists, throw_if_exists, conflicting_id))
{
changes_notifier->sendNotifications();
return true;
}
return false;
}

bool AccessControl::removeImpl(const UUID & id, bool throw_if_not_exists)
bool AccessControl::removeImpl(const UUID & id, const CheckFunc & check_func, bool throw_if_not_exists)
{
bool removed = MultipleAccessStorage::removeImpl(id, throw_if_not_exists);
bool removed = MultipleAccessStorage::removeImpl(id, check_func, throw_if_not_exists);
if (removed)
changes_notifier->sendNotifications();
return removed;
Expand Down Expand Up @@ -666,6 +666,23 @@ void AccessControl::setDefaultProfileName(const String & default_profile_name)
settings_profiles_cache->setDefaultProfileName(default_profile_name);
}

std::optional<UUID> AccessControl::getDefaultProfileId() const
{
return settings_profiles_cache->getDefaultProfileId();
}


bool AccessControl::isDefaultProfileOrDescendant(const UUID & profile_id) const
{
return settings_profiles_cache->isDefaultProfileOrDescendant(profile_id);
}


bool AccessControl::isExpectedProfileOrDescendant(const UUID & profile_id, const UUID & expected_id) const
{
return settings_profiles_cache->isExpectedProfileOrDescendant(profile_id, expected_id);
}


void AccessControl::setCustomSettingsPrefixes(const Strings & prefixes)
{
Expand Down
8 changes: 6 additions & 2 deletions src/Access/AccessControl.h
Original file line number Diff line number Diff line change
Expand Up @@ -139,6 +139,10 @@ class AccessControl : public MultipleAccessStorage
/// The default profile's settings are always applied before any other profile's.
void setDefaultProfileName(const String & default_profile_name);

std::optional<UUID> getDefaultProfileId() const;
bool isDefaultProfileOrDescendant(const UUID & profile_id) const;
bool isExpectedProfileOrDescendant(const UUID & profile_id, const UUID & expected_id) const;

/// Sets prefixes which should be used for custom settings.
/// This function also enables custom prefixes to be used.
void setCustomSettingsPrefixes(const Strings & prefixes);
Expand Down Expand Up @@ -264,8 +268,8 @@ class AccessControl : public MultipleAccessStorage
class CustomSettingsPrefixes;
class PasswordComplexityRules;

bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) override;
bool removeImpl(const UUID & id, bool throw_if_not_exists) override;
bool insertImpl(const UUID & id, const AccessEntityPtr & entity, const CheckFunc & check_func, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) override;
bool removeImpl(const UUID & id, const CheckFunc & check_func, bool throw_if_not_exists) override;
bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override;

std::unique_ptr<ContextAccessCache> context_access_cache;
Expand Down
1 change: 1 addition & 0 deletions src/Access/Common/AccessType.h
Original file line number Diff line number Diff line change
Expand Up @@ -284,6 +284,7 @@ enum class AccessType : uint8_t
M(ALTER_SETTINGS_PROFILE, "ALTER PROFILE", GLOBAL, ACCESS_MANAGEMENT) \
M(DROP_SETTINGS_PROFILE, "DROP PROFILE", GLOBAL, ACCESS_MANAGEMENT) \
M(ALLOW_SQL_SECURITY_NONE, "CREATE SQL SECURITY NONE, ALLOW SQL SECURITY NONE, SQL SECURITY NONE, SECURITY NONE", GLOBAL, ACCESS_MANAGEMENT) \
M(PROTECTED_ACCESS_MANAGEMENT, "PROTECTED", GLOBAL, ALL) \
M(SHOW_USERS, "SHOW CREATE USER", GLOBAL, SHOW_ACCESS) \
M(SHOW_ROLES, "SHOW CREATE ROLE", GLOBAL, SHOW_ACCESS) \
M(SHOW_ROW_POLICIES, "SHOW POLICIES, SHOW CREATE ROW POLICY, SHOW CREATE POLICY", TABLE, SHOW_ACCESS) \
Expand Down
30 changes: 20 additions & 10 deletions src/Access/DiskAccessStorage.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -415,7 +415,7 @@ void DiskAccessStorage::setAllInMemory(const std::vector<std::pair<UUID, AccessE

/// Insert or update entities.
for (const auto & [id, entity] : entities_without_conflicts)
insertNoLock(id, entity, /* replace_if_exists = */ true, /* throw_if_exists = */ false, /* conflicting_id = */ nullptr, /* write_on_disk= */ false);
insertNoLock(id, entity, CheckFunc{}, /* replace_if_exists = */ true, /* throw_if_exists = */ false, /* conflicting_id = */ nullptr, /* write_on_disk= */ false);
}

void DiskAccessStorage::removeAllExceptInMemory(const boost::container::flat_set<UUID> & ids_to_keep)
Expand All @@ -425,7 +425,7 @@ void DiskAccessStorage::removeAllExceptInMemory(const boost::container::flat_set
const auto & id = it->first;
++it; /// We must go to the next element in the map `entries_by_id` here because otherwise removeNoLock() can invalidate our iterator.
if (!ids_to_keep.contains(id))
(void)removeNoLock(id, /* throw_if_not_exists */ true, /* write_on_disk= */ false);
(void)removeNoLock(id, CheckFunc{}, /* throw_if_not_exists */ true, /* write_on_disk= */ false);
}
}

Expand Down Expand Up @@ -504,14 +504,14 @@ std::optional<std::pair<String, AccessEntityType>> DiskAccessStorage::readNameWi
}


bool DiskAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id)
bool DiskAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, const CheckFunc & check_func, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id)
{
std::lock_guard lock{mutex};
return insertNoLock(id, new_entity, replace_if_exists, throw_if_exists, conflicting_id, /* write_on_disk = */ true);
return insertNoLock(id, new_entity, check_func, replace_if_exists, throw_if_exists, conflicting_id, /* write_on_disk = */ true);
}


bool DiskAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id, bool write_on_disk)
bool DiskAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, const CheckFunc & check_func, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id, bool write_on_disk)
{
const String & name = new_entity->getName();
AccessEntityType type = new_entity->getType();
Expand Down Expand Up @@ -565,7 +565,8 @@ bool DiskAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & ne
if (name_collision && (id_by_name != id))
{
assert(replace_if_exists);
removeNoLock(id_by_name, /* throw_if_not_exists= */ false, write_on_disk); // NOLINT
// 25.3 pattern: Pass check_func to removeNoLock when replacing by name
removeNoLock(id_by_name, check_func, /* throw_if_not_exists= */ false, write_on_disk); // NOLINT
}

if (id_collision)
Expand All @@ -576,6 +577,11 @@ bool DiskAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & ne
{
if (!existing_entry.entity || (*existing_entry.entity != *new_entity))
{
// 25.3 pattern: Check EXISTING entity before replacing (avoids regressions)
if (existing_entry.entity)
{
check_func(existing_entry.entity);
}
if (write_on_disk)
writeAccessEntityToDisk(id, *new_entity);
if (existing_entry.name != new_entity->getName())
Expand All @@ -590,7 +596,8 @@ bool DiskAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & ne
return true;
}

removeNoLock(id, /* throw_if_not_exists= */ false, write_on_disk); // NOLINT
// 25.3 pattern: Pass check_func to removeNoLock when replacing by ID
removeNoLock(id, check_func, /* throw_if_not_exists= */ false, write_on_disk); // NOLINT
}

/// Do insertion.
Expand All @@ -609,14 +616,14 @@ bool DiskAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & ne
}


bool DiskAccessStorage::removeImpl(const UUID & id, bool throw_if_not_exists)
bool DiskAccessStorage::removeImpl(const UUID & id, const CheckFunc & check_func, bool throw_if_not_exists)
{
std::lock_guard lock{mutex};
return removeNoLock(id, throw_if_not_exists, /* write_on_disk= */ true);
return removeNoLock(id, check_func, throw_if_not_exists, /* write_on_disk= */ true);
}


bool DiskAccessStorage::removeNoLock(const UUID & id, bool throw_if_not_exists, bool write_on_disk)
bool DiskAccessStorage::removeNoLock(const UUID & id, const CheckFunc & check_func, bool throw_if_not_exists, bool write_on_disk)
{
auto it = entries_by_id.find(id);
if (it == entries_by_id.end())
Expand All @@ -627,6 +634,9 @@ bool DiskAccessStorage::removeNoLock(const UUID & id, bool throw_if_not_exists,
return false;
}

if (check_func && it->second.entity)
check_func(it->second.entity);

Entry & entry = it->second;
AccessEntityType type = entry.type;

Expand Down
8 changes: 4 additions & 4 deletions src/Access/DiskAccessStorage.h
Original file line number Diff line number Diff line change
Expand Up @@ -40,8 +40,8 @@ class DiskAccessStorage : public IAccessStorage
std::vector<UUID> findAllImpl(AccessEntityType type) const override;
AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override;
std::optional<std::pair<String, AccessEntityType>> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const override;
bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) override;
bool removeImpl(const UUID & id, bool throw_if_not_exists) override;
bool insertImpl(const UUID & id, const AccessEntityPtr & entity, const CheckFunc & check_func, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) override;
bool removeImpl(const UUID & id, const CheckFunc & check_func, bool throw_if_not_exists) override;
bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override;

bool readLists() TSA_REQUIRES(mutex);
Expand All @@ -54,9 +54,9 @@ class DiskAccessStorage : public IAccessStorage
void listsWritingThreadFunc() TSA_NO_THREAD_SAFETY_ANALYSIS;
void stopListsWritingThread();

bool insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id, bool write_on_disk) TSA_REQUIRES(mutex);
bool insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, const CheckFunc & check_func, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id, bool write_on_disk) TSA_REQUIRES(mutex);
bool updateNoLock(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists, bool write_on_disk) TSA_REQUIRES(mutex);
bool removeNoLock(const UUID & id, bool throw_if_not_exists, bool write_on_disk) TSA_REQUIRES(mutex);
bool removeNoLock(const UUID & id, const CheckFunc & check_func, bool throw_if_not_exists, bool write_on_disk) TSA_REQUIRES(mutex);

AccessEntityPtr readAccessEntityFromDisk(const UUID & id) const;
void writeAccessEntityToDisk(const UUID & id, const IAccessEntity & entity) const;
Expand Down
5 changes: 4 additions & 1 deletion src/Access/HTTPAuthClient.h
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,10 @@ class HTTPAuthClient

Result authenticateRequest(Poco::Net::HTTPRequest & request) const
{
auto session = makeHTTPSession(HTTPConnectionGroupType::HTTP, uri, timeouts);
// Note: makeHTTPSession signature was updated to support custom CA certificates for S3.
// This call site was already using makeHTTPSession, but needed to be updated to match the new signature.
// For HTTP authentication requests, we pass an empty context (default) since we don't need custom CA certificates.
auto session = makeHTTPSession(HTTPConnectionGroupType::HTTP, uri, timeouts, {}, nullptr, {});
Poco::Net::HTTPResponse response;

auto milliseconds_to_wait = retry_initial_backoff_ms;
Expand Down
2 changes: 2 additions & 0 deletions src/Access/IAccessEntity.h
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,8 @@ struct IAccessEntity
virtual void setName(const String & name_) { name = name_; }
const String & getName() const { return name; }

virtual bool isProtected() const { return false; }

friend bool operator ==(const IAccessEntity & lhs, const IAccessEntity & rhs) { return lhs.equal(rhs); }
friend bool operator !=(const IAccessEntity & lhs, const IAccessEntity & rhs) { return !(lhs == rhs); }

Expand Down
Loading