Commit Graph

568 Commits

Author SHA1 Message Date
Marcin Maliszkiewicz
3df951bc9c Merge 'audit: set audit_info for native-protocol BATCH messages' from Andrzej Jackowski
Commit 16b56c2451 ("Audit: avoid dynamic_cast on a hot path") moved
audit info into batch_statement via set_audit_info(), but only wired it
for the CQL-text BATCH path (raw::batch_statement::prepare()).
Native-protocol BATCH messages (opcode 0x0D), handled by
process_batch_internal in transport/server.cc, construct a
batch_statement without setting audit_info. This causes audit to
silently skip the entire batch.

Set audit_info on the batch_statement so these batches are audited.

Fixes SCYLLADB-1652

No backport - bug introduced recently.

Closes scylladb/scylladb#29570

* github.com:scylladb/scylladb:
  test/audit: add reproducer for native-protocol batch not being audited
  audit: set audit_info for native-protocol BATCH messages
  test/audit: rename internal test methods to avoid CI misdetection
2026-04-22 18:56:28 +02:00
Andrzej Jackowski
f5bb9b6282 audit: set audit_info for native-protocol BATCH messages
Commit 16b56c2451 ("Audit: avoid dynamic_cast on a hot path") moved
audit info into batch_statement via set_audit_info(), but only wired it
for the CQL-text BATCH path (raw::batch_statement::prepare()).
Native-protocol BATCH messages (opcode 0x0D), handled by
process_batch_internal in transport/server.cc, construct a
batch_statement without setting audit_info. This causes audit to
silently skip the entire batch.

Set audit_info on the batch_statement so these batches are audited.

Fixes SCYLLADB-1652
2026-04-21 21:52:26 +02:00
Marcin Maliszkiewicz
935e6a495d Merge 'transport: add per-service-level cql_requests_serving metric' from Piotr Smaron
The existing scylla_transport_requests_serving metric is a single global per-shard gauge counting outstanding CQL requests. When debugging latency spikes, it's useful to know which service level is contributing the most in-flight requests.
This PR adds a new per-scheduling-group gauge scylla_transport_cql_requests_serving (with the scheduling_group_name label), using the existing cql_sg_stats per-SG infrastructure. The cql_ prefix is intentional — it follows the convention of all other per-SG transport metrics (cql_requests_count, cql_request_bytes, etc.) and avoids Prometheus confusion with the global requests_serving metric (which lacks the scheduling_group_name label).

Fixes: SCYLLADB-1340

New feature, no backport.

Closes scylladb/scylladb#29493

* github.com:scylladb/scylladb:
  transport: add per-service-level cql_requests_serving metric
  transport: move requests_serving decrement to after response is sent
2026-04-21 12:35:50 +02:00
Piotr Smaron
218f8adc8f transport: add per-service-level cql_requests_serving metric
Add a per-scheduling-group gauge that tracks the number of in-flight CQL
requests for each service level. The existing scylla_transport_requests_serving
metric is a single global per-shard counter; the new metric breaks it down
by scheduling group so operators can see which service level contributes
the most in-flight requests when debugging latency.

The metric is named cql_requests_serving (exposed as
scylla_transport_cql_requests_serving) following the cql_ prefix convention
used by all other per-scheduling-group transport metrics (cql_requests_count,
cql_request_bytes, cql_response_bytes, cql_pending_response_memory). Using
a cql_ prefix avoids Prometheus confusion with the global requests_serving
metric, which lacks the scheduling_group_name label.

The counter is incremented when a request enters process_request() and
decremented in the same 'leave' defer block as the global requests_serving,
ensuring the request is counted as in-flight until the response is sent.
2026-04-17 15:07:14 +02:00
Piotr Smaron
4988077249 transport: move requests_serving decrement to after response is sent
The requests_serving metric was decremented right after query processing
completed, but before the response was written to the client. This means
requests whose responses were queued in the write pipeline were no longer
counted as in-flight, understating the actual load.

Move the decrement into the 'leave' defer block, which fires after the
response is fully sent via _ready_to_respond. This makes the shedding
check (max_concurrent_requests_per_shard) more accurate: requests that
have finished processing but are still waiting in the response queue now
correctly count toward the in-flight limit.
2026-04-17 15:05:29 +02:00
Alex
0f6d9ffd22 cql: expose stable result metadata for prepared LIST statements
Prepared LIST statements were not calculating metadata in PREPARE path, and sent empty string hash to client causing problematic behaviour where metadat_id was not recalculated correctly.
This patch moves metadata construction into get_result_metadata() for the affected LIST statements and reuse that metadata when building the result set.
This gives PREPARE a stable metadata id for LIST ROLES, LIST USERS, LIST PERMISSIONS and the service-level variants.
This patch also adds a new boost test that verifies that when an EXECUTE request carries an empty result metadata id while the server has a real metadata id for the result set, the response is marked METADATA_CHANGED and includes the full result metadata plus the server metadata id.
This covers the recovery path for clients that send an empty or otherwise unusable metadata id instead of a matching cached one.
2026-04-13 17:49:27 +03:00
Avi Kivity
0ae22a09d4 LICENSE: Update to version 1.1
Updated terms of non-commercial use (must be a never-customer).
2026-04-12 19:46:33 +03:00
Wojciech Mitros
163c6f71d6 transport: refactor result_message bounce interface
Replace move_to_shard()/move_to_host() with as_bounce()/target_shard()/
target_host() to clarify the interface after bounce was extended to
support cross-node bouncing.

- Add virtual as_bounce() returning const bounce* to the base class
  (nullptr by default, overridden in bounce to return this), replacing
  the virtual move_to_shard() which conflated bounce detection with
  shard access
- Rename move_to_shard() -> target_shard() (now non-virtual, returns
  unsigned directly) and move_to_host() -> target_host() on bounce
- Replace dynamic_pointer_cast with static_pointer_cast at call sites
  that already checked as_bounce()
- Move forward declarations of message types before the virtual
  methods so as_bounce() can reference bounce

Fixes: SCYLLADB-1066

Closes scylladb/scylladb#29367
2026-04-10 12:17:43 +02:00
Piotr Dulikowski
32e3a01718 Merge 'service: strong_consistency: Allow for aborting operations' from Dawid Mędrek
Motivation
----------

Since strongly consistent tables are based on the concept of Raft
groups, operations on them can get stuck for indefinite amounts of
time. That may be problematic, and so we'd like to implement a way
to cancel those operations at suitable times.

Description of solution
-----------------------

The situations we focus on are the following:

* Timed-out queries
* Leader changes
* Tablet migrations
* Table drops
* Node shutdowns

We handle each of them and provide validation tests.

Implementation strategy
-----------------------

1. Auxiliary commits.
2. Abort operations on timeout.
3. Abort operations on tablet removal.
4. Extend `client_state`.
5. Abort operation on shutdown.
6. Help `state_machine` be aborted as soon as possible.

Tests
-----

We provide tests that validate the correctness of the solution.

The total time spent on `test_strong_consistency.py`
(measured on my local machine, dev mode):

Before:
```
real    0m31.809s
user    1m3.048s
sys     0m21.812s
```

After:
```
real    0m34.523s
user    1m10.307s
sys     0m27.223s
```

The incremental differences in time can be found in the commit messages.

Fixes SCYLLADB-429

Backport: not needed. This is an enhancement to an experimental feature.

Closes scylladb/scylladb#28526

* github.com:scylladb/scylladb:
  service: strong_consistency: Abort state_machine::apply when aborting server
  service: strong_consistency: Abort ongoing operations when shutting down
  service: client_state: Extend with abort_source
  service: strong_consistency: Handle abort when removing Raft group
  service: strong_consistency: Abort Raft operations on timeout
  service: strong_consistency: Use timeout when mutating
  service: strong_consistency: Fix indentation
  service: strong_consistency: Enclose coordinator methods with try-catch
  service: strong_consistency: Crash at unexpected exception
  test: cluster: Extract default config & cmdline in test_strong_consistency.py
2026-04-10 11:11:21 +02:00
Botond Dénes
86417d49de Merge 'transport: improve memory accounting for big responses and slow network' from Marcin Maliszkiewicz
After obtaining the CQL response, check if its actual size exceeds the initially acquired memory permit. If so, acquire additional semaphore units and adopt them into the permit, ensuring accurate memory accounting for large responses.

Additionally, move the permit into a .then() continuation so that the semaphore units are kept alive until write_message finishes, preventing premature release of memory permit. This is especially important with slow networks and big responses when buffers can accumulate and deplete a node's memory.

Fixes: https://scylladb.atlassian.net/browse/SCYLLADB-1306
Related https://scylladb.atlassian.net/browse/SCYLLADB-740

Backport: all supported versions

Closes scylladb/scylladb#29288

* github.com:scylladb/scylladb:
  transport: add per-service-level pending response memory metric
  transport: hold memory permit until response write completes
  transport: account for response size exceeding initial memory estimate
2026-04-09 13:36:31 +03:00
Dawid Mędrek
4a87bdc778 service: client_state: Extend with abort_source
We make `client_state` store a pointer to an `abort_source`. This will
be useful in the following commit that will implement aborting ongoing
requests to strongly consistent tables upon connection shutdowns.
It might also be useful in some other places in the code in the future.

We set the abort source for client states in relevant places.
2026-04-09 11:35:35 +02:00
Marcin Maliszkiewicz
a74665b300 transport: add per-service-level pending response memory metric
Track the total memory consumed by responses waiting to be
written to the socket, exposed as a per-scheduling-group gauge
(cql_pending_response_memory). This complements the response
memory accounting added in the previous commits by giving
visibility into how much memory each service level is holding
in unsent response buffers.
2026-04-01 17:15:28 +02:00
Marcin Maliszkiewicz
a26ca0f5f7 transport: hold memory permit until response write completes
Capture the memory permit in the leave lambda's .finally()
continuation so that the semaphore units are kept alive until
write_response finishes, preventing premature release of
memory accounting.

This is especially important with slow network and big responses
when buffers can accumulate and deplete node's memory.
2026-03-31 14:05:00 +02:00
Marcin Maliszkiewicz
2645b95888 transport: account for response size exceeding initial memory estimate
After obtaining the CQL response, check if its actual size exceeds
the initially acquired memory permit. If so, take semaphore units
and adopt them into the permit (non blocking).

This doesn't fully prevent from allocating too much memory as
size is known when buffer is already allocated but improves
memory accounting for big responses.
2026-03-31 11:57:41 +02:00
Pavel Emelyanov
2d8540f1ee transport: fix process_startup cert-auth path missing connection-ready setup
When authenticate() returns a user directly (certificate-based auth,
introduced in 20e9619bb1), process_startup was missing the same
post-authentication bookkeeping that the no-auth and SASL paths perform:

  - update_scheduling_group(): without it, the connection runs under the
    default scheduling group instead of the one mapped to the user's
    service level.

  - _authenticating = false / _ready = true: without them,
    system.clients reports connection_stage = AUTHENTICATING forever
    instead of READY.

  - on_connection_ready(): without it, the connection never releases its
    slot in the uninitialized-connections concurrency semaphore (acquired
    at connection creation), leaking one unit per cert-authenticated
    connection for the lifetime of the connection.

The omission was introduced when on_connection_ready() was added to the
else and SASL branches in 474e84199c but the cert-auth branch was missed.

Fixes: 20e9619bb1 ("auth: support certificate-based authentication")

Signed-off-by: Pavel Emelyanov <xemul@scylladb.com>
Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com>
2026-03-24 18:02:46 +03:00
Piotr Dulikowski
d8b283e1fb Merge 'Add CQL forwarding for strongly consistent tables' from Wojciech Mitros
In this series we add support for forwarding strongly consistent CQL requests to suitable replicas, so that clients can issue reads/writes to any node and have the request executed on an appropriate tablet replica (and, for writes, on the Raft leader). We return the same CQL response as what the user would get while sending the request to the correct replica and we perform the same logging/stats updates on the request coordinator as if the coordinator was the appropriate replica.

The core mechanism of forwarding a strongly consistent request is sending an RPC containing the user's cql request frame to the appropriate replica and returning back a ready, serialized `cql_transport::response`. We do this in the CQL server - it is most prepared for handling these types and forwarding a request containing a CQL frame allows us to reuse near-top-level methods for CQL request handling in the new RPC handler (such as the general `process`)

For sending the RPC, the CQL server needs to obtain the information about who should it forward the request to. This requires knowledge about the tablet raft group members and leader. We obtain this information during the execution of a `cql3/strong_consistency` statement, and we return this information back to the CQL server using the generalized `bounce_to_shard` `response_message`, where we now store the information about either a shard, or a specific replica to which we should forward to. Similarly to `bounce_to_shard`, we need to handle this `result_message` in a loop - a replica may move during statement execution, or the Raft leader can change. We also use it for forwarding strongly consistent writes when we're not a member of the affected tablet raft group - in that case we need to forward the statement twice - once to any replica of the affected tablet, then that replica can find the leader and return this information to the coordinator, which allows the second request to be directed to the leader.

This feature also allows passing through exception messages which happened on the target replica while executing the statement. For that, many methods of the `cql_transport::cql_server::connection` for creating error responses needed to be moved to `cql_transport::cql_server`. And for final exception handling on the coordinator, we added additional error info to the RPC response, so that the handling can be performed without having the `result_message::exception` or `exception_ptr` itself.

Fixes [SCYLLADB-71](https://scylladb.atlassian.net/browse/SCYLLADB-71)

[SCYLLADB-71]: https://scylladb.atlassian.net/browse/SCYLLADB-71?atlOrigin=eyJpIjoiNWRkNTljNzYxNjVmNDY3MDlhMDU5Y2ZhYzA5YTRkZjUiLCJwIjoiZ2l0aHViLWNvbS1KU1cifQ

Closes scylladb/scylladb#27517

* github.com:scylladb/scylladb:
  test: add tests for CQL forwarding
  transport: enable CQL forwarding for strong consistency statements
  transport: add remote statement preparation for CQL forwarding
  transport: handle redirect responses in CQL forwarding
  transport: add exception handling for forwarded CQL requests
  transport: add basic CQL request forwarding
  idl: add a representation of client_state for forwarding
  cql_server: handle query, execute, batch in one case
  transport: inline process_on_shard in cql_server::process
  transport: extract process() to cql_server
  transport: add messaging_service to cql_server
  transport: add response reconstruction helpers for forwarding
  transport: generalize the bounce result message for bouncing to other nodes
  strong consistency: redirect requests to live replicas from the same rack
  transport: pass foreign_ptr into sleep_until_timeout_passes and move it to cql_server
  transport: extract the error handling from process_request_one
  transport: move error response helpers from connection to cql_server
2026-03-13 15:03:10 +01:00
Avi Kivity
e2eeef3e01 Merge 'service level: remove remnants of version 1 service level' from Gleb Natapov
can_use_effective_service_level_cache() always returns true now, so the function can be dropped entirely and all the code that assumes it may return false can be dropped as well. Also drop async versions of find_effective_service_level and get_user_scheduling_group since they are unused.

No need to backport, code removal,

Closes scylladb/scylladb#29002

* github.com:scylladb/scylladb:
  service level: make maybe_update_per_service_level_params synchronous
  service level: remove unused get_user_scheduling_group function
  service level: drop async find_effective_service_level
  service level: remove remnants of version 1 service level
2026-03-12 23:39:41 +02:00
Wojciech Mitros
32974770b0 test: add tests for CQL forwarding
Add basic cluster tests for CQL forwarding.
The test cases include:
- basic reads and writes
- prepared statements with binds
- forwarding from a non-replica
- exception passthrough during forwarding (using an injection)
- re-preparing a statement on the target node, even if the user
  query is also an EXECUTE request on a prepared statement
- verification metric updates

The existing test_basic_write_read was modified so that a few extra
cases could be validated on the same cluster.
2026-03-12 19:43:35 +01:00
Wojciech Mitros
21a7b036a5 transport: add remote statement preparation for CQL forwarding
During forwarding of CQL EXECUTE requests, the target node may
not have the prepared statement in its cache. If we do have this
statement as a coordinator, instead of returning PREPARED NOT FOUND
to the client, we want to prepare the statement ourselves on target
node.
For that, we add a new FORWARD_CQL_PREPARE RPC. We use the new RPC
after gettting the prepared_not_found status during forwarding. When
we try to forward a request, we always have the query string (we
decide whether to forward based on this query), so we can always use
the new RPC when getting the prepared_not_found status.
After receiving the response, we try forwarding the EXECUTE request
again.
2026-03-12 19:43:35 +01:00
Wojciech Mitros
96a5e1c7ce transport: handle redirect responses in CQL forwarding
During CQL forwarding, when the target node can't handle the request,
it will find another node which can execute the request or which knows
where the request can be executed. We return this information in
responses to CQL forwarding, and in this patch, we add handling of
this kind of a response.
After getting a redirect response, we retry forwarding to the returned
host/shard until success or timeout. This can happen many times during
a single request, when we first forward to a replica and later to the
coordinator, or when a replica/coordinator migrated while we were
performing the forwarding
2026-03-12 19:43:31 +01:00
Wojciech Mitros
8816d3038c transport: add exception handling for forwarded CQL requests
When a forwarded request fails on the remote node, we can't use the
exception handling that happens in process_request_one because we
don't go through this code path. Instead, we use the previously
extracted cql_server::handle_exception handler, which performs
all accounting on the forwarded-to node, and which prepares the
response. For the read_failure_exception_with_timeout exception,
we need to perform the sleep on the source node, so we return the
timeout in the forwarding response and use it on the source node
to know how long to sleep without any extra calculations.

The handle_forward_execute() method is extracted from the inline handler
lambda to make the error catching wrapper cleaner.
2026-03-12 19:41:37 +01:00
Wojciech Mitros
23bff5dfef transport: add basic CQL request forwarding
Add the infrastructure for forwarding CQL requests to other nodes.
When a process() call results in a node bounce (as opposed to a shard
bounce), the coordinator serializes the request and sends it via the
FORWARD_CQL_EXECUTE RPC verb to the target node.

In this patch we omit several features that allow handling more
scenarios that can happen when trying to forward a CQL request,
but the RPC request and response are already prepared for them.
They will be handled in the following commits.
2026-03-12 19:41:35 +01:00
Wojciech Mitros
b4a7fefe20 cql_server: handle query, execute, batch in one case
Currently we perform the same steps when handling query, execute
and batch CQL requests. So instead of creating multiple functions
performing these steps, we can handle them all in one fallthrough
case in cql_server::connection::process_request_one.
2026-03-12 17:48:58 +01:00
Wojciech Mitros
dadb87047c transport: inline process_on_shard in cql_server::process
The process_on_shard method is relatively short, it's only used
in the process() method and the Process concept that is uses
is as long as the function itself. This area will be made more
complex by the following patches for cql forwarding, so we simplify
it by inlining process_on_shard in cql_server::process.
2026-03-12 17:48:58 +01:00
Wojciech Mitros
24cdc3a10d transport: extract process() to cql_server
Move process() and process_on_shard() from cql_server::connection to
cql_server. The process() method is no longer a template - instead, it
takes an opcode parameter and uses get_process_fn_for_opcode() to select
the appropriate internal processing function.

The process_query, process_execute, and process_batch wrappers on
connection now delegate to _server.process() with the appropriate opcode.

This refactoring is preparation for CQL request forwarding, where
process() will need to be called from a context other than connection
- the forwarding RPC handler).
2026-03-12 17:48:57 +01:00
Wojciech Mitros
0e3469e89c transport: add messaging_service to cql_server
The messaging service will be used by cql_server to register RPC
handlers for forwarding CQL requests between nodes.
We pass it through the controller to cql_server.
2026-03-12 17:48:57 +01:00
Wojciech Mitros
e44820ba1f transport: generalize the bounce result message for bouncing to other nodes
In the following patches, we'll start allowing forwarding requests to strongly
consistent tables so that they'll get executed on the suitable tablet Raft group
members. For that we'll reuse the approach that we already have for bouncing
requests to other shards - we'll try to execute a request locally, and the
result of that will be a bounce message with another replica as the target.
In this patch we generalize the former bounce_to_shard result message so that
it will be able to specify the target of the bounce as another shard or specific
replica.
We also rename it to result_message::bounce so that it stops implying that only
another shard may be its target.
Aside from the host_id and the shard, the new message also includes the timeout,
because in the service handling the forwarding we won't have the access to it,
and it's needed for specifying how long we should wait for the forwarded
requests. It also includes an information whether this is a write request
to return correct timeout response in case the deadline is exceeded.
We will return other hosts in the new bounce message when executing requests to
strongly consistent tables when we can't handle the request because we aren't
a suitable replica. We can't handle this message yet, so we don't return it
anywhere and we still assume that every bounce message is a bounce to the same
host.
2026-03-12 17:48:57 +01:00
Wojciech Mitros
309abc44d9 transport: pass foreign_ptr into sleep_until_timeout_passes and move it to cql_server
Change sleep_until_timeout_passes() to accept a foreign_ptr<std::unique_ptr<response>>.
We can easily create the foreign_ptr for the responses created in the CQL server,
but we'll need this when we get responses when forwarding CQL statements - the responses
may come from other shards.
We also move it from cql_server::connection to cql_server, because for forwarded CQL
requests, we'll need to handle it at the cql_server level.
The method also loses its const qualifier - the abort_source that we pass into
sleep_abortable needs to be non-const. Apparently, we could still use it in a const
method of cql_server::connection because we passed it as _server._abort_source which
caused the const qualifier to be lost.
2026-03-12 16:03:14 +01:00
Gleb Natapov
c67f876893 service level: make maybe_update_per_service_level_params synchronous
It does not call async functions any more.
2026-03-12 15:53:08 +02:00
Gleb Natapov
f888f2dced service level: remove remnants of version 1 service level
can_use_effective_service_level_cache() always returns true now, so the
function can be dropped entirely and all the code that assumes it may
return false can be dropped as well.
2026-03-12 12:27:52 +02:00
Marcin Maliszkiewicz
b277d9d9aa cql3: track CQL parsing memory cost and use it for admission control
Use rolling_max_tracker to record gross bytes allocated during each
CQL parse.  The rolling maximum is then added to the memory estimate
for incoming QUERY and PREPARE requests so that the admission control
in the CQL transport layer accounts for parsing overhead.

The measured memory footprint serves as upper bound rather than
exact number but it's purpose is to prevent OOMs under unprepared
statements heavy load.

In benchmark 1G memory node shows decrease of non-LSA memory usage
from peak 320MB (our coordinator budget is 10% of 1G) to 96MB. While
tps drops from 1.2 kops to 0.8 kops. Drop in tps is expected as
memory admission kicks in trying to prevent OOM.
2026-03-12 10:16:10 +01:00
Wojciech Mitros
b1bd206147 transport: extract the error handling from process_request_one
When we forward CQL statements, we'll need to handle the errors
on the destination node. Only for read_failure_exception_with_timeout
exception, we'll still need to wait until timeout passes on the
source node.
For that we extract the exception handling to a separate method.
Additionally, we separate the waiting and all other handling,
so that all handling aside from waiting will be reusable after
forwarding, and we'll also be able to sleep on the source node
if necessary.
2026-03-11 19:40:47 +01:00
Wojciech Mitros
6184b1d5ea transport: move error response helpers from connection to cql_server
These methods are used only in the error handler in the cql server,
and outside of 3 cases, they don't need any information from the
cql_server::connection. We move them from cql_server::connection
to cql_server, so that they can be used in the following patches
for methods for CQL request forwarding where we'll have no instance
of cql_server::connection on the node forwarded to.

After the change the methods require no access to the server's
or connection's fields, so we also make them static methods.
2026-03-11 19:40:47 +01:00
Gleb Natapov
b59b3d4f8a service level: remove version 1 service level code 2026-03-10 10:46:48 +02:00
Dario Mirovic
d765b5b309 client_state: add _bypass_auth_checks flag
Authorization checks were previously skipped based on the
_is_internal flag. This couples two concerns: marking client
state as internal and bypassing authorization.

Introduce _bypass_auth_checks to handle only the authorization
bypass. Internal client state sets it to true, preserving current
behavior. External client state accepts it as a constructor
parameter, defaulting to false.

This will allow maintenance socket connections to skip
authorization without being marked as internal.

Refs SCYLLADB-409
2026-03-03 22:31:35 +01:00
Amnon Heiman
3175540e87 transport/server: to bytes_histogram
This patch replaces simple counters with bytes_histogram for tracking
CQL request and response sizes, enabling better visibility into message
size distribution.

Changes:
- Replace request_size and response_size metrics with bytes_histogram in
  cql_sg_stats::request_kind_stats
- Per-shard metrics continue to be reported as before
- QUERY, EXECUTE, and BATCH operations now report per-node, per-scheduling-group
  histograms of bytes sent and received, providing detailed insight into these
  operations

Other CQL operations (e.g., PREPARE, OPTIONS) are not included in per-node
histogram reporting as they are less performance-critical, but can be added
in the future if proven useful.

Metrics example:

```
 # HELP scylla_transport_cql_request_bytes Counts the total number of received bytes in CQL messages of a specific kind.
 # TYPE scylla_transport_cql_request_bytes counter
scylla_transport_cql_request_bytes{kind="BATCH",scheduling_group_name="sl:default",shard="0"} 129808
scylla_transport_cql_request_bytes{kind="EXECUTE",scheduling_group_name="sl:default",shard="0"} 227409
scylla_transport_cql_request_bytes{kind="PREPARE",scheduling_group_name="sl:default",shard="0"} 631
scylla_transport_cql_request_bytes{kind="QUERY",scheduling_group_name="sl:default",shard="0"} 2809
scylla_transport_cql_request_bytes{kind="QUERY",scheduling_group_name="sl:driver",shard="0"} 4079
scylla_transport_cql_request_bytes{kind="REGISTER",scheduling_group_name="sl:default",shard="0"} 98
scylla_transport_cql_request_bytes{kind="STARTUP",scheduling_group_name="sl:driver",shard="0"} 432
 # HELP scylla_transport_cql_request_histogram_bytes A histogram of received bytes in CQL messages of a specific kind and specific scheduling group.
 # TYPE scylla_transport_cql_request_histogram_bytes histogram
scylla_transport_cql_request_histogram_bytes_sum{kind="QUERY",scheduling_group_name="sl:driver"} 4079
scylla_transport_cql_request_histogram_bytes_count{kind="QUERY",scheduling_group_name="sl:driver"} 57
scylla_transport_cql_request_histogram_bytes_bucket{kind="QUERY",le="1024.000000",scheduling_group_name="sl:driver"} 57
scylla_transport_cql_request_histogram_bytes_bucket{kind="QUERY",le="2048.000000",scheduling_group_name="sl:driver"} 57
scylla_transport_cql_request_histogram_bytes_bucket{kind="QUERY",le="4096.000000",scheduling_group_name="sl:driver"} 57
scylla_transport_cql_request_histogram_bytes_bucket{kind="QUERY",le="8192.000000",scheduling_group_name="sl:driver"} 57
scylla_transport_cql_request_histogram_bytes_bucket{kind="QUERY",le="16384.000000",scheduling_group_name="sl:driver"} 57
scylla_transport_cql_request_histogram_bytes_bucket{kind="QUERY",le="32768.000000",scheduling_group_name="sl:driver"} 57
scylla_transport_cql_request_histogram_bytes_bucket{kind="QUERY",le="65536.000000",scheduling_group_name="sl:driver"} 57
scylla_transport_cql_request_histogram_bytes_bucket{kind="QUERY",le="131072.000000",scheduling_group_name="sl:driver"} 57
scylla_transport_cql_request_histogram_bytes_bucket{kind="QUERY",le="262144.000000",scheduling_group_name="sl:driver"} 57
scylla_transport_cql_request_histogram_bytes_bucket{kind="QUERY",le="524288.000000",scheduling_group_name="sl:driver"} 57
scylla_transport_cql_request_histogram_bytes_bucket{kind="QUERY",le="1048576.000000",scheduling_group_name="sl:driver"} 57
scylla_transport_cql_request_histogram_bytes_bucket{kind="QUERY",le="2097152.000000",scheduling_group_name="sl:driver"} 57
scylla_transport_cql_request_histogram_bytes_bucket{kind="QUERY",le="4194304.000000",scheduling_group_name="sl:driver"} 57
scylla_transport_cql_request_histogram_bytes_bucket{kind="QUERY",le="8388608.000000",scheduling_group_name="sl:driver"} 57
scylla_transport_cql_request_histogram_bytes_bucket{kind="QUERY",le="16777216.000000",scheduling_group_name="sl:driver"} 57
scylla_transport_cql_request_histogram_bytes_bucket{kind="QUERY",le="33554432.000000",scheduling_group_name="sl:driver"} 57
scylla_transport_cql_request_histogram_bytes_bucket{kind="QUERY",le="67108864.000000",scheduling_group_name="sl:driver"} 57
scylla_transport_cql_request_histogram_bytes_bucket{kind="QUERY",le="134217728.000000",scheduling_group_name="sl:driver"} 57
scylla_transport_cql_request_histogram_bytes_bucket{kind="QUERY",le="268435456.000000",scheduling_group_name="sl:driver"} 57
scylla_transport_cql_request_histogram_bytes_bucket{kind="QUERY",le="536870912.000000",scheduling_group_name="sl:driver"} 57
scylla_transport_cql_request_histogram_bytes_bucket{kind="QUERY",le="1073741824.000000",scheduling_group_name="sl:driver"} 57
```
2026-01-28 13:53:47 +02:00
Andrzej Jackowski
6eca7e4ff6 transport: unify lambda capture lifetime for control connections
Workload prioritization was added in scylladb/scylladb#22031.
The functionality of updating service levels was implemented as
a lambda coroutine, leaving room for the lambda coroutine fiasco.

The problem was noticed and addressed in scylladb/scylladb#26404.
There are currently three functions that call switch_tenant:
 - update_user_scheduling_group_v1 and update_user_scheduling_group_v2
   use the deducing this (this auto self) to ensure the proper
   lifecycle of the lambda capture.
 - update_control_connection_scheduling_group doesn’t use the deducing
   this, but the lambda captures only `this`, which is used before
   the first possible coroutine preemption. Therefore, it doesn’t seem
   that any memory corruption or undefined behavior is possible here.

Nevertheless, it seems better to start using the deducing this in
update_control_connection_scheduling_group as well, to avoid problems
in the future if someone modifies the code and forgets to add it.

Fixes: SCYLLADB-284

Closes scylladb/scylladb#28158
2026-01-17 20:36:31 +02:00
Avi Kivity
c6dfae5661 treewide: #include Seastar headers with angle brackets
Seastar is an external library from the point of view of
ScyllaDB, so should be included with angle brackets.

Closes scylladb/scylladb#27947
2026-01-13 14:56:15 +02:00
Vlad Zolotarov
ea95cdaaec transport/server: declare a new "CLIENT_OPTIONS" option as supported
Declare support for a 'CLIENT_OPTIONS' startup key.
This key is meant to be used by drivers for sending client-specific
configurations like request timeouts values, retry policy configuration, etc.
The value of this key can be any string in general (according to the CQL binary protocol),
however, it's expected to be some structured format, e.g. JSON.

Signed-off-by: Vlad Zolotarov <vladz@scylladb.com>
2025-12-20 12:26:22 -05:00
Vlad Zolotarov
28cbaef110 service/client_state and alternator/server: use cached values for driver_name and driver_version fields
Optimize memory usage changing types of driver_name and driver_version be
a reference to a cached value instead of an sstring.

These fields very often have the same value among different connections hence
it makes sense to cache these values and use references to them instead of duplicating
such strings in each connection state.

Signed-off-by: Vlad Zolotarov <vladz@scylladb.com>
2025-12-20 12:26:22 -05:00
Vlad Zolotarov
85adf6bdb1 system.clients: add a client_options column
This new column is going to contain all OPTIONS sent in the
STARTUP frame of the corresponding CQL session.

The new column has a `frozen<map<text, text>>` type, and
we are also optimizing the amount of required memory for storing
corresponding keys and values by caching them on each shard level.

Signed-off-by: Vlad Zolotarov <vladz@scylladb.com>
2025-12-20 12:26:15 -05:00
Vlad Zolotarov
3a54bab193 controller: update get_client_data to use foreign_ptr for client_data
get_client_data() is used to assemble `client_data` objects from each connection
on each CPU in the context of generation of the `system.clients` virtual table data.

After collected, `client_data` objects were std::moved and arranged into a
different structure to match the table's sorting requirements.

This didn't allow having not-cross-shard-movable objects as fields in the `client_data`,
e.g. lw_shared_ptr objects.

Since we are planning to add such fields to `client_data` in following patches this patch
is solving the limitation above by making get_client_data() return `foreign_ptr<std::unique_ptr<client_data>>`
objects instead of naked `client_data` ones.

Signed-off-by: Vlad Zolotarov <vladz@scylladb.com>
2025-12-19 11:01:41 -05:00
Piotr Dulikowski
7900aa5319 Merge 'server: fix scheduling group update timing in system.clients' from Alex Dathskovsky
Previously, the scheduling_group column was updated during the switch_tenant function, which meant the update occurred only after the tenant change operation completed—updating rows one by one. With this change, the scheduling_group column is now updated before the switch_tenant logic runs, ensuring that the table reflects the correct scheduling groups for all rows as early as possible.

fixes: #26060
fixes: #27295

backport: not required
this is a minor bug fix. Internal logic worked but the user couldnt see the change in the table if they would read the system.clients table

Closes scylladb/scylladb#26404

* github.com:scylladb/scylladb:
  test: cqlpy: Remove test_switch_tenants and add test in cluster testing. The test needs to run twice, in two separate Scylla runs, using two different modes: gossip and raft. The cluster framework supports this setup, while cqlpy only runs against Scylla instances in raft mode. Therefore, the test was moved from cqlpy to the cluster-based framework. This commit both adds the test in cluster/ and removes the old version in cqlpy/.
  server: Refactor update_control_connection_scheduling_group functionality This refactoring moves the logic that retrieves the scheduling group for driver_service_level_name out of switch_tenant. This change is possible because the scheduling group for the driver is retrieved from a map (LOOKUP). The lookup function is fully synchronized, non-coroutine, and returns immediately. For that reason, it’s better to perform this lookup outside of the switch_tenant function.
  server: Refactor scheduling group update functionality. This change generalizes the scheduling-group update functionality and removes some copy-paste code, improving overall readability and maintainability. To achieve this, capturing lambdas were introduced. As a result, self-deducing this was added to those lambdas to avoid coroutine-related issues (“coroutine fiasco”).
  server: Fix switch_tenant problem, When running on a V2 server, service-level data comes from service level cache. Because of this, we can use synchronized function to get the schedualing group. Since we are transitioning to a Raft-based architecture where all servers will be V2, we can safely implement this fix specifically for that case. This change adds get_cached_user_scheduling_group functionality and moves its usage out of switch_tenant function in update_scheduling_group_v2 usage.
  server: Add update_service_level_scheduling_group_v1 functions to create placehholder for functionality that will introduce v2 implementation. The new functionality will allow usage of service level cache
2025-12-16 15:39:49 +01:00
Andrzej Jackowski
c2b1b10ca0 service: transport: add CLIENT_ROUTES_CHANGE event
Introduce the CLIENT_ROUTES_CHANGE event to let drivers refresh
connections when `system.client_routes` is modified. Some deployments
(e.g., Private Link) require specific address/port mappings that can
change without topology changes and drivers need to adapt promptly
to avoid connectivity issues.

This new EVENT type carries a change indicator plus the affected
`connection_ids` and `host_ids`. The only change value is
`UPDATE_NODES`, meaning one or more client routes were inserted,
updated, or deleted.

Drivers subscribe using the existing events mechanism, so no additional
`cql_protocol_extension` key is required.

Ref: scylladb/scylla-enterprise#5699
2025-12-15 18:19:37 +01:00
Alex
30f6a40ae6 server: Refactor update_control_connection_scheduling_group functionality This refactoring moves the logic that retrieves the scheduling group for driver_service_level_name out of switch_tenant.
This change is possible because the scheduling group for the driver is retrieved from a map (LOOKUP).
The lookup function is fully synchronized, non-coroutine, and returns immediately.
For that reason, it’s better to perform this lookup outside of the switch_tenant function.
2025-12-14 18:46:06 +02:00
Alex
5579489c4c server: Refactor scheduling group update functionality. This change generalizes the scheduling-group update functionality and removes some copy-paste code, improving overall readability and maintainability.
To achieve this, capturing lambdas were introduced. As a result, self-deducing this was added to those lambdas to avoid coroutine-related issues (“coroutine fiasco”).
2025-12-14 18:46:05 +02:00
Alex
17c9d640fe server: Fix switch_tenant problem, When running on a V2 server, service-level data comes from service level cache. Because of this, we can use synchronized function to get the schedualing group.
Since we are transitioning to a Raft-based architecture where all servers will be V2, we can safely implement this fix specifically for that case.
This change adds get_cached_user_scheduling_group functionality and moves its usage out of switch_tenant function in update_scheduling_group_v2 usage.
2025-12-14 16:27:40 +02:00
Alex
f98af582a7 server: Add update_service_level_scheduling_group_v1 functions to create placehholder for functionality that will introduce v2 implementation.
The new functionality will allow usage of service level cache
2025-12-14 16:09:18 +02:00
Pavel Emelyanov
068d788084 transport: Don't use scattered_message
The API to put scattered_message into output_stream() is gone in seastar
API level 9, transport is the only place in Scylla that still uses it.

The change is to put the response as a sequence of temporary_buffer-s.
This preserves the zero-copy-ness of the reply, but needs few things to
care about.

First, the response header frame needs to be put as zero-copy buffer
too. Despite output_stream() supports semi-mixed mode, where z.c.
buffers can follow the buffered writes, it won't apply here. The socket
is flushed() in batched mode, so even if the first reply populates the
stream with data and flushes it, the next response may happen to start
putting the header frame before delayed flush took place.

Second, because socket is flushed in batch-flush poller, the temporary
buffers that are put into it must hold the foreigh_ptr with the response
object. With scattered message this was implemented with the help of a
delter that was attached to the message, now the deleter is shared
between all buffers.

Signed-off-by: Pavel Emelyanov <xemul@scylladb.com>
2025-10-17 10:17:08 +03:00
Andrzej Jackowski
f99b8c4a55 transport: use sl:driver to handle driver's control connections
Before `sl:driver` was introduced, service levels were assigned as
follows:
 1. New connections were processed in `main`.
 2. After user authentication was completed, the connection's SL was
    changed to the user's SL (or `sl:default` if the user had no SL).

This commit introduces `service_level_state` to `client_state` and
implements the following logic in `transport/server`:

 1. If `sl:driver` is not present in the system (for example, it was
    removed), service levels behave as described above.
 2. If `sl:driver` is present, the flow is:
   I.   New connections use `sl:driver`.
   II.  After user authentication is completed, the connection's SL is
        changed to the user's SL (or `sl:default`).
   III. If a REGISTER (to events) request is handled, the client is
        processing the control connection. We mark the client_state
        to permanently use `sl:driver`.

The aforementioned state `2.III` is represented by
`_control_connection` flag in `client_state`.

Fixes: scylladb/scylladb#24411
2025-10-08 08:25:28 +02:00