From f9e208b4f6734012e926050f3c7f6935dda9c5b4 Mon Sep 17 00:00:00 2001 From: Lucas Kent Date: Mon, 6 Jan 2025 10:41:36 +1100 Subject: [PATCH 1/3] Dont log message contents (#1862) --- Cargo.toml | 4 +- shotover/src/codec/cassandra.rs | 3 +- shotover/src/codec/kafka.rs | 4 +- shotover/src/connection.rs | 4 +- shotover/src/frame/cassandra.rs | 20 ++++++ shotover/src/frame/kafka.rs | 4 +- shotover/src/frame/mod.rs | 8 +-- shotover/src/frame/valkey.rs | 5 +- shotover/src/message/mod.rs | 2 +- shotover/src/server.rs | 20 +++--- .../transforms/cassandra/sink_cluster/mod.rs | 4 +- .../cassandra/sink_cluster/rewrite.rs | 41 +++++++----- .../cassandra/sink_cluster/topology.rs | 30 ++++----- shotover/src/transforms/chain.rs | 8 +-- shotover/src/transforms/debug/log_to_file.rs | 2 +- .../src/transforms/kafka/sink_cluster/mod.rs | 31 ++++----- .../kafka/sink_cluster/scram_over_mtls.rs | 2 +- .../scram_over_mtls/create_token.rs | 66 +++++++++++-------- shotover/src/transforms/protect/crypto.rs | 2 +- shotover/src/transforms/tee.rs | 3 +- .../util/cluster_connection_pool.rs | 9 +-- .../valkey/cluster_ports_rewrite.rs | 4 +- .../src/transforms/valkey/sink_cluster.rs | 31 +++++---- 23 files changed, 166 insertions(+), 141 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index be69dde5f..aca6cbaf7 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -36,7 +36,9 @@ reqwest = "0.12.0" redis = { version = "0.24.0", features = ["tokio-comp", "cluster"] } cdrs-tokio = "8.0" cassandra-protocol = "3.0" -tracing = "0.1.15" +# https://docs.rs/tracing/latest/tracing/level_filters/index.html#compile-time-filters +# `trace` level is considered development only, and may contain sensitive data, do not include it in release builds. +tracing = { version = "0.1.15", features = ["release_max_level_debug"] } tracing-subscriber = { version = "0.3.1", features = ["env-filter", "json"] } tracing-appender = "0.2.0" serde_json = "1.0" diff --git a/shotover/src/codec/cassandra.rs b/shotover/src/codec/cassandra.rs index 995ace34d..c8fa86c78 100644 --- a/shotover/src/codec/cassandra.rs +++ b/shotover/src/codec/cassandra.rs @@ -642,8 +642,7 @@ impl Decoder for CassandraDecoder { Err(CodecReadError::Parser(anyhow!(msg))) } err => Err(CodecReadError::Parser(anyhow!( - "Failed to parse frame {:?}", - err + "Failed to parse frame {err:?}" ))), } } diff --git a/shotover/src/codec/kafka.rs b/shotover/src/codec/kafka.rs index f310deeab..b1685b801 100644 --- a/shotover/src/codec/kafka.rs +++ b/shotover/src/codec/kafka.rs @@ -297,7 +297,9 @@ impl Encoder for KafkaEncoder { }) => { dst.extend_from_slice(&body.auth_bytes); } - _ => unreachable!("not expected {frame:?}"), + _ => unreachable!( + "Expected kafka sasl authenticate request or response but was not" + ), } Ok(()) } else { diff --git a/shotover/src/connection.rs b/shotover/src/connection.rs index 9a01fedf6..47079c8ce 100644 --- a/shotover/src/connection.rs +++ b/shotover/src/connection.rs @@ -385,8 +385,8 @@ async fn reader_task { - if let Err(err) = out_tx.send(messages) { - error!("Failed to send RespondAndThenCloseConnection message: {:?}", err); + if out_tx.send(messages).is_err() { + error!("Failed to send RespondAndThenCloseConnection message"); } return Err(ConnectionError::ShotoverClosed); } diff --git a/shotover/src/frame/cassandra.rs b/shotover/src/frame/cassandra.rs index ff522ee07..b476c1bc7 100644 --- a/shotover/src/frame/cassandra.rs +++ b/shotover/src/frame/cassandra.rs @@ -459,6 +459,26 @@ impl CassandraFrame { } } } +pub(crate) fn operation_name(operation: &CassandraOperation) -> &'static str { + match operation { + CassandraOperation::Query { .. } => "Query", + CassandraOperation::Result(_) => "Result", + CassandraOperation::Error(_) => "Error", + CassandraOperation::Prepare(_) => "Prepare", + CassandraOperation::Execute(_) => "Execute", + CassandraOperation::Register(_) => "Register", + CassandraOperation::Event(_) => "Event", + CassandraOperation::Batch(_) => "Batch", + CassandraOperation::Startup(_) => "Startup", + CassandraOperation::Ready(_) => "Ready", + CassandraOperation::Authenticate(_) => "Authenticate", + CassandraOperation::Options(_) => "Options", + CassandraOperation::Supported(_) => "Supported", + CassandraOperation::AuthChallenge(_) => "AuthChallenge", + CassandraOperation::AuthResponse(_) => "AuthResponse", + CassandraOperation::AuthSuccess(_) => "AuthSuccess", + } +} #[derive(PartialEq, Debug, Clone)] pub enum CassandraOperation { diff --git a/shotover/src/frame/kafka.rs b/shotover/src/frame/kafka.rs index e393190f1..8b4200fec 100644 --- a/shotover/src/frame/kafka.rs +++ b/shotover/src/frame/kafka.rs @@ -44,7 +44,7 @@ impl Display for KafkaFrame { header.unknown_tagged_fields )?; } - write!(f, " {:?}", body)?; + write!(f, " {body:?}")?; } KafkaFrame::Response { version, @@ -63,7 +63,7 @@ impl Display for KafkaFrame { header.unknown_tagged_fields )?; } - write!(f, " {body:?}",)?; + write!(f, " {body:?}")?; } } Ok(()) diff --git a/shotover/src/frame/mod.rs b/shotover/src/frame/mod.rs index 098a4a59b..cfe57bc03 100644 --- a/shotover/src/frame/mod.rs +++ b/shotover/src/frame/mod.rs @@ -237,14 +237,14 @@ impl Display for Frame { fn fmt(&self, f: &mut Formatter<'_>) -> FmtResult { match self { #[cfg(feature = "cassandra")] - Frame::Cassandra(frame) => write!(f, "Cassandra {}", frame), + Frame::Cassandra(frame) => write!(f, "Cassandra {frame}"), #[cfg(feature = "valkey")] - Frame::Valkey(frame) => write!(f, "Valkey {:?}", frame), + Frame::Valkey(frame) => write!(f, "Valkey {frame:?}"), #[cfg(feature = "kafka")] - Frame::Kafka(frame) => write!(f, "Kafka {}", frame), + Frame::Kafka(frame) => write!(f, "Kafka {frame}"), Frame::Dummy => write!(f, "Shotover internal dummy message"), #[cfg(feature = "opensearch")] - Frame::OpenSearch(frame) => write!(f, "OpenSearch: {:?}", frame), + Frame::OpenSearch(frame) => write!(f, "OpenSearch: {frame:?}"), } } } diff --git a/shotover/src/frame/valkey.rs b/shotover/src/frame/valkey.rs index d70d0ede1..ef17008b7 100644 --- a/shotover/src/frame/valkey.rs +++ b/shotover/src/frame/valkey.rs @@ -27,10 +27,7 @@ pub fn valkey_query_name(frame: &ValkeyFrame) -> Option { return Some(query_type); } Err(err) => { - tracing::error!( - "Failed to convert valkey bulkstring to string, err: {:?}", - err - ) + tracing::error!("Failed to convert valkey bulkstring to string, err: {err:?}") } } } diff --git a/shotover/src/message/mod.rs b/shotover/src/message/mod.rs index 1845c91d4..a9de459d9 100644 --- a/shotover/src/message/mod.rs +++ b/shotover/src/message/mod.rs @@ -553,7 +553,7 @@ impl Message { message_type, }) = &self.inner { - format!("Unparseable {:?} message {:?}", message_type, bytes) + format!("Unparseable {message_type:?} message {bytes:?}") } else { unreachable!("self.frame() failed so MessageInner must still be RawBytes") } diff --git a/shotover/src/server.rs b/shotover/src/server.rs index 43d495f82..6c619091b 100644 --- a/shotover/src/server.rs +++ b/shotover/src/server.rs @@ -26,8 +26,8 @@ use tokio_tungstenite::tungstenite::{ protocol::Message as WsMessage, }; use tokio_util::codec::{Decoder, Encoder, FramedRead, FramedWrite}; -use tracing::Instrument; use tracing::{debug, error, warn}; +use tracing::{trace, Instrument}; pub struct TcpCodecListener { chain_builder: TransformChainBuilder, @@ -358,13 +358,13 @@ async fn spawn_websocket_read_write_tasks< Err(CodecReadError::RespondAndThenCloseConnection(messages)) => { if let Err(err) = out_tx.send(messages) { // TODO we need to send a close message to the client - error!("Failed to send RespondAndThenCloseConnection message: {:?}", err); + error!("Failed to send RespondAndThenCloseConnection message: {err}"); } return; } Err(CodecReadError::Parser(err)) => { // TODO we need to send a close message to the client, protocol error - warn!("failed to decode message: {:?}", err); + warn!("failed to decode message: {err:?}"); return; } Err(CodecReadError::Io(_err)) => { @@ -480,12 +480,12 @@ pub fn spawn_read_write_tasks< } Err(CodecReadError::RespondAndThenCloseConnection(messages)) => { if let Err(err) = out_tx.send(messages) { - error!("Failed to send RespondAndThenCloseConnection message: {:?}", err); + error!("Failed to send RespondAndThenCloseConnection message: {err}"); } return; } Err(CodecReadError::Parser(err)) => { - warn!("failed to decode message: {:?}", err); + warn!("failed to decode message: {err:?}"); return; } Err(CodecReadError::Io(err)) => { @@ -494,7 +494,7 @@ pub fn spawn_read_write_tasks< // We shouldnt report that as a warning because its common for clients to do // that for performance reasons. if !matches!(err.kind(), ErrorKind::UnexpectedEof) { - warn!("failed to receive message on tcp stream: {:?}", err); + warn!("failed to receive message on tcp stream: {err:?}"); } return; } @@ -699,7 +699,7 @@ impl Handler { while let Ok(x) = in_rx.try_recv() { requests.extend(x); } - debug!("A transform in the chain requested that a chain run occur, requests {:?}", requests); + debug!("running transform chain because a transform in the chain requested that a chain run occur"); if let Some(close_reason) = self.send_receive_chain(local_addr, &out_tx, requests).await? { return Ok(close_reason) } @@ -710,7 +710,7 @@ impl Handler { while let Ok(x) = in_rx.try_recv() { requests.extend(x); } - debug!("Received requests from client {:?}", requests); + debug!("running transform chain because requests received from client"); if let Some(close_reason) = self.send_receive_chain(local_addr, &out_tx, requests).await? { return Ok(close_reason) } @@ -731,6 +731,7 @@ impl Handler { out_tx: &mpsc::UnboundedSender, requests: Messages, ) -> Result> { + trace!("running transform chain with requests: {requests:?}"); let mut wrapper = ChainState::new_with_addr(requests, local_addr); self.pending_requests.process_requests(&wrapper.requests); @@ -748,7 +749,8 @@ impl Handler { // send the result of the process up stream if !responses.is_empty() { - debug!("sending response to client: {:?}", responses); + debug!("sending {} responses to client", responses.len()); + trace!("sending response to client: {responses:?}"); if out_tx.send(responses).is_err() { // the client has disconnected so we should terminate this connection return Ok(Some(CloseReason::ClientClosed)); diff --git a/shotover/src/transforms/cassandra/sink_cluster/mod.rs b/shotover/src/transforms/cassandra/sink_cluster/mod.rs index 772794848..7e9e695d5 100644 --- a/shotover/src/transforms/cassandra/sink_cluster/mod.rs +++ b/shotover/src/transforms/cassandra/sink_cluster/mod.rs @@ -559,7 +559,7 @@ impl CassandraSinkCluster { } Err(GetReplicaErr::NoPreparedMetadata) => { let id = execute.id.clone(); - tracing::info!("forcing re-prepare on {:?}", id); + tracing::info!("forcing re-prepare on {id:?}"); // this shotover node doesn't have the metadata. // send an unprepared error in response to force // the client to reprepare the query @@ -632,7 +632,7 @@ impl CassandraSinkCluster { self.set_control_connection(connection, address); } tracing::info!( - "Control connection finalized against node at: {:?}", + "Control connection finalized against node at: {}", self.control_connection_address.unwrap() ); diff --git a/shotover/src/transforms/cassandra/sink_cluster/rewrite.rs b/shotover/src/transforms/cassandra/sink_cluster/rewrite.rs index e181deff6..451be5095 100644 --- a/shotover/src/transforms/cassandra/sink_cluster/rewrite.rs +++ b/shotover/src/transforms/cassandra/sink_cluster/rewrite.rs @@ -1,6 +1,7 @@ use super::node::ConnectionFactory; use super::node_pool::NodePool; use super::ShotoverNode; +use crate::frame::cassandra::operation_name; use crate::frame::{CassandraFrame, CassandraOperation, CassandraResult, Frame}; use crate::message::{Message, MessageIdMap, Messages}; use crate::{ @@ -350,9 +351,16 @@ impl MessageRewriter { CassandraOperation::Error(_), .. })) => None, - other => { - tracing::error!("Response to Prepare query was not a Prepared, was instead: {other:?}"); - warnings.push(format!("Shotover: Response to Prepare query was not a Prepared, was instead: {other:?}")); + Some(Frame::Cassandra(CassandraFrame { operation, .. })) => { + let operation_name = operation_name(operation); + tracing::error!("Response to Prepare query was not a Prepared, was instead: {operation_name}"); + warnings.push(format!("Shotover: Response to Prepare query was not a Prepared, was instead: {operation_name}")); + None + } + Some(_) => unreachable!("Response to prepare was not cassandra message"), + None => { + tracing::error!("Response to Prepare query was not parseable"); + warnings.push("Shotover: Response to Prepare query was not parseable".to_owned()); None } }) @@ -366,12 +374,15 @@ impl MessageRewriter { output }); - tracing::error!( - "Nodes did not return the same response to PREPARE statement {err_str}" + tracing::warn!( + "Nodes did not return the same response to PREPARE statement" + ); + tracing::trace!( + "Nodes did not return the same response to PREPARE statement:{err_str}" ); warnings.push(format!( - "Shotover: Nodes did not return the same response to PREPARE statement {err_str}" - )); + "Shotover: Nodes did not return the same response to PREPARE statement {err_str}" + )); } } @@ -544,10 +555,7 @@ impl MessageRewriter { } Ok(()) } else { - Err(anyhow!( - "Failed to parse system.local response {:?}", - peers_response - )) + Err(anyhow!("Failed to parse system.local response")) } } @@ -670,10 +678,7 @@ impl MessageRewriter { } Ok(()) } else { - Err(anyhow!( - "Failed to parse system.local response {:?}", - local_response - )) + Err(anyhow!("Failed to parse system.local response")) } } } @@ -831,13 +836,13 @@ fn parse_system_nodes(mut response: Message) -> Result, MessagePar "system.local returned error: {error:?}", ))), operation => Err(MessageParseError::ParseFailure(anyhow!( - "system.local returned unexpected cassandra operation: {operation:?}", + "system.local returned unexpected cassandra operation: {:?}", + operation_name(operation) ))), } } else { Err(MessageParseError::ParseFailure(anyhow!( - "Failed to parse system.local response {:?}", - response + "Failed to parse system.local response" ))) } } diff --git a/shotover/src/transforms/cassandra/sink_cluster/topology.rs b/shotover/src/transforms/cassandra/sink_cluster/topology.rs index 0027c8b50..85767da47 100644 --- a/shotover/src/transforms/cassandra/sink_cluster/topology.rs +++ b/shotover/src/transforms/cassandra/sink_cluster/topology.rs @@ -2,6 +2,7 @@ use super::node::{CassandraNode, ConnectionFactory}; use super::node_pool::KeyspaceMetadata; use super::KeyspaceChanTx; use crate::connection::SinkConnection; +use crate::frame::cassandra::operation_name; use crate::frame::{ cassandra::{parse_statement_single, Tracing}, value::GenericValue, @@ -93,7 +94,7 @@ async fn topology_task_process( register_for_topology_and_status_events(&mut connection, version).await?; tracing::info!( - "Topology task control connection finalized against node at: {:?}", + "Topology task control connection finalized against node at: {}", connection_info.address ); @@ -208,7 +209,7 @@ async fn register_for_topology_and_status_events( if let Some(Frame::Cassandra(CassandraFrame { operation, .. })) = response.frame() { match operation { CassandraOperation::Ready(_) => Ok(()), - operation => Err(anyhow!("Expected Cassandra to respond to a Register with a Ready. Instead it responded with {:?}", operation)) + operation => Err(anyhow!("Expected Cassandra to respond to a Register with a Ready. Instead it responded with {}", operation_name(operation))) } } else { Err(anyhow!("Failed to parse cassandra message")) @@ -231,7 +232,10 @@ async fn fetch_current_nodes( } mod system_keyspaces { - use crate::transforms::cassandra::sink_cluster::node_pool::ReplicationStrategy; + use crate::{ + frame::cassandra::operation_name, + transforms::cassandra::sink_cluster::node_pool::ReplicationStrategy, + }; use super::*; use std::str::FromStr; @@ -272,8 +276,8 @@ mod system_keyspaces { .map(|row| build_keyspace(row, data_center)) .collect(), operation => Err(anyhow!( - "keyspace query returned unexpected cassandra operation: {:?}", - operation + "keyspace query returned unexpected cassandra operation: {}", + operation_name(operation) )), } } else { @@ -463,15 +467,12 @@ mod system_local { }) .collect(), operation => Err(anyhow!( - "system.peers returned unexpected cassandra operation: {:?}", - operation + "system.peers returned unexpected cassandra operation: {}", + operation_name(operation) )), } } else { - Err(anyhow!( - "Failed to parse system.local response {:?}", - response - )) + Err(anyhow!("Failed to parse system.local response")) } } } @@ -604,14 +605,13 @@ mod system_peers { }) .collect(), operation => Err(anyhow!( - "system.peers or system.peers_v2 returned unexpected cassandra operation: {:?}", - operation + "system.peers or system.peers_v2 returned unexpected cassandra operation: {}", + operation_name(operation) )), } } else { Err(anyhow!( - "Failed to parse system.peers or system.peers_v2 response {:?}", - response + "Failed to parse system.peers or system.peers_v2 response", )) } } diff --git a/shotover/src/transforms/chain.rs b/shotover/src/transforms/chain.rs index 35a9b2e91..9fdd18ebd 100644 --- a/shotover/src/transforms/chain.rs +++ b/shotover/src/transforms/chain.rs @@ -96,7 +96,7 @@ impl BufferedChain { chain_state.flush, one_tx, )) - .map_err(|e| anyhow!("Couldn't send message to wrapped chain {:?}", e)) + .map_err(|e| anyhow!("Couldn't send message to wrapped chain {e}")) .await? } Some(timeout) => { @@ -110,7 +110,7 @@ impl BufferedChain { ), Duration::from_micros(timeout), ) - .map_err(|e| anyhow!("Couldn't send message to wrapped chain {:?}", e)) + .map_err(|e| anyhow!("Couldn't send message to wrapped chain {e}")) .await? } } @@ -137,7 +137,7 @@ impl BufferedChain { chain_state.requests, chain_state.local_addr, )) - .map_err(|e| anyhow!("Couldn't send message to wrapped chain {:?}", e)) + .map_err(|e| anyhow!("Couldn't send message to wrapped chain {e}")) .await? } Some(timeout) => { @@ -149,7 +149,7 @@ impl BufferedChain { ), Duration::from_micros(timeout), ) - .map_err(|e| anyhow!("Couldn't send message to wrapped chain {:?}", e)) + .map_err(|e| anyhow!("Couldn't send message to wrapped chain {e}")) .await? } } diff --git a/shotover/src/transforms/debug/log_to_file.rs b/shotover/src/transforms/debug/log_to_file.rs index 70711dff9..1fc734a24 100644 --- a/shotover/src/transforms/debug/log_to_file.rs +++ b/shotover/src/transforms/debug/log_to_file.rs @@ -115,7 +115,7 @@ impl Transform for DebugLogToFile { } async fn log_message(message: &Message, path: &Path) -> Result<()> { - info!("Logged message to {:?}", path); + info!("Logged message to {path:?}"); match message.clone().into_encodable() { Encodable::Bytes(bytes) => { tokio::fs::write(path, bytes) diff --git a/shotover/src/transforms/kafka/sink_cluster/mod.rs b/shotover/src/transforms/kafka/sink_cluster/mod.rs index 62a4a8e8e..8619a7cc1 100644 --- a/shotover/src/transforms/kafka/sink_cluster/mod.rs +++ b/shotover/src/transforms/kafka/sink_cluster/mod.rs @@ -841,9 +841,8 @@ impl KafkaSinkCluster { { Ok(node) => { tracing::debug!( - "Storing group_to_coordinator_broker metadata, group {:?} -> broker {}", - group.0, - node.broker_id.0 + "Storing group_to_coordinator_broker metadata, group {group:?} -> broker {:?}", + node.broker_id ); self.group_to_coordinator_broker .insert(group, node.broker_id); @@ -866,9 +865,8 @@ impl KafkaSinkCluster { { Ok(node) => { tracing::debug!( - "Storing transaction_to_coordinator_broker metadata, transaction {:?} -> broker {}", - transaction.0, - node.broker_id.0 + "Storing transaction_to_coordinator_broker metadata, transaction {transaction:?} -> broker {:?}", + node.broker_id ); self.transaction_to_coordinator_broker .insert(transaction, node.broker_id); @@ -1200,8 +1198,6 @@ The connection to the client has been closed." Some(Frame::Kafka(KafkaFrame::Request { header, .. })) => { let request_type = format!("{:?}", ApiKey::try_from(header.request_api_key).unwrap()); - // remove Key postfix, since its not part of the actual message name which is confusing. - let request_type = request_type.trim_end_matches("Key"); tracing::warn!("Routing for request of type {request_type:?} has not been implemented yet."); self.route_to_random_broker(request) } @@ -1267,7 +1263,7 @@ The connection to the client has been closed." ty: PendingRequestTy::Other, combine_responses: 1, }); - tracing::debug!("Routing request to single broker {:?}", destination.0); + tracing::debug!("Routing request to single broker {destination:?}"); } else { // The request has been split so it may be delivered to multiple destinations. // We must generate a unique request for each destination. @@ -1988,9 +1984,11 @@ The connection to the client has been closed." "Unexpected server error from FindCoordinator {err}" ))), }, - other => Err(anyhow!( - "Unexpected response returned to findcoordinator request {other:?}" + Some(Frame::Kafka(_)) => Err(anyhow!( + "Unexpected response returned to findcoordinator request" ))?, + None => Err(anyhow!("Response to FindCoordinator could not be parsed"))?, + _ => unreachable!("response to FindCoordinator was not a kafka response"), } } @@ -3018,8 +3016,7 @@ The connection to the client has been closed." // The broker doesnt know who the new leader is, clear the entire topic. self.topic_by_name.remove(topic_name); tracing::info!( - "Produce response included error NOT_LEADER_OR_FOLLOWER and so cleared topic {:?}", - topic_name, + "Produce response included error NOT_LEADER_OR_FOLLOWER and so cleared topic {topic_name:?}" ); break; } @@ -3430,9 +3427,7 @@ The connection to the client has been closed." }; tracing::debug!( - "Routing request relating to group id {:?} to broker {}", - group_id.0, - destination.0 + "Routing request relating to group id {group_id:?} to broker {destination:?}", ); self.pending_requests.push_back(PendingRequest { @@ -3458,9 +3453,7 @@ The connection to the client has been closed." }; tracing::debug!( - "Routing request relating to transaction id {:?} to broker {}", - transaction_id.0, - destination.0 + "Routing request relating to transaction id {transaction_id:?} to broker {destination:?}" ); self.pending_requests.push_back(PendingRequest { diff --git a/shotover/src/transforms/kafka/sink_cluster/scram_over_mtls.rs b/shotover/src/transforms/kafka/sink_cluster/scram_over_mtls.rs index 7fc3186c8..dee825eb8 100644 --- a/shotover/src/transforms/kafka/sink_cluster/scram_over_mtls.rs +++ b/shotover/src/transforms/kafka/sink_cluster/scram_over_mtls.rs @@ -125,7 +125,7 @@ async fn task( &username, delegation_token_lifetime ).await - .with_context(|| format!("Failed to recreate delegation token for {:?}", username))?; + .with_context(|| format!("Failed to recreate delegation token for {username:?}"))?; username_to_token.insert(username.clone(), token); recreate_queue.push(username.clone()); diff --git a/shotover/src/transforms/kafka/sink_cluster/scram_over_mtls/create_token.rs b/shotover/src/transforms/kafka/sink_cluster/scram_over_mtls/create_token.rs index 543a0a174..2b9442e8c 100644 --- a/shotover/src/transforms/kafka/sink_cluster/scram_over_mtls/create_token.rs +++ b/shotover/src/transforms/kafka/sink_cluster/scram_over_mtls/create_token.rs @@ -98,9 +98,11 @@ async fn find_new_brokers(nodes: &mut Vec, rng: &mut SmallRng) -> Result<( nodes.extend(new_nodes); Ok(()) } - other => Err(anyhow!( - "Unexpected message returned to metadata request {other:?}" - )), + Some(Frame::Kafka(_)) => Err(anyhow!( + "Unexpected response returned to findcoordinator request" + ))?, + None => Err(anyhow!("Response to FindCoordinator could not be parsed"))?, + _ => unreachable!("response to FindCoordinator was not a kafka response"), } } @@ -152,9 +154,13 @@ async fn create_delegation_token_for_user( Ok(response) } } - response => Err(anyhow!( - "Unexpected response to CreateDelegationToken {response:?}" - )), + Some(Frame::Kafka(_)) => Err(anyhow!( + "Unexpected response returned to CreateDelegationToken request" + ))?, + None => Err(anyhow!( + "Response to CreateDelegationToken could not be parsed" + ))?, + _ => unreachable!("response to CreateDelegationToken was not a kafka response"), } } @@ -253,28 +259,32 @@ async fn is_delegation_token_ready( }, ))])?; let mut response = connection.recv().await?.pop().unwrap(); - if let Some(Frame::Kafka(KafkaFrame::Response { - body: ResponseBody::DescribeDelegationToken(response), - .. - })) = response.frame() - { - if let Some(err) = ResponseError::try_from_code(response.error_code) { - return Err(anyhow!( - "Kafka's response to DescribeDelegationToken was an error: {err}" - )); - } - if response - .tokens - .iter() - .any(|x| x.hmac == create_response.hmac && x.token_id == create_response.token_id) - { - Ok(true) - } else { - Ok(false) + match response.frame() { + Some(Frame::Kafka(KafkaFrame::Response { + body: ResponseBody::DescribeDelegationToken(response), + .. + })) => { + if let Some(err) = ResponseError::try_from_code(response.error_code) { + return Err(anyhow!( + "Kafka's response to DescribeDelegationToken was an error: {err}" + )); + } + if response + .tokens + .iter() + .any(|x| x.hmac == create_response.hmac && x.token_id == create_response.token_id) + { + Ok(true) + } else { + Ok(false) + } } - } else { - Err(anyhow!( - "Unexpected response to CreateDelegationToken {response:?}" - )) + Some(Frame::Kafka(_)) => Err(anyhow!( + "Unexpected response returned to DescribeDelegationToken request" + ))?, + None => Err(anyhow!( + "Response to DescribeDelegationToken could not be parsed" + ))?, + _ => unreachable!("response to DescribeDelegationToken was not a kafka response"), } } diff --git a/shotover/src/transforms/protect/crypto.rs b/shotover/src/transforms/protect/crypto.rs index 49f15dc9f..e95fcbf67 100644 --- a/shotover/src/transforms/protect/crypto.rs +++ b/shotover/src/transforms/protect/crypto.rs @@ -51,7 +51,7 @@ pub async fn decrypt( ) -> Result { let bytes = match value { GenericValue::Bytes(bytes) => bytes, - _ => bail!("expected varchar to decrypt but was {:?}", value), + _ => bail!("expected varchar to decrypt but was not varchar"), }; let protected: Protected = bincode::deserialize(bytes)?; diff --git a/shotover/src/transforms/tee.rs b/shotover/src/transforms/tee.rs index 427a948dd..1bac9729c 100644 --- a/shotover/src/transforms/tee.rs +++ b/shotover/src/transforms/tee.rs @@ -574,8 +574,7 @@ async fn put_result_source( "regular-chain" => ResultSource::RegularChain, _ => { return Err(HttpServerError(anyhow!( - r"Invalid value for result source: {:?}, should be 'tee-chain' or 'regular-chain'", - new_result_source + r"Invalid value for result source: {new_result_source:?}, should be 'tee-chain' or 'regular-chain'" ))); } }; diff --git a/shotover/src/transforms/util/cluster_connection_pool.rs b/shotover/src/transforms/util/cluster_connection_pool.rs index 444e9fce9..744fb2b8e 100644 --- a/shotover/src/transforms/util/cluster_connection_pool.rs +++ b/shotover/src/transforms/util/cluster_connection_pool.rs @@ -90,10 +90,7 @@ impl, T: Token> ConnectionPool, connection_count: usize, ) -> Result, ConnectionError> { - debug!( - "getting {} pool connections to {} with token: {:?}", - connection_count, address, token - ); + debug!("getting {connection_count} pool connections to {address} with token: {token:?}",); let mut lanes = self.lanes.lock().await; let lane = lanes.entry(token.clone()).or_default(); @@ -205,7 +202,7 @@ pub fn spawn_read_write_tasks< tokio::spawn(async move { tokio::select! { result = tx_process(dummy_request_tx, stream_tx, out_rx, return_tx, encoder) => if let Err(e) = result { - trace!("connection write-closed with error: {:?}", e); + trace!("connection write-closed with error: {e:?}"); } else { trace!("connection write-closed gracefully"); }, @@ -218,7 +215,7 @@ pub fn spawn_read_write_tasks< tokio::spawn( async move { if let Err(e) = rx_process(dummy_request_rx, stream_rx, return_rx, decoder).await { - trace!("connection read-closed with error: {:?}", e); + trace!("connection read-closed with error: {e:?}"); } else { trace!("connection read-closed gracefully"); } diff --git a/shotover/src/transforms/valkey/cluster_ports_rewrite.rs b/shotover/src/transforms/valkey/cluster_ports_rewrite.rs index 048849a6e..2b9a22af1 100644 --- a/shotover/src/transforms/valkey/cluster_ports_rewrite.rs +++ b/shotover/src/transforms/valkey/cluster_ports_rewrite.rs @@ -137,9 +137,9 @@ fn rewrite_port_slot(frame: &mut Frame, new_port: u16) -> Result<()> { [ValkeyFrame::BulkString(_ip), ValkeyFrame::Integer(port), ..] => { *port = new_port.into(); } - _ => bail!("expected host-port in slot map but was: {:?}", frame), + _ => bail!("expected slot to start with bulkstring followed by integer, but was something else"), }, - _ => bail!("unexpected value in slot map: {:?}", frame), + _ => bail!("non array value in slot map"), } } }; diff --git a/shotover/src/transforms/valkey/sink_cluster.rs b/shotover/src/transforms/valkey/sink_cluster.rs index fc7313834..96439cb2f 100644 --- a/shotover/src/transforms/valkey/sink_cluster.rs +++ b/shotover/src/transforms/valkey/sink_cluster.rs @@ -221,7 +221,7 @@ impl ValkeySinkCluster { let command = match message.frame() { Some(Frame::Valkey(ValkeyFrame::Array(ref command))) => command, None => bail!("Failed to parse valkey frame"), - message => bail!("syntax error: bad command: {message:?}"), + _ => bail!("Invalid redis command, must be an array but was not"), }; let routing_info = RoutingInfo::for_command_frame(command)?; @@ -357,12 +357,12 @@ impl ValkeySinkCluster { self.connection_pool .new_unpooled_connection(address, token) .map_err(move |err| { - trace!("error fetching slot map from {}: {:?}", address, err); + trace!("error fetching slot map from {address}: {err:?}"); TransformError::from(err) }) .and_then(get_topology_from_node) .map_ok(move |slots| { - trace!("fetched slot map from {}: {:?}", address, slots); + trace!("fetched slot map from {address}: {slots:?}"); slots }), ); @@ -429,7 +429,7 @@ impl ValkeySinkCluster { } Err(e) => { // Intentional debug! Some errors should be silently passed through. - debug!("failed to connect to {}: {:?}", node, e); + debug!("failed to connect to {node}: {e:?}"); errors.push(e.into()); } } @@ -599,7 +599,7 @@ impl ValkeySinkCluster { let command = match message.frame() { Some(Frame::Valkey(ValkeyFrame::Array(ref command))) => command, None => bail!("Failed to parse valkey frame"), - message => bail!("syntax error: bad command: {message:?}"), + _ => bail!("syntax error: bad command"), }; let mut args = command.iter().skip(1).rev().map(|f| match f { @@ -952,9 +952,9 @@ async fn get_topology_from_node( ValkeyFrame::Error(message) => Err(TransformError::Upstream(ValkeyError::from_message( &message, ))), - frame => Err(TransformError::Protocol(format!( - "unexpected response for cluster slots: {frame:?}" - ))), + _ => Err(TransformError::Protocol( + "unexpected response for cluster slots to be an array but was not".to_owned(), + )), } } @@ -993,7 +993,7 @@ async fn receive_frame_response(receiver: oneshot::Receiver) -> Result match response?.frame() { Some(Frame::Valkey(frame)) => Ok(frame.take()), None => Err(anyhow!("Failed to parse valkey frame")), - response => Err(anyhow!("Unexpected valkey response: {response:?}")), + _ => Err(anyhow!("Expected valkey response but was not valkey.")), } } @@ -1070,7 +1070,6 @@ impl Transform for ValkeySinkCluster { while let Some(s) = responses.next().await { let original = requests.pop().unwrap(); - trace!("Got resp {:?}", s); let Response { response } = s.or_else(|e| -> Result { Ok(Response { response: Ok(Message::from_frame(Frame::Valkey(ValkeyFrame::Error( @@ -1184,13 +1183,13 @@ impl Authenticator for ValkeyAuthenticator { trace!("authenticated upstream as user: {:?}", token.username); Ok(()) } - ValkeyFrame::SimpleString(s) => Err(TransformError::Protocol(format!( - "expected OK but got: {s:?}" - ))), + ValkeyFrame::SimpleString(_) => Err(TransformError::Protocol( + "expected auth response to be \"OK\" but was a different SimpleString".to_owned(), + )), ValkeyFrame::Error(e) => Err(TransformError::Upstream(ValkeyError::from_message(&e))), - f => Err(TransformError::Protocol(format!( - "unexpected response type: {f:?}" - ))), + _ => Err(TransformError::Protocol( + "Expected auth response to be a SimpleString but was something else".to_owned(), + )), } } } From ac573cd282dd0690c9f2185135c21798266fcfc2 Mon Sep 17 00:00:00 2001 From: Lucas Kent Date: Mon, 6 Jan 2025 11:14:59 +1100 Subject: [PATCH 2/3] Update deps (#1863) --- Cargo.lock | 572 ++++++++---------- Cargo.toml | 2 +- shotover-proxy/Cargo.toml | 8 +- shotover/Cargo.toml | 4 +- shotover/src/server.rs | 2 +- test-helpers/Cargo.toml | 8 +- .../src/connection/cassandra/cql_ws.rs | 4 +- 7 files changed, 270 insertions(+), 330 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index c70ace228..946e5827c 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -151,9 +151,9 @@ dependencies = [ [[package]] name = "anyhow" -version = "1.0.94" +version = "1.0.95" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c1fd03a028ef38ba2276dce7e33fcd6369c158a1bca17946c4b1b701891c1ff7" +checksum = "34ac096ce696dc2fcabef30516bb13c0a68a11d30131d3df6f04711467681b04" [[package]] name = "arc-swap" @@ -182,7 +182,7 @@ checksum = "721cae7de5c34fbb2acd27e21e6d2cf7b886dce0c27388d46c4e6c47ea4318dd" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.94", ] [[package]] @@ -217,7 +217,7 @@ checksum = "99e1aca718ea7b89985790c94aad72d77533063fe00bc497bb79a7c2dae6a661" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.94", ] [[package]] @@ -228,9 +228,9 @@ checksum = "ace50bade8e6234aa140d9a2f552bbee1db4d353f69b8217bc503490fc1a9f26" [[package]] name = "aws-config" -version = "1.5.10" +version = "1.5.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9b49afaa341e8dd8577e1a2200468f98956d6eda50bcf4a53246cc00174ba924" +checksum = "649316840239f4e58df0b7f620c428f5fababbbca2d504488c641534050bd141" dependencies = [ "aws-credential-types", "aws-runtime", @@ -239,7 +239,7 @@ dependencies = [ "aws-sdk-sts", "aws-smithy-async", "aws-smithy-http", - "aws-smithy-json 0.60.7", + "aws-smithy-json", "aws-smithy-runtime", "aws-smithy-runtime-api", "aws-smithy-types", @@ -270,9 +270,9 @@ dependencies = [ [[package]] name = "aws-runtime" -version = "1.4.4" +version = "1.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b5ac934720fbb46206292d2c75b57e67acfc56fe7dfd34fb9a02334af08409ea" +checksum = "44f6f1124d6e19ab6daf7f2e615644305dc6cb2d706892a8a8c0b98db35de020" dependencies = [ "aws-credential-types", "aws-sigv4", @@ -295,15 +295,15 @@ dependencies = [ [[package]] name = "aws-sdk-kms" -version = "1.51.0" +version = "1.53.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3c30f6fd5646b99d9b45ec3a0c22e67112c175b2383100c960d7ee39d96c8d96" +checksum = "e349416a1998fde638deed85c18efeefd81af293439c16d676b7fce992904389" dependencies = [ "aws-credential-types", "aws-runtime", "aws-smithy-async", "aws-smithy-http", - "aws-smithy-json 0.61.1", + "aws-smithy-json", "aws-smithy-runtime", "aws-smithy-runtime-api", "aws-smithy-types", @@ -317,15 +317,15 @@ dependencies = [ [[package]] name = "aws-sdk-sso" -version = "1.50.0" +version = "1.52.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "05ca43a4ef210894f93096039ef1d6fa4ad3edfabb3be92b80908b9f2e4b4eab" +checksum = "cb25f7129c74d36afe33405af4517524df8f74b635af8c2c8e91c1552b8397b2" dependencies = [ "aws-credential-types", "aws-runtime", "aws-smithy-async", "aws-smithy-http", - "aws-smithy-json 0.61.1", + "aws-smithy-json", "aws-smithy-runtime", "aws-smithy-runtime-api", "aws-smithy-types", @@ -339,15 +339,15 @@ dependencies = [ [[package]] name = "aws-sdk-ssooidc" -version = "1.51.0" +version = "1.53.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "abaf490c2e48eed0bb8e2da2fb08405647bd7f253996e0f93b981958ea0f73b0" +checksum = "d03a3d5ef14851625eafd89660a751776f938bf32f309308b20dcca41c44b568" dependencies = [ "aws-credential-types", "aws-runtime", "aws-smithy-async", "aws-smithy-http", - "aws-smithy-json 0.61.1", + "aws-smithy-json", "aws-smithy-runtime", "aws-smithy-runtime-api", "aws-smithy-types", @@ -361,15 +361,15 @@ dependencies = [ [[package]] name = "aws-sdk-sts" -version = "1.51.0" +version = "1.53.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b68fde0d69c8bfdc1060ea7da21df3e39f6014da316783336deff0a9ec28f4bf" +checksum = "cf3a9f073ae3a53b54421503063dfb87ff1ea83b876f567d92e8b8d9942ba91b" dependencies = [ "aws-credential-types", "aws-runtime", "aws-smithy-async", "aws-smithy-http", - "aws-smithy-json 0.61.1", + "aws-smithy-json", "aws-smithy-query", "aws-smithy-runtime", "aws-smithy-runtime-api", @@ -407,9 +407,9 @@ dependencies = [ [[package]] name = "aws-smithy-async" -version = "1.2.1" +version = "1.2.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62220bc6e97f946ddd51b5f1361f78996e704677afc518a4ff66b7a72ea1378c" +checksum = "427cb637d15d63d6f9aae26358e1c9a9c09d5aa490d64b09354c8217cfef0f28" dependencies = [ "futures-util", "pin-project-lite", @@ -436,15 +436,6 @@ dependencies = [ "tracing", ] -[[package]] -name = "aws-smithy-json" -version = "0.60.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4683df9469ef09468dad3473d129960119a0d3593617542b7d52086c8486f2d6" -dependencies = [ - "aws-smithy-types", -] - [[package]] name = "aws-smithy-json" version = "0.61.1" @@ -466,9 +457,9 @@ dependencies = [ [[package]] name = "aws-smithy-runtime" -version = "1.7.4" +version = "1.7.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9f20685047ca9d6f17b994a07f629c813f08b5bce65523e47124879e60103d45" +checksum = "a05dd41a70fc74051758ee75b5c4db2c0ca070ed9229c3df50e9475cda1cb985" dependencies = [ "aws-smithy-async", "aws-smithy-http", @@ -481,7 +472,7 @@ dependencies = [ "http-body 0.4.6", "http-body 1.0.1", "httparse", - "hyper 0.14.31", + "hyper 0.14.32", "hyper-rustls 0.24.2", "once_cell", "pin-project-lite", @@ -510,9 +501,9 @@ dependencies = [ [[package]] name = "aws-smithy-types" -version = "1.2.9" +version = "1.2.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4fbd94a32b3a7d55d3806fe27d98d3ad393050439dd05eb53ece36ec5e3d3510" +checksum = "38ddc9bd6c28aeb303477170ddd183760a956a03e083b3902a990238a7e3792d" dependencies = [ "base64-simd", "bytes", @@ -580,18 +571,17 @@ dependencies = [ [[package]] name = "axum" -version = "0.7.9" +version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "edca88bc138befd0323b20752846e6587272d3b03b0343c8ea28a6f819e6e71f" +checksum = "6d6fd624c75e18b3b4c6b9caf42b1afe24437daaee904069137d8bab077be8b8" dependencies = [ - "async-trait", "axum-core", "bytes", "futures-util", "http 1.2.0", "http-body 1.0.1", "http-body-util", - "hyper 1.5.1", + "hyper 1.5.2", "hyper-util", "itoa", "matchit", @@ -601,7 +591,7 @@ dependencies = [ "pin-project-lite", "rustversion", "serde", - "sync_wrapper 1.0.2", + "sync_wrapper", "tokio", "tower", "tower-layer", @@ -611,11 +601,10 @@ dependencies = [ [[package]] name = "axum-core" -version = "0.4.5" +version = "0.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "09f2bd6146b97ae3359fa0cc6d6b376d9539582c7b4220f041a33ec24c226199" +checksum = "df1362f362fd16024ae199c1970ce98f9661bf5ef94b9808fee734bc3698b733" dependencies = [ - "async-trait", "bytes", "futures-util", "http 1.2.0", @@ -624,7 +613,7 @@ dependencies = [ "mime", "pin-project-lite", "rustversion", - "sync_wrapper 1.0.2", + "sync_wrapper", "tower-layer", "tower-service", "tracing", @@ -642,7 +631,7 @@ dependencies = [ "miniz_oxide", "object", "rustc-demangle", - "windows-targets 0.52.6", + "windows-targets", ] [[package]] @@ -710,9 +699,9 @@ dependencies = [ [[package]] name = "bigdecimal" -version = "0.4.6" +version = "0.4.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8f850665a0385e070b64c38d2354e6c104c8479c59868d1e48a0c13ee2c7a1c1" +checksum = "7f31f3af01c5c65a07985c804d3366560e6fa7883d640a122819b14ec327482c" dependencies = [ "autocfg", "libm", @@ -773,22 +762,22 @@ checksum = "79296716171880943b8470b5f8d03aa55eb2e645a4874bdbb28adb49162e012c" [[package]] name = "bytemuck" -version = "1.20.0" +version = "1.21.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8b37c88a63ffd85d15b406896cc343916d7cf57838a847b3a6f2ca5d39a5695a" +checksum = "ef657dfab802224e671f5818e9a4935f9b1957ed18e58292690cc39e7a4092a3" dependencies = [ "bytemuck_derive", ] [[package]] name = "bytemuck_derive" -version = "1.8.0" +version = "1.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bcfcc3cd946cb52f0bbfdbbcfa2f4e24f75ebb6c0e1002f7c25904fada18b9ec" +checksum = "3fa76293b4f7bb636ab88fd78228235b5248b4d05cc589aed610f954af5d7c7a" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.94", ] [[package]] @@ -843,7 +832,7 @@ dependencies = [ "darling", "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.94", ] [[package]] @@ -895,7 +884,7 @@ dependencies = [ "semver", "serde", "serde_json", - "thiserror 2.0.6", + "thiserror 2.0.9", ] [[package]] @@ -960,9 +949,9 @@ dependencies = [ [[package]] name = "cc" -version = "1.2.3" +version = "1.2.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "27f657647bcff5394bf56c7317665bbf790a137a50eaaa5c6bfbb9e27a518f2d" +checksum = "8d6dbb628b8f8555f86d0323c2eb39e3ec81901f4b83e091db8a6a76d316a333" dependencies = [ "jobserver", "libc", @@ -987,7 +976,7 @@ dependencies = [ "lazy_static", "rand", "serde_json", - "thiserror 2.0.6", + "thiserror 2.0.9", "tokio", "tracing", "uuid", @@ -1043,9 +1032,9 @@ dependencies = [ [[package]] name = "chrono" -version = "0.4.38" +version = "0.4.39" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a21f936df1771bf62b77f047b726c4625ff2e8aa607c01ec06e5a05bd8463401" +checksum = "7e36cc9d416881d2e24f9a963be5fb1cd90966419ac844274161d10488b3e825" dependencies = [ "android-tzdata", "iana-time-zone", @@ -1053,7 +1042,7 @@ dependencies = [ "num-traits", "serde", "wasm-bindgen", - "windows-targets 0.52.6", + "windows-targets", ] [[package]] @@ -1125,7 +1114,7 @@ dependencies = [ "heck", "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.94", ] [[package]] @@ -1184,12 +1173,12 @@ checksum = "5b63caa9aa9397e2d9480a9b13673856c78d8ac123288526c37d7839f2a86990" [[package]] name = "colored" -version = "2.1.0" +version = "2.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cbf2150cce219b664a8a70df7a1f933836724b503f8a413af9365b4dcc4d90b8" +checksum = "117725a109d387c937a1533ce01b450cbde6b88abceea8473c4d7a85853cda3c" dependencies = [ "lazy_static", - "windows-sys 0.48.0", + "windows-sys 0.59.0", ] [[package]] @@ -1208,15 +1197,15 @@ dependencies = [ [[package]] name = "console" -version = "0.15.8" +version = "0.15.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0e1f83fc076bd6dd27517eacdf25fef6c4dfe5f1d7448bafaaf3a26f13b5e4eb" +checksum = "ea3c6ecd8059b57859df5c69830340ed3c41d30e3da0c1cbed90a96ac853041b" dependencies = [ "encode_unicode", - "lazy_static", "libc", - "unicode-width", - "windows-sys 0.52.0", + "once_cell", + "unicode-width 0.2.0", + "windows-sys 0.59.0", ] [[package]] @@ -1371,18 +1360,18 @@ dependencies = [ [[package]] name = "crossbeam-channel" -version = "0.5.13" +version = "0.5.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "33480d6946193aa8033910124896ca395333cae7e2d1113d1fef6c3272217df2" +checksum = "06ba6d68e24814cb8de6bb986db8222d3a027d15872cabc0d18817bc3c0e4471" dependencies = [ "crossbeam-utils", ] [[package]] name = "crossbeam-deque" -version = "0.8.5" +version = "0.8.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "613f8cc01fe9cf1a3eb3d7f488fd2fa8388403e97039e2f73692932e291a770d" +checksum = "9dd111b7b7f7d55b72c0a6ae361660ee5853c9af73f70c3c2ef6858b950e2e51" dependencies = [ "crossbeam-epoch", "crossbeam-utils", @@ -1399,9 +1388,9 @@ dependencies = [ [[package]] name = "crossbeam-utils" -version = "0.8.20" +version = "0.8.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "22ec99545bb0ed0ea7bb9b8e1e9122ea386ff8a48c0922e43f36d45ab09e0e80" +checksum = "d0a5c400df2834b80a4c3327b3aad3a4c4cd4de0629063962b03235697506a28" [[package]] name = "crunchy" @@ -1486,7 +1475,7 @@ checksum = "f46882e17999c6cc590af592290432be3bce0428cb0d5f8b6715e4dc7b383eb3" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.94", ] [[package]] @@ -1526,7 +1515,7 @@ dependencies = [ "proc-macro2", "quote", "strsim", - "syn 2.0.90", + "syn 2.0.94", ] [[package]] @@ -1537,7 +1526,7 @@ checksum = "d336a2a514f6ccccaa3e09b02d41d35330c07ddf03a62165fcec10bb561c7806" dependencies = [ "darling_core", "quote", - "syn 2.0.90", + "syn 2.0.94", ] [[package]] @@ -1622,7 +1611,7 @@ checksum = "cb7330aeadfbe296029522e6c40f315320aba36fc43a5b3632f3795348f3bd22" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.94", "unicode-xid", ] @@ -1667,7 +1656,7 @@ checksum = "97369cbbc041bc366949bc74d34658d6cda5621039731c6310521892a3a20ae0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.94", ] [[package]] @@ -1777,9 +1766,9 @@ dependencies = [ [[package]] name = "encode_unicode" -version = "0.3.6" +version = "1.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a357d28ed41a50f9c765dbfe56cbc04a64e53e5fc58ba79fbc34c10ef3df831f" +checksum = "34aa73646ffb006b8f5147f3dc182bd4bcb190227ce861fc4a4844bf8e3cb2c0" [[package]] name = "encoding_rs" @@ -1904,9 +1893,9 @@ checksum = "3f9eec918d3f24069decb9af1554cad7c880e2da24a9afd88aca000531ab82c1" [[package]] name = "foldhash" -version = "0.1.3" +version = "0.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f81ec6369c545a7d40e4589b5597581fa1c441fe1cce96dd1de43159910a36a2" +checksum = "a0d2fde1f7b3d48b8395d5f2de76c18a528bd6a9cdde438df747bfcba3e05d6f" [[package]] name = "foreign-types" @@ -1934,9 +1923,9 @@ dependencies = [ [[package]] name = "fred" -version = "10.0.1" +version = "10.0.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0f5fbcd7118f15ce0ed032105c91137efa563996788a76a770e2fd928ddb243a" +checksum = "391bb3c769c0caea1c536b1e0802572740b8d686a9d363991277c6bc25b24eba" dependencies = [ "arc-swap", "async-trait", @@ -1949,7 +1938,7 @@ dependencies = [ "parking_lot", "rand", "redis-protocol", - "rustls 0.23.19", + "rustls 0.23.20", "rustls-native-certs 0.8.1", "semver", "socket2 0.5.8", @@ -1969,7 +1958,7 @@ checksum = "1458c6e22d36d61507034d5afecc64f105c1d39712b7ac6ec3b352c423f715cc" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.94", ] [[package]] @@ -2034,7 +2023,7 @@ checksum = "162ee34ebcb7c64a8abebc059ce0fee27c2262618d7b60ed8faf72fef13c3650" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.94", ] [[package]] @@ -2125,9 +2114,9 @@ checksum = "07e28edb80900c19c28f1072f2e8aeca7fa06b23cd4169cefe1af5aa3260783f" [[package]] name = "glob" -version = "0.3.1" +version = "0.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2fabcfbdc87f4758337ca535fb41a6d701b65693ce38287d856d1674551ec9b" +checksum = "a8d1add55171497b4705a648c6b583acafb01d58050a51727785f0b2c8e0a2b2" [[package]] name = "governor" @@ -2283,11 +2272,11 @@ dependencies = [ [[package]] name = "home" -version = "0.5.9" +version = "0.5.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e3d1354bf6b7235cb4a0576c2619fd4ed18183f689b12b006a0ee7329eeff9a5" +checksum = "589533453244b0995c858700322199b2becb13b627df2851f64a2775d024abcf" dependencies = [ - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] @@ -2369,9 +2358,9 @@ dependencies = [ [[package]] name = "hyper" -version = "0.14.31" +version = "0.14.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8c08302e8fa335b151b788c775ff56e7a03ae64ff85c548ee820fecb70356e85" +checksum = "41dfc780fdec9373c01bae43289ea34c972e40ee3c9f6b3c8801a35f35586ce7" dependencies = [ "bytes", "futures-channel", @@ -2393,9 +2382,9 @@ dependencies = [ [[package]] name = "hyper" -version = "1.5.1" +version = "1.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "97818827ef4f364230e16705d4706e2897df2bb60617d6ca15d598025a3c481f" +checksum = "256fb8d4bd6413123cc9d91832d78325c48ff41677595be797d90f42969beae0" dependencies = [ "bytes", "futures-channel", @@ -2420,7 +2409,7 @@ checksum = "ec3efd23720e2049821a693cbc7e65ea87c72f1c58ff2f9522ff332b1491e590" dependencies = [ "futures-util", "http 0.2.12", - "hyper 0.14.31", + "hyper 0.14.32", "log", "rustls 0.21.12", "rustls-native-certs 0.6.3", @@ -2430,15 +2419,15 @@ dependencies = [ [[package]] name = "hyper-rustls" -version = "0.27.3" +version = "0.27.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08afdbb5c31130e3034af566421053ab03787c640246a446327f550d11bcb333" +checksum = "2d191583f3da1305256f22463b9bb0471acad48a4e534a5218b9963e9c1f59b2" dependencies = [ "futures-util", "http 1.2.0", - "hyper 1.5.1", + "hyper 1.5.2", "hyper-util", - "rustls 0.23.19", + "rustls 0.23.20", "rustls-pki-types", "tokio", "tokio-rustls 0.26.1", @@ -2454,7 +2443,7 @@ checksum = "70206fc6890eaca9fde8a0bf71caa2ddfc9fe045ac9e5c70df101a7dbde866e0" dependencies = [ "bytes", "http-body-util", - "hyper 1.5.1", + "hyper 1.5.2", "hyper-util", "native-tls", "tokio", @@ -2473,7 +2462,7 @@ dependencies = [ "futures-util", "http 1.2.0", "http-body 1.0.1", - "hyper 1.5.1", + "hyper 1.5.2", "pin-project-lite", "socket2 0.5.8", "tokio", @@ -2619,7 +2608,7 @@ checksum = "1ec89e9337638ecdc08744df490b221a7399bf8d164eb52a665454e60e075ad6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.94", ] [[package]] @@ -2689,9 +2678,12 @@ checksum = "0d762194228a2f1c11063e46e32e5acb96e66e906382b9eb5441f2e0504bbd5a" [[package]] name = "inventory" -version = "0.3.15" +version = "0.3.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f958d3d68f4167080a18141e10381e7634563984a537f2a49a30fd8e53ac5767" +checksum = "e5d80fade88dd420ce0d9ab6f7c58ef2272dde38db874657950f827d4982c817" +dependencies = [ + "rustversion", +] [[package]] name = "ipnet" @@ -2743,6 +2735,15 @@ dependencies = [ "either", ] +[[package]] +name = "itertools" +version = "0.14.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2b192c782037fadd9cfa75548310488aabdbf3d2da73885b31bd0abd03351285" +dependencies = [ + "either", +] + [[package]] name = "itoa" version = "1.0.14" @@ -2842,9 +2843,9 @@ dependencies = [ [[package]] name = "libc" -version = "0.2.167" +version = "0.2.169" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "09d6582e104315a817dff97f75133544b2e094ee22447d2acf4a74e189ba06fc" +checksum = "b5aba8db14291edd000dfcc4d620c7ebfb122c613afb886ca8803fa4e128a20a" [[package]] name = "libloading" @@ -2853,7 +2854,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fc2f4eb4bc735547cfed7c0a4922cbd04a4655978c09b54f1f7b228750664c34" dependencies = [ "cfg-if", - "windows-targets 0.52.6", + "windows-targets", ] [[package]] @@ -2922,9 +2923,9 @@ dependencies = [ [[package]] name = "matchit" -version = "0.7.3" +version = "0.8.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0e7465ac9959cc2b1404e8e2367b43684a6d13790fe23056cc8c6c5a6b7bcb94" +checksum = "47e1ffaa40ddd1f3ed91f717a33c8c0ee23fff369e3aa8772b9605cc1d22f4c3" [[package]] name = "md5" @@ -3000,9 +3001,9 @@ checksum = "68354c5c6bd36d73ff3feceb05efa59b6acb7626617f4962be322a825e61f79a" [[package]] name = "miniz_oxide" -version = "0.8.0" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e2d80299ef12ff69b16a84bb182e3b9df68b5a91574d3d4fa6e41b65deec4df1" +checksum = "4ffbe83022cedc1d264172192511ae958937694cd57ce297164951b8b3568394" dependencies = [ "adler2", ] @@ -3192,14 +3193,14 @@ dependencies = [ "proc-macro-crate", "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.94", ] [[package]] name = "object" -version = "0.36.5" +version = "0.36.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "aedf0a2d09c573ed1d8d85b30c119153926a2b36dce0ab28322c09a117a4683e" +checksum = "62948e14d923ea95ea2c7c86c71013138b66525b86bdc08d2dcc262bdb497b87" dependencies = [ "memchr", ] @@ -3265,7 +3266,7 @@ checksum = "a948666b637a0f465e8564c73e89d4dde00d72d4d473cc972f390fc3dcee7d9c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.94", ] [[package]] @@ -3298,9 +3299,9 @@ dependencies = [ [[package]] name = "ordered-float" -version = "4.5.0" +version = "4.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c65ee1f9701bf938026630b455d5315f490640234259037edb259798b3bcf85e" +checksum = "7bb71e1b3fa6ca1c61f383464aaf2bb0e2f8e772a1f01d486832464de363b951" dependencies = [ "num-traits", "rand", @@ -3377,7 +3378,7 @@ dependencies = [ "libc", "redox_syscall", "smallvec", - "windows-targets 0.52.6", + "windows-targets", ] [[package]] @@ -3629,9 +3630,9 @@ dependencies = [ [[package]] name = "quanta" -version = "0.12.3" +version = "0.12.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8e5167a477619228a0b284fac2674e3c388cba90631d7b7de620e6f1fcd08da5" +checksum = "3bd1fe6824cea6538803de3ff1bc0cf3949024db3d43c9643024bfb33a807c0e" dependencies = [ "crossbeam-utils", "libc", @@ -3653,9 +3654,9 @@ dependencies = [ "quinn-proto", "quinn-udp", "rustc-hash", - "rustls 0.23.19", + "rustls 0.23.20", "socket2 0.5.8", - "thiserror 2.0.6", + "thiserror 2.0.9", "tokio", "tracing", ] @@ -3671,10 +3672,10 @@ dependencies = [ "rand", "ring", "rustc-hash", - "rustls 0.23.19", + "rustls 0.23.20", "rustls-pki-types", "slab", - "thiserror 2.0.6", + "thiserror 2.0.9", "tinyvec", "tracing", "web-time", @@ -3682,9 +3683,9 @@ dependencies = [ [[package]] name = "quinn-udp" -version = "0.5.7" +version = "0.5.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7d5a626c6807713b15cac82a6acaccd6043c9a5408c24baae07611fec3f243da" +checksum = "1c40286217b4ba3a71d644d752e6a0b71f13f1b6a2c5311acfcbe0c2418ed904" dependencies = [ "cfg_aliases 0.2.1", "libc", @@ -3696,9 +3697,9 @@ dependencies = [ [[package]] name = "quote" -version = "1.0.37" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b5b9d34b8991d19d98081b46eacdd8eb58c6f2b201139f7c5f643cc155a633af" +checksum = "0e4dccaaaf89514f546c693ddc140f729f958c247918a13380cccc6078391acc" dependencies = [ "proc-macro2", ] @@ -3785,9 +3786,9 @@ dependencies = [ [[package]] name = "rcgen" -version = "0.13.1" +version = "0.13.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "54077e1872c46788540de1ea3d7f4ccb1983d12f9aa909b234468676c1a36779" +checksum = "75e669e5202259b5314d1ea5397316ad400819437857b90861765f24c4cf80a2" dependencies = [ "pem", "ring", @@ -3798,9 +3799,9 @@ dependencies = [ [[package]] name = "rdkafka" -version = "0.36.2" +version = "0.37.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1beea247b9a7600a81d4cc33f659ce1a77e1988323d7d2809c7ed1c21f4c316d" +checksum = "14b52c81ac3cac39c9639b95c20452076e74b8d9a71bc6fc4d83407af2ea6fff" dependencies = [ "futures-channel", "futures-util", @@ -3866,9 +3867,9 @@ dependencies = [ [[package]] name = "redox_syscall" -version = "0.5.7" +version = "0.5.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9b6dfecf2c74bce2466cabf93f6664d6998a69eb21e39f4207930065b27b771f" +checksum = "03a862b389f93e68874fbf580b9de08dd02facb9a788ebadaf4a3fd33cf58834" dependencies = [ "bitflags", ] @@ -3931,9 +3932,9 @@ checksum = "ba39f3699c378cd8970968dcbff9c43159ea4cfbd88d43c00b22f2ef10a435d2" [[package]] name = "reqwest" -version = "0.12.9" +version = "0.12.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a77c62af46e79de0a562e1a9849205ffcb7fc1238876e9bd743357570e04046f" +checksum = "43e734407157c3c2034e0258f5e4473ddb361b1e85f95a66690d67264d7cd1da" dependencies = [ "async-compression", "base64 0.22.1", @@ -3945,8 +3946,8 @@ dependencies = [ "http 1.2.0", "http-body 1.0.1", "http-body-util", - "hyper 1.5.1", - "hyper-rustls 0.27.3", + "hyper 1.5.2", + "hyper-rustls 0.27.5", "hyper-tls", "hyper-util", "ipnet", @@ -3958,18 +3959,19 @@ dependencies = [ "percent-encoding", "pin-project-lite", "quinn", - "rustls 0.23.19", + "rustls 0.23.20", "rustls-pemfile 2.2.0", "rustls-pki-types", "serde", "serde_json", "serde_urlencoded", - "sync_wrapper 1.0.2", + "sync_wrapper", "system-configuration", "tokio", "tokio-native-tls", "tokio-rustls 0.26.1", "tokio-util", + "tower", "tower-service", "url", "wasm-bindgen", @@ -4031,7 +4033,7 @@ dependencies = [ "rinja_parser", "rustc-hash", "serde", - "syn 2.0.90", + "syn 2.0.94", ] [[package]] @@ -4068,21 +4070,21 @@ dependencies = [ [[package]] name = "rstest" -version = "0.22.0" +version = "0.24.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b423f0e62bdd61734b67cd21ff50871dfaeb9cc74f869dcd6af974fbcb19936" +checksum = "03e905296805ab93e13c1ec3a03f4b6c4f35e9498a3d5fa96dc626d22c03cd89" dependencies = [ - "futures", "futures-timer", + "futures-util", "rstest_macros", "rustc_version", ] [[package]] name = "rstest_macros" -version = "0.22.0" +version = "0.24.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c5e1711e7d14f74b12a58411c542185ef7fb7f2e7f8ee6e2940a883628522b42" +checksum = "ef0053bbffce09062bee4bcc499b0fbe7a57b879f1efe088d6d8d4c7adcdef9b" dependencies = [ "cfg-if", "glob", @@ -4092,7 +4094,7 @@ dependencies = [ "regex", "relative-path", "rustc_version", - "syn 2.0.90", + "syn 2.0.94", "unicode-ident", ] @@ -4104,7 +4106,7 @@ checksum = "b3a8fb4672e840a587a66fc577a5491375df51ddb88f2a2c2a792598c326fe14" dependencies = [ "quote", "rand", - "syn 2.0.90", + "syn 2.0.94", ] [[package]] @@ -4234,15 +4236,15 @@ dependencies = [ [[package]] name = "rustix" -version = "0.38.41" +version = "0.38.42" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d7f649912bc1495e167a6edee79151c84b1bad49748cb4f1f1167f459f6224f6" +checksum = "f93dc38ecbab2eb790ff964bb77fa94faf256fd3e73285fd7ba0903b76bedb85" dependencies = [ "bitflags", "errno", "libc", "linux-raw-sys", - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] @@ -4259,9 +4261,9 @@ dependencies = [ [[package]] name = "rustls" -version = "0.23.19" +version = "0.23.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "934b404430bb06b3fae2cba809eb45a1ab1aecd64491213d7c3301b88393f8d1" +checksum = "5065c3f250cbd332cd894be57c40fa52387247659b14a2d6041d121547903b1b" dependencies = [ "log", "once_cell", @@ -4293,7 +4295,7 @@ dependencies = [ "openssl-probe", "rustls-pki-types", "schannel", - "security-framework 3.0.1", + "security-framework 3.1.0", ] [[package]] @@ -4316,9 +4318,9 @@ dependencies = [ [[package]] name = "rustls-pki-types" -version = "1.10.0" +version = "1.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "16f1201b3c9a7ee8039bcadc17b7e605e2945b27eee7631788c1bd2b0643674b" +checksum = "d2bf47e6ff922db3825eb750c4e2ff784c6ff8fb9e13046ef6a1d1c5401b0b37" dependencies = [ "web-time", ] @@ -4346,9 +4348,9 @@ dependencies = [ [[package]] name = "rustversion" -version = "1.0.18" +version = "1.0.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0e819f2bc632f285be6d7cd36e25940d45b2391dd6d9b939e79de557f7014248" +checksum = "f7c45b9784283f1b2e7fb61b42047c2fd678ef0960d4f6f1eba131594cc369d4" [[package]] name = "rustyline" @@ -4367,7 +4369,7 @@ dependencies = [ "nix 0.28.0", "radix_trie", "unicode-segmentation", - "unicode-width", + "unicode-width 0.1.14", "utf8parse", "windows-sys 0.52.0", ] @@ -4505,7 +4507,7 @@ dependencies = [ "darling", "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.94", ] [[package]] @@ -4537,9 +4539,9 @@ dependencies = [ [[package]] name = "security-framework" -version = "3.0.1" +version = "3.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e1415a607e92bec364ea2cf9264646dcce0f91e6d65281bd6f2819cca3bf39c8" +checksum = "81d3f8c9bfcc3cbb6b0179eb57042d75b1582bdc65c3cb95f3fa999509c03cbc" dependencies = [ "bitflags", "core-foundation 0.10.0", @@ -4550,9 +4552,9 @@ dependencies = [ [[package]] name = "security-framework-sys" -version = "2.12.1" +version = "2.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fa39c7303dc58b5543c94d22c1766b0d31f2ee58306363ea622b10bbc075eaa2" +checksum = "1863fd3768cd83c56a7f60faa4dc0d403f1b6df0a38c3c25f44b7894e45370d5" dependencies = [ "core-foundation-sys", "libc", @@ -4560,38 +4562,38 @@ dependencies = [ [[package]] name = "semver" -version = "1.0.23" +version = "1.0.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "61697e0a1c7e512e84a621326239844a24d8207b4669b41bc18b32ea5cbf988b" +checksum = "3cb6eb87a131f756572d7fb904f6e7b68633f09cca868c5df1c4b8d1a694bbba" dependencies = [ "serde", ] [[package]] name = "serde" -version = "1.0.215" +version = "1.0.217" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6513c1ad0b11a9376da888e3e0baa0077f1aed55c17f50e7b2397136129fb88f" +checksum = "02fc4265df13d6fa1d00ecff087228cc0a2b5f3c0e87e258d8b94a156e984c70" dependencies = [ "serde_derive", ] [[package]] name = "serde_derive" -version = "1.0.215" +version = "1.0.217" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ad1e866f866923f252f05c889987993144fb74e722403468a4ebd70c3cd756c0" +checksum = "5a9bf7cf98d04a2b28aead066b7496853d4779c9cc183c440dbac457641e19a0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.94", ] [[package]] name = "serde_json" -version = "1.0.133" +version = "1.0.134" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c7fceb2473b9166b2294ef05efcb65a3db80803f0b03ef86a5fc88a2b85ee377" +checksum = "d00f4175c42ee48b15416f6193a959ba3a0d67fc699a0db9ad12df9f83991c7d" dependencies = [ "itoa", "memchr", @@ -4613,9 +4615,9 @@ dependencies = [ [[package]] name = "serde_with" -version = "3.11.0" +version = "3.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8e28bdad6db2b8340e449f7108f020b3b092e8583a9e3fb82713e1d4e71fe817" +checksum = "d6b6f7f2fcb69f747921f79f3926bd1e203fce4fef62c268dd3abfb6d86029aa" dependencies = [ "base64 0.22.1", "chrono", @@ -4631,14 +4633,14 @@ dependencies = [ [[package]] name = "serde_with_macros" -version = "3.11.0" +version = "3.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9d846214a9854ef724f3da161b426242d8de7c1fc7de2f89bb1efcb154dca79d" +checksum = "8d00caa5193a3c8362ac2b73be6b9e768aa5a4b2f721d8f4b339600c3cb51f8e" dependencies = [ "darling", "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.94", ] [[package]] @@ -4752,7 +4754,7 @@ dependencies = [ "hex-literal", "http 1.2.0", "httparse", - "itertools 0.13.0", + "itertools 0.14.0", "kafka-protocol", "lz4_flex", "metrics", @@ -4764,14 +4766,14 @@ dependencies = [ "pretty_assertions", "rand", "redis-protocol", - "rustls 0.23.19", + "rustls 0.23.20", "rustls-pemfile 2.2.0", "rustls-pki-types", "sasl", "serde", "serde_json", "serde_yaml", - "thiserror 2.0.6", + "thiserror 2.0.9", "tokio", "tokio-rustls 0.26.1", "tokio-stream", @@ -4802,7 +4804,7 @@ dependencies = [ "fred", "futures", "hex", - "itertools 0.13.0", + "itertools 0.14.0", "opensearch", "pretty_assertions", "prometheus-parse", @@ -5010,7 +5012,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.90", + "syn 2.0.94", ] [[package]] @@ -5042,21 +5044,15 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.90" +version = "2.0.94" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "919d3b74a5dd0ccd15aeb8f93e7006bd9e14c295087c9896a110f490752bcf31" +checksum = "987bc0be1cdea8b10216bd06e2ca407d40b9543468fafd3ddfb02f36e77f71f3" dependencies = [ "proc-macro2", "quote", "unicode-ident", ] -[[package]] -name = "sync_wrapper" -version = "0.1.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2047c6ded9c721764247e62cd3b03c09ffc529b2ba5b10ec482ae507a4a70160" - [[package]] name = "sync_wrapper" version = "1.0.2" @@ -5074,7 +5070,7 @@ checksum = "c8af7666ab7b6390ab78131fb5b0fce11d6b7a6951602017c35fa82800708971" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.94", ] [[package]] @@ -5123,7 +5119,7 @@ dependencies = [ "docker-compose-runner", "futures-util", "http 1.2.0", - "itertools 0.13.0", + "itertools 0.14.0", "j4rs", "openssl", "ordered-float", @@ -5132,7 +5128,7 @@ dependencies = [ "rdkafka", "redis", "reqwest", - "rustls 0.23.19", + "rustls 0.23.20", "rustls-pemfile 2.2.0", "rustls-pki-types", "scylla", @@ -5159,11 +5155,11 @@ dependencies = [ [[package]] name = "thiserror" -version = "2.0.6" +version = "2.0.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8fec2a1820ebd077e2b90c4df007bebf344cd394098a13c563957d0afc83ea47" +checksum = "f072643fd0190df67a8bab670c20ef5d8737177d6ac6b2e9a236cb096206b2cc" dependencies = [ - "thiserror-impl 2.0.6", + "thiserror-impl 2.0.9", ] [[package]] @@ -5174,18 +5170,18 @@ checksum = "4fee6c4efc90059e10f81e6d42c60a18f76588c3d74cb83a0b242a2b6c7504c1" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.94", ] [[package]] name = "thiserror-impl" -version = "2.0.6" +version = "2.0.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d65750cab40f4ff1929fb1ba509e9914eb756131cef4210da8d5d700d26f6312" +checksum = "7b50fa271071aae2e6ee85f842e2e28ba8cd2c5fb67f11fcb1fd70b276f9e7d4" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.94", ] [[package]] @@ -5251,9 +5247,9 @@ dependencies = [ [[package]] name = "tinyvec" -version = "1.8.0" +version = "1.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "445e881f4f6d382d5f27c034e25eb92edd7c784ceab92a0937db7f2e9471b938" +checksum = "022db8904dfa342efe721985167e9fcd16c29b226db4397ed752a761cfce81e8" dependencies = [ "tinyvec_macros", ] @@ -5320,7 +5316,7 @@ checksum = "693d596312e88961bc67d7f1f97af8a70227d9f90c31bba5806eec004978d752" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.94", ] [[package]] @@ -5360,7 +5356,7 @@ version = "0.26.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5f6d0975eaace0cf0fcadee4e4aaa5da15b5c079146f2cffb67c113be122bf37" dependencies = [ - "rustls 0.23.19", + "rustls 0.23.20", "tokio", ] @@ -5377,13 +5373,13 @@ dependencies = [ [[package]] name = "tokio-tungstenite" -version = "0.24.0" +version = "0.26.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "edc5f74e248dc973e0dbb7b74c7e0d6fcc301c694ff50049504004ef4d0cdcd9" +checksum = "be4bf6fecd69fcdede0ec680aaf474cdab988f9de6bc73d3758f0160e3b7025a" dependencies = [ "futures-util", "log", - "rustls 0.23.19", + "rustls 0.23.20", "rustls-native-certs 0.8.1", "rustls-pki-types", "tokio", @@ -5423,14 +5419,14 @@ dependencies = [ [[package]] name = "tower" -version = "0.5.1" +version = "0.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2873938d487c3cfb9aed7546dc9f2711d867c9f90c46b889989a2cb84eba6b4f" +checksum = "d039ad9159c98b70ecfd540b2573b97f7f52c3e8d9f8ad57a24b916a536975f9" dependencies = [ "futures-core", "futures-util", "pin-project-lite", - "sync_wrapper 0.1.2", + "sync_wrapper", "tokio", "tower-layer", "tower-service", @@ -5479,7 +5475,7 @@ checksum = "395ae124c09f9e6918a2310af6038fba074bcf474ac352496d5910dd59a2226d" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.94", ] [[package]] @@ -5563,9 +5559,9 @@ checksum = "e421abadd41a4225275504ea4d6566923418b7f05506fbc9c0fe86ba7396114b" [[package]] name = "tungstenite" -version = "0.24.0" +version = "0.26.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "18e5b8366ee7a95b16d32197d0b2604b43a0be89dc5fac9f8e96ccafbaedda8a" +checksum = "413083a99c579593656008130e29255e54dcaae495be556cc26888f211648c24" dependencies = [ "byteorder", "bytes", @@ -5574,10 +5570,10 @@ dependencies = [ "httparse", "log", "rand", - "rustls 0.23.19", + "rustls 0.23.20", "rustls-pki-types", "sha1", - "thiserror 1.0.69", + "thiserror 2.0.9", "utf-8", ] @@ -5605,9 +5601,9 @@ checksum = "42ff0bf0c66b8238c6f3b578df37d0b7848e55df8577b3f74f92a69acceeb825" [[package]] name = "typetag" -version = "0.2.18" +version = "0.2.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "52ba3b6e86ffe0054b2c44f2d86407388b933b16cb0a70eea3929420db1d9bbe" +checksum = "044fc3365ddd307c297fe0fe7b2e70588cdab4d0f62dc52055ca0d11b174cf0e" dependencies = [ "erased-serde", "inventory", @@ -5618,20 +5614,20 @@ dependencies = [ [[package]] name = "typetag-impl" -version = "0.2.18" +version = "0.2.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "70b20a22c42c8f1cd23ce5e34f165d4d37038f5b663ad20fb6adbdf029172483" +checksum = "d9d30226ac9cbd2d1ff775f74e8febdab985dab14fb14aa2582c29a92d5555dc" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.94", ] [[package]] name = "unicase" -version = "2.8.0" +version = "2.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7e51b68083f157f853b6379db119d1c1be0e6e4dec98101079dec41f6f5cf6df" +checksum = "75b844d17643ee918803943289730bec8aac480150456169e647ed0b576ba539" [[package]] name = "unicode-ident" @@ -5651,6 +5647,12 @@ version = "0.1.14" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7dd6e30e90baa6f72411720665d41d89b9a3d039dc45b8faea1ddd07f617f6af" +[[package]] +name = "unicode-width" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1fc81956842c57dac11422a97c3b8195a1ff727f06e85c84ed2e8aa277c9a0fd" + [[package]] name = "unicode-xid" version = "0.2.6" @@ -5812,7 +5814,7 @@ dependencies = [ "log", "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.94", "wasm-bindgen-shared", ] @@ -5847,7 +5849,7 @@ checksum = "30d7a95b763d3c45903ed6c81f156801839e5ee968bb07e534c44df0fcd330c2" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.94", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -5936,7 +5938,7 @@ version = "0.52.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "33ab640c8d7e35bf8ba19b884ba838ceb4fba93a4e8c65a9059d08afcfc683d9" dependencies = [ - "windows-targets 0.52.6", + "windows-targets", ] [[package]] @@ -5947,7 +5949,7 @@ checksum = "e400001bb720a623c1c69032f8e3e4cf09984deec740f007dd2b03ec864804b0" dependencies = [ "windows-result", "windows-strings", - "windows-targets 0.52.6", + "windows-targets", ] [[package]] @@ -5956,7 +5958,7 @@ version = "0.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1d1043d8214f791817bab27572aaa8af63732e11bf84aa21a45a78d6c317ae0e" dependencies = [ - "windows-targets 0.52.6", + "windows-targets", ] [[package]] @@ -5966,16 +5968,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4cd9b125c486025df0eabcb585e62173c6c9eddcec5d117d3b6e8c30e2ee4d10" dependencies = [ "windows-result", - "windows-targets 0.52.6", -] - -[[package]] -name = "windows-sys" -version = "0.48.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "677d2418bec65e3338edb076e806bc1ec15693c5d0104683f2efe857f61056a9" -dependencies = [ - "windows-targets 0.48.5", + "windows-targets", ] [[package]] @@ -5984,7 +5977,7 @@ version = "0.52.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "282be5f36a8ce781fad8c8ae18fa3f9beff57ec1b52cb3de0789201425d9a33d" dependencies = [ - "windows-targets 0.52.6", + "windows-targets", ] [[package]] @@ -5993,22 +5986,7 @@ version = "0.59.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1e38bc4d79ed67fd075bcc251a1c39b32a1776bbe92e5bef1f0bf1f8c531853b" dependencies = [ - "windows-targets 0.52.6", -] - -[[package]] -name = "windows-targets" -version = "0.48.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9a2fa6e2155d7247be68c096456083145c183cbbbc2764150dda45a87197940c" -dependencies = [ - "windows_aarch64_gnullvm 0.48.5", - "windows_aarch64_msvc 0.48.5", - "windows_i686_gnu 0.48.5", - "windows_i686_msvc 0.48.5", - "windows_x86_64_gnu 0.48.5", - "windows_x86_64_gnullvm 0.48.5", - "windows_x86_64_msvc 0.48.5", + "windows-targets", ] [[package]] @@ -6017,46 +5995,28 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9b724f72796e036ab90c1021d4780d4d3d648aca59e491e6b98e725b84e99973" dependencies = [ - "windows_aarch64_gnullvm 0.52.6", - "windows_aarch64_msvc 0.52.6", - "windows_i686_gnu 0.52.6", + "windows_aarch64_gnullvm", + "windows_aarch64_msvc", + "windows_i686_gnu", "windows_i686_gnullvm", - "windows_i686_msvc 0.52.6", - "windows_x86_64_gnu 0.52.6", - "windows_x86_64_gnullvm 0.52.6", - "windows_x86_64_msvc 0.52.6", + "windows_i686_msvc", + "windows_x86_64_gnu", + "windows_x86_64_gnullvm", + "windows_x86_64_msvc", ] -[[package]] -name = "windows_aarch64_gnullvm" -version = "0.48.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2b38e32f0abccf9987a4e3079dfb67dcd799fb61361e53e2882c3cbaf0d905d8" - [[package]] name = "windows_aarch64_gnullvm" version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "32a4622180e7a0ec044bb555404c800bc9fd9ec262ec147edd5989ccd0c02cd3" -[[package]] -name = "windows_aarch64_msvc" -version = "0.48.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dc35310971f3b2dbbf3f0690a219f40e2d9afcf64f9ab7cc1be722937c26b4bc" - [[package]] name = "windows_aarch64_msvc" version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "09ec2a7bb152e2252b53fa7803150007879548bc709c039df7627cabbd05d469" -[[package]] -name = "windows_i686_gnu" -version = "0.48.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a75915e7def60c94dcef72200b9a8e58e5091744960da64ec734a6c6e9b3743e" - [[package]] name = "windows_i686_gnu" version = "0.52.6" @@ -6069,48 +6029,24 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0eee52d38c090b3caa76c563b86c3a4bd71ef1a819287c19d586d7334ae8ed66" -[[package]] -name = "windows_i686_msvc" -version = "0.48.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8f55c233f70c4b27f66c523580f78f1004e8b5a8b659e05a4eb49d4166cca406" - [[package]] name = "windows_i686_msvc" version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "240948bc05c5e7c6dabba28bf89d89ffce3e303022809e73deaefe4f6ec56c66" -[[package]] -name = "windows_x86_64_gnu" -version = "0.48.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "53d40abd2583d23e4718fddf1ebec84dbff8381c07cae67ff7768bbf19c6718e" - [[package]] name = "windows_x86_64_gnu" version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "147a5c80aabfbf0c7d901cb5895d1de30ef2907eb21fbbab29ca94c5b08b1a78" -[[package]] -name = "windows_x86_64_gnullvm" -version = "0.48.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0b7b52767868a23d5bab768e390dc5f5c55825b6d30b86c844ff2dc7414044cc" - [[package]] name = "windows_x86_64_gnullvm" version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "24d5b23dc417412679681396f2b49f3de8c1473deb516bd34410872eff51ed0d" -[[package]] -name = "windows_x86_64_msvc" -version = "0.48.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ed94fce61571a4006852b7389a063ab983c02eb1bb37b47f8272ce92d06d9538" - [[package]] name = "windows_x86_64_msvc" version = "0.52.6" @@ -6145,9 +6081,9 @@ dependencies = [ [[package]] name = "winnow" -version = "0.6.20" +version = "0.6.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "36c1fec1a2bb5866f07c25f68c26e565c4c200aebb96d7e55710c19d3e8ac49b" +checksum = "e6f5bb5257f2407a5425c6e749bfd9692192a73e70a6060516ac04f889087d68" dependencies = [ "memchr", ] @@ -6172,9 +6108,9 @@ checksum = "66fee0b777b0f5ac1c69bb06d361268faafa61cd4682ae064a171c16c433e9e4" [[package]] name = "xxhash-rust" -version = "0.8.12" +version = "0.8.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6a5cbf750400958819fb6178eaa83bee5cd9c29a26a40cc241df8c70fdd46984" +checksum = "fdd20c5420375476fbd4394763288da7eb0cc0b8c11deed431a91562af7335d3" [[package]] name = "yansi" @@ -6211,7 +6147,7 @@ checksum = "2380878cad4ac9aac1e2435f3eb4020e8374b5f13c296cb75b4620ff8e229154" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.94", "synstructure", ] @@ -6233,7 +6169,7 @@ checksum = "fa4f8080344d4671fb4e831a13ad1e68092748387dfc4f55e356242fae12ce3e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.94", ] [[package]] @@ -6253,7 +6189,7 @@ checksum = "595eed982f7d355beb85837f651fa22e90b3c044842dc7f2c2842c086f295808" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.94", "synstructure", ] @@ -6282,5 +6218,5 @@ checksum = "6eafa6dfb17584ea3e2bd6e76e0cc15ad7af12b09abdd1ca55961bed9b1063c6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.94", ] diff --git a/Cargo.toml b/Cargo.toml index aca6cbaf7..21955c355 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -26,7 +26,7 @@ bytes = { version = "1.0.0", features = ["serde"] } tokio = { version = "1.25.0", features = ["full"] } tokio-util = { version = "0.7.7", features = ["codec"] } tokio-openssl = "0.6.2" -itertools = "0.13.0" +itertools = "0.14.0" openssl = { version = "0.10.36", features = ["vendored"] } anyhow = "1.0.76" serde = { version = "1.0.111", features = ["derive"] } diff --git a/shotover-proxy/Cargo.toml b/shotover-proxy/Cargo.toml index 3a1be85fd..a37bf559d 100644 --- a/shotover-proxy/Cargo.toml +++ b/shotover-proxy/Cargo.toml @@ -8,7 +8,7 @@ license = "Apache-2.0" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html [dependencies] -shotover = { path = "../shotover", default-features = false} +shotover = { path = "../shotover", default-features = false } [dev-dependencies] prometheus-parse = "0.2.4" @@ -17,7 +17,7 @@ scylla.workspace = true anyhow.workspace = true tokio.workspace = true tracing.workspace = true -rstest = "0.22.0" +rstest = "0.24.0" rstest_reuse = "0.7.0" test-helpers = { path = "../test-helpers" } redis.workspace = true @@ -44,7 +44,9 @@ rustls-pki-types = "1.1.0" aws-throwaway.workspace = true windsock = "0.2.0" regex = "1.7.0" -opensearch = { version = "2.1.0", default-features = false, features = ["rustls-tls"] } +opensearch = { version = "2.1.0", default-features = false, features = [ + "rustls-tls", +] } serde_json = "1.0.103" time = { version = "0.3.25" } shell-quote.workspace = true diff --git a/shotover/Cargo.toml b/shotover/Cargo.toml index b9bed9dab..9607550f5 100644 --- a/shotover/Cargo.toml +++ b/shotover/Cargo.toml @@ -48,7 +48,7 @@ default = ["cassandra", "valkey", "kafka", "opensearch"] [dependencies] atomic_enum = "0.3.0" -axum = { version = "0.7", default-features = false, features = ["tokio", "tracing", "http1"] } +axum = { version = "0.8", default-features = false, features = ["tokio", "tracing", "http1"] } pretty-hex = "0.4.0" tokio-stream = "0.1.2" derivative = "2.1.1" @@ -76,7 +76,7 @@ csv = { workspace = true, optional = true } hex = { workspace = true, optional = true } async-trait.workspace = true typetag.workspace = true -tokio-tungstenite = "0.24.0" +tokio-tungstenite = "0.26.0" # Error handling thiserror = "2.0" diff --git a/shotover/src/server.rs b/shotover/src/server.rs index 6c619091b..abd074ee7 100644 --- a/shotover/src/server.rs +++ b/shotover/src/server.rs @@ -343,7 +343,7 @@ async fn spawn_websocket_read_write_tasks< Ok(WsMessage::Binary(ws_message_data)) => { // Entire message is reallocated and copied here due to // incompatibility between tokio codecs and tungstenite. - let message = decoder.decode(&mut BytesMut::from(ws_message_data.as_slice())); + let message = decoder.decode(&mut BytesMut::from(ws_message_data.as_ref())); match message { Ok(Some(message)) => { if in_tx.send(message).await.is_err() { diff --git a/test-helpers/Cargo.toml b/test-helpers/Cargo.toml index d7ac22bc5..7842c6316 100644 --- a/test-helpers/Cargo.toml +++ b/test-helpers/Cargo.toml @@ -16,7 +16,9 @@ subprocess.workspace = true tokio-bin-process.workspace = true cdrs-tokio.workspace = true cassandra-protocol.workspace = true -cassandra-cpp = { version = "3.0.0", default-features = false, features = ["log"], optional = true } +cassandra-cpp = { version = "3.0.0", default-features = false, features = [ + "log", +], optional = true } scylla.workspace = true openssl.workspace = true bytes.workspace = true @@ -31,7 +33,7 @@ reqwest.workspace = true tracing-subscriber.workspace = true anyhow.workspace = true rcgen.workspace = true -rdkafka = { version = "0.36", features = ["cmake-build"], optional = true } +rdkafka = { version = "0.37", features = ["cmake-build"], optional = true } docker-compose-runner = "0.3.0" j4rs = "0.21.0" futures-util = "0.3.28" @@ -39,6 +41,6 @@ http = "1.1.0" rustls = { version = "0.23.18", default-features = false } rustls-pki-types = "1.0.1" rustls-pemfile = "2.0.0" -tokio-tungstenite = { version = "0.24", features = ["rustls-tls-native-roots"] } +tokio-tungstenite = { version = "0.26", features = ["rustls-tls-native-roots"] } pretty_assertions.workspace = true serde.workspace = true diff --git a/test-helpers/src/connection/cassandra/cql_ws.rs b/test-helpers/src/connection/cassandra/cql_ws.rs index 193f70c84..6a7371e65 100644 --- a/test-helpers/src/connection/cassandra/cql_ws.rs +++ b/test-helpers/src/connection/cassandra/cql_ws.rs @@ -159,13 +159,13 @@ impl CqlWsSession { fn encode(envelope: Envelope) -> Message { let data = envelope.encode_with(Compression::None).unwrap(); - Message::Binary(data) + Message::Binary(data.into()) } fn decode(ws_message: Message) -> Envelope { match ws_message { Message::Binary(data) => { - Envelope::from_buffer(data.as_slice(), Compression::None) + Envelope::from_buffer(data.as_ref(), Compression::None) .unwrap() .envelope } From ea0bf41a174aac3a8e08bf316cbb54646e6a631e Mon Sep 17 00:00:00 2001 From: Lucas Kent Date: Mon, 6 Jan 2025 11:52:17 +1100 Subject: [PATCH 3/3] Update scylla-rust-driver (#1840) --- Cargo.lock | 135 +++++++++--------- Cargo.toml | 2 +- .../tests/cassandra_int_tests/mod.rs | 14 +- .../connection/cassandra/connection/scylla.rs | 35 +++-- 4 files changed, 104 insertions(+), 82 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 946e5827c..b2c00a011 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -176,13 +176,13 @@ dependencies = [ [[package]] name = "async-trait" -version = "0.1.83" +version = "0.1.84" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "721cae7de5c34fbb2acd27e21e6d2cf7b886dce0c27388d46c4e6c47ea4318dd" +checksum = "1b1244b10dcd56c92219da4e14caa97e312079e185f04ba3eea25061561dc0a0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.94", + "syn 2.0.95", ] [[package]] @@ -217,7 +217,7 @@ checksum = "99e1aca718ea7b89985790c94aad72d77533063fe00bc497bb79a7c2dae6a661" dependencies = [ "proc-macro2", "quote", - "syn 2.0.94", + "syn 2.0.95", ] [[package]] @@ -228,9 +228,9 @@ checksum = "ace50bade8e6234aa140d9a2f552bbee1db4d353f69b8217bc503490fc1a9f26" [[package]] name = "aws-config" -version = "1.5.12" +version = "1.5.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "649316840239f4e58df0b7f620c428f5fababbbca2d504488c641534050bd141" +checksum = "c03a50b30228d3af8865ce83376b4e99e1ffa34728220fe2860e4df0bb5278d6" dependencies = [ "aws-credential-types", "aws-runtime", @@ -270,9 +270,9 @@ dependencies = [ [[package]] name = "aws-runtime" -version = "1.5.2" +version = "1.5.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "44f6f1124d6e19ab6daf7f2e615644305dc6cb2d706892a8a8c0b98db35de020" +checksum = "b16d1aa50accc11a4b4d5c50f7fb81cc0cf60328259c587d0e6b0f11385bde46" dependencies = [ "aws-credential-types", "aws-sigv4", @@ -295,9 +295,9 @@ dependencies = [ [[package]] name = "aws-sdk-kms" -version = "1.53.0" +version = "1.54.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e349416a1998fde638deed85c18efeefd81af293439c16d676b7fce992904389" +checksum = "a6cf16c0e5853312995505557b876dd3f9fb9941e96d031383528ccef14ace57" dependencies = [ "aws-credential-types", "aws-runtime", @@ -317,9 +317,9 @@ dependencies = [ [[package]] name = "aws-sdk-sso" -version = "1.52.0" +version = "1.53.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cb25f7129c74d36afe33405af4517524df8f74b635af8c2c8e91c1552b8397b2" +checksum = "1605dc0bf9f0a4b05b451441a17fcb0bda229db384f23bf5cead3adbab0664ac" dependencies = [ "aws-credential-types", "aws-runtime", @@ -339,9 +339,9 @@ dependencies = [ [[package]] name = "aws-sdk-ssooidc" -version = "1.53.0" +version = "1.54.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d03a3d5ef14851625eafd89660a751776f938bf32f309308b20dcca41c44b568" +checksum = "59f3f73466ff24f6ad109095e0f3f2c830bfb4cd6c8b12f744c8e61ebf4d3ba1" dependencies = [ "aws-credential-types", "aws-runtime", @@ -361,9 +361,9 @@ dependencies = [ [[package]] name = "aws-sdk-sts" -version = "1.53.0" +version = "1.54.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cf3a9f073ae3a53b54421503063dfb87ff1ea83b876f567d92e8b8d9942ba91b" +checksum = "249b2acaa8e02fd4718705a9494e3eb633637139aa4bb09d70965b0448e865db" dependencies = [ "aws-credential-types", "aws-runtime", @@ -777,7 +777,7 @@ checksum = "3fa76293b4f7bb636ab88fd78228235b5248b4d05cc589aed610f954af5d7c7a" dependencies = [ "proc-macro2", "quote", - "syn 2.0.94", + "syn 2.0.95", ] [[package]] @@ -832,7 +832,7 @@ dependencies = [ "darling", "proc-macro2", "quote", - "syn 2.0.94", + "syn 2.0.95", ] [[package]] @@ -949,9 +949,9 @@ dependencies = [ [[package]] name = "cc" -version = "1.2.6" +version = "1.2.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8d6dbb628b8f8555f86d0323c2eb39e3ec81901f4b83e091db8a6a76d316a333" +checksum = "a012a0df96dd6d06ba9a1b29d6402d1a5d77c6befd2566afdc26e10603dc93d7" dependencies = [ "jobserver", "libc", @@ -1114,7 +1114,7 @@ dependencies = [ "heck", "proc-macro2", "quote", - "syn 2.0.94", + "syn 2.0.95", ] [[package]] @@ -1475,7 +1475,7 @@ checksum = "f46882e17999c6cc590af592290432be3bce0428cb0d5f8b6715e4dc7b383eb3" dependencies = [ "proc-macro2", "quote", - "syn 2.0.94", + "syn 2.0.95", ] [[package]] @@ -1515,7 +1515,7 @@ dependencies = [ "proc-macro2", "quote", "strsim", - "syn 2.0.94", + "syn 2.0.95", ] [[package]] @@ -1526,7 +1526,7 @@ checksum = "d336a2a514f6ccccaa3e09b02d41d35330c07ddf03a62165fcec10bb561c7806" dependencies = [ "darling_core", "quote", - "syn 2.0.94", + "syn 2.0.95", ] [[package]] @@ -1611,7 +1611,7 @@ checksum = "cb7330aeadfbe296029522e6c40f315320aba36fc43a5b3632f3795348f3bd22" dependencies = [ "proc-macro2", "quote", - "syn 2.0.94", + "syn 2.0.95", "unicode-xid", ] @@ -1656,7 +1656,7 @@ checksum = "97369cbbc041bc366949bc74d34658d6cda5621039731c6310521892a3a20ae0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.94", + "syn 2.0.95", ] [[package]] @@ -1958,7 +1958,7 @@ checksum = "1458c6e22d36d61507034d5afecc64f105c1d39712b7ac6ec3b352c423f715cc" dependencies = [ "proc-macro2", "quote", - "syn 2.0.94", + "syn 2.0.95", ] [[package]] @@ -2023,7 +2023,7 @@ checksum = "162ee34ebcb7c64a8abebc059ce0fee27c2262618d7b60ed8faf72fef13c3650" dependencies = [ "proc-macro2", "quote", - "syn 2.0.94", + "syn 2.0.95", ] [[package]] @@ -2608,7 +2608,7 @@ checksum = "1ec89e9337638ecdc08744df490b221a7399bf8d164eb52a665454e60e075ad6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.94", + "syn 2.0.95", ] [[package]] @@ -3193,7 +3193,7 @@ dependencies = [ "proc-macro-crate", "proc-macro2", "quote", - "syn 2.0.94", + "syn 2.0.95", ] [[package]] @@ -3266,7 +3266,7 @@ checksum = "a948666b637a0f465e8564c73e89d4dde00d72d4d473cc972f390fc3dcee7d9c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.94", + "syn 2.0.95", ] [[package]] @@ -4033,7 +4033,7 @@ dependencies = [ "rinja_parser", "rustc-hash", "serde", - "syn 2.0.94", + "syn 2.0.95", ] [[package]] @@ -4094,7 +4094,7 @@ dependencies = [ "regex", "relative-path", "rustc_version", - "syn 2.0.94", + "syn 2.0.95", "unicode-ident", ] @@ -4106,7 +4106,7 @@ checksum = "b3a8fb4672e840a587a66fc577a5491375df51ddb88f2a2c2a792598c326fe14" dependencies = [ "quote", "rand", - "syn 2.0.94", + "syn 2.0.95", ] [[package]] @@ -4450,9 +4450,9 @@ dependencies = [ [[package]] name = "scylla" -version = "0.14.0" +version = "0.15.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8139623d3fb0c8205b15e84fa587f3aa0ba61f876c19a9157b688f7c1763a7c5" +checksum = "f0408e59e11f589071d1cefc3928270aa8fe4d03f654cb118e0c24d16013ea82" dependencies = [ "arc-swap", "async-trait", @@ -4474,7 +4474,7 @@ dependencies = [ "smallvec", "snap", "socket2 0.5.8", - "thiserror 1.0.69", + "thiserror 2.0.9", "tokio", "tokio-openssl", "tracing", @@ -4483,9 +4483,9 @@ dependencies = [ [[package]] name = "scylla-cql" -version = "0.3.0" +version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "de7020bcd1f6fdbeaed356cd426bf294b2071bd7120d48d2e8e319295e2acdcd" +checksum = "a0cefd8b924bb8f67525937a811038d5662f9febc30c74c778a8205f63c4b365" dependencies = [ "async-trait", "byteorder", @@ -4493,21 +4493,23 @@ dependencies = [ "lz4_flex", "scylla-macros", "snap", - "thiserror 1.0.69", + "stable_deref_trait", + "thiserror 2.0.9", "tokio", "uuid", + "yoke", ] [[package]] name = "scylla-macros" -version = "0.6.0" +version = "0.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3859b6938663fc5062e3b26f3611649c9bd26fb252e85f6fdfa581e0d2ce74b6" +checksum = "e878bfb8a235207864ac3fb0b51d7954c77fd38486e0e4fb4e037935ff7eb46c" dependencies = [ "darling", "proc-macro2", "quote", - "syn 2.0.94", + "syn 2.0.95", ] [[package]] @@ -4586,7 +4588,7 @@ checksum = "5a9bf7cf98d04a2b28aead066b7496853d4779c9cc183c440dbac457641e19a0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.94", + "syn 2.0.95", ] [[package]] @@ -4640,7 +4642,7 @@ dependencies = [ "darling", "proc-macro2", "quote", - "syn 2.0.94", + "syn 2.0.95", ] [[package]] @@ -4695,9 +4697,9 @@ dependencies = [ [[package]] name = "shell-quote" -version = "0.7.1" +version = "0.7.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ae4c63bdcc11eea49b562941b914d5ac30d42cad982e3f6e846a513ee6a3ce7e" +checksum = "fb502615975ae2365825521fa1529ca7648fd03ce0b0746604e0683856ecd7e4" [[package]] name = "shellfish" @@ -5012,7 +5014,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.94", + "syn 2.0.95", ] [[package]] @@ -5044,9 +5046,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.94" +version = "2.0.95" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "987bc0be1cdea8b10216bd06e2ca407d40b9543468fafd3ddfb02f36e77f71f3" +checksum = "46f71c0377baf4ef1cc3e3402ded576dccc315800fbc62dfc7fe04b009773b4a" dependencies = [ "proc-macro2", "quote", @@ -5070,7 +5072,7 @@ checksum = "c8af7666ab7b6390ab78131fb5b0fce11d6b7a6951602017c35fa82800708971" dependencies = [ "proc-macro2", "quote", - "syn 2.0.94", + "syn 2.0.95", ] [[package]] @@ -5096,12 +5098,13 @@ dependencies = [ [[package]] name = "tempfile" -version = "3.14.0" +version = "3.15.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "28cce251fcbc87fac86a866eeb0d6c2d536fc16d06f184bb61aeae11aa4cee0c" +checksum = "9a8a559c81686f576e8cd0290cd2a24a2a9ad80c98b3478856500fcbd7acd704" dependencies = [ "cfg-if", "fastrand", + "getrandom", "once_cell", "rustix", "windows-sys 0.59.0", @@ -5170,7 +5173,7 @@ checksum = "4fee6c4efc90059e10f81e6d42c60a18f76588c3d74cb83a0b242a2b6c7504c1" dependencies = [ "proc-macro2", "quote", - "syn 2.0.94", + "syn 2.0.95", ] [[package]] @@ -5181,7 +5184,7 @@ checksum = "7b50fa271071aae2e6ee85f842e2e28ba8cd2c5fb67f11fcb1fd70b276f9e7d4" dependencies = [ "proc-macro2", "quote", - "syn 2.0.94", + "syn 2.0.95", ] [[package]] @@ -5316,7 +5319,7 @@ checksum = "693d596312e88961bc67d7f1f97af8a70227d9f90c31bba5806eec004978d752" dependencies = [ "proc-macro2", "quote", - "syn 2.0.94", + "syn 2.0.95", ] [[package]] @@ -5475,7 +5478,7 @@ checksum = "395ae124c09f9e6918a2310af6038fba074bcf474ac352496d5910dd59a2226d" dependencies = [ "proc-macro2", "quote", - "syn 2.0.94", + "syn 2.0.95", ] [[package]] @@ -5620,7 +5623,7 @@ checksum = "d9d30226ac9cbd2d1ff775f74e8febdab985dab14fb14aa2582c29a92d5555dc" dependencies = [ "proc-macro2", "quote", - "syn 2.0.94", + "syn 2.0.95", ] [[package]] @@ -5814,7 +5817,7 @@ dependencies = [ "log", "proc-macro2", "quote", - "syn 2.0.94", + "syn 2.0.95", "wasm-bindgen-shared", ] @@ -5849,7 +5852,7 @@ checksum = "30d7a95b763d3c45903ed6c81f156801839e5ee968bb07e534c44df0fcd330c2" dependencies = [ "proc-macro2", "quote", - "syn 2.0.94", + "syn 2.0.95", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -6081,9 +6084,9 @@ dependencies = [ [[package]] name = "winnow" -version = "0.6.21" +version = "0.6.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e6f5bb5257f2407a5425c6e749bfd9692192a73e70a6060516ac04f889087d68" +checksum = "39281189af81c07ec09db316b302a3e67bf9bd7cbf6c820b50e35fee9c2fa980" dependencies = [ "memchr", ] @@ -6147,7 +6150,7 @@ checksum = "2380878cad4ac9aac1e2435f3eb4020e8374b5f13c296cb75b4620ff8e229154" dependencies = [ "proc-macro2", "quote", - "syn 2.0.94", + "syn 2.0.95", "synstructure", ] @@ -6169,7 +6172,7 @@ checksum = "fa4f8080344d4671fb4e831a13ad1e68092748387dfc4f55e356242fae12ce3e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.94", + "syn 2.0.95", ] [[package]] @@ -6189,7 +6192,7 @@ checksum = "595eed982f7d355beb85837f651fa22e90b3c044842dc7f2c2842c086f295808" dependencies = [ "proc-macro2", "quote", - "syn 2.0.94", + "syn 2.0.95", "synstructure", ] @@ -6218,5 +6221,5 @@ checksum = "6eafa6dfb17584ea3e2bd6e76e0cc15ad7af12b09abdd1ca55961bed9b1063c6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.94", + "syn 2.0.95", ] diff --git a/Cargo.toml b/Cargo.toml index 21955c355..a4c36336b 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -21,7 +21,7 @@ inherits = "release" debug = true [workspace.dependencies] -scylla = { version = "0.14.0", features = ["ssl"] } +scylla = { version = "0.15.0", features = ["ssl"] } bytes = { version = "1.0.0", features = ["serde"] } tokio = { version = "1.25.0", features = ["full"] } tokio-util = { version = "0.7.7", features = ["codec"] } diff --git a/shotover-proxy/tests/cassandra_int_tests/mod.rs b/shotover-proxy/tests/cassandra_int_tests/mod.rs index f85f0d089..f464148d7 100644 --- a/shotover-proxy/tests/cassandra_int_tests/mod.rs +++ b/shotover-proxy/tests/cassandra_int_tests/mod.rs @@ -9,6 +9,10 @@ use futures::Future; use pretty_assertions::assert_eq; use rstest::rstest; use rstest_reuse::{self, *}; +use scylla::transport::errors::{ + ConnectionError, ConnectionPoolError, ConnectionSetupRequestError, + ConnectionSetupRequestErrorKind, DbError, NewSessionError, +}; use scylla::SessionBuilder; use std::net::SocketAddr; #[cfg(feature = "cassandra-cpp-driver-tests")] @@ -141,10 +145,16 @@ async fn passthrough_cassandra_down() { .await .unwrap_err(); match err { - scylla::transport::errors::NewSessionError::IoError(err) => { + NewSessionError::ConnectionPoolError(ConnectionPoolError::Broken { + last_connection_error: + ConnectionError::ConnectionSetupRequestError(ConnectionSetupRequestError { + error: ConnectionSetupRequestErrorKind::DbError(DbError::ServerError, err), + .. + }), + }) => { assert_eq!( format!("{err}"), - format!("No connections in the pool; last connection failed with: Database returned an error: Internal server error. This indicates a server-side bug, Error message: Internal shotover (or custom transform) bug: Chain failed to send and/or receive messages, the connection will now be closed. + format!("Internal shotover (or custom transform) bug: Chain failed to send and/or receive messages, the connection will now be closed. Caused by: 0: CassandraSinkSingle transform failed diff --git a/test-helpers/src/connection/cassandra/connection/scylla.rs b/test-helpers/src/connection/cassandra/connection/scylla.rs index 5668b219d..8e7a7d927 100644 --- a/test-helpers/src/connection/cassandra/connection/scylla.rs +++ b/test-helpers/src/connection/cassandra/connection/scylla.rs @@ -2,6 +2,7 @@ use super::{Compression, Consistency, PreparedQuery, ProtocolVersion, Tls}; use crate::connection::cassandra::ResultValue; use cdrs_tokio::frame::message_error::{ErrorBody, ErrorType}; use scylla::batch::Batch; +use scylla::frame::response::result::Row; use scylla::frame::types::Consistency as ScyllaConsistency; use scylla::frame::value::{CqlDate, CqlDecimal, CqlTime, CqlTimestamp}; use scylla::serialize::value::SerializeValue; @@ -79,7 +80,7 @@ impl ScyllaConnection { .execute_unpaged(statement, values) .await .unwrap(); - let tracing_id = response.tracing_id.unwrap(); + let tracing_id = response.tracing_id().unwrap(); tokio::time::sleep(tokio::time::Duration::from_millis(100)).await; self.session @@ -145,18 +146,26 @@ impl ScyllaConnection { response: Result, ) -> Result>, ErrorBody> { match response { - Ok(value) => Ok(match value.rows { - Some(rows) => rows - .into_iter() - .map(|x| { - x.columns - .into_iter() - .map(ResultValue::new_from_scylla) - .collect() - }) - .collect(), - None => vec![], - }), + Ok(value) => { + if value.is_rows() { + Ok(value + .into_rows_result() + .unwrap() + .rows::() + .unwrap() + .map(|x| { + x.unwrap() + .columns + .into_iter() + .map(ResultValue::new_from_scylla) + .collect() + }) + .collect()) + } else { + value.result_not_rows().unwrap(); + Ok(vec![]) + } + } Err(QueryError::DbError(code, message)) => Err(ErrorBody { ty: match code { DbError::Overloaded => ErrorType::Overloaded,