Skip to content

Conversation

@tilman-aiven
Copy link

@tilman-aiven tilman-aiven commented Dec 4, 2025

Applying patches #1, #2, #5, #8, #14, #15, #16, #23, #37, #55

Patches

Core replication (patch group 1)

(1) Advertise host from config for replicated databases

(2) Enable internal replication for DatabaseReplicated clusters

(5) Enable ALTER DATABASE MODIFY SETTING for Replicated databases

(8) Replace MergeTree with ReplicatedMergeTree in Replicated databases

(14) Tolerate ZooKeeper restart with increased retries and exponential backoff

  • The backoff is applied in the catch block inside the inner loop, which means it runs after each node failure, not just between retry attempts.

(15) Fix ClickHouse restart with replicated tables containing {shard} macro

(16) Add missing settings to recoverLostReplica

(23) Fix unbounded replication queue growth

(37) Replicate ALTER TABLE MOVE PARTITION queries through DatabaseReplicated

(55) Change default logs_to_keep from 1000 to 300 for DatabaseReplicated

Security features (patch group 2)

(13) Restrict SHOW CREATE DATABASE access

(20) Allow custom CA certificate path for S3 connections

  • new structs S3AuthSettings, S3RequestSettings

(22) Allow custom CA certificate path for Azure Blob Storage connections

(26) Fix default profile escape vulnerability

(32) Allow delegating S3 signature to a separate process

(33) Allow delegating Azure signature to a separate process

(35) Enforce SSL in the MySQL handler

(45) Enforce HTTPS for URL storage and HTTPDictionarySource

(54) Allow avnadmin creating database using sql

(57) Add CHECK TABLE to default privileges

(58) Add SSL/TLS configuration support for PostgreSQL and MySQL connections

Appendix

tilman-aiven and others added 9 commits December 4, 2025 15:31
In containerized/cloud environments, the system hostname (e.g., pod-abc123) is not the network-accessible address. This causes replica communication to fail.

Use the explicitly configured interserver_http_host from config instead of system hostname.

The hostname is part of the Host ID hostname:port:database_uuid
and replicated database uses the Host ID for Zookeeper registration, replica discovery and DDL coordination.

Co-authored-by: Kevin Michel <[email protected]>
Enable internal_replication=true for clusters created programmatically by
DatabaseReplicated to improve performance and consistency of Distributed
tables over Replicated databases.

Unlike statically configured clusters in config.xml, DatabaseReplicated
creates clusters dynamically at runtime. This change ensures these
programmatically created clusters also benefit from internal replication.

Changes:
- Add internal_replication field to ClusterConnectionParameters struct
- Set internal_replication=true when creating cluster in DatabaseReplicated
- Update both Cluster constructors to pass internal_replication to addShard()

This ensures that Distributed tables over Replicated databases write to one
replica per shard, allowing ReplicatedMergeTree to handle replication
asynchronously, instead of writing to all replicas directly.

Benefits:
- Reduces network traffic (1× instead of N× writes per shard)
- Improves performance (one replica processes INSERT, others replicate async)
- Better consistency (uses ReplicatedMergeTree replication mechanism)

Co-authored-by: Kevin Michel <[email protected]>
Add support for altering DatabaseReplicated settings at runtime without
requiring database recreation. This is particularly important for
rotating cluster_secret, which previously required dropping and
recreating the entire database.

Changes:
- Override applySettingsChanges() in DatabaseReplicated to handle
  MODIFY SETTING commands
- Add applyChange() and has() methods to DatabaseReplicatedSettings
  wrapper to expose BaseSettings functionality
- Invalidate cached cluster when cluster_secret is changed to ensure
  new authentication credentials are used

Implementation details:
- Thread-safe: protected by DatabaseReplicated::mutex
- Validates setting existence before applying changes
- Automatically resets cluster cache when cluster_secret changes
- Supports all DatabaseReplicatedSettings (max_broken_tables_ratio,
  max_replication_lag_to_enqueue, collection_name, etc.)

Usage:
    ALTER DATABASE  MODIFY SETTING cluster_secret='new_secret'

Co-authored-by: Kevin Michel <[email protected]>
Automatically rewrite non-replicated MergeTree engine names to their
Replicated equivalents when creating tables in DatabaseReplicated
databases. This ensures all tables in a replicated database are
properly replicated, even if users specify ENGINE = MergeTree.

Changes:
- Add rewriteUnreplicatedMergeTreeEngines() method to StorageFactory
- Call rewrite method before engine name extraction in StorageFactory::get()
- Rewrite applies only on secondary queries (replica execution)
- Supports all MergeTree variants (ReplacingMergeTree, SummingMergeTree, etc.)

Implementation details:
- Checks if engine name ends with MergeTree (catches all variants)
- Verifies engine is not already replicated (starts with Replicated)
- Only rewrites when query_kind is SECONDARY_QUERY and database engine is Replicated
- Modifies AST in-place before storage engine instantiation

This allows users to write ENGINE = MergeTree in replicated databases
and automatically get ReplicatedMergeTree behavior without manual
engine name specification.

Co-authored-by: Kevin Michel <[email protected]>
Co-authored-by: Joe Lynch <[email protected]>
Co-authored-by: Dmitry Potepalov <[email protected]>
…koff

When ZooKeeper restarts (especially during version upgrades), it can be
unavailable for approximately 6 seconds. ClickHouse previously failed
queries if ZooKeeper was not available within ~3 seconds, leading to
inconsistent database state because DDL operations are not fully atomic.

This change improves ZooKeeper connection resilience by:
- Increasing minimum retry attempts from 3 to 6
- Adding exponential backoff between retry attempts (100ms, 200ms, 400ms...)
- Capping maximum backoff at 10 seconds to prevent excessive delays

The total retry window now covers typical ZooKeeper restart times (~6
seconds), allowing ClickHouse to successfully reconnect after ZooKeeper
restarts without requiring manual intervention.

This is particularly important during version upgrades when ZooKeeper
nodes restart sequentially, as it prevents DDL operations from failing
mid-execution and leaving the database in an inconsistent state.

Co-authored-by: Kevin Michel <[email protected]>
When ClickHouse restarts, tables in DatabaseReplicated databases are loaded
from metadata files using ATTACH operations. If a table's ZooKeeper path
contains the {shard} macro (e.g., /clickhouse/tables/{uuid}/{shard}),
the macro expansion fails during server startup because shard and replica
information is not populated in the MacroExpansionInfo structure.

The issue occurs because is_replicated_database is only true for
SECONDARY_QUERY operations (queries executed from the replicated queries
log in ZooKeeper). During server restart, ATTACH operations are not
SECONDARY_QUERY operations, so is_replicated_database is false, and the
shard/replica information is not populated, causing the error:

  Code: 62. DB::Exception: No macro 'shard' in config while processing
  substitutions in '/clickhouse/tables/{uuid}/{shard}'

This fix adds query.attach as an additional condition to populate shard
and replica information during ATTACH operations, ensuring that macro
expansion succeeds during server startup while maintaining the existing
behavior for replication queries.

The change is safe because:
- query.attach is only true during server startup when loading tables
  from disk metadata
- getReplicatedDatabaseShardName() and getReplicatedDatabaseReplicaName()
  use assert_cast, so they will fail fast if called on a non-replicated
  database
- This pattern is already used elsewhere in the codebase (e.g., line 28
  for allow_uuid_macro)

This restores functionality that previously worked in ClickHouse 21.x with
the custom {shard_name} macro, which was removed in later versions.

Co-authored-by: Kevin Michel <[email protected]>
When adding a new replica to an existing DatabaseReplicated cluster,
the recoverLostReplica() function reads table metadata from ZooKeeper
and recreates tables. However, the metadata stored in ZooKeeper contains
only the CREATE TABLE statement and table settings, not the global
settings that were active during original table creation.

If a table was created with the DEFLATE_QPL codec (which requires the
enable_deflate_qpl_codec global setting), a new replica would fail to
create the table during recovery because this setting is not enabled.

This fix explicitly enables enable_deflate_qpl_codec in the recovery
query context, ensuring that tables using this codec can be successfully
recreated on new replicas.

Co-authored-by: Kevin Michel <[email protected]>
If a replica is struggling to execute all tasks in its replication queue,
the queue can grow to a huge size and becomes even slower to manage when
it's larger (it's an std::list). This queue is also present in ZooKeeper,
for each replica. There is also a shared queue, which is called the log.
It's used to communicate changes between replicas. One replica will push
an item on the log as part of executing a query, then all other replicas
will read that log item and repush it in their own queue.

Each replica monitors the log size and which replicas are deemed lost
when they are too late to consume log items. However, the replicas do not
monitor the size of the queue of the other replicas. If a replica is
pushing lots of items in the log and another replica is able to copy them
to their own queue, but too slow to consume the queue items, then we have
a problem. At some point, if the imbalance persists, ZooKeeper will crash
because it won't have enough memory to hold all the queues. It will be
very hard to recover: ZooKeeper itself often corrupts its state when OOMing.
Manually cleaning hundreds of thousands of items in the queues to know
which ones can be removed and which ones are necessary to keep the
replicated data in a consistent state is almost impossible.

To fix that, we reuse the infrastructure that delays insert queries when
there are too many parts. The same logic is applied when any replica queue
size is too large for the table, or the grand total of all replicas' queue
sizes, over all tables, is too large. We also do the same to fail queries
if either counter reaches a second threshold, exactly like it's done with
the parts count.

In each replica, a thread is updating a local copy of the maximum queue
size in all replicas. A map is added to the context to keep track of these
per-storage maxima. These values are then used to gate the inserts. We do
it like that to avoid adding many ZooKeeper queries in the hot path, and
we don't need a very accurate queue size to get the feedback loop we need.

Changes:
- Add ReplicatedMergeTreeQueueSizeThread to periodically monitor queue sizes
- Add settings: queue_size_to_delay_insert, queue_size_to_throw_insert,
  queues_total_size_to_delay_insert, queues_total_size_to_throw_insert
- Add metric: ReplicatedQueuesTotalSize to track total queue size across
  all tables
- Extend delayInsertOrThrowIfNeeded() to check queue sizes in addition
  to parts count
- Add Context methods to track per-storage maximum queue sizes
- Integrate queue size monitoring into StorageReplicatedMergeTree startup
  and shutdown lifecycle

Co-authored-by: Kevin Michel <[email protected]>
Previously, MOVE PARTITION commands were not replicated through the
DatabaseReplicated DDL log, causing data inconsistency when moving
partitions between tables across replicas in a DatabaseReplicated cluster.

The issue occurred because shouldReplicateQuery() only checked for
AlterCommand (metadata changes) but not for PartitionCommand::MOVE_PARTITION.
Other partition commands like DROP PARTITION and ATTACH PARTITION are
handled by ReplicatedMergeTree's own replication mechanism, but MOVE
PARTITION TO TABLE requires database-level coordination to ensure both
source and destination tables are updated consistently across all replicas.

Changes:
- Add PartitionCommands.h include
- Extend shouldReplicateQuery() to detect MOVE_PARTITION commands
- Return true for MOVE_PARTITION to enable database-level replication

This ensures that when a partition is moved between tables, all replicas
execute the operation atomically, maintaining data consistency across
the cluster.

Co-authored-by: Joe Lynch <[email protected]>
@tilman-aiven tilman-aiven marked this pull request as ready for review December 8, 2025 08:46
Reduce the default value of  setting from 1000 to 300 for
DatabaseReplicated databases. This reduces ZooKeeper resource consumption
by ~70% while maintaining a 6x safety margin over max_replication_lag_to_enqueue (50).

Context:
Previously, we implemented a server-level setting (replicated_database_logs_to_keep)
to centralize control of this value. However, after analysis, we determined that:
1. Customers do not have ALTER_DATABASE_SETTINGS permission, so they cannot
   modify database settings via ALTER DATABASE MODIFY SETTING
2. The simpler approach of changing the database-level default is sufficient
3. No additional readonly checks are needed since access control already
   prevents customer modifications

This change affects only newly created databases. Existing databases retain
their current logs_to_keep value stored in ZooKeeper.

The default value of 300 provides adequate recovery buffer while significantly
reducing ZooKeeper memory usage in multi-database managed provider environments.

Co-authored-by: Khatskevich <[email protected]>
@tilman-aiven tilman-aiven force-pushed the v25.8.12.129-lts-aiven-core-replication branch from 244dd99 to 4f53d4e Compare December 8, 2025 13:02
tilman-aiven and others added 3 commits December 10, 2025 10:40
Main service users are allowed the SHOW DATABASES access because it is necessary for their operation. It is implicitly granted by ClickHouse when giving access to a database.

However, we do not want to give them access to SHOW CREATE DATABASE.
This query shows the entire create statement, unredacted. This is actually a useful feature for superusers, but can leak credentials to other users.

Also only show the create query in system.tables to users that were able to create that table.

Co-authored-by: Kevin Michel <[email protected]>
This change enables S3 object storage to use custom CA certificates
for HTTPS connections, which is useful for testing environments or
private deployments where certificates are not signed by public CAs.

The implementation adds a  configuration option that can be
specified in S3 disk settings. When provided, this path is used to
create a custom SSL context for all HTTPS connections to that S3
endpoint.

Key changes:
- Added  field to S3Settings struct with config loading,
  serialization, and deserialization support
- Extended PocoHTTPClientConfiguration to store and pass ca_path
- Updated ClientFactory::createClientConfiguration to accept ca_path
- Modified PocoHTTPClient to create SSL context from ca_path when
  provided
- Extended HTTP layer (HTTPCommon, HTTPConnectionPool) to accept and
  use custom SSL contexts
- Updated all makeHTTPSession call sites to match new signature
- Preserved ca_path through credentials provider chain

Backwards compatibility:
- ca_path is optional and defaults to std::nullopt
- Serialization includes backwards compatibility handling for old
  data formats without ca_path
- All existing call sites continue to work with default (empty)
  context parameter

Co-authored-by: Kevin Michel <[email protected]>
This change enables Azure Blob Storage to use custom CA certificates
for HTTPS connections, which is useful for testing environments or
private deployments where certificates are not signed by public CAs.

The implementation adds a  configuration option that can be
specified in Azure Blob Storage disk settings. When provided, this path
is used to configure Curl's CAInfo option (Azure SDK uses Curl
underneath) for all HTTPS connections to that Azure endpoint.

Key changes:
- Added  field to RequestSettings struct
- Updated getRequestSettings() to read ca_path from config
- Modified getClientOptions() to set curl_options.CAInfo when
  curl_ca_path is provided
- Endpoint-specific settings automatically supported through existing
  getRequestSettings() call

Implementation details:
- Uses Curl's built-in CAInfo option via Azure SDK's CurlTransportOptions
- Simpler than S3 implementation as no HTTP layer modifications needed
- Azure SDK handles HTTP/HTTPS layer, so we only configure Curl options

This commit is based on the original patch:
0049-Custom_Azure_certificate_authority.patch

Co-authored-by: Kevin Michel <[email protected]>
@tilman-aiven tilman-aiven changed the title [25.8 LTS] Apply Patch Group 1 (Core Replication) ClickHouse 25.8 LTS Dec 10, 2025
This change prevents users from bypassing default profile restrictions
by creating new users/roles that use less restrictive profiles (like
admin profile).

The security issue:
Normally, ClickHouse prevents switching to settings profiles that are
less restrictive than your current profile by verifying the new profiles
against the constraints of the current profile. However, if a user has
permission to create new users, they can create a user that directly
uses the admin profile, then connect directly with this user. This way
they can skip the verification that happens while switching profiles
and escape the constraints of the default profile.

The solution:
Enforce that all created profiles must inherit from the default profile
(or one of its descendants), unless the current user has the
`allow_non_default_profile` setting enabled. This works because the
admin profile is pre-created using users.xml and not via SQL, so it can
already exist and be configured with `allow_non_default_profile` set to
true, while the pre-created default profile has `allow_non_default_profile`
set to false.

Key changes:
- Added `allow_non_default_profile` setting to control profile creation
- Added methods to AccessControl and SettingsProfilesCache to check
  profile inheritance hierarchy
- Added circular dependency detection when creating/altering profiles
- Auto-inject default profile as parent if no parent is specified and
  user doesn't have `allow_non_default_profile` permission
- Added validation in SettingsConstraints to ensure parent profiles
  inherit from default

Co-authored-by: Kevin Michel <[email protected]>
@tilman-aiven tilman-aiven marked this pull request as draft December 10, 2025 14:04
tilman-aiven and others added 7 commits December 12, 2025 10:43
This commit adds support for delegating AWS S3 signature generation to
an external HTTP service. This enables use cases where signature
generation needs to be controlled by a proxy or external service for
security, compliance, or monitoring purposes.

The process is accessed over HTTP and the URL to access it can be
configured with the signature_delegation_url parameter in the S3 disk
configuration.

ClickHouse will make a POST request with a JSON body:
{
    "canonicalRequest": "PUT\n..."
}

And expects a JSON response with status 200 and this structure:
{
    "signature": "01234567890abcdef..."
}

The canonical request matches the format defined by AWS for signatures
and contains all the required information (path, host, operation...) to
let the proxy decide if the request is allowed and can be signed.

Changes:
- Added AWSAuthV4DelegatedSigner class that delegates signature
  generation to an external HTTP service
- Updated S3Client to use AWSAuthSignerProvider-based constructor
  (compatible with newer AWS SDK)
- Added signature_delegation_url configuration parameter to S3 disk
  settings
- Updated AWS SDK submodule to aiven/clickhouse-v25.8.12.129 branch
  which includes the necessary SDK changes for delegated signatures
- Updated all S3 client creation sites to pass the new parameter

This commit was adapted from the original patch to work with the newer
AWS SDK version (1.7.321) used in ClickHouse v25.8.12.129, which
requires using AWSAuthSignerProvider instead of the older constructor
signature.

Co-authored-by: Kevin Michel <[email protected]>
The process is accessed over HTTP and the URL to access it can be
configured with the signature_delegation_url parameter in the Azure
disk configuration. ClickHouse will make a POST request with a json body:
{
    "stringToSign": "..."
}
And expects a JSON response with status 200 and this structure:
{
    "signature": "01234567890abcdefg..."
}

The string to sign matches the format defined by Azure for signatures.
The canonical request also contains all the required information
(path, operation...) to let the proxy decide if the request is
allowed and can be signed. It's also enough information to know
the cost of the request.

This commit was applied from the patch file 0067-Delegated_signature_azure.patch
and adapted to work with ClickHouse v25.8.12.129 codebase structure.

Changes include:
- Updated Azure SDK submodule to aiven/azure-sdk-for-cpp fork
  with branch aiven/clickhouse-v25.8.12.129 containing the SDK
  modifications (virtual GetSignature method in SharedKeyPolicy)
- Added AzureDelegatedKeyPolicy class that extends SharedKeyPolicy
  to delegate signature generation via HTTP POST requests
- Added account_name and signature_delegation_url fields to
  RequestSettings for configuration
- Added delegated_signature flag to ConnectionParams to control
  client creation behavior
- Updated all ConnectionParams creation sites to include the
  delegated_signature flag
- Modified getClientOptions to inject AzureDelegatedKeyPolicy into
  PerRetryPolicies when signature delegation is enabled

Co-authored-by: Kevin Michel <[email protected]>
This commit enforces SSL/TLS encryption for all MySQL protocol connections
to ClickHouse. Previously, clients could connect without SSL and communicate
in plaintext. Now, any connection attempt without SSL is rejected with an
error message.

When a client attempts to connect without SSL:
- The connection is immediately rejected
- An error packet (MySQL error code 3159) is sent with a clear message
- The connection is closed

Error message: "SSL support for MySQL TCP protocol is required. If using
the MySQL CLI client, please connect with --ssl-mode=REQUIRED."

This is a security hardening change that ensures all MySQL protocol
traffic is encrypted, protecting credentials and data in transit. This
helps meet compliance requirements and prevents man-in-the-middle attacks.

This commit was applied from the patch file 0071-MySQL_enforce_SSL.patch

Co-authored-by: Joe Lynch <[email protected]>
This commit enforces HTTPS/TLS encryption for all HTTP/HTTPS connections
in two ClickHouse features:
1. HTTP Dictionary Sources - dictionaries loaded from remote URLs
2. URL Storage - tables that read/write data from remote URLs

Previously, both features allowed unencrypted HTTP connections, which
exposed credentials, queries, and data to potential interception.
Now, any attempt to use HTTP (non-HTTPS) URLs is rejected with a
clear error message.

Changes include:
- Changed HTTPDictionarySource::Configuration::url from std::string
  to Poco::URI for better type safety and scheme validation
- Added HTTPS validation in HTTPDictionarySource registration:
  throws UNSUPPORTED_METHOD error if scheme is not "https"
- Added HTTPS validation in StorageURL constructor:
  throws BAD_ARGUMENTS error if scheme is not "https"
- Simplified code by removing redundant Poco::URI object creations
  and using configuration.url directly throughout

Security impact:
- Prevents unencrypted transmission of dictionary data, table data,
  credentials, and query parameters
- Protects against man-in-the-middle attacks
- Helps meet compliance requirements (PCI-DSS, HIPAA, GDPR)
- Fails fast with clear error messages if HTTP is used

Breaking change: Existing configurations using http:// URLs will
fail and must be updated to use https:// URLs.

This commit was applied from the patch file 0091-HTTP-dictionary-source.patch

Co-authored-by: Joe Lynch <[email protected]>
This commit enables a special user (configured via
user_with_indirect_database_creation setting) to create and drop
Replicated databases through SQL with automatic parameter setup and
privilege management.

Main changes:
1. Allow special user calling CREATE DATABASE and DROP DATABASE, which
   overrides during execution phase, ensuring correct parameters are
   passed and running it with elevated privileges.

2. Introduce GRANT DEFAULT REPLICATED DATABASE PRIVILEGES statement,
   which grants a default set of privileges for a database. This
   maintains a common source of truth while creating databases from
   different environments.

3. Add server settings:
   - reserved_replicated_database_prefixes: Prohibits database names
     with certain prefixes
   - user_with_indirect_database_creation: User allowed simplified
     database creation
   - cluster_database: Reference database for cluster operations

4. Enhance DatabaseReplicated to store shard_macros for reuse when
   creating new databases.

5. Enforce ON CLUSTER requirement for non-admin users when dropping
   databases to ensure complete removal across the cluster.

Technical details:
- Added createReplicatedDatabaseByClient() to handle automatic database
  creation with proper cluster configuration
- Added checkDatabaseNameAllowed() to validate database names
- Modified executeDDLQueryOnCluster() to support skipping distributed
  checks for internal operations
- Added setGlobalContext() to Context for executing queries with
  elevated privileges

[DDB-1615] [DDB-1839] [DDB-1968]
CHECK TABLE requires an explicit grant since e96e0ae. This commit adds
CHECK to the default privileges granted when using GRANT DEFAULT
REPLICATED DATABASE PRIVILEGES, ensuring users can run CHECK TABLE on
tables in replicated databases without requiring an additional grant.

The change adds "CHECK, " to the privilege list in
InterpreterGrantQuery::execute() when handling default replicated
database privileges.

Co-authored-by: Aliaksei Khatskevich <[email protected]>
This commit adds comprehensive SSL/TLS configuration capabilities for
PostgreSQL and MySQL database connections in ClickHouse, along with a
security fix for the MariaDB connector.

Changes:

1. MariaDB Connector/C Security Fix:
   - Updated submodule to aiven/mariadb-connector-c fork
   - Fixed X509_check_host call to include hostname length parameter
   - Prevents potential certificate validation bypass vulnerabilities

2. PostgreSQL SSL Configuration:
   - Added SSLMode enum (DISABLE, ALLOW, PREFER, REQUIRE, VERIFY_CA, VERIFY_FULL)
   - Added server settings:
     * postgresql_connection_pool_ssl_mode (default: PREFER)
     * postgresql_connection_pool_ssl_root_cert (default: empty)
   - Updated PoolWithFailover to accept SSL mode and CA certificate path
   - Modified formatConnectionString to include sslmode and sslrootcert parameters
   - Integrated SSL settings across all PostgreSQL integration points:
     * DatabasePostgreSQL
     * DatabaseMaterializedPostgreSQL
     * StoragePostgreSQL
     * StorageMaterializedPostgreSQL
     * TableFunctionPostgreSQL
     * PostgreSQLDictionarySource

3. MySQL SSL Configuration:
   - Added MySQLSSLMode enum (DISABLE, PREFER, VERIFY_FULL)
   - Updated Connection, Pool, and PoolWithFailover classes to accept SSL mode
   - Added ssl_mode and ssl_root_cert to StorageMySQL::Configuration
   - Enhanced MySQL dictionary source to support ssl_mode in named collections
   - Integrated SSL settings in MySQLHelpers and StorageMySQL

Security Benefits:
- Enables encrypted connections to prevent data interception
- Supports certificate validation to prevent man-in-the-middle attacks
- Provides flexible SSL mode selection for different security requirements
- Fixes critical certificate hostname validation bug in MariaDB connector

The changes maintain backward compatibility with default SSL mode set to
PREFER, which attempts SSL but falls back gracefully if unavailable.

Co-authored-by: Joe Lynch <[email protected]>
@tilman-aiven tilman-aiven force-pushed the v25.8.12.129-lts-aiven-core-replication branch from 1080c5a to 3132b13 Compare December 15, 2025 09:14
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants