Skip to content

Commit

Permalink
perf(1024): Remove serialization from tx execution thread (#1027)
Browse files Browse the repository at this point in the history
Closes #1024.

Before this change,
we would serialize messages **before** inserting into the send queue.

Because we commit the tx only after inserting into the send queue,
this meant we were holding onto the database lock unnecessarily.

After this change,
we serialize messages **after** inserting into the send queue.
This means we serialize only after committing the tx.
  • Loading branch information
joshua-spacetime committed Mar 27, 2024
1 parent 9141a42 commit 99bd7ac
Show file tree
Hide file tree
Showing 6 changed files with 70 additions and 85 deletions.
12 changes: 8 additions & 4 deletions crates/client-api/src/routes/subscribe.rs
Original file line number Diff line number Diff line change
Expand Up @@ -9,9 +9,10 @@ use futures::{FutureExt, SinkExt, StreamExt};
use http::{HeaderValue, StatusCode};
use scopeguard::ScopeGuard;
use serde::Deserialize;
use spacetimedb::client::messages::{IdentityTokenMessage, ServerMessage};
use spacetimedb::client::messages::{IdentityTokenMessage, SerializableMessage, ServerMessage};
use spacetimedb::client::{ClientActorId, ClientConnection, DataMessage, MessageHandleError, Protocol};
use spacetimedb::util::{also_poll, future_queue};
use spacetimedb::worker_metrics::WORKER_METRICS;
use spacetimedb_lib::address::AddressForUrl;
use spacetimedb_lib::Address;
use std::time::Instant;
Expand Down Expand Up @@ -168,7 +169,7 @@ where

const LIVELINESS_TIMEOUT: Duration = Duration::from_secs(60);

async fn ws_client_actor(client: ClientConnection, ws: WebSocketStream, sendrx: mpsc::Receiver<DataMessage>) {
async fn ws_client_actor(client: ClientConnection, ws: WebSocketStream, sendrx: mpsc::Receiver<SerializableMessage>) {
// ensure that even if this task gets cancelled, we always cleanup the connection
let client = scopeguard::guard(client, |client| {
tokio::spawn(client.disconnect());
Expand All @@ -182,7 +183,7 @@ async fn ws_client_actor(client: ClientConnection, ws: WebSocketStream, sendrx:
async fn ws_client_actor_inner(
client: &ClientConnection,
mut ws: WebSocketStream,
mut sendrx: mpsc::Receiver<DataMessage>,
mut sendrx: mpsc::Receiver<SerializableMessage>,
) {
let mut liveness_check_interval = tokio::time::interval(LIVELINESS_TIMEOUT);
let mut got_pong = true;
Expand Down Expand Up @@ -240,7 +241,10 @@ async fn ws_client_actor_inner(
log::info!("dropping messages due to ws already being closed: {:?}", &rx_buf[..n]);
} else {
let send_all = async {
for msg in rx_buf.drain(..n).map(datamsg_to_wsmsg) {
let id = client.id.identity;
for msg in rx_buf.drain(..n).map(|msg| datamsg_to_wsmsg(msg.serialize(client.protocol))) {
WORKER_METRICS.websocket_sent.with_label_values(&id).inc();
WORKER_METRICS.websocket_sent_msg_size.with_label_values(&id).observe(msg.len() as f64);
// feed() buffers the message, but does not necessarily send it
ws.feed(msg).await?;
}
Expand Down
23 changes: 7 additions & 16 deletions crates/core/src/client/client_connection.rs
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,7 @@ use std::sync::atomic::{AtomicBool, Ordering::Relaxed};
use std::sync::Arc;
use std::time::Instant;

use super::messages::{OneOffQueryResponseMessage, ServerMessage};
use super::messages::{OneOffQueryResponseMessage, SerializableMessage};
use super::{message_handlers, ClientActorId, MessageHandleError};
use crate::error::DBError;
use crate::host::{ModuleHost, ReducerArgs, ReducerCallError, ReducerCallResult};
Expand All @@ -26,7 +26,7 @@ pub enum Protocol {
pub struct ClientConnectionSender {
pub id: ClientActorId,
pub protocol: Protocol,
sendtx: mpsc::Sender<DataMessage>,
sendtx: mpsc::Sender<SerializableMessage>,
abort_handle: AbortHandle,
cancelled: AtomicBool,
}
Expand Down Expand Up @@ -56,13 +56,11 @@ impl ClientConnectionSender {
}
}

pub fn send_message(&self, message: impl ServerMessage) -> Result<(), ClientSendError> {
self.send(message.serialize(self.protocol))
pub fn send_message(&self, message: impl Into<SerializableMessage>) -> Result<(), ClientSendError> {
self.send(message.into())
}

fn send(&self, message: DataMessage) -> Result<(), ClientSendError> {
let bytes_len = message.len();

fn send(&self, message: SerializableMessage) -> Result<(), ClientSendError> {
if self.cancelled.load(Relaxed) {
return Err(ClientSendError::Cancelled);
}
Expand All @@ -77,13 +75,6 @@ impl ClientConnectionSender {
mpsc::error::TrySendError::Closed(_) => ClientSendError::Disconnected,
})?;

WORKER_METRICS.websocket_sent.with_label_values(&self.id.identity).inc();

WORKER_METRICS
.websocket_sent_msg_size
.with_label_values(&self.id.identity)
.observe(bytes_len as f64);

Ok(())
}
}
Expand Down Expand Up @@ -137,7 +128,7 @@ impl ClientConnection {
actor: F,
) -> Result<ClientConnection, ReducerCallError>
where
F: FnOnce(ClientConnection, mpsc::Receiver<DataMessage>) -> Fut,
F: FnOnce(ClientConnection, mpsc::Receiver<SerializableMessage>) -> Fut,
Fut: Future<Output = ()> + Send + 'static,
{
// Add this client as a subscriber
Expand All @@ -148,7 +139,7 @@ impl ClientConnection {
.call_identity_connected_disconnected(id.identity, id.address, true)
.await?;

let (sendtx, sendrx) = mpsc::channel::<DataMessage>(CLIENT_CHANNEL_CAPACITY);
let (sendtx, sendrx) = mpsc::channel::<SerializableMessage>(CLIENT_CHANNEL_CAPACITY);

let db = module.info().address;

Expand Down
5 changes: 3 additions & 2 deletions crates/core/src/client/message_handlers.rs
Original file line number Diff line number Diff line change
@@ -1,3 +1,4 @@
use std::sync::Arc;
use std::time::{Duration, Instant};

use crate::energy::EnergyQuanta;
Expand Down Expand Up @@ -249,15 +250,15 @@ impl MessageExecutionError {
impl ServerMessage for MessageExecutionError {
fn serialize_text(self) -> crate::json::client_api::MessageJson {
TransactionUpdateMessage::<DatabaseUpdate> {
event: &mut self.into_event(),
event: Arc::new(self.into_event()),
database_update: Default::default(),
}
.serialize_text()
}

fn serialize_binary(self) -> Message {
TransactionUpdateMessage::<DatabaseUpdate> {
event: &mut self.into_event(),
event: Arc::new(self.into_event()),
database_update: Default::default(),
}
.serialize_binary()
Expand Down
104 changes: 45 additions & 59 deletions crates/core/src/client/messages.rs
Original file line number Diff line number Diff line change
@@ -1,9 +1,11 @@
use base64::Engine;
use derive_more::From;
use prost::Message as _;
use spacetimedb_lib::identity::RequestId;
use std::sync::Arc;
use std::time::Instant;

use crate::host::module_host::{EventStatus, ModuleEvent, ProtocolDatabaseUpdate};
use crate::host::module_host::{DatabaseUpdate, EventStatus, ModuleEvent, ProtocolDatabaseUpdate};
use crate::identity::Identity;
use crate::json::client_api::{
EventJson, FunctionCallJson, IdentityTokenJson, MessageJson, OneOffQueryResponseJson, OneOffTableJson,
Expand All @@ -14,6 +16,7 @@ use spacetimedb_client_api_messages::client_api::{OneOffQueryResponse, OneOffTab
use spacetimedb_lib::Address;
use spacetimedb_vm::relation::MemTable;

use super::message_handlers::MessageExecutionError;
use super::{DataMessage, Protocol};

/// A message sent from the server to the client. Because clients can request either text or binary messages,
Expand All @@ -29,6 +32,41 @@ pub trait ServerMessage: Sized {
fn serialize_binary(self) -> Message;
}

#[derive(Debug, From)]
pub enum SerializableMessage {
Query(OneOffQueryResponseMessage),
Error(MessageExecutionError),
Identity(IdentityTokenMessage),
Subscribe(SubscriptionUpdateMessage),
DatabaseUpdate(TransactionUpdateMessage<DatabaseUpdate>),
ProtocolUpdate(TransactionUpdateMessage<ProtocolDatabaseUpdate>),
}

impl ServerMessage for SerializableMessage {
fn serialize_text(self) -> MessageJson {
match self {
SerializableMessage::Query(msg) => msg.serialize_text(),
SerializableMessage::Error(msg) => msg.serialize_text(),
SerializableMessage::Identity(msg) => msg.serialize_text(),
SerializableMessage::Subscribe(msg) => msg.serialize_text(),
SerializableMessage::DatabaseUpdate(msg) => msg.serialize_text(),
SerializableMessage::ProtocolUpdate(msg) => msg.serialize_text(),
}
}

fn serialize_binary(self) -> Message {
match self {
SerializableMessage::Query(msg) => msg.serialize_binary(),
SerializableMessage::Error(msg) => msg.serialize_binary(),
SerializableMessage::Identity(msg) => msg.serialize_binary(),
SerializableMessage::Subscribe(msg) => msg.serialize_binary(),
SerializableMessage::DatabaseUpdate(msg) => msg.serialize_binary(),
SerializableMessage::ProtocolUpdate(msg) => msg.serialize_binary(),
}
}
}

#[derive(Debug)]
pub struct IdentityTokenMessage {
pub identity: Identity,
pub identity_token: String,
Expand All @@ -54,12 +92,13 @@ impl ServerMessage for IdentityTokenMessage {
}
}

pub struct TransactionUpdateMessage<'a, U> {
pub event: &'a ModuleEvent,
#[derive(Debug)]
pub struct TransactionUpdateMessage<U> {
pub event: Arc<ModuleEvent>,
pub database_update: SubscriptionUpdate<U>,
}

impl<U: Into<Vec<TableUpdate>> + Into<Vec<TableUpdateJson>>> ServerMessage for TransactionUpdateMessage<'_, U> {
impl<U: Into<Vec<TableUpdate>> + Into<Vec<TableUpdateJson>>> ServerMessage for TransactionUpdateMessage<U> {
fn serialize_text(self) -> MessageJson {
let Self { event, database_update } = self;
let (status_str, errmsg) = match &event.status {
Expand Down Expand Up @@ -123,25 +162,7 @@ impl<U: Into<Vec<TableUpdate>> + Into<Vec<TableUpdateJson>>> ServerMessage for T
}
}

impl<U: Clone + Into<Vec<TableUpdate>> + Into<Vec<TableUpdateJson>>> ServerMessage
for &mut TransactionUpdateMessage<'_, U>
{
fn serialize_text(self) -> MessageJson {
TransactionUpdateMessage {
event: self.event,
database_update: self.database_update.clone(),
}
.serialize_text()
}
fn serialize_binary(self) -> Message {
TransactionUpdateMessage {
event: self.event,
database_update: self.database_update.clone(),
}
.serialize_binary()
}
}

#[derive(Debug)]
pub struct SubscriptionUpdateMessage {
pub subscription_update: SubscriptionUpdate<ProtocolDatabaseUpdate>,
}
Expand Down Expand Up @@ -185,42 +206,7 @@ impl<T: Into<Vec<TableUpdateJson>>> SubscriptionUpdate<T> {
}
}

pub struct CachedMessage<M> {
msg: M,
text: Option<String>,
binary: Option<Vec<u8>>,
}

impl<M> CachedMessage<M> {
pub fn new(msg: M) -> Self {
Self {
msg,
text: None,
binary: None,
}
}
}

impl<M> CachedMessage<M>
where
for<'b> &'b mut M: ServerMessage,
{
pub fn serialize(&mut self, protocol: Protocol) -> DataMessage {
match protocol {
Protocol::Text => self
.text
.get_or_insert_with(|| self.msg.serialize_text().to_json())
.clone()
.into(),
Protocol::Binary => self
.binary
.get_or_insert_with(|| self.msg.serialize_binary().encode_to_vec())
.clone()
.into(),
}
}
}

#[derive(Debug)]
pub struct OneOffQueryResponseMessage {
pub message_id: Vec<u8>,
pub error: Option<String>,
Expand Down
4 changes: 2 additions & 2 deletions crates/core/src/subscription/module_subscription_actor.rs
Original file line number Diff line number Diff line change
Expand Up @@ -140,7 +140,7 @@ impl ModuleSubscriptions {
EventStatus::Failed(_) => {
if let Some(client) = client {
let message = TransactionUpdateMessage::<DatabaseUpdate> {
event: &event,
event,
database_update: <_>::default(),
};
let _ = client.send_message(message);
Expand Down Expand Up @@ -169,7 +169,7 @@ impl ModuleSubscriptions {
/// it resolves, it's guaranteed that if you call `subscriber.send(x)` the client will receive
/// x after they receive this subscription update).
fn broadcast_commit_event(&self, subscriptions: &SubscriptionManager, event: Arc<ModuleEvent>) {
subscriptions.eval_updates(&self.relational_db, &event)
subscriptions.eval_updates(&self.relational_db, event)
}
}

Expand Down
7 changes: 5 additions & 2 deletions crates/core/src/subscription/module_subscription_manager.rs
Original file line number Diff line number Diff line change
Expand Up @@ -112,7 +112,7 @@ impl SubscriptionManager {
/// evaluates only the necessary queries for those delta tables,
/// and then sends the results to each client.
#[tracing::instrument(skip_all)]
pub fn eval_updates(&self, db: &RelationalDB, event: &ModuleEvent) {
pub fn eval_updates(&self, db: &RelationalDB, event: Arc<ModuleEvent>) {
let tables = &event.status.database_update().unwrap().tables;

let tx = scopeguard::guard(db.begin_tx(), |tx| {
Expand Down Expand Up @@ -246,7 +246,10 @@ impl SubscriptionManager {
request_id: event.request_id,
timer: event.timer,
};
let message = TransactionUpdateMessage { event, database_update };
let message = TransactionUpdateMessage {
event: event.clone(),
database_update,
};
if let Err(e) = client.send_message(message) {
tracing::warn!(%client.id, "failed to send update message to client: {e}")
}
Expand Down

1 comment on commit 99bd7ac

@github-actions
Copy link

@github-actions github-actions bot commented on 99bd7ac Mar 27, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Criterion benchmark results

Criterion benchmark report

YOU SHOULD PROBABLY IGNORE THESE RESULTS.

Criterion is a wall time based benchmarking system that is extremely noisy when run on CI. We collect these results for longitudinal analysis, but they are not reliable for comparing individual PRs.

Go look at the callgrind report instead.

empty

db on disk new latency old latency new throughput old throughput
sqlite 💿 432.8±1.87ns 432.8±1.87ns - -
sqlite 🧠 422.8±3.15ns 422.8±3.15ns - -
stdb_raw 💿 706.0±1.36ns 706.0±1.36ns - -
stdb_raw 🧠 685.4±0.83ns 685.4±0.83ns - -

insert_1

db on disk schema indices preload new latency old latency new throughput old throughput

insert_bulk

db on disk schema indices preload count new latency old latency new throughput old throughput
sqlite 💿 u32_u64_str btree_each_column 2048 256 515.0±0.49µs 515.0±0.49µs 1941 tx/sec 1941 tx/sec
sqlite 💿 u32_u64_str unique_0 2048 256 136.2±0.44µs 136.2±0.44µs 7.2 Ktx/sec 7.2 Ktx/sec
sqlite 💿 u32_u64_u64 btree_each_column 2048 256 421.8±0.84µs 421.8±0.84µs 2.3 Ktx/sec 2.3 Ktx/sec
sqlite 💿 u32_u64_u64 unique_0 2048 256 127.1±0.51µs 127.1±0.51µs 7.7 Ktx/sec 7.7 Ktx/sec
sqlite 🧠 u32_u64_str btree_each_column 2048 256 447.5±0.56µs 447.5±0.56µs 2.2 Ktx/sec 2.2 Ktx/sec
sqlite 🧠 u32_u64_str unique_0 2048 256 120.1±0.63µs 120.1±0.63µs 8.1 Ktx/sec 8.1 Ktx/sec
sqlite 🧠 u32_u64_u64 btree_each_column 2048 256 370.0±0.26µs 370.0±0.26µs 2.6 Ktx/sec 2.6 Ktx/sec
sqlite 🧠 u32_u64_u64 unique_0 2048 256 109.3±1.04µs 109.3±1.04µs 8.9 Ktx/sec 8.9 Ktx/sec
stdb_raw 💿 u32_u64_str btree_each_column 2048 256 721.7±0.26µs 721.7±0.26µs 1385 tx/sec 1385 tx/sec
stdb_raw 💿 u32_u64_str unique_0 2048 256 628.0±0.51µs 628.0±0.51µs 1592 tx/sec 1592 tx/sec
stdb_raw 💿 u32_u64_u64 btree_each_column 2048 256 416.9±0.12µs 416.9±0.12µs 2.3 Ktx/sec 2.3 Ktx/sec
stdb_raw 💿 u32_u64_u64 unique_0 2048 256 365.6±0.38µs 365.6±0.38µs 2.7 Ktx/sec 2.7 Ktx/sec
stdb_raw 🧠 u32_u64_str btree_each_column 2048 256 497.3±0.16µs 497.3±0.16µs 2011 tx/sec 2011 tx/sec
stdb_raw 🧠 u32_u64_str unique_0 2048 256 407.7±0.83µs 407.7±0.83µs 2.4 Ktx/sec 2.4 Ktx/sec
stdb_raw 🧠 u32_u64_u64 btree_each_column 2048 256 311.9±0.24µs 311.9±0.24µs 3.1 Ktx/sec 3.1 Ktx/sec
stdb_raw 🧠 u32_u64_u64 unique_0 2048 256 269.0±0.86µs 269.0±0.86µs 3.6 Ktx/sec 3.6 Ktx/sec

iterate

db on disk schema indices new latency old latency new throughput old throughput
sqlite 💿 u32_u64_str unique_0 22.0±0.10µs 22.0±0.10µs 44.3 Ktx/sec 44.3 Ktx/sec
sqlite 💿 u32_u64_u64 unique_0 20.2±0.10µs 20.2±0.10µs 48.3 Ktx/sec 48.3 Ktx/sec
sqlite 🧠 u32_u64_str unique_0 20.7±0.17µs 20.7±0.17µs 47.1 Ktx/sec 47.1 Ktx/sec
sqlite 🧠 u32_u64_u64 unique_0 18.9±0.20µs 18.9±0.20µs 51.7 Ktx/sec 51.7 Ktx/sec
stdb_raw 💿 u32_u64_str unique_0 18.7±0.00µs 18.7±0.00µs 52.3 Ktx/sec 52.3 Ktx/sec
stdb_raw 💿 u32_u64_u64 unique_0 15.8±0.00µs 15.8±0.00µs 61.7 Ktx/sec 61.7 Ktx/sec
stdb_raw 🧠 u32_u64_str unique_0 18.6±0.00µs 18.6±0.00µs 52.4 Ktx/sec 52.4 Ktx/sec
stdb_raw 🧠 u32_u64_u64 unique_0 15.8±0.00µs 15.8±0.00µs 61.8 Ktx/sec 61.8 Ktx/sec

find_unique

db on disk key type preload new latency old latency new throughput old throughput

filter

db on disk key type index strategy load count new latency old latency new throughput old throughput
sqlite 💿 string index 2048 256 67.7±0.15µs 67.7±0.15µs 14.4 Ktx/sec 14.4 Ktx/sec
sqlite 💿 u64 index 2048 256 63.3±0.32µs 63.3±0.32µs 15.4 Ktx/sec 15.4 Ktx/sec
sqlite 🧠 string index 2048 256 65.6±0.24µs 65.6±0.24µs 14.9 Ktx/sec 14.9 Ktx/sec
sqlite 🧠 u64 index 2048 256 58.9±0.40µs 58.9±0.40µs 16.6 Ktx/sec 16.6 Ktx/sec
stdb_raw 💿 string index 2048 256 5.6±0.00µs 5.6±0.00µs 173.5 Ktx/sec 173.5 Ktx/sec
stdb_raw 💿 u64 index 2048 256 5.5±0.00µs 5.5±0.00µs 178.1 Ktx/sec 178.1 Ktx/sec
stdb_raw 🧠 string index 2048 256 5.6±0.00µs 5.6±0.00µs 174.4 Ktx/sec 174.4 Ktx/sec
stdb_raw 🧠 u64 index 2048 256 5.4±0.00µs 5.4±0.00µs 179.6 Ktx/sec 179.6 Ktx/sec

serialize

schema format count new latency old latency new throughput old throughput
u32_u64_str bflatn_to_bsatn_fast_path 100 4.0±0.01µs 4.0±0.01µs 24.1 Mtx/sec 24.1 Mtx/sec
u32_u64_str bflatn_to_bsatn_slow_path 100 3.7±0.01µs 3.7±0.01µs 26.0 Mtx/sec 26.0 Mtx/sec
u32_u64_str bsatn 100 2.4±0.01µs 2.4±0.01µs 40.4 Mtx/sec 40.4 Mtx/sec
u32_u64_str json 100 5.4±0.03µs 5.4±0.03µs 17.6 Mtx/sec 17.6 Mtx/sec
u32_u64_str product_value 100 646.8±8.28ns 646.8±8.28ns 147.5 Mtx/sec 147.5 Mtx/sec
u32_u64_u64 bflatn_to_bsatn_fast_path 100 1388.3±1.72ns 1388.3±1.72ns 68.7 Mtx/sec 68.7 Mtx/sec
u32_u64_u64 bflatn_to_bsatn_slow_path 100 2.9±0.01µs 2.9±0.01µs 32.9 Mtx/sec 32.9 Mtx/sec
u32_u64_u64 bsatn 100 1630.7±67.15ns 1630.7±67.15ns 58.5 Mtx/sec 58.5 Mtx/sec
u32_u64_u64 json 100 3.2±0.03µs 3.2±0.03µs 30.0 Mtx/sec 30.0 Mtx/sec
u32_u64_u64 product_value 100 560.1±0.61ns 560.1±0.61ns 170.3 Mtx/sec 170.3 Mtx/sec
u64_u64_u32 bflatn_to_bsatn_fast_path 100 1083.1±3.11ns 1083.1±3.11ns 88.0 Mtx/sec 88.0 Mtx/sec
u64_u64_u32 bflatn_to_bsatn_slow_path 100 2.9±0.01µs 2.9±0.01µs 32.7 Mtx/sec 32.7 Mtx/sec
u64_u64_u32 bsatn 100 1712.5±3.10ns 1712.5±3.10ns 55.7 Mtx/sec 55.7 Mtx/sec
u64_u64_u32 json 100 3.4±0.03µs 3.4±0.03µs 27.8 Mtx/sec 27.8 Mtx/sec
u64_u64_u32 product_value 100 598.6±0.47ns 598.6±0.47ns 159.3 Mtx/sec 159.3 Mtx/sec

stdb_module_large_arguments

arg size new latency old latency new throughput old throughput
64KiB 91.2±10.26µs 91.2±10.26µs - -

stdb_module_print_bulk

line count new latency old latency new throughput old throughput
1 41.3±6.16µs 41.3±6.16µs - -
100 339.7±11.17µs 339.7±11.17µs - -
1000 2.8±0.20ms 2.8±0.20ms - -

remaining

name new latency old latency new throughput old throughput
sqlite/💿/update_bulk/u32_u64_str/unique_0/load=2048/count=256 50.1±0.16µs 50.1±0.16µs 19.5 Ktx/sec 19.5 Ktx/sec
sqlite/💿/update_bulk/u32_u64_u64/unique_0/load=2048/count=256 43.3±0.55µs 43.3±0.55µs 22.5 Ktx/sec 22.5 Ktx/sec
sqlite/🧠/update_bulk/u32_u64_str/unique_0/load=2048/count=256 43.0±0.23µs 43.0±0.23µs 22.7 Ktx/sec 22.7 Ktx/sec
sqlite/🧠/update_bulk/u32_u64_u64/unique_0/load=2048/count=256 35.8±0.21µs 35.8±0.21µs 27.3 Ktx/sec 27.3 Ktx/sec
stdb_module/💿/update_bulk/u32_u64_str/unique_0/load=2048/count=256 2.8±0.01ms 2.8±0.01ms 352 tx/sec 352 tx/sec
stdb_module/💿/update_bulk/u32_u64_u64/unique_0/load=2048/count=256 1834.4±3.82µs 1834.4±3.82µs 545 tx/sec 545 tx/sec
stdb_raw/💿/update_bulk/u32_u64_str/unique_0/load=2048/count=256 1117.9±1.45µs 1117.9±1.45µs 894 tx/sec 894 tx/sec
stdb_raw/💿/update_bulk/u32_u64_u64/unique_0/load=2048/count=256 729.1±0.40µs 729.1±0.40µs 1371 tx/sec 1371 tx/sec
stdb_raw/🧠/update_bulk/u32_u64_str/unique_0/load=2048/count=256 790.6±0.50µs 790.6±0.50µs 1264 tx/sec 1264 tx/sec
stdb_raw/🧠/update_bulk/u32_u64_u64/unique_0/load=2048/count=256 534.4±0.30µs 534.4±0.30µs 1871 tx/sec 1871 tx/sec

Please sign in to comment.