diff --git a/Cargo.lock b/Cargo.lock index e0a426c1fd..2799b0164e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8238,6 +8238,7 @@ dependencies = [ "bilrost", "bytes", "bytestring", + "derive_more", "enum-map", "prost 0.14.1", "restate-invoker-api", diff --git a/crates/types/src/identifiers.rs b/crates/types/src/identifiers.rs index e4e850679f..79eea440f4 100644 --- a/crates/types/src/identifiers.rs +++ b/crates/types/src/identifiers.rs @@ -349,15 +349,26 @@ impl From for opentelemetry::trace::SpanId { /// Services are isolated by key. This means that there cannot be two concurrent /// invocations for the same service instance (service name, key). #[derive( - Eq, Hash, PartialEq, PartialOrd, Ord, Clone, Debug, serde::Serialize, serde::Deserialize, + Eq, + Hash, + PartialEq, + PartialOrd, + Ord, + Clone, + Debug, + serde::Serialize, + serde::Deserialize, + bilrost::Message, )] pub struct ServiceId { // TODO rename this to KeyedServiceId. This type can be used only by keyed service types (virtual objects and workflows) /// Identifies the grpc service + #[bilrost(1)] pub service_name: ByteString, /// Identifies the service instance for the given service name + #[bilrost(2)] pub key: ByteString, - + #[bilrost(3)] partition_key: PartitionKey, } diff --git a/crates/types/src/invocation/mod.rs b/crates/types/src/invocation/mod.rs index 90fc59722a..e3afbf65bd 100644 --- a/crates/types/src/invocation/mod.rs +++ b/crates/types/src/invocation/mod.rs @@ -12,6 +12,18 @@ pub mod client; +use std::borrow::Cow; +use std::hash::Hash; +use std::ops::Deref; +use std::str::FromStr; +use std::time::Duration; +use std::{cmp, fmt}; + +use bytes::Bytes; +use bytestring::ByteString; +use opentelemetry::trace::{SpanContext, SpanId, TraceFlags, TraceState}; +use serde_with::{DisplayFromStr, FromInto, serde_as}; + use crate::errors::InvocationError; use crate::identifiers::{ DeploymentId, EntryIndex, IdempotencyId, InvocationId, PartitionKey, @@ -21,17 +33,6 @@ use crate::journal_v2::{CompletionId, GetInvocationOutputResult, Signal}; use crate::time::MillisSinceEpoch; use crate::{GenerationalNodeId, RestateVersion}; -use bytes::Bytes; -use bytestring::ByteString; -use opentelemetry::trace::{SpanContext, SpanId, TraceFlags, TraceState}; -use serde_with::{DisplayFromStr, FromInto, serde_as}; -use std::borrow::Cow; -use std::hash::Hash; -use std::ops::Deref; -use std::str::FromStr; -use std::time::Duration; -use std::{cmp, fmt}; - // Re-exporting opentelemetry [`TraceId`] to avoid having to import opentelemetry in all crates. pub use opentelemetry::trace::TraceId; diff --git a/crates/types/src/state_mut.rs b/crates/types/src/state_mut.rs index 7a28be51cf..6426aca6e1 100644 --- a/crates/types/src/state_mut.rs +++ b/crates/types/src/state_mut.rs @@ -22,11 +22,14 @@ use crate::identifiers::ServiceId; /// ExternalStateMutation /// /// represents an external request to mutate a user's state. -#[derive(Debug, Clone, Eq, PartialEq, serde::Serialize, serde::Deserialize)] +#[derive(Debug, Clone, Eq, PartialEq, serde::Serialize, serde::Deserialize, bilrost::Message)] pub struct ExternalStateMutation { + #[bilrost(1)] pub service_id: ServiceId, + #[bilrost(2)] pub version: Option, // flexbuffers only supports string-keyed maps :-( --> so we store it as vector of kv pairs + #[bilrost(3)] #[serde_as(as = "serde_with::Seq<(_, _)>")] pub state: HashMap, } diff --git a/crates/types/src/storage.rs b/crates/types/src/storage.rs index e57e97c146..4cb7026b6e 100644 --- a/crates/types/src/storage.rs +++ b/crates/types/src/storage.rs @@ -27,6 +27,8 @@ use crate::journal_v2::raw::{RawEntry, RawEntryError, TryFromEntry}; use crate::journal_v2::{Decoder, EntryMetadata, EntryType}; use crate::time::MillisSinceEpoch; +pub use tracing; + #[derive(Debug, thiserror::Error)] pub enum StorageEncodeError { #[error("encoding failed: {0}")] @@ -187,7 +189,7 @@ macro_rules! flexbuffers_storage_encode_decode { Self: Sized, { $crate::storage::decode::decode_serde(buf, kind).map_err(|err| { - ::tracing::error!(%err, "{} decode failure (decoding {})", kind, stringify!($name)); + $crate::storage::tracing::error!(%err, "{} decode failure (decoding {})", kind, stringify!($name)); err }) @@ -196,6 +198,40 @@ macro_rules! flexbuffers_storage_encode_decode { }; } +/// Implements the [`StorageEncode`] and [`StorageDecode`] by encoding/decoding the implementing +/// type using [`bilrost`]. +#[macro_export] +macro_rules! bilrost_storage_encode_decode { + ($name:tt) => { + impl $crate::storage::StorageEncode for $name { + fn default_codec(&self) -> $crate::storage::StorageCodecKind { + $crate::storage::StorageCodecKind::Bilrost + } + + fn encode( + &self, + buf: &mut ::bytes::BytesMut, + ) -> Result<(), $crate::storage::StorageEncodeError> { + bytes::BufMut::put(buf, $crate::storage::encode::encode_bilrost(self)); + Ok(()) + } + } + + impl $crate::storage::StorageDecode for $name { + fn decode( + buf: &mut B, + kind: $crate::storage::StorageCodecKind, + ) -> Result + where + Self: Sized, + { + debug_assert_eq!(kind, $crate::storage::StorageCodecKind::Bilrost); + $crate::storage::decode::decode_bilrost(buf) + } + } + }; +} + /// A polymorphic container of a buffer or a cached storage-encodeable object #[derive(Clone, derive_more::Debug, BilrostAs)] #[bilrost_as(dto::PolyBytes)] diff --git a/crates/wal-protocol/Cargo.toml b/crates/wal-protocol/Cargo.toml index 82941fc1f2..2083ae1bd5 100644 --- a/crates/wal-protocol/Cargo.toml +++ b/crates/wal-protocol/Cargo.toml @@ -13,7 +13,6 @@ serde = ["dep:serde", "enum-map/serde", "bytestring/serde", "restate-invoker-api [dependencies] restate-workspace-hack = { workspace = true } - restate-invoker-api = { workspace = true } restate-storage-api = { workspace = true } restate-types = { workspace = true } @@ -22,6 +21,7 @@ anyhow = { workspace = true } bytes = { workspace = true } bytestring = { workspace = true } bilrost = { workspace = true } +derive_more = { workspace = true } enum-map = { workspace = true } serde = { workspace = true, optional = true } strum = { workspace = true } diff --git a/crates/wal-protocol/src/control.rs b/crates/wal-protocol/src/control.rs index 01bfa59368..3a9ad7a434 100644 --- a/crates/wal-protocol/src/control.rs +++ b/crates/wal-protocol/src/control.rs @@ -11,25 +11,38 @@ use std::ops::RangeInclusive; use restate_types::identifiers::{LeaderEpoch, PartitionId, PartitionKey}; -use restate_types::logs::{Keys, Lsn}; +use restate_types::logs::{HasRecordKeys, Keys, Lsn}; use restate_types::schema::Schema; use restate_types::time::MillisSinceEpoch; -use restate_types::{GenerationalNodeId, SemanticRestateVersion}; +use restate_types::{ + GenerationalNodeId, SemanticRestateVersion, bilrost_storage_encode_decode, + flexbuffers_storage_encode_decode, +}; /// Announcing a new leader. This message can be written by any component to make the specified /// partition processor the leader. -#[derive(Debug, Clone, PartialEq, Eq)] +#[derive(Debug, Clone, PartialEq, Eq, bilrost::Message)] #[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))] pub struct AnnounceLeader { /// Sender of the announce leader message. /// /// This became non-optional in v1.5. Noting that it has always been set in previous versions, /// it's safe to assume that it's always set. + #[bilrost(1)] pub node_id: GenerationalNodeId, + #[bilrost(2)] pub leader_epoch: LeaderEpoch, + #[bilrost(3)] pub partition_key_range: RangeInclusive, } +bilrost_storage_encode_decode!(AnnounceLeader); + +impl HasRecordKeys for AnnounceLeader { + fn record_keys(&self) -> Keys { + Keys::RangeInclusive(self.partition_key_range.clone()) + } +} /// A version barrier to fence off state machine changes that require a certain minimum /// version of restate server. /// @@ -40,12 +53,23 @@ pub struct AnnounceLeader { #[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))] pub struct VersionBarrier { /// The minimum version required (inclusive) to progress after this barrier. + #[bilrost(1)] pub version: SemanticRestateVersion, /// A human-readable reason for why this barrier exists. + #[bilrost(2)] pub human_reason: Option, + #[bilrost(3)] pub partition_key_range: Keys, } +bilrost_storage_encode_decode!(VersionBarrier); + +impl HasRecordKeys for VersionBarrier { + fn record_keys(&self) -> Keys { + self.partition_key_range.clone() + } +} + /// Updates the `PARTITION_DURABILITY` FSM variable to the given value. Note that durability /// only applies to partitions with the same `partition_id`. At replay time, the partition will /// ignore updates that are not targeted to its own ID. @@ -56,12 +80,26 @@ pub struct VersionBarrier { #[derive(Debug, Clone, PartialEq, Eq, bilrost::Message)] #[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))] pub struct PartitionDurability { + #[bilrost(1)] pub partition_id: PartitionId, /// The partition has applied this LSN durably to the replica-set and/or has been /// persisted in a snapshot in the snapshot repository. + #[bilrost(2)] pub durable_point: Lsn, /// Timestamp which the durability point was updated + #[bilrost(3)] pub modification_time: MillisSinceEpoch, + /// partition key range + #[bilrost(4)] + pub partition_key_range: Keys, +} + +bilrost_storage_encode_decode!(PartitionDurability); + +impl HasRecordKeys for PartitionDurability { + fn record_keys(&self) -> Keys { + self.partition_key_range.clone() + } } /// Consistently store schema across partition replicas. @@ -73,3 +111,11 @@ pub struct UpsertSchema { pub partition_key_range: Keys, pub schema: Schema, } + +flexbuffers_storage_encode_decode!(UpsertSchema); + +impl HasRecordKeys for UpsertSchema { + fn record_keys(&self) -> Keys { + self.partition_key_range.clone() + } +} diff --git a/crates/wal-protocol/src/lib.rs b/crates/wal-protocol/src/lib.rs index 43d9491006..867c908cce 100644 --- a/crates/wal-protocol/src/lib.rs +++ b/crates/wal-protocol/src/lib.rs @@ -8,244 +8,9 @@ // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0. -use restate_storage_api::deduplication_table::DedupInformation; -use restate_types::identifiers::{LeaderEpoch, PartitionId, PartitionKey, WithPartitionKey}; -use restate_types::invocation::{ - AttachInvocationRequest, GetInvocationOutputResponse, InvocationResponse, - InvocationTermination, NotifySignalRequest, PurgeInvocationRequest, - RestartAsNewInvocationRequest, ResumeInvocationRequest, ServiceInvocation, -}; -use restate_types::logs::{self, HasRecordKeys, Keys, MatchKeyQuery}; -use restate_types::message::MessageIndex; -use restate_types::state_mut::ExternalStateMutation; - -use crate::control::{AnnounceLeader, UpsertSchema, VersionBarrier}; -use crate::timer::TimerKeyValue; - -use self::control::PartitionDurability; - pub mod control; pub mod timer; +mod v1; +pub mod v2; -/// The primary envelope for all messages in the system. -#[derive(Debug, Clone)] -#[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))] -pub struct Envelope { - pub header: Header, - pub command: Command, -} - -#[cfg(feature = "serde")] -restate_types::flexbuffers_storage_encode_decode!(Envelope); - -impl Envelope { - pub fn new(header: Header, command: Command) -> Self { - Self { header, command } - } -} - -/// Header is set on every message -#[derive(Debug, Clone, PartialEq, Eq)] -#[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))] -pub struct Header { - pub source: Source, - pub dest: Destination, -} - -/// Identifies the source of a message -#[derive(Debug, Clone, PartialEq, Eq)] -#[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))] -pub enum Source { - /// Message is sent from another partition processor - Processor { - /// if possible, this is used to reroute responses in case of splits/merges - /// v1.4 requires this to be set. - /// v1.5 Marked as `Option`. - /// v1.6 always set to `None`. - /// Will be removed in v1.7. - #[cfg_attr(feature = "serde", serde(default))] - partition_id: Option, - #[cfg_attr(feature = "serde", serde(default))] - partition_key: Option, - /// The current epoch of the partition leader. Readers should observe this to decide which - /// messages to accept. Readers should ignore messages coming from - /// epochs lower than the max observed for a given partition id. - leader_epoch: LeaderEpoch, - // Which node is this message from? - // First deprecation in v1.1, but since v1.5 we switched to Option and it's - // still being set to Some(v) to maintain compatibility with v1.4. - // - // v1.6 field is removed. -- Kept here for reference only. - // #[cfg_attr(feature = "serde", serde(default))] - // node_id: Option, - - // From v1.1 this is always set, but maintained to support rollback to v1.0. - // Deprecated(v1.5): It's set to Some(v) to maintain support for v1.4 but - // will be removed in v1.6. Commands that need the node-id of the sender should - // include the node-id in the command payload itself (e.g. in the [`AnnounceLeader`]) - // v1.6 field is removed. -- Kept here for reference only. - // #[cfg_attr(feature = "serde", serde(default))] - // generational_node_id: Option, - }, - /// Message is sent from an ingress node - Ingress { - // The identity of the sender node. Generational for fencing. Ingress is - // stateless, so we shouldn't respond to requests from older generation - // if a new generation is alive. - // - // Deprecated(v1.5): This field is set to Some(v) to maintain compatibility with v1.4. - // but will be removed in v1.6. - // v1.6 field is removed. -- Kept here for reference only. - // #[cfg_attr(feature = "serde", serde(default))] - // node_id: Option, - - // Last config version observed by sender. If this is a newer generation - // or an unknown ID, we might need to update our config. - // - // Deprecated(v1.5): This field is set to Some(v) to maintain compatibility with v1.4. - // but will be removed in v1.6. - // v1.6 field is removed. -- Kept here for reference only. - // #[cfg_attr(feature = "serde", serde(default))] - // nodes_config_version: Option, - }, - /// Message is sent from some control plane component (controller, cli, etc.) - ControlPlane { - // Reserved for future use. - }, -} - -/// Identifies the intended destination of the message -#[derive(Debug, Clone, PartialEq, Eq)] -#[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))] -pub enum Destination { - /// Message is sent to partition processor - Processor { - partition_key: PartitionKey, - #[cfg_attr(feature = "serde", serde(default))] - dedup: Option, - }, -} - -/// State machine input commands -#[derive(Debug, Clone, strum::EnumDiscriminants, strum::VariantNames)] -#[strum_discriminants(derive(strum::IntoStaticStr))] -#[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))] -pub enum Command { - /// Updates the `PARTITION_DURABILITY` FSM variable to the given value. - /// See [`PartitionDurability`] for more details. - /// - /// *Since v1.4.2* - UpdatePartitionDurability(PartitionDurability), - /// A version barrier to fence off state machine changes that require a certain minimum - /// version of restate server. - /// *Since v1.4.0* - VersionBarrier(VersionBarrier), - // -- Control-plane related events - AnnounceLeader(Box), - - // -- Partition processor commands - /// Manual patching of storage state - PatchState(ExternalStateMutation), - /// Terminate an ongoing invocation - TerminateInvocation(InvocationTermination), - /// Purge a completed invocation - PurgeInvocation(PurgeInvocationRequest), - /// Purge a completed invocation journal - PurgeJournal(PurgeInvocationRequest), - /// Start an invocation on this partition - Invoke(Box), - /// Truncate the message outbox up to, and including, the specified index. - TruncateOutbox(MessageIndex), - /// Proxy a service invocation through this partition processor, to reuse the deduplication id map. - ProxyThrough(Box), - /// Attach to an existing invocation - AttachInvocation(AttachInvocationRequest), - /// Resume an invocation - ResumeInvocation(ResumeInvocationRequest), - /// Restart as new invocation from prefix - RestartAsNewInvocation(RestartAsNewInvocationRequest), - - // -- Partition processor events for PP - /// Invoker is reporting effect(s) from an ongoing invocation. - InvokerEffect(Box), - /// Timer has fired - Timer(TimerKeyValue), - /// Schedule timer - ScheduleTimer(TimerKeyValue), - /// Another partition processor is reporting a response of an invocation we requested. - /// - /// KINDA DEPRECATED: When Journal Table V1 is removed, this command should be used only to reply to invocations. - /// Now it's abused for a bunch of other scenarios, like replying to get promise and get invocation output. - /// - /// For more details see `OnNotifyInvocationResponse`. - InvocationResponse(InvocationResponse), - - // -- New PP <-> PP commands using Journal V2 - /// Notify Get invocation output - NotifyGetInvocationOutputResponse(GetInvocationOutputResponse), - /// Notify a signal. - NotifySignal(NotifySignalRequest), - - /// Upsert schema for consistent schema across replicas - /// *Since v1.6.0 - UpsertSchema(UpsertSchema), -} - -impl Command { - pub fn name(&self) -> &'static str { - CommandDiscriminants::from(self).into() - } -} - -impl WithPartitionKey for Envelope { - fn partition_key(&self) -> PartitionKey { - match self.header.dest { - Destination::Processor { partition_key, .. } => partition_key, - } - } -} - -impl HasRecordKeys for Envelope { - fn record_keys(&self) -> logs::Keys { - match &self.command { - // the partition_key is used as key here since the command targets the partition by ID. - // Partitions will ignore this message at read time if the paritition ID (in body) - // does not match. Alternatively, we could use the partition key range or `Keys::None` - // but this would just be a waste of effort for readers after a partition has been - // split or if the log is shared between multiple partitions. - Command::UpdatePartitionDurability(_) => Keys::Single(self.partition_key()), - Command::VersionBarrier(barrier) => barrier.partition_key_range.clone(), - Command::AnnounceLeader(announce) => { - Keys::RangeInclusive(announce.partition_key_range.clone()) - } - Command::PatchState(mutation) => Keys::Single(mutation.service_id.partition_key()), - Command::TerminateInvocation(terminate) => { - Keys::Single(terminate.invocation_id.partition_key()) - } - Command::PurgeInvocation(purge) => Keys::Single(purge.invocation_id.partition_key()), - Command::PurgeJournal(purge) => Keys::Single(purge.invocation_id.partition_key()), - Command::Invoke(invoke) => Keys::Single(invoke.partition_key()), - // todo: Remove this, or pass the partition key range but filter based on partition-id - // on read if needed. - Command::TruncateOutbox(_) => Keys::Single(self.partition_key()), - Command::ProxyThrough(_) => Keys::Single(self.partition_key()), - Command::AttachInvocation(_) => Keys::Single(self.partition_key()), - Command::ResumeInvocation(req) => Keys::Single(req.partition_key()), - Command::RestartAsNewInvocation(req) => Keys::Single(req.partition_key()), - // todo: Handle journal entries that request cross-partition invocations - Command::InvokerEffect(effect) => Keys::Single(effect.invocation_id.partition_key()), - Command::Timer(timer) => Keys::Single(timer.invocation_id().partition_key()), - Command::ScheduleTimer(timer) => Keys::Single(timer.invocation_id().partition_key()), - Command::InvocationResponse(response) => Keys::Single(response.partition_key()), - Command::NotifySignal(sig) => Keys::Single(sig.partition_key()), - Command::NotifyGetInvocationOutputResponse(res) => Keys::Single(res.partition_key()), - Command::UpsertSchema(schema) => schema.partition_key_range.clone(), - } - } -} - -impl MatchKeyQuery for Envelope { - fn matches_key_query(&self, query: &logs::KeyFilter) -> bool { - self.record_keys().matches_key_query(query) - } -} +pub use v1::{Command, Destination, Envelope, Header, Source}; diff --git a/crates/wal-protocol/src/v1.rs b/crates/wal-protocol/src/v1.rs new file mode 100644 index 0000000000..66ff9f95fd --- /dev/null +++ b/crates/wal-protocol/src/v1.rs @@ -0,0 +1,246 @@ +// Copyright (c) 2023 - 2025 Restate Software, Inc., Restate GmbH. +// All rights reserved. +// +// Use of this software is governed by the Business Source License +// included in the LICENSE file. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0. + +use restate_storage_api::deduplication_table::DedupInformation; +use restate_types::identifiers::{LeaderEpoch, PartitionId, PartitionKey, WithPartitionKey}; +use restate_types::invocation::{ + AttachInvocationRequest, GetInvocationOutputResponse, InvocationResponse, + InvocationTermination, NotifySignalRequest, PurgeInvocationRequest, + RestartAsNewInvocationRequest, ResumeInvocationRequest, ServiceInvocation, +}; +use restate_types::logs::{self, HasRecordKeys, Keys, MatchKeyQuery}; +use restate_types::message::MessageIndex; +use restate_types::state_mut::ExternalStateMutation; + +use crate::control::{AnnounceLeader, PartitionDurability, UpsertSchema, VersionBarrier}; +use crate::timer::TimerKeyValue; + +/// The primary envelope for all messages in the system. +#[derive(Debug, Clone)] +#[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))] +pub struct Envelope { + pub header: Header, + pub command: Command, +} + +#[cfg(feature = "serde")] +restate_types::flexbuffers_storage_encode_decode!(Envelope); + +impl Envelope { + pub fn new(header: Header, command: Command) -> Self { + Self { header, command } + } +} + +/// Header is set on every message +#[derive(Debug, Clone, PartialEq, Eq)] +#[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))] +pub struct Header { + pub source: Source, + pub dest: Destination, +} + +/// Identifies the source of a message +#[derive(Debug, Clone, PartialEq, Eq)] +#[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))] +pub enum Source { + /// Message is sent from another partition processor + Processor { + /// if possible, this is used to reroute responses in case of splits/merges + /// v1.4 requires this to be set. + /// v1.5 Marked as `Option`. + /// v1.6 always set to `None`. + /// Will be removed in v1.7. + #[cfg_attr(feature = "serde", serde(default))] + partition_id: Option, + #[cfg_attr(feature = "serde", serde(default))] + partition_key: Option, + /// The current epoch of the partition leader. Readers should observe this to decide which + /// messages to accept. Readers should ignore messages coming from + /// epochs lower than the max observed for a given partition id. + leader_epoch: LeaderEpoch, + // Which node is this message from? + // First deprecation in v1.1, but since v1.5 we switched to Option and it's + // still being set to Some(v) to maintain compatibility with v1.4. + // + // v1.6 field is removed. -- Kept here for reference only. + // #[cfg_attr(feature = "serde", serde(default))] + // node_id: Option, + + // From v1.1 this is always set, but maintained to support rollback to v1.0. + // Deprecated(v1.5): It's set to Some(v) to maintain support for v1.4 but + // will be removed in v1.6. Commands that need the node-id of the sender should + // include the node-id in the command payload itself (e.g. in the [`AnnounceLeader`]) + // v1.6 field is removed. -- Kept here for reference only. + // #[cfg_attr(feature = "serde", serde(default))] + // generational_node_id: Option, + }, + /// Message is sent from an ingress node + Ingress { + // The identity of the sender node. Generational for fencing. Ingress is + // stateless, so we shouldn't respond to requests from older generation + // if a new generation is alive. + // + // Deprecated(v1.5): This field is set to Some(v) to maintain compatibility with v1.4. + // but will be removed in v1.6. + // v1.6 field is removed. -- Kept here for reference only. + // #[cfg_attr(feature = "serde", serde(default))] + // node_id: Option, + + // Last config version observed by sender. If this is a newer generation + // or an unknown ID, we might need to update our config. + // + // Deprecated(v1.5): This field is set to Some(v) to maintain compatibility with v1.4. + // but will be removed in v1.6. + // v1.6 field is removed. -- Kept here for reference only. + // #[cfg_attr(feature = "serde", serde(default))] + // nodes_config_version: Option, + }, + /// Message is sent from some control plane component (controller, cli, etc.) + ControlPlane { + // Reserved for future use. + }, +} + +/// Identifies the intended destination of the message +#[derive(Debug, Clone, PartialEq, Eq)] +#[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))] +pub enum Destination { + /// Message is sent to partition processor + Processor { + partition_key: PartitionKey, + #[cfg_attr(feature = "serde", serde(default))] + dedup: Option, + }, +} + +/// State machine input commands +#[derive(Debug, Clone, strum::EnumDiscriminants, strum::VariantNames)] +#[strum_discriminants(derive(strum::IntoStaticStr))] +#[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))] +pub enum Command { + /// Updates the `PARTITION_DURABILITY` FSM variable to the given value. + /// See [`PartitionDurability`] for more details. + /// + /// *Since v1.4.2* + UpdatePartitionDurability(PartitionDurability), + /// A version barrier to fence off state machine changes that require a certain minimum + /// version of restate server. + /// *Since v1.4.0* + VersionBarrier(VersionBarrier), + // -- Control-plane related events + AnnounceLeader(Box), + + // -- Partition processor commands + /// Manual patching of storage state + PatchState(ExternalStateMutation), + /// Terminate an ongoing invocation + TerminateInvocation(InvocationTermination), + /// Purge a completed invocation + PurgeInvocation(PurgeInvocationRequest), + /// Purge a completed invocation journal + PurgeJournal(PurgeInvocationRequest), + /// Start an invocation on this partition + Invoke(Box), + /// Truncate the message outbox up to, and including, the specified index. + TruncateOutbox(MessageIndex), + /// Proxy a service invocation through this partition processor, to reuse the deduplication id map. + ProxyThrough(Box), + /// Attach to an existing invocation + AttachInvocation(AttachInvocationRequest), + /// Resume an invocation + ResumeInvocation(ResumeInvocationRequest), + /// Restart as new invocation from prefix + RestartAsNewInvocation(RestartAsNewInvocationRequest), + + // -- Partition processor events for PP + /// Invoker is reporting effect(s) from an ongoing invocation. + InvokerEffect(Box), + /// Timer has fired + Timer(TimerKeyValue), + /// Schedule timer + ScheduleTimer(TimerKeyValue), + /// Another partition processor is reporting a response of an invocation we requested. + /// + /// KINDA DEPRECATED: When Journal Table V1 is removed, this command should be used only to reply to invocations. + /// Now it's abused for a bunch of other scenarios, like replying to get promise and get invocation output. + /// + /// For more details see `OnNotifyInvocationResponse`. + InvocationResponse(InvocationResponse), + + // -- New PP <-> PP commands using Journal V2 + /// Notify Get invocation output + NotifyGetInvocationOutputResponse(GetInvocationOutputResponse), + /// Notify a signal. + NotifySignal(NotifySignalRequest), + + /// Upsert schema for consistent schema across replicas + /// *Since v1.6.0 + UpsertSchema(UpsertSchema), +} + +impl Command { + pub fn name(&self) -> &'static str { + CommandDiscriminants::from(self).into() + } +} + +impl WithPartitionKey for Envelope { + fn partition_key(&self) -> PartitionKey { + match self.header.dest { + Destination::Processor { partition_key, .. } => partition_key, + } + } +} + +impl HasRecordKeys for Envelope { + fn record_keys(&self) -> logs::Keys { + match &self.command { + // the partition_key is used as key here since the command targets the partition by ID. + // Partitions will ignore this message at read time if the paritition ID (in body) + // does not match. Alternatively, we could use the partition key range or `Keys::None` + // but this would just be a waste of effort for readers after a partition has been + // split or if the log is shared between multiple partitions. + Command::UpdatePartitionDurability(_) => Keys::Single(self.partition_key()), + Command::VersionBarrier(barrier) => barrier.partition_key_range.clone(), + Command::AnnounceLeader(announce) => { + Keys::RangeInclusive(announce.partition_key_range.clone()) + } + Command::PatchState(mutation) => Keys::Single(mutation.service_id.partition_key()), + Command::TerminateInvocation(terminate) => { + Keys::Single(terminate.invocation_id.partition_key()) + } + Command::PurgeInvocation(purge) => Keys::Single(purge.invocation_id.partition_key()), + Command::PurgeJournal(purge) => Keys::Single(purge.invocation_id.partition_key()), + Command::Invoke(invoke) => Keys::Single(invoke.partition_key()), + // todo: Remove this, or pass the partition key range but filter based on partition-id + // on read if needed. + Command::TruncateOutbox(_) => Keys::Single(self.partition_key()), + Command::ProxyThrough(_) => Keys::Single(self.partition_key()), + Command::AttachInvocation(req) => Keys::Single(req.partition_key()), + Command::ResumeInvocation(req) => Keys::Single(req.partition_key()), + Command::RestartAsNewInvocation(req) => Keys::Single(req.partition_key()), + // todo: Handle journal entries that request cross-partition invocations + Command::InvokerEffect(effect) => Keys::Single(effect.invocation_id.partition_key()), + Command::Timer(timer) => Keys::Single(timer.invocation_id().partition_key()), + Command::ScheduleTimer(timer) => Keys::Single(timer.invocation_id().partition_key()), + Command::InvocationResponse(response) => Keys::Single(response.partition_key()), + Command::NotifySignal(sig) => Keys::Single(sig.partition_key()), + Command::NotifyGetInvocationOutputResponse(res) => Keys::Single(res.partition_key()), + Command::UpsertSchema(schema) => schema.partition_key_range.clone(), + } + } +} + +impl MatchKeyQuery for Envelope { + fn matches_key_query(&self, query: &logs::KeyFilter) -> bool { + self.record_keys().matches_key_query(query) + } +} diff --git a/crates/wal-protocol/src/v2.rs b/crates/wal-protocol/src/v2.rs new file mode 100644 index 0000000000..da0450af80 --- /dev/null +++ b/crates/wal-protocol/src/v2.rs @@ -0,0 +1,509 @@ +// Copyright (c) 2023 - 2025 Restate Software, Inc., Restate GmbH. +// All rights reserved. +// +// Use of this software is governed by the Business Source License +// included in the LICENSE file. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0. + +use std::marker::PhantomData; +use std::sync::Arc; + +use bilrost::encoding::encoded_len_varint; +use bilrost::{Message, OwnedMessage}; +use bytes::{BufMut, BytesMut}; + +use restate_types::identifiers::{LeaderEpoch, PartitionId, PartitionKey}; +use restate_types::storage::{ + PolyBytes, StorageCodec, StorageCodecKind, StorageDecode, StorageDecodeError, StorageEncode, + StorageEncodeError, +}; + +use crate::v1; + +mod compatibility; +pub mod records; + +mod sealed { + pub trait Sealed {} +} + +/// Metadata that accompanies every WAL record and carries routing, deduplication, +/// and serialization details required to interpret the payload. +#[derive(Debug, Clone, bilrost::Message)] +pub struct Header { + #[bilrost(1)] + source: Source, + #[bilrost(3)] + dedup: Dedup, + #[bilrost(5)] + kind: RecordKind, +} + +impl Header { + pub fn source(&self) -> &Source { + &self.source + } + + pub fn dedup(&self) -> &Dedup { + &self.dedup + } + + pub fn kind(&self) -> RecordKind { + self.kind + } +} + +impl StorageDecode for Header { + fn decode( + buf: &mut B, + kind: StorageCodecKind, + ) -> Result + where + Self: Sized, + { + // we use custom encoding because it's the length delimited version + // of bilrost + debug_assert_eq!(kind, StorageCodecKind::Custom); + + Self::decode_length_delimited(buf) + .map_err(|err| StorageDecodeError::DecodeValue(err.into())) + } +} + +/// Outgoing envelope used when you are sending out records +/// over bifrost. +#[derive(Clone, derive_more::Deref)] +pub struct Envelope { + #[deref] + header: Header, + payload: PolyBytes, + _p: PhantomData, +} + +impl Envelope { + pub fn header(&self) -> &Header { + &self.header + } +} + +impl Envelope { + pub fn new(source: Source, dedup: Dedup, payload: R::Payload) -> Self { + Self { + header: Header { + source, + dedup, + kind: R::KIND, + }, + payload: PolyBytes::Typed(Arc::new(payload)), + _p: PhantomData, + } + } +} + +impl StorageEncode for Envelope { + fn default_codec(&self) -> StorageCodecKind { + StorageCodecKind::Custom + } + + fn encode(&self, buf: &mut BytesMut) -> Result<(), StorageEncodeError> { + let len = self.header.encoded_len(); + buf.reserve(encoded_len_varint(len as u64) + len); + + self.header + .encode_length_delimited(buf) + .map_err(|err| StorageEncodeError::EncodeValue(err.into()))?; + + match &self.payload { + PolyBytes::Bytes(bytes) => buf.put_slice(bytes), + PolyBytes::Typed(payload) => StorageCodec::encode(payload.as_ref(), buf)?, + } + + Ok(()) + } +} + +/// Marker type used with [`IncomingEnvelope`] to signal that the payload has not been +/// decoded into a typed record yet. +#[derive(Clone, Copy)] +pub struct Raw; + +impl StorageDecode for Envelope { + fn decode( + buf: &mut B, + kind: StorageCodecKind, + ) -> Result + where + Self: Sized, + { + match kind { + StorageCodecKind::FlexbuffersSerde => { + let envelope = v1::Envelope::decode(buf, kind)?; + Self::try_from(envelope).map_err(|err| StorageDecodeError::DecodeValue(err.into())) + } + StorageCodecKind::Custom => { + let header = StorageDecode::decode(buf, StorageCodecKind::Custom)?; + + Ok(Self { + header, + payload: PolyBytes::Bytes(buf.copy_to_bytes(buf.remaining())), + _p: PhantomData, + }) + } + _ => { + panic!("unsupported encoding"); + } + } + } +} + +impl Envelope { + /// Converts Raw Envelope into a Typed envelope. Panics + /// if the record kind does not match the M::KIND + pub fn into_typed(self) -> Envelope { + assert_eq!(self.header.kind, M::KIND); + + let Self { + header, payload, .. + } = self; + + Envelope { + header, + payload, + _p: PhantomData, + } + } +} + +impl Envelope +where + M::Payload: Clone, +{ + /// return the envelope payload + pub fn split(self) -> Result<(Header, M::Payload), StorageDecodeError> { + let payload = match self.payload { + PolyBytes::Bytes(mut bytes) => StorageCodec::decode(&mut bytes)?, + PolyBytes::Typed(typed) => { + let typed = typed.downcast_arc::().map_err(|_| { + StorageDecodeError::DecodeValue("Type mismatch. Original value in PolyBytes::Typed does not match requested type".into()) + })?; + + match Arc::try_unwrap(typed) { + Ok(value) => value, + Err(arc) => arc.as_ref().clone(), + } + } + }; + + Ok((self.header, payload)) + } + + pub fn into_inner(self) -> Result { + self.split().map(|v| v.1) + } +} + +impl Envelope { + pub fn into_raw(self) -> Envelope { + Envelope { + header: self.header, + payload: self.payload, + _p: PhantomData, + } + } +} + +/// Enumerates the logical categories of WAL records that the partition +/// processor understands. +#[derive(Debug, Clone, Copy, PartialEq, Eq, bilrost::Enumeration, strum::Display)] +pub enum RecordKind { + Unknown = 0, + + AnnounceLeader = 1, + /// A version barrier to fence off state machine changes that require a certain minimum + /// version of restate server. + /// *Since v1.4.0* + VersionBarrier = 2, + /// Updates the `PARTITION_DURABILITY` FSM variable to the given value. + /// See [`PartitionDurability`] for more details. + /// + /// *Since v1.4.2* + UpdatePartitionDurability = 3, + + // -- Partition processor commands + /// Manual patching of storage state + PatchState = 4, + /// Terminate an ongoing invocation + TerminateInvocation = 5, + /// Purge a completed invocation + PurgeInvocation = 6, + /// Purge a completed invocation journal + PurgeJournal = 7, + /// Start an invocation on this partition + Invoke = 8, + /// Truncate the message outbox up to, and including, the specified index. + TruncateOutbox = 9, + /// Proxy a service invocation through this partition processor, to reuse the deduplication id map. + ProxyThrough = 10, + /// Attach to an existing invocation + AttachInvocation = 11, + /// Resume an invocation + ResumeInvocation = 12, + /// Restart as new invocation from prefix + RestartAsNewInvocation = 13, + // -- Partition processor events for PP + /// Invoker is reporting effect(s) from an ongoing invocation. + InvokerEffect = 14, + /// Timer has fired + Timer = 15, + /// Schedule timer + ScheduleTimer = 16, + /// Another partition processor is reporting a response of an invocation we requested. + /// + /// KINDA DEPRECATED: When Journal Table V1 is removed, this command should be used only to reply to invocations. + /// Now it's abused for a bunch of other scenarios, like replying to get promise and get invocation output. + /// + /// For more details see `OnNotifyInvocationResponse`. + InvocationResponse = 17, + + // -- New PP <-> PP commands using Journal V2 + /// Notify Get invocation output + NotifyGetInvocationOutputResponse = 18, + /// Notify a signal. + NotifySignal = 19, + + /// UpsertSchema + UpsertSchema = 20, +} + +/// Identifies which subsystem produced a given WAL record. +#[derive(Debug, Clone, PartialEq, Eq, bilrost::Oneof, bilrost::Message)] +pub enum Source { + /// Message is sent from an ingress node + #[bilrost(empty)] + Ingress, + + /// Message is sent from some control plane component (controller, cli, etc.) + #[bilrost(tag = 1, message)] + ControlPlane, + + /// Message is sent from another partition processor + #[bilrost(tag = 2, message)] + Processor { + /// if possible, this is used to reroute responses in case of splits/merges + /// Marked as `Option` in v1.5. Note that v1.4 requires this to be set but as of v1.6 + /// this can be safely set to `None`. + #[bilrost(1)] + partition_id: Option, + #[bilrost(2)] + partition_key: Option, + /// The current epoch of the partition leader. Readers should observe this to decide which + /// messages to accept. Readers should ignore messages coming from + /// epochs lower than the max observed for a given partition id. + #[bilrost(3)] + leader_epoch: LeaderEpoch, + }, +} + +/// Specifies the deduplication strategy that allows receivers to discard +/// duplicate WAL records safely. +#[derive(Debug, Clone, PartialEq, Eq, Default, bilrost::Oneof, bilrost::Message)] +pub enum Dedup { + #[default] + None, + /// Sequence number to deduplicate messages sent by the same partition or a successor + /// of a previous partition (a successor partition will inherit the leader epoch of its + /// predecessor). + #[bilrost(tag(1), message)] + SelfProposal { + #[bilrost(0)] + leader_epoch: LeaderEpoch, + #[bilrost(1)] + seq: u64, + }, + /// Sequence number to deduplicate messages from a foreign partition. + #[bilrost(tag(2), message)] + ForeignPartition { + #[bilrost(0)] + partition: PartitionId, + #[bilrost(1)] + seq: u64, + }, + /// Sequence number to deduplicate messages from an arbitrary string prefix. + #[bilrost(tag(3), message)] + Arbitrary { + #[bilrost(0)] + prefix: String, + #[bilrost(1)] + seq: u64, + }, +} + +/// Marker trait implemented by strongly-typed representations of WAL record +/// payloads. +pub trait Record: sealed::Sealed + Send + Sync + Clone + Copy + Sized { + const KIND: RecordKind; + type Payload: StorageEncode + StorageDecode + 'static; + + /// Create an envelope with `this` record kind + /// given the header, keys and payload + fn new(source: Source, dedup: Dedup, payload: impl Into) -> Envelope + where + Self::Payload: StorageEncode, + { + Envelope::new(source, dedup, payload.into()) + } + + /// Creates a new test envelope. Shortcut for new(Source::Ingress, Dedup::None, payload) + // #[cfg(test)] + fn new_test(payload: impl Into) -> Envelope + where + Self::Payload: StorageEncode, + { + let record = Self::new(Source::Ingress, Dedup::None, payload); + record.into_raw() + } +} + +#[cfg(test)] +mod test { + + use bytes::BytesMut; + + use restate_types::{GenerationalNodeId, storage::StorageCodec}; + + use super::{Dedup, Header, Source, records}; + use crate::{ + control::AnnounceLeader, + v1, + v2::{Envelope, Raw, Record, RecordKind}, + }; + + #[test] + fn envelope_encode_decode() { + let payload = AnnounceLeader { + leader_epoch: 11.into(), + node_id: GenerationalNodeId::new(1, 3), + partition_key_range: 0..=u64::MAX, + }; + + let envelope = records::AnnounceLeader::new( + Source::Ingress, + Dedup::SelfProposal { + leader_epoch: 10.into(), + seq: 120, + }, + payload.clone(), + ); + + let mut buf = BytesMut::new(); + StorageCodec::encode(&envelope, &mut buf).expect("to encode"); + + let envelope: Envelope = StorageCodec::decode(&mut buf).expect("to decode"); + + assert_eq!(envelope.kind(), RecordKind::AnnounceLeader); + let typed = envelope.into_typed::(); + + let (_, loaded_payload) = typed.split().expect("to decode"); + + assert_eq!(payload, loaded_payload); + } + + #[test] + fn header_decode_discard_payload() { + let payload = AnnounceLeader { + leader_epoch: 11.into(), + node_id: GenerationalNodeId::new(1, 3), + partition_key_range: 0..=u64::MAX, + }; + + let envelope = records::AnnounceLeader::new( + Source::Ingress, + Dedup::SelfProposal { + leader_epoch: 10.into(), + seq: 120, + }, + payload.clone(), + ); + + let mut buf = BytesMut::new(); + StorageCodec::encode(&envelope, &mut buf).expect("to encode"); + + let mut slice = &buf[..]; + // decode header only and discard the rest of the envelope + let header: Header = StorageCodec::decode(&mut slice).expect("to decode"); + + assert_eq!(header.source, Source::Ingress); + + assert_eq!( + header.dedup, + Dedup::SelfProposal { + leader_epoch: 10.into(), + seq: 120 + } + ); + } + + #[test] + fn envelope_skip_encode() { + let payload = AnnounceLeader { + leader_epoch: 11.into(), + node_id: GenerationalNodeId::new(1, 3), + partition_key_range: 0..=u64::MAX, + }; + + let envelope = records::AnnounceLeader::new( + Source::Ingress, + Dedup::SelfProposal { + leader_epoch: 10.into(), + seq: 120, + }, + payload.clone(), + ); + + // assert_eq!(envelope.record_keys(), Keys::RangeInclusive(0..=u64::MAX)); + + let envelope = envelope.into_raw(); + + assert_eq!(envelope.kind(), RecordKind::AnnounceLeader); + let typed = envelope.into_typed::(); + + let (_, loaded_payload) = typed.split().expect("to decode"); + + assert_eq!(payload, loaded_payload); + } + + #[test] + fn envelope_from_to_v1() { + let payload = AnnounceLeader { + leader_epoch: 11.into(), + node_id: GenerationalNodeId::new(1, 3), + partition_key_range: 0..=u64::MAX, + }; + + let envelope = records::AnnounceLeader::new( + Source::Ingress, + Dedup::SelfProposal { + leader_epoch: 10.into(), + seq: 120, + }, + payload.clone(), + ); + + let env_v1: v1::Envelope = envelope.try_into().expect("to work"); + + let mut buf = BytesMut::new(); + StorageCodec::encode(&env_v1, &mut buf).expect("to encode"); + + let envelope: Envelope = StorageCodec::decode(&mut buf).expect("to decode"); + + assert_eq!(envelope.kind(), RecordKind::AnnounceLeader); + let typed = envelope.into_typed::(); + + let (_, loaded_payload) = typed.split().expect("to decode"); + + assert_eq!(payload, loaded_payload); + } +} diff --git a/crates/wal-protocol/src/v2/compatibility.rs b/crates/wal-protocol/src/v2/compatibility.rs new file mode 100644 index 0000000000..e1dd656e3c --- /dev/null +++ b/crates/wal-protocol/src/v2/compatibility.rs @@ -0,0 +1,507 @@ +// Copyright (c) 2023 - 2025 Restate Software, Inc., Restate GmbH. +// All rights reserved. +// +// Use of this software is governed by the Business Source License +// included in the LICENSE file. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0. + +use restate_storage_api::deduplication_table::{ + DedupInformation, DedupSequenceNumber, EpochSequenceNumber, ProducerId, +}; +use restate_types::{ + logs::{HasRecordKeys, Keys}, + storage::StorageDecodeError, +}; + +use super::{Raw, records}; +use crate::{ + v1, + v2::{ + self, Record, + records::{ProxyThroughRequest, TruncateOutboxRequest}, + }, +}; + +impl TryFrom for v2::Envelope { + type Error = anyhow::Error; + + fn try_from(value: v1::Envelope) -> Result { + let source = match value.header.source { + v1::Source::Ingress {} => v2::Source::Ingress, + v1::Source::Processor { + partition_id, + partition_key, + leader_epoch, + } => v2::Source::Processor { + partition_id, + partition_key, + leader_epoch, + }, + v1::Source::ControlPlane {} => v2::Source::ControlPlane, + }; + + let v1::Destination::Processor { + dedup, + partition_key, + } = value.header.dest; + + let dedup = match dedup { + None => v2::Dedup::None, + Some(info) => match (info.producer_id, info.sequence_number) { + (ProducerId::Partition(id), DedupSequenceNumber::Sn(seq)) => { + v2::Dedup::ForeignPartition { partition: id, seq } + } + (ProducerId::Partition(_), _) => anyhow::bail!("invalid deduplication information"), + (ProducerId::Other(_), DedupSequenceNumber::Esn(sn)) => v2::Dedup::SelfProposal { + leader_epoch: sn.leader_epoch, + seq: sn.sequence_number, + }, + (ProducerId::Other(prefix), DedupSequenceNumber::Sn(seq)) => v2::Dedup::Arbitrary { + prefix: prefix.into(), + seq, + }, + }, + }; + + let envelope = match value.command { + v1::Command::AnnounceLeader(payload) => { + records::AnnounceLeader::new(source, dedup, *payload).into_raw() + } + v1::Command::AttachInvocation(payload) => { + records::AttachInvocation::new(source, dedup, payload).into_raw() + } + v1::Command::InvocationResponse(payload) => { + records::InvocationResponse::new(source, dedup, payload).into_raw() + } + v1::Command::Invoke(payload) => records::Invoke::new(source, dedup, payload).into_raw(), + v1::Command::InvokerEffect(payload) => { + records::InvokerEffect::new(source, dedup, payload).into_raw() + } + v1::Command::NotifyGetInvocationOutputResponse(payload) => { + records::NotifyGetInvocationOutputResponse::new(source, dedup, payload).into_raw() + } + v1::Command::NotifySignal(payload) => { + records::NotifySignal::new(source, dedup, payload).into_raw() + } + v1::Command::PatchState(payload) => { + records::PatchState::new(source, dedup, payload).into_raw() + } + v1::Command::ProxyThrough(payload) => records::ProxyThrough::new( + source, + dedup, + ProxyThroughRequest { + invocation: payload.into(), + proxy_partition: Keys::Single(partition_key), + }, + ) + .into_raw(), + v1::Command::PurgeInvocation(payload) => { + records::PurgeInvocation::new(source, dedup, payload).into_raw() + } + v1::Command::PurgeJournal(payload) => { + records::PurgeJournal::new(source, dedup, payload).into_raw() + } + v1::Command::RestartAsNewInvocation(payload) => { + records::RestartAsNewInvocation::new(source, dedup, payload).into_raw() + } + v1::Command::ResumeInvocation(payload) => { + records::ResumeInvocation::new(source, dedup, payload).into_raw() + } + v1::Command::ScheduleTimer(payload) => { + records::ScheduleTimer::new(source, dedup, payload).into_raw() + } + v1::Command::TerminateInvocation(payload) => { + records::TerminateInvocation::new(source, dedup, payload).into_raw() + } + v1::Command::Timer(payload) => records::Timer::new(source, dedup, payload).into_raw(), + v1::Command::TruncateOutbox(payload) => records::TruncateOutbox::new( + source, + dedup, + TruncateOutboxRequest { + index: payload, + // this actually should be a key-range but v1 unfortunately + // only hold the "start" of the range. + // will be fixed in v2 + partition_key_range: Keys::Single(partition_key), + }, + ) + .into_raw(), + v1::Command::UpdatePartitionDurability(payload) => { + records::UpdatePartitionDurability::new(source, dedup, payload).into_raw() + } + v1::Command::UpsertSchema(payload) => { + records::UpsertSchema::new(source, dedup, payload).into_raw() + } + v1::Command::VersionBarrier(payload) => { + records::VersionBarrier::new(source, dedup, payload).into_raw() + } + }; + + Ok(envelope) + } +} + +impl From<(Keys, v2::Header)> for v1::Header { + fn from((keys, value): (Keys, v2::Header)) -> Self { + let source = match value.source { + v2::Source::Ingress => v1::Source::Ingress {}, + v2::Source::ControlPlane => v1::Source::ControlPlane {}, + v2::Source::Processor { + partition_id, + partition_key, + leader_epoch, + } => v1::Source::Processor { + partition_id, + partition_key, + leader_epoch, + }, + }; + + // this is only for backward compatibility + // but in reality the partition_id in the dest + // should never be used. Instead the associated + // record keys should. + let partition_key = match keys { + Keys::None => 0, + Keys::Single(pk) => pk, + Keys::Pair(pk, _) => pk, + Keys::RangeInclusive(range) => *range.start(), + }; + + v1::Header { + source, + dest: v1::Destination::Processor { + partition_key, + dedup: value.dedup.into(), + }, + } + } +} + +impl From for Option { + fn from(value: v2::Dedup) -> Self { + match value { + v2::Dedup::None => None, + v2::Dedup::SelfProposal { leader_epoch, seq } => { + Some(DedupInformation::self_proposal(EpochSequenceNumber { + leader_epoch, + sequence_number: seq, + })) + } + v2::Dedup::ForeignPartition { partition, seq } => { + Some(DedupInformation::cross_partition(partition, seq)) + } + v2::Dedup::Arbitrary { prefix, seq } => Some(DedupInformation::ingress(prefix, seq)), + } + } +} + +// compatibility from v2 to v1. We will keep writing v1 envelops +// until the following release. + +impl TryFrom> for v1::Envelope { + type Error = StorageDecodeError; + + fn try_from(value: v2::Envelope) -> Result { + let (header, inner) = value.split()?; + + let header = v1::Header::from((inner.record_keys(), header)); + + Ok(Self { + header, + command: v1::Command::AnnounceLeader(inner.into()), + }) + } +} + +impl TryFrom> for v1::Envelope { + type Error = StorageDecodeError; + + fn try_from(value: v2::Envelope) -> Result { + let (header, inner) = value.split()?; + + let header = v1::Header::from((inner.record_keys(), header)); + + Ok(Self { + header, + command: v1::Command::VersionBarrier(inner), + }) + } +} + +impl TryFrom> for v1::Envelope { + type Error = StorageDecodeError; + + fn try_from( + value: v2::Envelope, + ) -> Result { + let (header, inner) = value.split()?; + + let header = v1::Header::from((inner.record_keys(), header)); + + Ok(Self { + header, + command: v1::Command::UpdatePartitionDurability(inner), + }) + } +} + +impl TryFrom> for v1::Envelope { + type Error = StorageDecodeError; + + fn try_from(value: v2::Envelope) -> Result { + let (header, inner) = value.split()?; + + let header = v1::Header::from((inner.record_keys(), header)); + + Ok(Self { + header, + command: v1::Command::PatchState(inner.into()), + }) + } +} + +impl TryFrom> for v1::Envelope { + type Error = StorageDecodeError; + + fn try_from(value: v2::Envelope) -> Result { + let (header, inner) = value.split()?; + + let header = v1::Header::from((inner.record_keys(), header)); + + Ok(Self { + header, + command: v1::Command::TerminateInvocation(inner.into()), + }) + } +} + +impl TryFrom> for v1::Envelope { + type Error = StorageDecodeError; + + fn try_from(value: v2::Envelope) -> Result { + let (header, inner) = value.split()?; + + let header = v1::Header::from((inner.record_keys(), header)); + + Ok(Self { + header, + command: v1::Command::PurgeInvocation(inner.into()), + }) + } +} + +impl TryFrom> for v1::Envelope { + type Error = StorageDecodeError; + + fn try_from(value: v2::Envelope) -> Result { + let (header, inner) = value.split()?; + + let header = v1::Header::from((inner.record_keys(), header)); + + Ok(Self { + header, + command: v1::Command::PurgeJournal(inner.into()), + }) + } +} + +impl TryFrom> for v1::Envelope { + type Error = StorageDecodeError; + + fn try_from(value: v2::Envelope) -> Result { + let (header, inner) = value.split()?; + + let header = v1::Header::from((inner.record_keys(), header)); + + Ok(Self { + header, + command: v1::Command::Invoke(inner.into()), + }) + } +} + +impl TryFrom> for v1::Envelope { + type Error = StorageDecodeError; + + fn try_from(value: v2::Envelope) -> Result { + let (header, inner) = value.split()?; + + let header = v1::Header::from((inner.record_keys(), header)); + + Ok(Self { + header, + command: v1::Command::TruncateOutbox(inner.index), + }) + } +} + +impl TryFrom> for v1::Envelope { + type Error = StorageDecodeError; + + fn try_from(value: v2::Envelope) -> Result { + let (header, inner) = value.split()?; + + let header = v1::Header::from((inner.record_keys(), header)); + + Ok(Self { + header, + command: v1::Command::ProxyThrough(inner.invocation.into()), + }) + } +} + +impl TryFrom> for v1::Envelope { + type Error = StorageDecodeError; + + fn try_from(value: v2::Envelope) -> Result { + let (header, inner) = value.split()?; + + let header = v1::Header::from((inner.record_keys(), header)); + + Ok(Self { + header, + command: v1::Command::AttachInvocation(inner.into()), + }) + } +} + +impl TryFrom> for v1::Envelope { + type Error = StorageDecodeError; + + fn try_from(value: v2::Envelope) -> Result { + let (header, inner) = value.split()?; + + let header = v1::Header::from((inner.record_keys(), header)); + + Ok(Self { + header, + command: v1::Command::ResumeInvocation(inner.into()), + }) + } +} + +impl TryFrom> for v1::Envelope { + type Error = StorageDecodeError; + + fn try_from(value: v2::Envelope) -> Result { + let (header, inner) = value.split()?; + + let header = v1::Header::from((inner.record_keys(), header)); + + Ok(Self { + header, + command: v1::Command::RestartAsNewInvocation(inner.into()), + }) + } +} + +impl TryFrom> for v1::Envelope { + type Error = StorageDecodeError; + + fn try_from(value: v2::Envelope) -> Result { + let (header, inner) = value.split()?; + + let header = v1::Header::from((inner.record_keys(), header)); + + Ok(Self { + header, + command: v1::Command::InvokerEffect(inner.into()), + }) + } +} + +impl TryFrom> for v1::Envelope { + type Error = StorageDecodeError; + + fn try_from(value: v2::Envelope) -> Result { + let (header, inner) = value.split()?; + + let header = v1::Header::from((inner.record_keys(), header)); + + Ok(Self { + header, + command: v1::Command::Timer(inner.into()), + }) + } +} + +impl TryFrom> for v1::Envelope { + type Error = StorageDecodeError; + + fn try_from(value: v2::Envelope) -> Result { + let (header, inner) = value.split()?; + + let header = v1::Header::from((inner.record_keys(), header)); + + Ok(Self { + header, + command: v1::Command::ScheduleTimer(inner.into()), + }) + } +} + +impl TryFrom> for v1::Envelope { + type Error = StorageDecodeError; + + fn try_from(value: v2::Envelope) -> Result { + let (header, inner) = value.split()?; + + let header = v1::Header::from((inner.record_keys(), header)); + + Ok(Self { + header, + command: v1::Command::InvocationResponse(inner.into()), + }) + } +} + +impl TryFrom> for v1::Envelope { + type Error = StorageDecodeError; + + fn try_from( + value: v2::Envelope, + ) -> Result { + let (header, inner) = value.split()?; + + let header = v1::Header::from((inner.record_keys(), header)); + + Ok(Self { + header, + command: v1::Command::NotifyGetInvocationOutputResponse(inner.into()), + }) + } +} + +impl TryFrom> for v1::Envelope { + type Error = StorageDecodeError; + + fn try_from(value: v2::Envelope) -> Result { + let (header, inner) = value.split()?; + + let header = v1::Header::from((inner.record_keys(), header)); + + Ok(Self { + header, + command: v1::Command::NotifySignal(inner.into()), + }) + } +} + +impl TryFrom> for v1::Envelope { + type Error = StorageDecodeError; + + fn try_from(value: v2::Envelope) -> Result { + let (header, inner) = value.split()?; + + let header = v1::Header::from((inner.record_keys(), header)); + + Ok(Self { + header, + command: v1::Command::UpsertSchema(inner), + }) + } +} diff --git a/crates/wal-protocol/src/v2/records.rs b/crates/wal-protocol/src/v2/records.rs new file mode 100644 index 0000000000..7ce3bf7b6f --- /dev/null +++ b/crates/wal-protocol/src/v2/records.rs @@ -0,0 +1,431 @@ +// Copyright (c) 2023 - 2025 Restate Software, Inc., Restate GmbH. +// All rights reserved. +// +// Use of this software is governed by the Business Source License +// included in the LICENSE file. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0. + +use restate_types::{ + bilrost_storage_encode_decode, flexbuffers_storage_encode_decode, + identifiers::WithPartitionKey, + invocation, + logs::{HasRecordKeys, Keys}, + message::MessageIndex, + state_mut, +}; +use serde::{Deserialize, Serialize}; + +use super::sealed::Sealed; +use super::{Record, RecordKind}; +use crate::timer::{self}; + +// Create type wrappers to implement storage encode/decode +// and HasRecordKeys +#[derive( + Debug, + Clone, + Eq, + PartialEq, + bilrost::Message, + derive_more::Deref, + derive_more::Into, + derive_more::From, +)] +pub struct ExternalStateMutation(state_mut::ExternalStateMutation); +bilrost_storage_encode_decode!(ExternalStateMutation); + +impl HasRecordKeys for ExternalStateMutation { + fn record_keys(&self) -> Keys { + Keys::Single(self.0.service_id.partition_key()) + } +} + +#[derive( + Clone, + Eq, + PartialEq, + Serialize, + Deserialize, + derive_more::Deref, + derive_more::Into, + derive_more::From, +)] +pub struct InvocationTermination(invocation::InvocationTermination); +flexbuffers_storage_encode_decode!(InvocationTermination); + +impl HasRecordKeys for InvocationTermination { + fn record_keys(&self) -> Keys { + Keys::Single(self.invocation_id.partition_key()) + } +} + +#[derive( + Clone, + Eq, + PartialEq, + Serialize, + Deserialize, + derive_more::Deref, + derive_more::Into, + derive_more::From, +)] +pub struct PurgeInvocationRequest(invocation::PurgeInvocationRequest); +flexbuffers_storage_encode_decode!(PurgeInvocationRequest); + +impl HasRecordKeys for PurgeInvocationRequest { + fn record_keys(&self) -> Keys { + Keys::Single(self.invocation_id.partition_key()) + } +} + +#[derive( + Clone, + Eq, + PartialEq, + Serialize, + Deserialize, + derive_more::Deref, + derive_more::Into, + derive_more::From, +)] +pub struct ServiceInvocation(Box); + +flexbuffers_storage_encode_decode!(ServiceInvocation); + +impl HasRecordKeys for ServiceInvocation { + fn record_keys(&self) -> Keys { + Keys::Single(self.invocation_id.partition_key()) + } +} + +#[derive(Debug, Clone, bilrost::Message)] +pub struct TruncateOutboxRequest { + #[bilrost(1)] + pub index: MessageIndex, + + #[bilrost(2)] + pub partition_key_range: Keys, +} + +impl HasRecordKeys for TruncateOutboxRequest { + fn record_keys(&self) -> Keys { + self.partition_key_range.clone() + } +} + +bilrost_storage_encode_decode!(TruncateOutboxRequest); + +#[derive( + Clone, + Eq, + PartialEq, + Serialize, + Deserialize, + derive_more::Deref, + derive_more::Into, + derive_more::From, +)] +pub struct AttachInvocationRequest(invocation::AttachInvocationRequest); + +flexbuffers_storage_encode_decode!(AttachInvocationRequest); + +impl HasRecordKeys for AttachInvocationRequest { + fn record_keys(&self) -> Keys { + Keys::Single(self.partition_key()) + } +} + +#[derive( + Clone, + Eq, + PartialEq, + Serialize, + Deserialize, + derive_more::Deref, + derive_more::Into, + derive_more::From, +)] +pub struct ResumeInvocationRequest(invocation::ResumeInvocationRequest); + +flexbuffers_storage_encode_decode!(ResumeInvocationRequest); + +impl HasRecordKeys for ResumeInvocationRequest { + fn record_keys(&self) -> Keys { + Keys::Single(self.partition_key()) + } +} + +#[derive( + Clone, + Eq, + PartialEq, + Serialize, + Deserialize, + derive_more::Deref, + derive_more::Into, + derive_more::From, +)] +pub struct RestartAsNewInvocationRequest(invocation::RestartAsNewInvocationRequest); + +flexbuffers_storage_encode_decode!(RestartAsNewInvocationRequest); + +impl HasRecordKeys for RestartAsNewInvocationRequest { + fn record_keys(&self) -> Keys { + Keys::Single(self.partition_key()) + } +} + +#[derive( + Clone, + Eq, + PartialEq, + Serialize, + Deserialize, + derive_more::Deref, + derive_more::Into, + derive_more::From, +)] +pub struct Effect(Box); + +flexbuffers_storage_encode_decode!(Effect); + +impl HasRecordKeys for Effect { + fn record_keys(&self) -> restate_types::logs::Keys { + Keys::Single(self.invocation_id.partition_key()) + } +} + +#[derive( + Clone, + Eq, + PartialEq, + Serialize, + Deserialize, + derive_more::Deref, + derive_more::Into, + derive_more::From, +)] +pub struct TimerKeyValue(timer::TimerKeyValue); + +flexbuffers_storage_encode_decode!(TimerKeyValue); + +impl HasRecordKeys for TimerKeyValue { + fn record_keys(&self) -> Keys { + Keys::Single(self.invocation_id().partition_key()) + } +} + +#[derive( + Clone, + Eq, + PartialEq, + Serialize, + Deserialize, + derive_more::Deref, + derive_more::Into, + derive_more::From, +)] +pub struct InvocationResponsePayload(invocation::InvocationResponse); + +flexbuffers_storage_encode_decode!(InvocationResponsePayload); + +impl HasRecordKeys for InvocationResponsePayload { + fn record_keys(&self) -> Keys { + Keys::Single(self.partition_key()) + } +} + +#[derive( + Clone, + Eq, + PartialEq, + Serialize, + Deserialize, + derive_more::Deref, + derive_more::Into, + derive_more::From, +)] +pub struct GetInvocationOutputResponse(invocation::GetInvocationOutputResponse); + +flexbuffers_storage_encode_decode!(GetInvocationOutputResponse); + +impl HasRecordKeys for GetInvocationOutputResponse { + fn record_keys(&self) -> Keys { + Keys::Single(self.partition_key()) + } +} + +#[derive( + Clone, + Eq, + PartialEq, + Serialize, + Deserialize, + derive_more::Deref, + derive_more::Into, + derive_more::From, +)] +pub struct NotifySignalRequest(invocation::NotifySignalRequest); + +flexbuffers_storage_encode_decode!(NotifySignalRequest); + +impl HasRecordKeys for NotifySignalRequest { + fn record_keys(&self) -> Keys { + Keys::Single(self.partition_key()) + } +} + +#[derive(Clone, Serialize, Deserialize)] +pub struct ProxyThroughRequest { + pub invocation: ServiceInvocation, + + pub proxy_partition: Keys, +} + +flexbuffers_storage_encode_decode!(ProxyThroughRequest); + +impl HasRecordKeys for ProxyThroughRequest { + fn record_keys(&self) -> Keys { + self.proxy_partition.clone() + } +} + +// end types + +// define record types + +macro_rules! record { + {@name=$name:ident, @kind=$type:expr, @payload=$payload:path} => { + #[allow(dead_code)] + #[derive(Clone, Copy)] + pub struct $name; + impl Sealed for $name{} + impl Record for $name { + const KIND: RecordKind = $type; + type Payload = $payload; + } + }; + } + +record! { + @name=AnnounceLeader, + @kind=RecordKind::AnnounceLeader, + @payload=crate::control::AnnounceLeader +} + +record! { + @name=VersionBarrier, + @kind=RecordKind::VersionBarrier, + @payload=crate::control::VersionBarrier +} + +record! { + @name=UpdatePartitionDurability, + @kind=RecordKind::UpdatePartitionDurability, + @payload=crate::control::PartitionDurability +} + +record! { + @name=PatchState, + @kind=RecordKind::PatchState, + @payload=ExternalStateMutation +} + +record! { + @name=TerminateInvocation, + @kind=RecordKind::TerminateInvocation, + @payload=InvocationTermination +} + +record! { + @name=PurgeInvocation, + @kind=RecordKind::PurgeInvocation, + @payload=PurgeInvocationRequest +} + +record! { + @name=PurgeJournal, + @kind=RecordKind::PurgeJournal, + @payload=PurgeInvocationRequest +} + +record! { + @name=Invoke, + @kind=RecordKind::Invoke, + @payload=ServiceInvocation +} + +record! { + @name=TruncateOutbox, + @kind=RecordKind::TruncateOutbox, + @payload=TruncateOutboxRequest +} + +record! { + @name=ProxyThrough, + @kind=RecordKind::ProxyThrough, + @payload=ProxyThroughRequest +} + +record! { + @name=AttachInvocation, + @kind=RecordKind::AttachInvocation, + @payload=AttachInvocationRequest +} + +record! { + @name=ResumeInvocation, + @kind=RecordKind::ResumeInvocation, + @payload=ResumeInvocationRequest +} + +record! { + @name=RestartAsNewInvocation, + @kind=RecordKind::RestartAsNewInvocation, + @payload=RestartAsNewInvocationRequest +} + +record! { + @name=InvokerEffect, + @kind=RecordKind::InvokerEffect, + @payload=Effect +} + +record! { + @name=Timer, + @kind=RecordKind::Timer, + @payload=TimerKeyValue +} + +record! { + @name=ScheduleTimer, + @kind=RecordKind::ScheduleTimer, + @payload=TimerKeyValue +} + +record! { + @name=InvocationResponse, + @kind=RecordKind::InvocationResponse, + @payload=InvocationResponsePayload +} + +record! { + @name=NotifyGetInvocationOutputResponse, + @kind=RecordKind::NotifyGetInvocationOutputResponse, + @payload=GetInvocationOutputResponse +} + +record! { + @name=NotifySignal, + @kind=RecordKind::NotifySignal, + @payload=NotifySignalRequest +} + +record! { + @name=UpsertSchema, + @kind=RecordKind::UpsertSchema, + @payload=crate::control::UpsertSchema +} diff --git a/crates/worker/src/partition/leadership/durability_tracker.rs b/crates/worker/src/partition/leadership/durability_tracker.rs index a59200c4a8..886fc84d37 100644 --- a/crates/worker/src/partition/leadership/durability_tracker.rs +++ b/crates/worker/src/partition/leadership/durability_tracker.rs @@ -9,21 +9,22 @@ // by the Apache License, Version 2.0. use std::pin::Pin; +use std::sync::Arc; use std::task::Poll; use std::time::Duration; use futures::{Stream, StreamExt}; -use restate_partition_store::snapshots::ArchivedLsn; use tokio::sync::watch; use tokio::time::{Instant, MissedTickBehavior}; use tokio_stream::wrappers::{IntervalStream, WatchStream}; use tracing::{debug, warn}; use restate_core::Metadata; +use restate_partition_store::snapshots::ArchivedLsn; use restate_types::config::{Configuration, DurabilityMode}; -use restate_types::identifiers::PartitionId; -use restate_types::logs::{Lsn, SequenceNumber}; +use restate_types::logs::{Keys, Lsn, SequenceNumber}; use restate_types::nodes_config::Role; +use restate_types::partitions::Partition; use restate_types::partitions::state::PartitionReplicaSetStates; use restate_types::time::MillisSinceEpoch; use restate_wal_protocol::control::PartitionDurability; @@ -37,7 +38,7 @@ const WARN_PERIOD: Duration = Duration::from_secs(60); /// changed in the replica-set, but it'll react immediately to changes in the archived Lsn /// watch. pub struct DurabilityTracker { - partition_id: PartitionId, + partition: Arc, last_reported_durable_lsn: Lsn, replica_set_states: PartitionReplicaSetStates, archived_lsn_watch: WatchStream>, @@ -50,7 +51,7 @@ pub struct DurabilityTracker { impl DurabilityTracker { pub fn new( - partition_id: PartitionId, + partition: Arc, last_reported_durable_lsn: Option, replica_set_states: PartitionReplicaSetStates, archived_lsn_watch: watch::Receiver>, @@ -62,7 +63,7 @@ impl DurabilityTracker { let check_timer = IntervalStream::new(check_timer); Self { - partition_id, + partition, last_reported_durable_lsn: last_reported_durable_lsn.unwrap_or(Lsn::INVALID), replica_set_states, archived_lsn_watch: WatchStream::new(archived_lsn_watch), @@ -184,11 +185,11 @@ impl Stream for DurabilityTracker { } DurabilityMode::ReplicaSetOnly => self .replica_set_states - .get_min_durable_lsn(self.partition_id), + .get_min_durable_lsn(self.partition.partition_id), DurabilityMode::SnapshotAndReplicaSet => { let min_durable_lsn = self .replica_set_states - .get_min_durable_lsn(self.partition_id); + .get_min_durable_lsn(self.partition.partition_id); self.last_archived.min(min_durable_lsn) } // disabled until ad-hoc snapshot sharing is supported @@ -202,7 +203,7 @@ impl Stream for DurabilityTracker { DurabilityMode::Balanced => { let max_durable_lsn = self .replica_set_states - .get_max_durable_lsn(self.partition_id); + .get_max_durable_lsn(self.partition.partition_id); self.last_archived.min(max_durable_lsn) } }; @@ -213,15 +214,16 @@ impl Stream for DurabilityTracker { } let partition_durability = PartitionDurability { - partition_id: self.partition_id, + partition_id: self.partition.partition_id, durable_point: suggested, modification_time: MillisSinceEpoch::now(), + partition_key_range: Keys::RangeInclusive(self.partition.key_range.clone()), }; // We don't want to keep reporting the same durable Lsn over and over. self.last_reported_durable_lsn = suggested; debug!( - partition_id = %self.partition_id, + partition_id = %self.partition.partition_id, durability_mode = %durability_mode, "Reporting {suggested:?} as a durable point for partition" ); diff --git a/crates/worker/src/partition/leadership/mod.rs b/crates/worker/src/partition/leadership/mod.rs index b356807625..1269897e12 100644 --- a/crates/worker/src/partition/leadership/mod.rs +++ b/crates/worker/src/partition/leadership/mod.rs @@ -405,7 +405,7 @@ where .map(|d| d.durable_point); let durability_tracker = DurabilityTracker::new( - self.partition.partition_id, + Arc::clone(&self.partition), last_reported_durable_lsn, replica_set_states, partition_store.partition_db().watch_archived_lsn(), diff --git a/crates/worker/src/partition/mod.rs b/crates/worker/src/partition/mod.rs index 04d161a1d2..9c2c3c27f1 100644 --- a/crates/worker/src/partition/mod.rs +++ b/crates/worker/src/partition/mod.rs @@ -7,15 +7,6 @@ // As of the Change Date specified in that file, in accordance with // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0. - -mod cleaner; -pub mod invoker_storage_reader; -mod leadership; -mod rpc; -pub mod shuffle; -mod state_machine; -pub mod types; - use std::fmt::Debug; use std::sync::Arc; use std::time::Duration; @@ -45,7 +36,7 @@ use restate_time_util::DurationExt; use restate_types::cluster::cluster_state::{PartitionProcessorStatus, ReplayStatus, RunMode}; use restate_types::config::Configuration; use restate_types::identifiers::LeaderEpoch; -use restate_types::logs::{KeyFilter, Lsn, Record, SequenceNumber}; +use restate_types::logs::{KeyFilter, Keys, Lsn, MatchKeyQuery, Record, SequenceNumber}; use restate_types::net::RpcRequest; use restate_types::net::partition_processor::{ PartitionLeaderService, PartitionProcessorRpcError, PartitionProcessorRpcRequest, @@ -58,7 +49,8 @@ use restate_types::storage::StorageDecodeError; use restate_types::time::{MillisSinceEpoch, NanosSinceEpoch}; use restate_types::{GenerationalNodeId, SemanticRestateVersion}; use restate_wal_protocol::control::AnnounceLeader; -use restate_wal_protocol::{Command, Destination, Envelope, Header}; +use restate_wal_protocol::v2; +use restate_wal_protocol::v2::{RecordKind, records}; use self::leadership::trim_queue::TrimQueue; use crate::metric_definitions::{ @@ -68,6 +60,14 @@ use crate::partition::invoker_storage_reader::InvokerStorageReader; use crate::partition::leadership::LeadershipState; use crate::partition::state_machine::{ActionCollector, StateMachine}; +mod cleaner; +pub mod invoker_storage_reader; +mod leadership; +mod rpc; +pub mod shuffle; +mod state_machine; +pub mod types; + /// Target leader state of the partition processor. #[derive(Clone, Copy, Debug, Default, PartialEq)] pub enum TargetLeaderState { @@ -163,6 +163,7 @@ where partition_id_str, leadership_state, state_machine, + key_filter: KeyFilter::Within(partition_store.partition_key_range().clone()), partition_store, bifrost, target_leader_state_rx, @@ -219,6 +220,7 @@ pub struct PartitionProcessor { replica_set_states: PartitionReplicaSetStates, partition_store: PartitionStore, + key_filter: KeyFilter, trim_queue: TrimQueue, } @@ -269,8 +271,9 @@ pub enum ProcessorError { struct LsnEnvelope { pub lsn: Lsn, + pub keys: Keys, pub created_at: NanosSinceEpoch, - pub envelope: Arc, + pub envelope: Arc>, } impl PartitionProcessor @@ -514,6 +517,7 @@ where let record = LsnEnvelope { lsn, + keys: record.keys().clone(), created_at: record.created_at(), envelope: record.decode_arc()?, }; @@ -686,37 +690,53 @@ where transaction: &mut PartitionStoreTransaction<'_>, action_collector: &mut ActionCollector, ) -> Result>, state_machine::Error> { - trace!(lsn = %record.lsn, "Processing bifrost record for '{}': {:?}", record.envelope.command.name(), record.envelope.header); - - if let Some(dedup_information) = self.is_targeted_to_me(&record.envelope.header) { - // deduplicate if deduplication information has been provided - if let Some(dedup_information) = dedup_information { - if Self::is_outdated_or_duplicate(dedup_information, transaction).await? { - debug!( - "Ignoring outdated or duplicate message: {:?}", - record.envelope.header - ); - return Ok(None); - } - transaction - .put_dedup_seq_number( - dedup_information.producer_id.clone(), - &dedup_information.sequence_number, - ) - .map_err(state_machine::Error::Storage)?; - } + trace!(lsn = %record.lsn, "Processing bifrost record for '{}': {:?}", record.envelope.kind(), record.envelope.header()); - // todo: redesign to pass the arc (or reference) further down - let record_created_at = record.created_at; - let record_lsn = record.lsn; - let envelope = Arc::unwrap_or_clone(record.envelope); + if !self.is_targeted_to_me(&record.keys) { + self.status.num_skipped_records += 1; + trace!( + "Ignore message which is not targeted to me: {:?}", + record.envelope.header() + ); + } - if let Command::AnnounceLeader(announce_leader) = envelope.command { - // leadership change detected, let's finish our transaction here - return Ok(Some(announce_leader)); - } else if let Command::UpdatePartitionDurability(partition_durability) = - envelope.command + // deduplicate if deduplication information has been provided + // todo(azmy): use record.envelope.dedup() directly instead of + // converting to dedup-information. + let dedup_information = record.envelope.dedup().clone().into(); + if let Some(dedup_information) = dedup_information { + if Self::is_outdated_or_duplicate(&dedup_information, transaction) + .await + .map_err(state_machine::Error::from)? { + debug!( + "Ignoring outdated or duplicate message: {:?}", + record.envelope.header() + ); + return Ok(None); + } + transaction + .put_dedup_seq_number( + dedup_information.producer_id.clone(), + &dedup_information.sequence_number, + ) + .map_err(state_machine::Error::Storage)?; + } + + // todo: redesign to pass the arc (or reference) further down + let record_created_at = record.created_at; + let record_lsn = record.lsn; + let envelope = Arc::unwrap_or_clone(record.envelope); + + match envelope.kind() { + RecordKind::AnnounceLeader => { + let envelope = envelope.into_typed::(); + let (_, payload) = envelope.split()?; + return Ok(Some(Box::new(payload))); + } + RecordKind::UpdatePartitionDurability => { + let envelope = envelope.into_typed::(); + let (_, partition_durability) = envelope.split()?; if partition_durability.partition_id != self.partition_store.partition_id() { self.status.num_skipped_records += 1; trace!( @@ -732,12 +752,15 @@ where durable_point: partition_durability.durable_point, }; if self.trim_queue.push(&partition_durability) { - transaction.put_partition_durability(&partition_durability)?; + transaction + .put_partition_durability(&partition_durability) + .map_err(state_machine::Error::from)?; } - } else { + } + _ => { self.state_machine .apply( - envelope.command, + envelope, record_created_at.into(), record_lsn, transaction, @@ -746,31 +769,13 @@ where ) .await?; } - } else { - self.status.num_skipped_records += 1; - trace!( - "Ignore message which is not targeted to me: {:?}", - record.envelope.header - ); } Ok(None) } - fn is_targeted_to_me<'a>(&self, header: &'a Header) -> Option<&'a Option> { - match &header.dest { - Destination::Processor { - partition_key, - dedup, - } if self - .partition_store - .partition_key_range() - .contains(partition_key) => - { - Some(dedup) - } - _ => None, - } + fn is_targeted_to_me(&self, keys: &Keys) -> bool { + keys.matches_key_query(&self.key_filter) } async fn is_outdated_or_duplicate( diff --git a/crates/worker/src/partition/state_machine/entries/attach_invocation_command.rs b/crates/worker/src/partition/state_machine/entries/attach_invocation_command.rs index 0ed66a5fab..4f343e10ee 100644 --- a/crates/worker/src/partition/state_machine/entries/attach_invocation_command.rs +++ b/crates/worker/src/partition/state_machine/entries/attach_invocation_command.rs @@ -8,14 +8,15 @@ // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0. -use crate::partition::state_machine::entries::ApplyJournalCommandEffect; -use crate::partition::state_machine::{CommandHandler, Error, StateMachineApplyContext}; use restate_storage_api::fsm_table::WriteFsmTable; use restate_storage_api::outbox_table::{OutboxMessage, WriteOutboxTable}; use restate_storage_api::timer_table::WriteTimerTable; use restate_types::invocation::{AttachInvocationRequest, ServiceInvocationResponseSink}; use restate_types::journal_v2::AttachInvocationCommand; +use crate::partition::state_machine::entries::ApplyJournalCommandEffect; +use crate::partition::state_machine::{CommandHandler, Error, StateMachineApplyContext}; + pub(super) type ApplyAttachInvocationCommand<'e> = ApplyJournalCommandEffect<'e, AttachInvocationCommand>; @@ -59,7 +60,7 @@ mod tests { AttachInvocationCommand, AttachInvocationCompletion, AttachInvocationResult, AttachInvocationTarget, CommandType, Entry, EntryMetadata, EntryType, }; - use restate_wal_protocol::Command; + use restate_wal_protocol::v2::{Record, records}; use rstest::rstest; #[rstest] @@ -87,7 +88,7 @@ mod tests { let actions = test_env .apply_multiple([ invoker_entry_effect(invocation_id, attach_invocation_command.clone()), - Command::InvocationResponse(InvocationResponse { + records::InvocationResponse::new_test(InvocationResponse { target: JournalCompletionTarget::from_parts(invocation_id, completion_id, 0), result: ResponseResult::Success(success_result.clone()), }), diff --git a/crates/worker/src/partition/state_machine/entries/call_commands.rs b/crates/worker/src/partition/state_machine/entries/call_commands.rs index c66b3b7403..95c4baa2f0 100644 --- a/crates/worker/src/partition/state_machine/entries/call_commands.rs +++ b/crates/worker/src/partition/state_machine/entries/call_commands.rs @@ -7,9 +7,8 @@ // As of the Change Date specified in that file, in accordance with // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0. +use std::collections::VecDeque; -use crate::partition::state_machine::entries::ApplyJournalCommandEffect; -use crate::partition::state_machine::{CommandHandler, Error, StateMachineApplyContext}; use restate_service_protocol_v4::entry_codec::ServiceProtocolV4Codec; use restate_storage_api::fsm_table::WriteFsmTable; use restate_storage_api::invocation_status_table::InvocationStatus; @@ -20,7 +19,9 @@ use restate_types::journal_v2::command::{CallCommand, CallRequest, OneWayCallCom use restate_types::journal_v2::raw::RawEntry; use restate_types::journal_v2::{CallInvocationIdCompletion, CompletionId, Entry}; use restate_types::time::MillisSinceEpoch; -use std::collections::VecDeque; + +use crate::partition::state_machine::entries::ApplyJournalCommandEffect; +use crate::partition::state_machine::{CommandHandler, Error, StateMachineApplyContext}; pub(super) type ApplyCallCommand<'e> = ApplyJournalCommandEffect<'e, CallCommand>; @@ -164,7 +165,7 @@ mod tests { CommandType, Entry, EntryMetadata, EntryType, OneWayCallCommand, }; use restate_types::time::MillisSinceEpoch; - use restate_wal_protocol::Command; + use restate_wal_protocol::v2::{Record, records}; use rstest::rstest; use std::time::{Duration, SystemTime}; @@ -194,7 +195,7 @@ mod tests { let actions = test_env .apply_multiple([ invoker_entry_effect(invocation_id, call_command.clone()), - Command::InvocationResponse(InvocationResponse { + records::InvocationResponse::new_test(InvocationResponse { target: JournalCompletionTarget::from_parts( invocation_id, result_completion_id, diff --git a/crates/worker/src/partition/state_machine/entries/get_invocation_output_command.rs b/crates/worker/src/partition/state_machine/entries/get_invocation_output_command.rs index 597fcdca47..1648e9d6ec 100644 --- a/crates/worker/src/partition/state_machine/entries/get_invocation_output_command.rs +++ b/crates/worker/src/partition/state_machine/entries/get_invocation_output_command.rs @@ -8,14 +8,15 @@ // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0. -use crate::partition::state_machine::entries::ApplyJournalCommandEffect; -use crate::partition::state_machine::{CommandHandler, Error, StateMachineApplyContext}; use restate_storage_api::fsm_table::WriteFsmTable; use restate_storage_api::outbox_table::{OutboxMessage, WriteOutboxTable}; use restate_storage_api::timer_table::WriteTimerTable; use restate_types::invocation::{AttachInvocationRequest, ServiceInvocationResponseSink}; use restate_types::journal_v2::GetInvocationOutputCommand; +use crate::partition::state_machine::entries::ApplyJournalCommandEffect; +use crate::partition::state_machine::{CommandHandler, Error, StateMachineApplyContext}; + pub(super) type ApplyGetInvocationOutputCommand<'e> = ApplyJournalCommandEffect<'e, GetInvocationOutputCommand>; @@ -60,7 +61,7 @@ mod tests { AttachInvocationTarget, CommandType, Entry, EntryMetadata, EntryType, GetInvocationOutputCommand, GetInvocationOutputCompletion, GetInvocationOutputResult, }; - use restate_wal_protocol::Command; + use restate_wal_protocol::v2::{Record, records}; use rstest::rstest; #[rstest] @@ -93,12 +94,12 @@ mod tests { completion_id, }; let response_command = if complete_using_notify_get_invocation_output { - Command::NotifyGetInvocationOutputResponse(GetInvocationOutputResponse { + records::NotifyGetInvocationOutputResponse::new_test(GetInvocationOutputResponse { target: JournalCompletionTarget::from_parts(invocation_id, completion_id, 0), result: expected_get_invocation_result.clone(), }) } else { - Command::InvocationResponse(InvocationResponse { + records::InvocationResponse::new_test(InvocationResponse { target: JournalCompletionTarget::from_parts(invocation_id, completion_id, 0), result: if complete_with_not_ready { ResponseResult::Failure(NOT_READY_INVOCATION_ERROR) diff --git a/crates/worker/src/partition/state_machine/entries/mod.rs b/crates/worker/src/partition/state_machine/entries/mod.rs index 3271ff1561..ee4315f08c 100644 --- a/crates/worker/src/partition/state_machine/entries/mod.rs +++ b/crates/worker/src/partition/state_machine/entries/mod.rs @@ -415,7 +415,7 @@ mod tests { Header, InvocationResponse, InvocationTarget, JournalCompletionTarget, ResponseResult, }; use restate_types::journal_v2::{CallCommand, CallRequest}; - use restate_wal_protocol::Command; + use restate_wal_protocol::v2::{Record, records}; #[restate_core::test] async fn update_journal_and_commands_length() { @@ -458,7 +458,7 @@ mod tests { ); let _ = test_env - .apply(Command::InvocationResponse(InvocationResponse { + .apply(records::InvocationResponse::new_test(InvocationResponse { target: JournalCompletionTarget { caller_id: invocation_id, caller_completion_id: result_completion_id, diff --git a/crates/worker/src/partition/state_machine/entries/notification.rs b/crates/worker/src/partition/state_machine/entries/notification.rs index 629314f19d..48942d8166 100644 --- a/crates/worker/src/partition/state_machine/entries/notification.rs +++ b/crates/worker/src/partition/state_machine/entries/notification.rs @@ -219,8 +219,8 @@ mod tests { SleepCommand, SleepCompletion, }; use restate_types::time::MillisSinceEpoch; - use restate_wal_protocol::Command; use restate_wal_protocol::timer::TimerKeyValue; + use restate_wal_protocol::v2::{Record, records}; use rstest::rstest; use std::time::Duration; @@ -237,7 +237,7 @@ mod tests { // Send signal notification let signal = Signal::new(SignalId::for_index(17), signal_result); let actions = test_env - .apply(Command::NotifySignal(NotifySignalRequest { + .apply(records::NotifySignal::new_test(NotifySignalRequest { invocation_id, signal: signal.clone(), })) @@ -270,7 +270,7 @@ mod tests { // Send signal notification before pinned deployment let signal = Signal::new(SignalId::for_index(17), SignalResult::Void); let actions = test_env - .apply(Command::NotifySignal(NotifySignalRequest { + .apply(records::NotifySignal::new_test(NotifySignalRequest { invocation_id, signal: signal.clone(), })) @@ -338,12 +338,14 @@ mod tests { // Send a completion notification for a command (e.g., Sleep) with completion_id = 1 let completion_id = 1; let _ = test_env - .apply(Command::Timer(TimerKeyValue::complete_journal_entry( - wake_up_time, - invocation_id, - completion_id, - 0, - ))) + .apply(records::Timer::new_test( + TimerKeyValue::complete_journal_entry( + wake_up_time, + invocation_id, + completion_id, + 0, + ), + )) .await; // The invocation should remain paused @@ -386,11 +388,13 @@ mod tests { // Apply the cancel signal notification let actions = test_env - .apply(Command::TerminateInvocation(InvocationTermination { - invocation_id, - flavor: TerminationFlavor::Cancel, - response_sink: None, - })) + .apply(records::TerminateInvocation::new_test( + InvocationTermination { + invocation_id, + flavor: TerminationFlavor::Cancel, + response_sink: None, + }, + )) .await; // The invocation should be resumed (invoke action dispatched) diff --git a/crates/worker/src/partition/state_machine/lifecycle/cancel.rs b/crates/worker/src/partition/state_machine/lifecycle/cancel.rs index 8b0c74f997..fd470a47b2 100644 --- a/crates/worker/src/partition/state_machine/lifecycle/cancel.rs +++ b/crates/worker/src/partition/state_machine/lifecycle/cancel.rs @@ -7,9 +7,8 @@ // As of the Change Date specified in that file, in accordance with // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0. +use tracing::{debug, trace}; -use crate::partition::state_machine::entries::OnJournalEntryCommand; -use crate::partition::state_machine::{CommandHandler, Error, StateMachineApplyContext}; use restate_storage_api::fsm_table::WriteFsmTable; use restate_storage_api::inbox_table::WriteInboxTable; use restate_storage_api::invocation_status_table::{ @@ -28,7 +27,9 @@ use restate_types::invocation::{ InvocationEpoch, InvocationMutationResponseSink, TerminationFlavor, }; use restate_types::journal_v2::CANCEL_SIGNAL; -use tracing::{debug, trace}; + +use crate::partition::state_machine::entries::OnJournalEntryCommand; +use crate::partition::state_machine::{CommandHandler, Error, StateMachineApplyContext}; pub struct OnCancelCommand { pub invocation_id: InvocationId, @@ -139,7 +140,7 @@ mod tests { use restate_types::journal_v2::CANCEL_SIGNAL; use restate_types::service_protocol::ServiceProtocolVersion; use restate_types::time::MillisSinceEpoch; - use restate_wal_protocol::Command; + use restate_wal_protocol::v2::{Record, records}; #[restate_core::test] async fn cancel_invoked_invocation() { @@ -149,11 +150,13 @@ mod tests { // Send signal notification let actions = test_env - .apply(Command::TerminateInvocation(InvocationTermination { - invocation_id, - flavor: TerminationFlavor::Cancel, - response_sink: None, - })) + .apply(records::TerminateInvocation::new_test( + InvocationTermination { + invocation_id, + flavor: TerminationFlavor::Cancel, + response_sink: None, + }, + )) .await; assert_that!( actions, @@ -174,7 +177,7 @@ mod tests { // Send signal notification let actions = test_env - .apply(Command::NotifySignal(NotifySignalRequest { + .apply(records::NotifySignal::new_test(NotifySignalRequest { invocation_id, signal: CANCEL_SIGNAL.try_into().unwrap(), })) @@ -197,7 +200,7 @@ mod tests { // Send signal notification before pinning the deployment let actions = test_env - .apply(Command::NotifySignal(NotifySignalRequest { + .apply(records::NotifySignal::new_test(NotifySignalRequest { invocation_id, signal: CANCEL_SIGNAL.try_into().unwrap(), })) @@ -212,7 +215,7 @@ mod tests { // Now pin to protocol v4, this should apply the cancel notification let actions = test_env - .apply(Command::InvokerEffect(Box::new(Effect { + .apply(records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::PinnedDeployment(PinnedDeployment { @@ -240,7 +243,7 @@ mod tests { let rpc_id = PartitionProcessorRpcRequestId::new(); let _ = test_env - .apply(Command::Invoke(Box::new(ServiceInvocation { + .apply(records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id, execution_time: Some(MillisSinceEpoch::MAX), response_sink: Some(ServiceInvocationResponseSink::ingress(rpc_id)), @@ -256,7 +259,7 @@ mod tests { assert!(let InvocationStatus::Scheduled(_) = current_invocation_status); let actions = test_env - .apply(Command::NotifySignal(NotifySignalRequest { + .apply(records::NotifySignal::new_test(NotifySignalRequest { invocation_id, signal: CANCEL_SIGNAL.try_into().unwrap(), })) @@ -294,7 +297,7 @@ mod tests { let caller_id = InvocationId::mock_random(); let _ = test_env - .apply(Command::Invoke(Box::new(ServiceInvocation { + .apply(records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id, invocation_target: invocation_target.clone(), ..ServiceInvocation::mock() @@ -302,7 +305,7 @@ mod tests { .await; let _ = test_env - .apply(Command::Invoke(Box::new(ServiceInvocation { + .apply(records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id: inboxed_id, invocation_target: inboxed_target, response_sink: Some(ServiceInvocationResponseSink::PartitionProcessor( @@ -321,7 +324,7 @@ mod tests { assert!(let InvocationStatus::Inboxed(_) = current_invocation_status); let actions = test_env - .apply(Command::NotifySignal(NotifySignalRequest { + .apply(records::NotifySignal::new_test(NotifySignalRequest { invocation_id: inboxed_id, signal: CANCEL_SIGNAL.try_into().unwrap(), })) diff --git a/crates/worker/src/partition/state_machine/lifecycle/event.rs b/crates/worker/src/partition/state_machine/lifecycle/event.rs index facbe8341c..08bbe8671e 100644 --- a/crates/worker/src/partition/state_machine/lifecycle/event.rs +++ b/crates/worker/src/partition/state_machine/lifecycle/event.rs @@ -8,12 +8,13 @@ // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0. -use crate::partition::state_machine::{CommandHandler, Error, StateMachineApplyContext}; use restate_storage_api::invocation_status_table::{InvocationStatus, WriteInvocationStatusTable}; use restate_storage_api::journal_events::{EventView, WriteJournalEventsTable}; use restate_types::identifiers::InvocationId; use restate_types::journal_events::raw::RawEvent; +use crate::partition::state_machine::{CommandHandler, Error, StateMachineApplyContext}; + pub struct OnInvokerEventCommand { pub invocation_id: InvocationId, pub invocation_status: InvocationStatus, @@ -87,7 +88,7 @@ mod tests { use restate_invoker_api::Effect; use restate_types::journal_events::raw::RawEvent; use restate_types::journal_events::{Event, TransientErrorEvent}; - use restate_wal_protocol::Command; + use restate_wal_protocol::v2::{Record, records}; #[restate_core::test] async fn store_event() { @@ -106,7 +107,7 @@ mod tests { }; let _ = test_env - .apply(Command::InvokerEffect(Box::new(Effect { + .apply(records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::JournalEvent { diff --git a/crates/worker/src/partition/state_machine/lifecycle/manual_resume.rs b/crates/worker/src/partition/state_machine/lifecycle/manual_resume.rs index 0ec1dc4c1d..fd025f8df9 100644 --- a/crates/worker/src/partition/state_machine/lifecycle/manual_resume.rs +++ b/crates/worker/src/partition/state_machine/lifecycle/manual_resume.rs @@ -8,15 +8,17 @@ // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0. -use crate::partition::state_machine::lifecycle::ResumeInvocationCommand; -use crate::partition::state_machine::{CommandHandler, Error, StateMachineApplyContext}; +use tracing::trace; + use restate_storage_api::invocation_status_table::{ InvocationStatus, ReadInvocationStatusTable, WriteInvocationStatusTable, }; use restate_types::identifiers::{DeploymentId, InvocationId}; use restate_types::invocation::InvocationMutationResponseSink; use restate_types::invocation::client::ResumeInvocationResponse; -use tracing::trace; + +use crate::partition::state_machine::lifecycle::ResumeInvocationCommand; +use crate::partition::state_machine::{CommandHandler, Error, StateMachineApplyContext}; pub struct OnManualResumeCommand { pub invocation_id: InvocationId, @@ -103,7 +105,7 @@ mod tests { use restate_types::invocation::{IngressInvocationResponseSink, ResumeInvocationRequest}; use restate_types::journal_v2::{NotificationId, SleepCommand}; use restate_types::service_protocol::ServiceProtocolVersion; - use restate_wal_protocol::Command; + use restate_wal_protocol::v2::{Record, records}; use std::time::{Duration, SystemTime}; #[restate_core::test] @@ -125,13 +127,15 @@ mod tests { // Now on manual resume, we should resume the suspended invocation let request_id = PartitionProcessorRpcRequestId::new(); let actions = test_env - .apply(Command::ResumeInvocation(ResumeInvocationRequest { - invocation_id, - update_pinned_deployment_id: None, - response_sink: Some(InvocationMutationResponseSink::Ingress( - IngressInvocationResponseSink { request_id }, - )), - })) + .apply(records::ResumeInvocation::new_test( + ResumeInvocationRequest { + invocation_id, + update_pinned_deployment_id: None, + response_sink: Some(InvocationMutationResponseSink::Ingress( + IngressInvocationResponseSink { request_id }, + )), + }, + )) .await; assert_that!( actions, @@ -162,7 +166,7 @@ mod tests { let initial_deployment_id = DeploymentId::new(); // Pin deployment let _ = test_env - .apply(Command::InvokerEffect(Box::new(Effect { + .apply(records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::PinnedDeployment(PinnedDeployment { @@ -188,13 +192,15 @@ mod tests { let request_id = PartitionProcessorRpcRequestId::new(); let new_deployment_id = DeploymentId::new(); let actions = test_env - .apply(Command::ResumeInvocation(ResumeInvocationRequest { - invocation_id, - update_pinned_deployment_id: Some(new_deployment_id), - response_sink: Some(InvocationMutationResponseSink::Ingress( - IngressInvocationResponseSink { request_id }, - )), - })) + .apply(records::ResumeInvocation::new_test( + ResumeInvocationRequest { + invocation_id, + update_pinned_deployment_id: Some(new_deployment_id), + response_sink: Some(InvocationMutationResponseSink::Ingress( + IngressInvocationResponseSink { request_id }, + )), + }, + )) .await; assert_that!( actions, @@ -258,13 +264,15 @@ mod tests { // Now on manual resume, we should resume the suspended invocation let request_id = PartitionProcessorRpcRequestId::new(); let actions = test_env - .apply(Command::ResumeInvocation(ResumeInvocationRequest { - invocation_id, - update_pinned_deployment_id: None, - response_sink: Some(InvocationMutationResponseSink::Ingress( - IngressInvocationResponseSink { request_id }, - )), - })) + .apply(records::ResumeInvocation::new_test( + ResumeInvocationRequest { + invocation_id, + update_pinned_deployment_id: None, + response_sink: Some(InvocationMutationResponseSink::Ingress( + IngressInvocationResponseSink { request_id }, + )), + }, + )) .await; assert_that!( actions, diff --git a/crates/worker/src/partition/state_machine/lifecycle/notify_invocation_response.rs b/crates/worker/src/partition/state_machine/lifecycle/notify_invocation_response.rs index 9260fd1d98..caf6f0bf81 100644 --- a/crates/worker/src/partition/state_machine/lifecycle/notify_invocation_response.rs +++ b/crates/worker/src/partition/state_machine/lifecycle/notify_invocation_response.rs @@ -8,9 +8,8 @@ // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0. -use crate::debug_if_leader; -use crate::partition::state_machine::invocation_status_ext::InvocationStatusExt; -use crate::partition::state_machine::{CommandHandler, Error, StateMachineApplyContext, entries}; +use tracing::error; + use restate_storage_api::fsm_table::WriteFsmTable; use restate_storage_api::invocation_status_table::{ InvocationStatus, ReadInvocationStatusTable, WriteInvocationStatusTable, @@ -30,7 +29,10 @@ use restate_types::journal_v2::{ CompletionId, GetInvocationOutputCompletion, GetInvocationOutputResult, GetPromiseCompletion, GetPromiseResult, SleepCompletion, }; -use tracing::error; + +use crate::debug_if_leader; +use crate::partition::state_machine::invocation_status_ext::InvocationStatusExt; +use crate::partition::state_machine::{CommandHandler, Error, StateMachineApplyContext, entries}; pub struct OnNotifyInvocationResponse { pub invocation_id: InvocationId, @@ -182,7 +184,7 @@ mod tests { use restate_types::journal_v2::{ CallCommand, CallInvocationIdCompletion, CallRequest, Entry, EntryType, }; - use restate_wal_protocol::Command; + use restate_wal_protocol::v2::{Record, records}; #[restate_core::test] async fn reply_to_call_with_failure_and_metadata() { @@ -208,7 +210,7 @@ mod tests { let actions = test_env .apply_multiple([ fixtures::invoker_entry_effect(invocation_id, call_command.clone()), - Command::InvocationResponse(InvocationResponse { + records::InvocationResponse::new_test(InvocationResponse { target: JournalCompletionTarget::from_parts( invocation_id, result_completion_id, diff --git a/crates/worker/src/partition/state_machine/lifecycle/paused.rs b/crates/worker/src/partition/state_machine/lifecycle/paused.rs index 7ebbc859ec..46e24de607 100644 --- a/crates/worker/src/partition/state_machine/lifecycle/paused.rs +++ b/crates/worker/src/partition/state_machine/lifecycle/paused.rs @@ -8,9 +8,6 @@ // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0. -use crate::debug_if_leader; -use crate::partition::state_machine::lifecycle::event::ApplyEventCommand; -use crate::partition::state_machine::{CommandHandler, Error, StateMachineApplyContext}; use restate_storage_api::invocation_status_table::{ InvocationStatus, ReadInvocationStatusTable, WriteInvocationStatusTable, }; @@ -18,6 +15,10 @@ use restate_storage_api::journal_events::WriteJournalEventsTable; use restate_types::identifiers::InvocationId; use restate_types::journal_events::raw::RawEvent; +use crate::debug_if_leader; +use crate::partition::state_machine::lifecycle::event::ApplyEventCommand; +use crate::partition::state_machine::{CommandHandler, Error, StateMachineApplyContext}; + pub struct OnPausedCommand { pub invocation_id: InvocationId, pub paused_event: RawEvent, @@ -78,7 +79,7 @@ mod tests { InFlightInvocationMetadata, InvocationStatusDiscriminants, ReadInvocationStatusTable, }; use restate_types::journal_events::{Event, PausedEvent, TransientErrorEvent}; - use restate_wal_protocol::Command; + use restate_wal_protocol::v2::{Record, records}; #[restate_core::test] async fn paused_with_pinned_deployment() { @@ -100,7 +101,7 @@ mod tests { // Check we just pause let _ = test_env - .apply(Command::InvokerEffect(Box::new( + .apply(records::InvokerEffect::new_test(Box::new( restate_invoker_api::Effect { invocation_id, invocation_epoch: 0, @@ -152,7 +153,7 @@ mod tests { // Check we just pause let _ = test_env - .apply(Command::InvokerEffect(Box::new( + .apply(records::InvokerEffect::new_test(Box::new( restate_invoker_api::Effect { invocation_id, invocation_epoch: 0, diff --git a/crates/worker/src/partition/state_machine/lifecycle/purge_journal.rs b/crates/worker/src/partition/state_machine/lifecycle/purge_journal.rs index 8d1424897d..7508095a6b 100644 --- a/crates/worker/src/partition/state_machine/lifecycle/purge_journal.rs +++ b/crates/worker/src/partition/state_machine/lifecycle/purge_journal.rs @@ -7,8 +7,8 @@ // As of the Change Date specified in that file, in accordance with // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0. +use tracing::trace; -use crate::partition::state_machine::{CommandHandler, Error, StateMachineApplyContext}; use restate_storage_api::invocation_status_table::{ InvocationStatus, ReadInvocationStatusTable, WriteInvocationStatusTable, }; @@ -19,7 +19,8 @@ use restate_types::identifiers::InvocationId; use restate_types::invocation::InvocationMutationResponseSink; use restate_types::invocation::client::PurgeInvocationResponse; use restate_types::service_protocol::ServiceProtocolVersion; -use tracing::trace; + +use crate::partition::state_machine::{CommandHandler, Error, StateMachineApplyContext}; pub struct OnPurgeJournalCommand { pub invocation_id: InvocationId, @@ -113,7 +114,7 @@ mod tests { InvocationTarget, PurgeInvocationRequest, ServiceInvocation, ServiceInvocationResponseSink, }; use restate_types::journal_v2::{CommandType, OutputCommand, OutputResult}; - use restate_wal_protocol::Command; + use restate_wal_protocol::v2::{Record, records}; use std::time::Duration; #[restate_core::test] @@ -131,7 +132,7 @@ mod tests { // Create and complete a fresh invocation let actions = test_env .apply_multiple([ - Command::Invoke(Box::new(ServiceInvocation { + records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id, invocation_target: invocation_target.clone(), response_sink: Some(ServiceInvocationResponseSink::Ingress { request_id }), @@ -188,7 +189,7 @@ mod tests { // Now let's purge the journal test_env - .apply(Command::PurgeJournal(PurgeInvocationRequest { + .apply(records::PurgeJournal::new_test(PurgeInvocationRequest { invocation_id, response_sink: None, })) @@ -197,7 +198,7 @@ mod tests { // At this point we should still be able to de-duplicate the invocation let request_id = PartitionProcessorRpcRequestId::default(); let actions = test_env - .apply(Command::Invoke(Box::new(ServiceInvocation { + .apply(records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id, invocation_target: invocation_target.clone(), response_sink: Some(ServiceInvocationResponseSink::Ingress { request_id }), @@ -232,7 +233,7 @@ mod tests { // Now purge completely test_env - .apply(Command::PurgeInvocation(PurgeInvocationRequest { + .apply(records::PurgeInvocation::new_test(PurgeInvocationRequest { invocation_id, response_sink: None, })) diff --git a/crates/worker/src/partition/state_machine/lifecycle/restart_as_new.rs b/crates/worker/src/partition/state_machine/lifecycle/restart_as_new.rs index a808b8ee08..f90fbe4e00 100644 --- a/crates/worker/src/partition/state_machine/lifecycle/restart_as_new.rs +++ b/crates/worker/src/partition/state_machine/lifecycle/restart_as_new.rs @@ -8,10 +8,9 @@ // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0. -use crate::debug_if_leader; -use crate::partition::state_machine::{Action, CommandHandler, Error, StateMachineApplyContext}; use ahash::HashSet; use opentelemetry::trace::Span; + use restate_service_protocol_v4::entry_codec::ServiceProtocolV4Codec; use restate_storage_api::fsm_table::WriteFsmTable; use restate_storage_api::idempotency_table::IdempotencyTable; @@ -35,6 +34,9 @@ use restate_types::invocation::{ use restate_types::journal_v2; use restate_types::journal_v2::{CommandMetadata, EntryMetadata, EntryType, NotificationId}; +use crate::debug_if_leader; +use crate::partition::state_machine::{Action, CommandHandler, Error, StateMachineApplyContext}; + pub struct OnRestartAsNewInvocationCommand { pub invocation_id: InvocationId, pub new_invocation_id: InvocationId, @@ -300,8 +302,8 @@ mod tests { }; use restate_types::service_protocol::ServiceProtocolVersion; use restate_types::time::MillisSinceEpoch; - use restate_wal_protocol::Command; use restate_wal_protocol::timer::TimerKeyValue; + use restate_wal_protocol::v2::{Record, records}; use std::time::Duration; #[restate_core::test] @@ -313,7 +315,7 @@ mod tests { let original_invocation_id = InvocationId::generate(&invocation_target, None); let _ = test_env .apply_multiple([ - Command::Invoke(Box::new(ServiceInvocation { + records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id: original_invocation_id, invocation_target: invocation_target.clone(), completion_retention_duration: Duration::from_secs(120), @@ -350,7 +352,7 @@ mod tests { let new_id = InvocationId::mock_generate(&InvocationTarget::mock_virtual_object()); let request_id = PartitionProcessorRpcRequestId::new(); let actions = test_env - .apply(Command::RestartAsNewInvocation( + .apply(records::RestartAsNewInvocation::new_test( RestartAsNewInvocationRequest { invocation_id: original_invocation_id, new_invocation_id: new_id, @@ -386,7 +388,7 @@ mod tests { let original_invocation_id = InvocationId::generate(&invocation_target, None); let _ = test_env .apply_multiple([ - Command::Invoke(Box::new(ServiceInvocation { + records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id: original_invocation_id, invocation_target: invocation_target.clone(), completion_retention_duration: Duration::from_secs(120), @@ -409,7 +411,7 @@ mod tests { let new_id = InvocationId::mock_generate(&invocation_target); let request_id = PartitionProcessorRpcRequestId::new(); let actions = test_env - .apply(Command::RestartAsNewInvocation( + .apply(records::RestartAsNewInvocation::new_test( RestartAsNewInvocationRequest { invocation_id: original_invocation_id, new_invocation_id: new_id, @@ -462,7 +464,7 @@ mod tests { let original_invocation_id = InvocationId::generate(&invocation_target, None); let _ = test_env .apply_multiple([ - Command::Invoke(Box::new(ServiceInvocation { + records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id: original_invocation_id, invocation_target: invocation_target.clone(), completion_retention_duration: Duration::from_secs(120), @@ -491,7 +493,7 @@ mod tests { // Now restart original into a new invocation while VO is locked by locker_id let new_id = InvocationId::mock_generate(&invocation_target); let _ = test_env - .apply(Command::RestartAsNewInvocation( + .apply(records::RestartAsNewInvocation::new_test( RestartAsNewInvocationRequest { invocation_id: original_invocation_id, new_invocation_id: new_id, @@ -537,7 +539,7 @@ mod tests { let completion_id = 1u32; let _ = test_env .apply_multiple([ - Command::Invoke(Box::new(ServiceInvocation { + records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id: original_invocation_id, invocation_target: invocation_target.clone(), completion_retention_duration: Duration::from_secs(120), @@ -553,7 +555,7 @@ mod tests { completion_id, }, ), - Command::NotifySignal(NotifySignalRequest { + records::NotifySignal::new_test(NotifySignalRequest { invocation_id: original_invocation_id, signal: Signal::new( SignalId::for_index(1), @@ -568,7 +570,7 @@ mod tests { completion_id: completion_id + 1, }, ), - Command::Timer(TimerKeyValue::complete_journal_entry( + records::Timer::new_test(TimerKeyValue::complete_journal_entry( wake_up_time, original_invocation_id, completion_id, @@ -601,7 +603,7 @@ mod tests { ); let new_deployment_id = DeploymentId::new(); let _ = test_env - .apply(Command::RestartAsNewInvocation( + .apply(records::RestartAsNewInvocation::new_test( RestartAsNewInvocationRequest { invocation_id: original_invocation_id, new_invocation_id, @@ -655,7 +657,7 @@ mod tests { let original_invocation_id = InvocationId::generate(&invocation_target, None); let _ = test_env .apply_multiple([ - Command::Invoke(Box::new(ServiceInvocation { + records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id: original_invocation_id, invocation_target: invocation_target.clone(), completion_retention_duration: Duration::from_secs(120), @@ -690,7 +692,7 @@ mod tests { let new_invocation_id = InvocationId::mock_generate(&invocation_target); let request_id = PartitionProcessorRpcRequestId::new(); let actions = test_env - .apply(Command::RestartAsNewInvocation( + .apply(records::RestartAsNewInvocation::new_test( RestartAsNewInvocationRequest { invocation_id: original_invocation_id, new_invocation_id, diff --git a/crates/worker/src/partition/state_machine/lifecycle/suspend.rs b/crates/worker/src/partition/state_machine/lifecycle/suspend.rs index 648ae6de20..134eb300bb 100644 --- a/crates/worker/src/partition/state_machine/lifecycle/suspend.rs +++ b/crates/worker/src/partition/state_machine/lifecycle/suspend.rs @@ -7,14 +7,16 @@ // As of the Change Date specified in that file, in accordance with // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0. +use std::collections::HashSet; + +use tracing::trace; -use crate::partition::state_machine::{CommandHandler, Error, StateMachineApplyContext}; use restate_storage_api::invocation_status_table::{InvocationStatus, WriteInvocationStatusTable}; use restate_storage_api::journal_table_v2::ReadJournalTable; use restate_types::identifiers::InvocationId; use restate_types::journal_v2::NotificationId; -use std::collections::HashSet; -use tracing::trace; + +use crate::partition::state_machine::{CommandHandler, Error, StateMachineApplyContext}; pub struct OnSuspendCommand { pub invocation_id: InvocationId, @@ -102,8 +104,8 @@ mod tests { CommandType, Entry, EntryMetadata, EntryType, NotificationId, SleepCommand, SleepCompletion, }; use restate_types::time::MillisSinceEpoch; - use restate_wal_protocol::Command; use restate_wal_protocol::timer::TimerKeyValue; + use restate_wal_protocol::v2::{Record, records}; use std::time::{Duration, SystemTime}; #[restate_core::test] @@ -138,7 +140,9 @@ mod tests { })) ); - let actions = test_env.apply(Command::Timer(timer_key_value)).await; + let actions = test_env + .apply(records::Timer::new_test(timer_key_value)) + .await; assert_that!( actions, contains(matchers::actions::invoke_for_id(invocation_id)) @@ -178,7 +182,7 @@ mod tests { let actions = test_env .apply_multiple([ invoker_entry_effect(invocation_id, sleep_command.clone()), - Command::Timer(timer_key_value.clone()), + records::Timer::new_test(timer_key_value.clone()), ]) .await; assert_that!( diff --git a/crates/worker/src/partition/state_machine/lifecycle/version_barrier.rs b/crates/worker/src/partition/state_machine/lifecycle/version_barrier.rs index e8cd206b14..acaf0f246f 100644 --- a/crates/worker/src/partition/state_machine/lifecycle/version_barrier.rs +++ b/crates/worker/src/partition/state_machine/lifecycle/version_barrier.rs @@ -48,8 +48,8 @@ mod tests { use restate_types::SemanticRestateVersion; use restate_types::identifiers::PartitionKey; use restate_types::logs::Keys; - use restate_wal_protocol::Command; use restate_wal_protocol::control::VersionBarrier; + use restate_wal_protocol::v2::{Record, records}; use crate::partition::state_machine::tests::TestEnv; use crate::partition::state_machine::{Action, StateMachine}; @@ -77,7 +77,7 @@ mod tests { ); let result = test_env - .apply_fallible(Command::VersionBarrier(VersionBarrier { + .apply_fallible(records::VersionBarrier::new_test(VersionBarrier { version: SemanticRestateVersion::parse("99.0.0").unwrap(), human_reason: Some("testing".to_string()), partition_key_range: Keys::RangeInclusive(PartitionKey::MIN..=PartitionKey::MAX), @@ -113,7 +113,7 @@ mod tests { let mut test_env = TestEnv::create_with_state_machine(state_machine).await; let result = test_env - .apply_fallible(Command::VersionBarrier(VersionBarrier { + .apply_fallible(records::VersionBarrier::new_test(VersionBarrier { version: SemanticRestateVersion::current().clone(), human_reason: Some("testing".to_string()), partition_key_range: Keys::RangeInclusive(PartitionKey::MIN..=PartitionKey::MAX), @@ -128,7 +128,7 @@ mod tests { } // re-apply the same version, no-op let result = test_env - .apply_fallible(Command::VersionBarrier(VersionBarrier { + .apply_fallible(records::VersionBarrier::new_test(VersionBarrier { version: SemanticRestateVersion::current().clone(), human_reason: Some("testing".to_string()), partition_key_range: Keys::RangeInclusive(PartitionKey::MIN..=PartitionKey::MAX), @@ -143,7 +143,7 @@ mod tests { // apply an older version, success but without effect. let result = test_env - .apply_fallible(Command::VersionBarrier(VersionBarrier { + .apply_fallible(records::VersionBarrier::new_test(VersionBarrier { version: SemanticRestateVersion::parse("0.1.0").unwrap(), human_reason: Some("testing".to_string()), partition_key_range: Keys::RangeInclusive(PartitionKey::MIN..=PartitionKey::MAX), diff --git a/crates/worker/src/partition/state_machine/mod.rs b/crates/worker/src/partition/state_machine/mod.rs index 26db066ca9..ef95e582bf 100644 --- a/crates/worker/src/partition/state_machine/mod.rs +++ b/crates/worker/src/partition/state_machine/mod.rs @@ -14,6 +14,8 @@ mod lifecycle; mod utils; pub use actions::{Action, ActionCollector}; +use restate_types::storage::StorageDecodeError; +use restate_wal_protocol::v2::{Raw, RecordKind, records}; use std::borrow::Cow; use std::collections::HashSet; @@ -74,7 +76,7 @@ use restate_types::invocation::client::{ PurgeInvocationResponse, ResumeInvocationResponse, }; use restate_types::invocation::{ - AttachInvocationRequest, IngressInvocationResponseSink, InvocationEpoch, + self, AttachInvocationRequest, IngressInvocationResponseSink, InvocationEpoch, InvocationMutationResponseSink, InvocationQuery, InvocationResponse, InvocationTarget, InvocationTargetType, InvocationTermination, JournalCompletionTarget, NotifySignalRequest, ResponseResult, ServiceInvocation, ServiceInvocationResponseSink, ServiceInvocationSpanContext, @@ -105,9 +107,9 @@ use restate_types::state_mut::StateMutationVersion; use restate_types::time::MillisSinceEpoch; use restate_types::{RestateVersion, SemanticRestateVersion}; use restate_types::{Versioned, journal::*}; -use restate_wal_protocol::Command; use restate_wal_protocol::timer::TimerKeyDisplay; use restate_wal_protocol::timer::TimerKeyValue; +use restate_wal_protocol::v2; use self::utils::SpanExt; use crate::metric_definitions::{PARTITION_APPLY_COMMAND, USAGE_LEADER_JOURNAL_ENTRY_COUNT}; @@ -168,6 +170,8 @@ pub enum Error { EntryEncoding(#[from] journal_v2::encoding::DecodingError), #[error("failed to deserialize entry: {0}")] EntryDecoding(#[from] journal_v2::raw::RawEntryError), + #[error("failed to decode envelope(v2): {0}")] + EnvelopeDecoding(#[from] StorageDecodeError), #[error( "error when trying to apply invocation response with completion id {1}, the entry type {0} is not expected to be completed through InvocationResponse command" )] @@ -176,6 +180,8 @@ pub enum Error { "error when trying to apply invocation response with completion id {0}, because no command was found for given completion id" )] MissingCommandForInvocationResponse(CompletionId), + #[error("received an unknown biforst record kind")] + UnknownEnvelopeKind, } #[macro_export] @@ -259,18 +265,18 @@ impl StateMachine { // - Accept `LsnEnvelope` by reference. pub async fn apply( &mut self, - command: Command, + envelope: v2::Envelope, record_created_at: MillisSinceEpoch, record_lsn: Lsn, transaction: &mut TransactionType, action_collector: &mut ActionCollector, is_leader: bool, ) -> Result<(), Error> { - let span = utils::state_machine_apply_command_span(is_leader, &command); + let span = utils::state_machine_apply_command_span(is_leader, envelope.kind()); async { let start = Instant::now(); // Apply the command - let command_type = command.name(); + let command_type = envelope.kind().to_string(); let res = StateMachineApplyContext { storage: transaction, record_created_at, @@ -285,7 +291,7 @@ impl StateMachine { experimental_features: &self.experimental_features, is_leader, } - .on_apply(command) + .on_apply(envelope) .await; histogram!(PARTITION_APPLY_COMMAND, "command" => command_type).record(start.elapsed()); res @@ -424,7 +430,7 @@ impl StateMachineApplyContext<'_, S> { }); } - async fn on_apply(&mut self, command: Command) -> Result<(), Error> + async fn on_apply(&mut self, envelope: v2::Envelope) -> Result<(), Error> where S: IdempotencyTable + ReadPromiseTable @@ -445,8 +451,9 @@ impl StateMachineApplyContext<'_, S> { + journal_table_v2::ReadJournalTable + WriteJournalEventsTable, { - match command { - Command::UpdatePartitionDurability(_) => { + match envelope.kind() { + RecordKind::Unknown => Err(Error::UnknownEnvelopeKind), + RecordKind::UpdatePartitionDurability | RecordKind::AnnounceLeader => { // no-op :-) // // This is a partition-level command that doesn't impact the state machine. @@ -454,7 +461,7 @@ impl StateMachineApplyContext<'_, S> { // on_apply() method. Ok(()) } - Command::VersionBarrier(barrier) => { + RecordKind::VersionBarrier => { // We have versions in play: // - Our binary's version (this process) // - `min_restate_version` coming from the FSM @@ -484,6 +491,9 @@ impl StateMachineApplyContext<'_, S> { // that's not blocking this partition if such replacement exists. // - Peers will not pick this node as leader candidate when performing // adhoc failovers. + let barrier = envelope + .into_typed::() + .into_inner()?; if SemanticRestateVersion::current().is_equal_or_newer_than(&barrier.version) { // Feels amazing to be running a new version of restate! lifecycle::OnVersionBarrierCommand { barrier } @@ -497,10 +507,16 @@ impl StateMachineApplyContext<'_, S> { }) } } - Command::Invoke(service_invocation) => { - self.on_service_invocation(service_invocation).await + RecordKind::Invoke => { + let service_invocation = envelope.into_typed::().into_inner()?; + self.on_service_invocation(service_invocation.into()).await } - Command::InvocationResponse(InvocationResponse { target, result }) => { + RecordKind::InvocationResponse => { + let response = envelope + .into_typed::() + .into_inner()?; + + let InvocationResponse { target, result } = response.into(); let status = self.get_invocation_status(&target.caller_id).await?; if should_use_journal_table_v2(&status) { @@ -523,16 +539,36 @@ impl StateMachineApplyContext<'_, S> { self.handle_completion(target.caller_id, status, completion) .await } - Command::ProxyThrough(service_invocation) => { - self.handle_outgoing_message(OutboxMessage::ServiceInvocation(service_invocation))?; + RecordKind::ProxyThrough => { + let proxy_request = envelope + .into_typed::() + .into_inner()?; + self.handle_outgoing_message(OutboxMessage::ServiceInvocation( + proxy_request.invocation.into(), + ))?; Ok(()) } - Command::AttachInvocation(attach_invocation_request) => { - self.handle_attach_invocation_request(attach_invocation_request) + RecordKind::AttachInvocation => { + let attach_invocation_request = envelope + .into_typed::() + .into_inner()?; + self.handle_attach_invocation_request(attach_invocation_request.into()) .await } - Command::InvokerEffect(effect) => self.try_invoker_effect(effect).await, - Command::TruncateOutbox(index) => { + RecordKind::InvokerEffect => { + self.try_invoker_effect( + envelope + .into_typed::() + .into_inner()? + .into(), + ) + .await + } + RecordKind::TruncateOutbox => { + let index = envelope + .into_typed::() + .into_inner()? + .index; self.do_truncate_outbox(RangeInclusive::new( (*self.outbox_head_seq_number).unwrap_or(index), index, @@ -541,11 +577,23 @@ impl StateMachineApplyContext<'_, S> { *self.outbox_head_seq_number = Some(index + 1); Ok(()) } - Command::Timer(timer) => self.on_timer(timer).await, - Command::TerminateInvocation(invocation_termination) => { - self.on_terminate_invocation(invocation_termination).await + RecordKind::Timer => { + self.on_timer(envelope.into_typed::().into_inner()?.into()) + .await } - Command::PurgeInvocation(purge_invocation_request) => { + RecordKind::TerminateInvocation => { + let invocation_termination = envelope + .into_typed::() + .into_inner()?; + self.on_terminate_invocation(invocation_termination.into()) + .await + } + RecordKind::PurgeInvocation => { + let purge_invocation_request: invocation::PurgeInvocationRequest = envelope + .into_typed::() + .into_inner()? + .into(); + lifecycle::OnPurgeCommand { invocation_id: purge_invocation_request.invocation_id, response_sink: purge_invocation_request.response_sink, @@ -554,7 +602,12 @@ impl StateMachineApplyContext<'_, S> { .await?; Ok(()) } - Command::PurgeJournal(purge_invocation_request) => { + RecordKind::PurgeJournal => { + let purge_invocation_request: invocation::PurgeInvocationRequest = envelope + .into_typed::() + .into_inner()? + .into(); + lifecycle::OnPurgeJournalCommand { invocation_id: purge_invocation_request.invocation_id, response_sink: purge_invocation_request.response_sink, @@ -563,7 +616,12 @@ impl StateMachineApplyContext<'_, S> { .await?; Ok(()) } - Command::ResumeInvocation(resume_invocation_request) => { + RecordKind::ResumeInvocation => { + let resume_invocation_request: invocation::ResumeInvocationRequest = envelope + .into_typed::() + .into_inner()? + .into(); + lifecycle::OnManualResumeCommand { invocation_id: resume_invocation_request.invocation_id, update_pinned_deployment_id: resume_invocation_request @@ -574,7 +632,13 @@ impl StateMachineApplyContext<'_, S> { .await?; Ok(()) } - Command::RestartAsNewInvocation(restart_as_new_invocation_request) => { + RecordKind::RestartAsNewInvocation => { + let restart_as_new_invocation_request: invocation::RestartAsNewInvocationRequest = + envelope + .into_typed::() + .into_inner()? + .into(); + lifecycle::OnRestartAsNewInvocationCommand { invocation_id: restart_as_new_invocation_request.invocation_id, new_invocation_id: restart_as_new_invocation_request.new_invocation_id, @@ -587,16 +651,31 @@ impl StateMachineApplyContext<'_, S> { .await?; Ok(()) } - Command::PatchState(mutation) => self.handle_external_state_mutation(mutation).await, - Command::AnnounceLeader(_) => { - // no-op :-) - Ok(()) + RecordKind::PatchState => { + self.handle_external_state_mutation( + envelope + .into_typed::() + .into_inner()? + .into(), + ) + .await } - Command::ScheduleTimer(timer) => { - self.register_timer(timer, Default::default())?; + RecordKind::ScheduleTimer => { + self.register_timer( + envelope + .into_typed::() + .into_inner()? + .into(), + Default::default(), + )?; Ok(()) } - Command::NotifySignal(notify_signal_request) => { + RecordKind::NotifySignal => { + let notify_signal_request: invocation::NotifySignalRequest = envelope + .into_typed::() + .into_inner()? + .into(); + lifecycle::OnNotifySignalCommand { invocation_id: notify_signal_request.invocation_id, invocation_status: self @@ -608,13 +687,21 @@ impl StateMachineApplyContext<'_, S> { .await?; Ok(()) } - Command::NotifyGetInvocationOutputResponse(get_invocation_output_response) => { - lifecycle::OnNotifyGetInvocationOutputResponse(get_invocation_output_response) - .apply(self) - .await?; + RecordKind::NotifyGetInvocationOutputResponse => { + let get_invocation_output_response = envelope + .into_typed::() + .into_inner()?; + lifecycle::OnNotifyGetInvocationOutputResponse( + get_invocation_output_response.into(), + ) + .apply(self) + .await?; Ok(()) } - Command::UpsertSchema(upsert) => { + RecordKind::UpsertSchema => { + let upsert = envelope + .into_typed::() + .into_inner()?; trace!( "Upsert schema record to version '{}'", upsert.schema.version() diff --git a/crates/worker/src/partition/state_machine/tests/delayed_send.rs b/crates/worker/src/partition/state_machine/tests/delayed_send.rs index 4e8b78a029..12deb2d0fd 100644 --- a/crates/worker/src/partition/state_machine/tests/delayed_send.rs +++ b/crates/worker/src/partition/state_machine/tests/delayed_send.rs @@ -7,14 +7,15 @@ // As of the Change Date specified in that file, in accordance with // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0. +use std::time::{Duration, SystemTime}; -use super::*; +use test_log::test; use restate_storage_api::inbox_table::ReadInboxTable; use restate_types::invocation::SubmitNotificationSink; use restate_types::time::MillisSinceEpoch; -use std::time::{Duration, SystemTime}; -use test_log::test; + +use super::*; #[test(restate_core::test)] async fn send_with_delay() { @@ -27,7 +28,7 @@ async fn send_with_delay() { let wake_up_time = MillisSinceEpoch::from(SystemTime::now() + Duration::from_secs(60)); let actions = test_env - .apply(Command::Invoke(Box::new(ServiceInvocation { + .apply(records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id, invocation_target: invocation_target.clone(), response_sink: None, @@ -52,7 +53,7 @@ async fn send_with_delay() { // Now fire the timer let actions = test_env - .apply(Command::Timer(TimerKeyValue::neo_invoke( + .apply(records::Timer::new_test(TimerKeyValue::neo_invoke( wake_up_time, invocation_id, ))) @@ -87,7 +88,7 @@ async fn send_with_delay_to_locked_virtual_object() { let wake_up_time = MillisSinceEpoch::from(SystemTime::now() + Duration::from_secs(60)); let actions = test_env - .apply(Command::Invoke(Box::new(ServiceInvocation { + .apply(records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id, invocation_target: invocation_target.clone(), response_sink: None, @@ -121,7 +122,7 @@ async fn send_with_delay_to_locked_virtual_object() { // Now fire the timer let actions = test_env - .apply(Command::Timer(TimerKeyValue::neo_invoke( + .apply(records::Timer::new_test(TimerKeyValue::neo_invoke( wake_up_time, invocation_id, ))) @@ -172,7 +173,7 @@ async fn send_with_delay_and_idempotency_key() { )); let actions = test_env - .apply(Command::Invoke(Box::new(ServiceInvocation { + .apply(records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id, invocation_target: invocation_target.clone(), idempotency_key: Some(idempotency_key.clone()), @@ -202,7 +203,7 @@ async fn send_with_delay_and_idempotency_key() { // Send another invocation which reattaches to the original one let request_id_2 = PartitionProcessorRpcRequestId::default(); let actions = test_env - .apply(Command::Invoke(Box::new(ServiceInvocation { + .apply(records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id, invocation_target: invocation_target.clone(), idempotency_key: Some(idempotency_key), diff --git a/crates/worker/src/partition/state_machine/tests/fixtures.rs b/crates/worker/src/partition/state_machine/tests/fixtures.rs index c66ef4f58a..949833b4f5 100644 --- a/crates/worker/src/partition/state_machine/tests/fixtures.rs +++ b/crates/worker/src/partition/state_machine/tests/fixtures.rs @@ -7,12 +7,11 @@ // As of the Change Date specified in that file, in accordance with // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0. +use std::collections::HashSet; -use crate::partition::state_machine::Action; -use crate::partition::state_machine::tests::TestEnv; -use crate::partition::types::InvokerEffectKind; use bytes::Bytes; use googletest::prelude::*; + use restate_invoker_api::{Effect, InvokeInputJournal}; use restate_service_protocol_v4::entry_codec::ServiceProtocolV4Codec; use restate_storage_api::journal_table::JournalEntry; @@ -29,8 +28,12 @@ use restate_types::journal::enriched::{ use restate_types::journal_v2; use restate_types::journal_v2::Entry; use restate_types::service_protocol::ServiceProtocolVersion; -use restate_wal_protocol::Command; -use std::collections::HashSet; +use restate_wal_protocol::v2; +use restate_wal_protocol::v2::{Raw, Record, records}; + +use crate::partition::state_machine::Action; +use crate::partition::state_machine::tests::TestEnv; +use crate::partition::types::InvokerEffectKind; pub fn completed_invoke_entry(invocation_id: InvocationId) -> JournalEntry { JournalEntry::Entry(EnrichedRawEntry::new( @@ -76,7 +79,10 @@ pub fn incomplete_invoke_entry(invocation_id: InvocationId) -> JournalEntry { )) } -pub fn invoker_entry_effect(invocation_id: InvocationId, entry: impl Into) -> Command { +pub fn invoker_entry_effect( + invocation_id: InvocationId, + entry: impl Into, +) -> v2::Envelope { invoker_entry_effect_for_epoch(invocation_id, 0, entry) } @@ -84,8 +90,8 @@ pub fn invoker_entry_effect_for_epoch( invocation_id: InvocationId, invocation_epoch: InvocationEpoch, entry: impl Into, -) -> Command { - Command::InvokerEffect(Box::new(Effect { +) -> v2::Envelope { + records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch, kind: InvokerEffectKind::journal_entry( @@ -95,15 +101,15 @@ pub fn invoker_entry_effect_for_epoch( })) } -pub fn invoker_end_effect(invocation_id: InvocationId) -> Command { +pub fn invoker_end_effect(invocation_id: InvocationId) -> v2::Envelope { invoker_end_effect_for_epoch(invocation_id, 0) } pub fn invoker_end_effect_for_epoch( invocation_id: InvocationId, invocation_epoch: InvocationEpoch, -) -> Command { - Command::InvokerEffect(Box::new(Effect { +) -> v2::Envelope { + records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch, kind: InvokerEffectKind::End, @@ -113,8 +119,8 @@ pub fn invoker_end_effect_for_epoch( pub fn pinned_deployment( invocation_id: InvocationId, service_protocol_version: ServiceProtocolVersion, -) -> Command { - Command::InvokerEffect(Box::new(Effect { +) -> v2::Envelope { + records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::PinnedDeployment(PinnedDeployment { @@ -127,8 +133,8 @@ pub fn pinned_deployment( pub fn invoker_suspended( invocation_id: InvocationId, waiting_for_notifications: impl Into>, -) -> Command { - Command::InvokerEffect(Box::new(Effect { +) -> v2::Envelope { + records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::SuspendedV2 { @@ -155,11 +161,13 @@ pub async fn mock_start_invocation_with_invocation_target( let invocation_id = InvocationId::mock_generate(&invocation_target); let actions = state_machine - .apply(Command::Invoke(Box::new(ServiceInvocation::initialize( - invocation_id, - invocation_target.clone(), - Source::Ingress(PartitionProcessorRpcRequestId::new()), - )))) + .apply(records::Invoke::new_test(Box::new( + ServiceInvocation::initialize( + invocation_id, + invocation_target.clone(), + Source::Ingress(PartitionProcessorRpcRequestId::new()), + ), + ))) .await; assert_that!( diff --git a/crates/worker/src/partition/state_machine/tests/idempotency.rs b/crates/worker/src/partition/state_machine/tests/idempotency.rs index a56f1c1a35..641561880f 100644 --- a/crates/worker/src/partition/state_machine/tests/idempotency.rs +++ b/crates/worker/src/partition/state_machine/tests/idempotency.rs @@ -7,10 +7,10 @@ // As of the Change Date specified in that file, in accordance with // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0. +use std::time::Duration; -use super::*; +use rstest::*; -use crate::partition::state_machine::tests::matchers::actions::invocation_response_to_partition_processor; use restate_invoker_api::Effect; use restate_storage_api::idempotency_table::{ IdempotencyMetadata, IdempotencyTable, ReadOnlyIdempotencyTable, @@ -24,8 +24,9 @@ use restate_types::invocation::{ AttachInvocationRequest, InvocationQuery, InvocationTarget, PurgeInvocationRequest, SubmitNotificationSink, }; -use rstest::*; -use std::time::Duration; + +use super::*; +use crate::partition::state_machine::tests::matchers::actions::invocation_response_to_partition_processor; #[restate_core::test] async fn start_and_complete_idempotent_invocation() { @@ -41,7 +42,7 @@ async fn start_and_complete_idempotent_invocation() { // Send fresh invocation with idempotency key let actions = test_env - .apply(Command::Invoke(Box::new(ServiceInvocation { + .apply(records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id, invocation_target: invocation_target.clone(), response_sink: Some(ServiceInvocationResponseSink::Ingress { request_id }), @@ -72,7 +73,7 @@ async fn start_and_complete_idempotent_invocation() { let response_bytes = Bytes::from_static(b"123"); let actions = test_env .apply_multiple([ - Command::InvokerEffect(Box::new(Effect { + records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::JournalEntry { @@ -82,7 +83,7 @@ async fn start_and_complete_idempotent_invocation() { )), }, })), - Command::InvokerEffect(Box::new(Effect { + records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::End, @@ -132,7 +133,7 @@ async fn start_and_complete_idempotent_invocation_neo_table() { // Send fresh invocation with idempotency key let actions = test_env - .apply(Command::Invoke(Box::new(ServiceInvocation { + .apply(records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id, invocation_target: invocation_target.clone(), response_sink: Some(ServiceInvocationResponseSink::Ingress { request_id }), @@ -163,7 +164,7 @@ async fn start_and_complete_idempotent_invocation_neo_table() { let response_bytes = Bytes::from_static(b"123"); let actions = test_env .apply_multiple([ - Command::InvokerEffect(Box::new(Effect { + records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::JournalEntry { @@ -173,7 +174,7 @@ async fn start_and_complete_idempotent_invocation_neo_table() { )), }, })), - Command::InvokerEffect(Box::new(Effect { + records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::End, @@ -258,7 +259,7 @@ async fn complete_already_completed_invocation() { // Send a request, should be completed immediately with result let request_id = PartitionProcessorRpcRequestId::default(); let actions = test_env - .apply(Command::Invoke(Box::new(ServiceInvocation { + .apply(records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id, invocation_target: invocation_target.clone(), response_sink: Some(ServiceInvocationResponseSink::Ingress { request_id }), @@ -294,7 +295,7 @@ async fn attach_with_service_invocation_command_while_executing() { // Send fresh invocation with idempotency key let actions = test_env - .apply(Command::Invoke(Box::new(ServiceInvocation { + .apply(records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id, invocation_target: invocation_target.clone(), response_sink: Some(ServiceInvocationResponseSink::Ingress { @@ -315,7 +316,7 @@ async fn attach_with_service_invocation_command_while_executing() { // Latch to existing invocation let actions = test_env - .apply(Command::Invoke(Box::new(ServiceInvocation { + .apply(records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id, invocation_target: invocation_target.clone(), response_sink: Some(ServiceInvocationResponseSink::Ingress { @@ -331,7 +332,7 @@ async fn attach_with_service_invocation_command_while_executing() { let response_bytes = Bytes::from_static(b"123"); let actions = test_env .apply_multiple([ - Command::InvokerEffect(Box::new(Effect { + records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::JournalEntry { @@ -341,7 +342,7 @@ async fn attach_with_service_invocation_command_while_executing() { )), }, })), - Command::InvokerEffect(Box::new(Effect { + records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::End, @@ -397,7 +398,7 @@ async fn attach_with_send_service_invocation(#[case] use_same_request_id: bool) // Send fresh invocation with idempotency key let actions = test_env - .apply(Command::Invoke(Box::new(ServiceInvocation { + .apply(records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id, invocation_target: invocation_target.clone(), response_sink: Some(ServiceInvocationResponseSink::Ingress { @@ -419,7 +420,7 @@ async fn attach_with_send_service_invocation(#[case] use_same_request_id: bool) // Latch to existing invocation, but with a send call let actions = test_env - .apply(Command::Invoke(Box::new(ServiceInvocation { + .apply(records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id, invocation_target: invocation_target.clone(), idempotency_key: Some(idempotency_key.clone()), @@ -447,7 +448,7 @@ async fn attach_with_send_service_invocation(#[case] use_same_request_id: bool) let response_bytes = Bytes::from_static(b"123"); let actions = test_env .apply_multiple([ - Command::InvokerEffect(Box::new(Effect { + records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::JournalEntry { @@ -457,7 +458,7 @@ async fn attach_with_send_service_invocation(#[case] use_same_request_id: bool) )), }, })), - Command::InvokerEffect(Box::new(Effect { + records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::End, @@ -523,7 +524,7 @@ async fn attach_inboxed_with_send_service_invocation() { let idempotency_key = ByteString::from_static("my-idempotency-key"); let invocation_id = InvocationId::generate(&invocation_target, Some(&idempotency_key)); let actions = test_env - .apply(Command::Invoke(Box::new(ServiceInvocation { + .apply(records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id, invocation_target: invocation_target.clone(), idempotency_key: Some(idempotency_key.clone()), @@ -566,7 +567,7 @@ async fn attach_inboxed_with_send_service_invocation() { // Now send the request that should get the submit notification let idempotency_key = ByteString::from_static("my-idempotency-key"); let actions = test_env - .apply(Command::Invoke(Box::new(ServiceInvocation { + .apply(records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id, invocation_target: invocation_target.clone(), idempotency_key: Some(idempotency_key.clone()), @@ -608,7 +609,7 @@ async fn attach_command() { // Send fresh invocation with idempotency key let actions = test_env - .apply(Command::Invoke(Box::new(ServiceInvocation { + .apply(records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id, invocation_target: invocation_target.clone(), response_sink: Some(ServiceInvocationResponseSink::Ingress { @@ -629,13 +630,15 @@ async fn attach_command() { // Latch to existing invocation, but with a send call let actions = test_env - .apply(Command::AttachInvocation(AttachInvocationRequest { - invocation_query: InvocationQuery::Invocation(invocation_id), - block_on_inflight: true, - response_sink: ServiceInvocationResponseSink::Ingress { - request_id: request_id_2, + .apply(records::AttachInvocation::new_test( + AttachInvocationRequest { + invocation_query: InvocationQuery::Invocation(invocation_id), + block_on_inflight: true, + response_sink: ServiceInvocationResponseSink::Ingress { + request_id: request_id_2, + }, }, - })) + )) .await; assert_that!( actions, @@ -646,7 +649,7 @@ async fn attach_command() { let response_bytes = Bytes::from_static(b"123"); let actions = test_env .apply_multiple([ - Command::InvokerEffect(Box::new(Effect { + records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::JournalEntry { @@ -656,7 +659,7 @@ async fn attach_command() { )), }, })), - Command::InvokerEffect(Box::new(Effect { + records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::End, @@ -700,7 +703,7 @@ async fn attach_command_without_blocking_inflight() { // Send fresh invocation with idempotency key let actions = test_env - .apply(Command::Invoke(Box::new(ServiceInvocation { + .apply(records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id, invocation_target: invocation_target.clone(), response_sink: Some(ServiceInvocationResponseSink::Ingress { @@ -722,13 +725,15 @@ async fn attach_command_without_blocking_inflight() { // Latch to existing invocation without blocking on inflight invocation let caller_invocation_id = InvocationId::mock_random(); let actions = test_env - .apply(Command::AttachInvocation(AttachInvocationRequest { - invocation_query: InvocationQuery::Invocation(invocation_id), - block_on_inflight: false, - response_sink: ServiceInvocationResponseSink::PartitionProcessor( - JournalCompletionTarget::from_parts(caller_invocation_id, 1, 0), - ), - })) + .apply(records::AttachInvocation::new_test( + AttachInvocationRequest { + invocation_query: InvocationQuery::Invocation(invocation_id), + block_on_inflight: false, + response_sink: ServiceInvocationResponseSink::PartitionProcessor( + JournalCompletionTarget::from_parts(caller_invocation_id, 1, 0), + ), + }, + )) .await; assert_that!( actions, @@ -773,7 +778,7 @@ async fn purge_completed_idempotent_invocation() { // Send purge command let _ = test_env - .apply(Command::PurgeInvocation(PurgeInvocationRequest { + .apply(records::PurgeInvocation::new_test(PurgeInvocationRequest { invocation_id, response_sink: None, })) diff --git a/crates/worker/src/partition/state_machine/tests/invocation_epoch_awareness.rs b/crates/worker/src/partition/state_machine/tests/invocation_epoch_awareness.rs index 43e4f0f5f6..d2d17c21bb 100644 --- a/crates/worker/src/partition/state_machine/tests/invocation_epoch_awareness.rs +++ b/crates/worker/src/partition/state_machine/tests/invocation_epoch_awareness.rs @@ -8,12 +8,12 @@ // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0. -use super::*; - use restate_types::journal_v2::{ CallCommand, CallCompletion, CallRequest, SleepCommand, SleepCompletion, raw::TryFromEntry, }; +use super::*; + #[restate_core::test] async fn fence_old_calls_and_completions() { let mut test_env = TestEnv::create().await; @@ -129,7 +129,7 @@ async fn fence_old_calls_and_completions() { // Completion with epoch 0 gets ignored let actions = test_env - .apply(Command::InvocationResponse(InvocationResponse { + .apply(records::InvocationResponse::new_test(InvocationResponse { target: JournalCompletionTarget::from_parts(invocation_id, 2, 0), result: ResponseResult::Success(Bytes::default()), })) @@ -143,7 +143,7 @@ async fn fence_old_calls_and_completions() { // Completion with epoch 1 gets accepted let actions = test_env - .apply(Command::InvocationResponse(InvocationResponse { + .apply(records::InvocationResponse::new_test(InvocationResponse { target: JournalCompletionTarget::from_parts(invocation_id, 2, 1), result: ResponseResult::Success(Bytes::default()), })) @@ -288,12 +288,9 @@ async fn fence_old_sleep_and_completions() { // Completion with epoch 0 gets ignored let actions = test_env - .apply(Command::Timer(TimerKeyValue::complete_journal_entry( - wake_up_time, - invocation_id, - 1, - 0, - ))) + .apply(records::Timer::new_test( + TimerKeyValue::complete_journal_entry(wake_up_time, invocation_id, 1, 0), + )) .await; assert_that!( actions, @@ -309,12 +306,9 @@ async fn fence_old_sleep_and_completions() { // Completion with epoch 1 gets accepted let actions = test_env - .apply(Command::Timer(TimerKeyValue::complete_journal_entry( - wake_up_time, - invocation_id, - 1, - 1, - ))) + .apply(records::Timer::new_test( + TimerKeyValue::complete_journal_entry(wake_up_time, invocation_id, 1, 1), + )) .await; assert_that!( actions, diff --git a/crates/worker/src/partition/state_machine/tests/kill_cancel.rs b/crates/worker/src/partition/state_machine/tests/kill_cancel.rs index 97312d2125..d155f1f6d7 100644 --- a/crates/worker/src/partition/state_machine/tests/kill_cancel.rs +++ b/crates/worker/src/partition/state_machine/tests/kill_cancel.rs @@ -8,12 +8,13 @@ // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0. -use super::{fixtures, matchers, *}; - use assert2::assert; use assert2::let_assert; use googletest::any; use prost::Message; +use rstest::rstest; +use test_log::test; + use restate_storage_api::journal_table::WriteJournalTable; use restate_storage_api::timer_table::{ ReadTimerTable, Timer, TimerKey, TimerKeyKind, WriteTimerTable, @@ -24,8 +25,8 @@ use restate_types::invocation::{IngressInvocationResponseSink, TerminationFlavor use restate_types::journal::enriched::EnrichedEntryHeader; use restate_types::journal_v2::NotificationId; use restate_types::service_protocol; -use rstest::rstest; -use test_log::test; + +use super::{fixtures, matchers, *}; #[restate_core::test] async fn kill_inboxed_invocation() -> anyhow::Result<()> { @@ -40,7 +41,7 @@ async fn kill_inboxed_invocation() -> anyhow::Result<()> { let caller_id = InvocationId::mock_random(); let _ = test_env - .apply(Command::Invoke(Box::new(ServiceInvocation { + .apply(records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id, invocation_target: invocation_target.clone(), ..ServiceInvocation::mock() @@ -48,7 +49,7 @@ async fn kill_inboxed_invocation() -> anyhow::Result<()> { .await; let _ = test_env - .apply(Command::Invoke(Box::new(ServiceInvocation { + .apply(records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id: inboxed_id, invocation_target: inboxed_target, response_sink: Some(ServiceInvocationResponseSink::PartitionProcessor( @@ -68,13 +69,15 @@ async fn kill_inboxed_invocation() -> anyhow::Result<()> { let request_id = PartitionProcessorRpcRequestId::new(); let actions = test_env - .apply(Command::TerminateInvocation(InvocationTermination { - invocation_id: inboxed_id, - flavor: TerminationFlavor::Kill, - response_sink: Some(InvocationMutationResponseSink::Ingress( - IngressInvocationResponseSink { request_id }, - )), - })) + .apply(records::TerminateInvocation::new_test( + InvocationTermination { + invocation_id: inboxed_id, + flavor: TerminationFlavor::Kill, + response_sink: Some(InvocationMutationResponseSink::Ingress( + IngressInvocationResponseSink { request_id }, + )), + }, + )) .await; let current_invocation_status = test_env @@ -133,7 +136,7 @@ async fn terminate_scheduled_invocation( let rpc_id = PartitionProcessorRpcRequestId::new(); let _ = test_env - .apply(Command::Invoke(Box::new(ServiceInvocation { + .apply(records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id, execution_time: Some(MillisSinceEpoch::MAX), response_sink: Some(ServiceInvocationResponseSink::ingress(rpc_id)), @@ -149,11 +152,13 @@ async fn terminate_scheduled_invocation( assert!(let InvocationStatus::Scheduled(_) = current_invocation_status); let actions = test_env - .apply(Command::TerminateInvocation(InvocationTermination { - invocation_id, - flavor: termination_flavor, - response_sink: None, - })) + .apply(records::TerminateInvocation::new_test( + InvocationTermination { + invocation_id, + flavor: termination_flavor, + response_sink: None, + }, + )) .await; assert_that!( actions, @@ -193,7 +198,7 @@ async fn kill_call_tree() -> anyhow::Result<()> { let enqueued_invocation_id_on_same_target = InvocationId::mock_generate(&invocation_target); let _ = test_env - .apply(Command::Invoke(Box::new(ServiceInvocation { + .apply(records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id, invocation_target: invocation_target.clone(), ..ServiceInvocation::mock() @@ -202,7 +207,7 @@ async fn kill_call_tree() -> anyhow::Result<()> { // Let's enqueue an invocation afterward let _ = test_env - .apply(Command::Invoke(Box::new(ServiceInvocation { + .apply(records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id: enqueued_invocation_id_on_same_target, invocation_target: invocation_target.clone(), ..ServiceInvocation::mock() @@ -233,11 +238,13 @@ async fn kill_call_tree() -> anyhow::Result<()> { // Now let's send the termination command let actions = test_env - .apply(Command::TerminateInvocation(InvocationTermination { - invocation_id, - flavor: TerminationFlavor::Kill, - response_sink: None, - })) + .apply(records::TerminateInvocation::new_test( + InvocationTermination { + invocation_id, + flavor: TerminationFlavor::Kill, + response_sink: None, + }, + )) .await; assert_that!( @@ -309,12 +316,12 @@ async fn cancel_invoked_invocation() -> Result<(), Error> { let _ = test_env .apply_multiple([ - Command::Invoke(Box::new(ServiceInvocation { + records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id, invocation_target: invocation_target.clone(), ..ServiceInvocation::mock() })), - Command::InvokerEffect(Box::new(Effect { + records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::PinnedDeployment(PinnedDeployment { @@ -366,11 +373,13 @@ async fn cancel_invoked_invocation() -> Result<(), Error> { tx.commit().await?; let actions = test_env - .apply(Command::TerminateInvocation(InvocationTermination { - invocation_id, - flavor: TerminationFlavor::Cancel, - response_sink: None, - })) + .apply(records::TerminateInvocation::new_test( + InvocationTermination { + invocation_id, + flavor: TerminationFlavor::Cancel, + response_sink: None, + }, + )) .await; // Invocation shouldn't be gone @@ -432,12 +441,12 @@ async fn cancel_suspended_invocation() -> Result<(), Error> { let _ = test_env .apply_multiple([ - Command::Invoke(Box::new(ServiceInvocation { + records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id, invocation_target: invocation_target.clone(), ..ServiceInvocation::mock() })), - Command::InvokerEffect(Box::new(Effect { + records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::PinnedDeployment(PinnedDeployment { @@ -504,13 +513,15 @@ async fn cancel_suspended_invocation() -> Result<(), Error> { let request_id = PartitionProcessorRpcRequestId::new(); let actions = test_env - .apply(Command::TerminateInvocation(InvocationTermination { - invocation_id, - flavor: TerminationFlavor::Cancel, - response_sink: Some(InvocationMutationResponseSink::Ingress( - IngressInvocationResponseSink { request_id }, - )), - })) + .apply(records::TerminateInvocation::new_test( + InvocationTermination { + invocation_id, + flavor: TerminationFlavor::Cancel, + response_sink: Some(InvocationMutationResponseSink::Ingress( + IngressInvocationResponseSink { request_id }, + )), + }, + )) .await; // Invocation shouldn't be gone @@ -572,7 +583,7 @@ async fn cancel_invocation_entry_referring_to_previous_entry() { let callee_2 = InvocationId::mock_random(); let _ = test_env - .apply(Command::Invoke(Box::new(ServiceInvocation { + .apply(records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id, invocation_target: invocation_target.clone(), ..ServiceInvocation::mock() @@ -602,7 +613,7 @@ async fn cancel_invocation_entry_referring_to_previous_entry() { // Now create cancel invocation entry let actions = test_env .apply_multiple(vec![ - Command::InvokerEffect(Box::new(Effect { + records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::JournalEntry { @@ -612,7 +623,7 @@ async fn cancel_invocation_entry_referring_to_previous_entry() { )), }, })), - Command::InvokerEffect(Box::new(Effect { + records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::JournalEntry { diff --git a/crates/worker/src/partition/state_machine/tests/mod.rs b/crates/worker/src/partition/state_machine/tests/mod.rs index 31c3cb59e4..30219e6186 100644 --- a/crates/worker/src/partition/state_machine/tests/mod.rs +++ b/crates/worker/src/partition/state_machine/tests/mod.rs @@ -8,27 +8,16 @@ // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0. -use super::*; - -mod delayed_send; -pub mod fixtures; -mod idempotency; -mod invocation_epoch_awareness; -mod kill_cancel; -pub mod matchers; -mod workflow; +use std::collections::{HashMap, HashSet}; -use crate::partition::state_machine::tests::fixtures::{ - background_invoke_entry, incomplete_invoke_entry, -}; -use crate::partition::state_machine::tests::matchers::storage::is_entry; -use crate::partition::state_machine::tests::matchers::success_completion; -use crate::partition::types::InvokerEffectKind; use ::tracing::info; use bytes::Bytes; use bytestring::ByteString; use futures::{StreamExt, TryStreamExt}; use googletest::{all, assert_that, pat, property}; +use test_log::test; +use tracing_subscriber::fmt::format::FmtSpan; + use restate_core::TaskCenter; use restate_invoker_api::{Effect, EffectKind, InvokeInputJournal}; use restate_partition_store::{PartitionStore, PartitionStoreManager}; @@ -65,12 +54,26 @@ use restate_types::journal::{ use restate_types::journal::{Entry, EntryType}; use restate_types::journal_events::Event; use restate_types::journal_v2::raw::TryFromEntry; -use restate_types::logs::SequenceNumber; +use restate_types::logs::{Keys, SequenceNumber}; use restate_types::partitions::Partition; use restate_types::state_mut::ExternalStateMutation; -use std::collections::{HashMap, HashSet}; -use test_log::test; -use tracing_subscriber::fmt::format::FmtSpan; +use restate_wal_protocol::v2::Record; + +use super::*; +use crate::partition::state_machine::tests::fixtures::{ + background_invoke_entry, incomplete_invoke_entry, +}; +use crate::partition::state_machine::tests::matchers::storage::is_entry; +use crate::partition::state_machine::tests::matchers::success_completion; +use crate::partition::types::InvokerEffectKind; + +mod delayed_send; +pub mod fixtures; +mod idempotency; +mod invocation_epoch_awareness; +mod kill_cancel; +pub mod matchers; +mod workflow; pub struct TestEnv { pub state_machine: StateMachine, @@ -151,12 +154,12 @@ impl TestEnv { } } - pub async fn apply(&mut self, command: Command) -> Vec { + pub async fn apply(&mut self, envelope: v2::Envelope) -> Vec { let mut transaction = self.storage.transaction(); let mut action_collector = ActionCollector::default(); self.state_machine .apply( - command, + envelope, MillisSinceEpoch::now(), Lsn::OLDEST, &mut transaction, @@ -171,12 +174,15 @@ impl TestEnv { action_collector } - pub async fn apply_fallible(&mut self, command: Command) -> Result, Error> { + pub async fn apply_fallible( + &mut self, + envelope: v2::Envelope, + ) -> Result, Error> { let mut transaction = self.storage.transaction(); let mut action_collector = ActionCollector::default(); self.state_machine .apply( - command, + envelope, MillisSinceEpoch::now(), Lsn::OLDEST, &mut transaction, @@ -192,11 +198,11 @@ impl TestEnv { pub async fn apply_multiple( &mut self, - commands: impl IntoIterator, + envelops: impl IntoIterator>, ) -> Vec { let mut actions = vec![]; - for command in commands { - actions.append(&mut self.apply(command).await) + for envelope in envelops { + actions.append(&mut self.apply(envelope).await) } actions } @@ -373,10 +379,10 @@ async fn shared_invocation_skips_inbox() -> TestResult { async fn awakeable_completion_received_before_entry() -> TestResult { let mut test_env = TestEnv::create().await; let invocation_id = fixtures::mock_start_invocation(&mut test_env).await; - // Send completion first + let _ = test_env - .apply(Command::InvocationResponse(InvocationResponse { + .apply(records::InvocationResponse::new_test(InvocationResponse { target: JournalCompletionTarget::from_parts(invocation_id, 1, 0), result: ResponseResult::Success(Bytes::default()), })) @@ -397,7 +403,7 @@ async fn awakeable_completion_received_before_entry() -> TestResult { // * If the awakeable entry has not been received yet, when receiving it the completion will be sent through. let actions = test_env - .apply(Command::InvokerEffect(Box::new(Effect { + .apply(records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::JournalEntry { @@ -437,7 +443,7 @@ async fn awakeable_completion_received_before_entry() -> TestResult { // If we try to send the completion again, it should not be forwarded! let actions = test_env - .apply(Command::InvocationResponse(InvocationResponse { + .apply(records::InvocationResponse::new_test(InvocationResponse { target: JournalCompletionTarget::from_parts(invocation_id, 1, 0), result: ResponseResult::Success(Bytes::default()), })) @@ -454,7 +460,7 @@ async fn awakeable_completion_received_before_entry() -> TestResult { ); let actions = test_env - .apply(Command::InvokerEffect(Box::new(Effect { + .apply(records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::Suspended { @@ -490,7 +496,7 @@ async fn complete_awakeable_with_success() { )); let actions = test_env - .apply(Command::InvokerEffect(Box::new(Effect { + .apply(records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: EffectKind::JournalEntry { @@ -536,7 +542,7 @@ async fn complete_awakeable_with_failure() { )); let actions = test_env - .apply(Command::InvokerEffect(Box::new(Effect { + .apply(records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: EffectKind::JournalEntry { @@ -576,7 +582,7 @@ async fn invoke_with_headers() -> TestResult { fixtures::mock_start_invocation_with_service_id(&mut test_env, service_id.clone()).await; let actions = test_env - .apply(Command::InvokerEffect(Box::new(Effect { + .apply(records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::JournalEntry { @@ -647,14 +653,14 @@ async fn mutate_state() -> anyhow::Result<()> { ); test_env - .apply(Command::PatchState(ExternalStateMutation { + .apply(records::PatchState::new_test(ExternalStateMutation { service_id: keyed_service_id.clone(), version: None, state: first_state_mutation, })) .await; test_env - .apply(Command::PatchState(ExternalStateMutation { + .apply(records::PatchState::new_test(ExternalStateMutation { service_id: keyed_service_id.clone(), version: None, state: second_state_mutation.clone(), @@ -664,7 +670,7 @@ async fn mutate_state() -> anyhow::Result<()> { // terminating the ongoing invocation should trigger popping from the inbox until the // next invocation is found test_env - .apply(Command::InvokerEffect(Box::new(Effect { + .apply(records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::End, @@ -699,7 +705,7 @@ async fn clear_all_user_states() -> anyhow::Result<()> { fixtures::mock_start_invocation_with_service_id(&mut test_env, service_id.clone()).await; test_env - .apply(Command::InvokerEffect(Box::new(Effect { + .apply(records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::JournalEntry { @@ -735,7 +741,7 @@ async fn get_state_keys() -> TestResult { txn.commit().await.unwrap(); let actions = test_env - .apply(Command::InvokerEffect(Box::new(Effect { + .apply(records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::JournalEntry { @@ -786,7 +792,7 @@ async fn get_invocation_id_entry() { let actions = test_env .apply_multiple(vec![ - Command::InvokerEffect(Box::new(Effect { + records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::JournalEntry { @@ -796,7 +802,7 @@ async fn get_invocation_id_entry() { ), }, })), - Command::InvokerEffect(Box::new(Effect { + records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::JournalEntry { @@ -862,7 +868,7 @@ async fn attach_invocation_entry() { let callee_invocation_id = InvocationId::mock_random(); let actions = test_env - .apply(Command::InvokerEffect(Box::new(Effect { + .apply(records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: EffectKind::JournalEntry { @@ -908,7 +914,7 @@ async fn get_invocation_output_entry() { let callee_invocation_id = InvocationId::mock_random(); let actions = test_env - .apply(Command::InvokerEffect(Box::new(Effect { + .apply(records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: EffectKind::JournalEntry { @@ -946,7 +952,7 @@ async fn get_invocation_output_entry() { // Let's try to complete it with not ready, this should forward empty let actions = test_env - .apply(Command::InvocationResponse(InvocationResponse { + .apply(records::InvocationResponse::new_test(InvocationResponse { target: JournalCompletionTarget::from_parts(invocation_id, 1, 0), result: NOT_READY_INVOCATION_ERROR.into(), })) @@ -973,7 +979,7 @@ async fn send_ingress_response_to_multiple_targets() -> TestResult { let request_id_3 = PartitionProcessorRpcRequestId::default(); let actions = test_env - .apply(Command::Invoke(Box::new(ServiceInvocation { + .apply(records::Invoke::new_test(Box::new(ServiceInvocation { response_sink: Some(ServiceInvocationResponseSink::Ingress { request_id: request_id_1, }), @@ -1011,7 +1017,7 @@ async fn send_ingress_response_to_multiple_targets() -> TestResult { // Now let's send the output entry let response_bytes = Bytes::from_static(b"123"); let actions = test_env - .apply(Command::InvokerEffect(Box::new(Effect { + .apply(records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::JournalEntry { @@ -1027,7 +1033,7 @@ async fn send_ingress_response_to_multiple_targets() -> TestResult { // Send the End Effect let actions = test_env - .apply(Command::InvokerEffect(Box::new(Effect { + .apply(records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::End, @@ -1072,7 +1078,14 @@ async fn truncate_outbox_from_empty() -> Result<(), Error> { let mut test_env = TestEnv::create().await; - let _ = test_env.apply(Command::TruncateOutbox(outbox_index)).await; + let _ = test_env + .apply(records::TruncateOutbox::new_test( + records::TruncateOutboxRequest { + index: outbox_index, + partition_key_range: Keys::RangeInclusive(0..=u64::MAX), + }, + )) + .await; assert_that!(test_env.storage.get_outbox_message(0).await?, none()); @@ -1105,7 +1118,12 @@ async fn truncate_outbox_with_gap() -> Result<(), Error> { .await; test_env - .apply(Command::TruncateOutbox(outbox_tail_index)) + .apply(records::TruncateOutbox::new_test( + records::TruncateOutboxRequest { + index: outbox_tail_index, + partition_key_range: Keys::RangeInclusive(0..=u64::MAX), + }, + )) .await; assert_that!(test_env.storage.get_outbox_message(3).await?, none()); @@ -1132,7 +1150,7 @@ async fn consecutive_exclusive_handler_invocations_will_use_inbox() -> TestResul // Let's start the first invocation let actions = test_env - .apply(Command::Invoke(Box::new(ServiceInvocation { + .apply(records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id: first_invocation_id, invocation_target: invocation_target.clone(), ..ServiceInvocation::mock() @@ -1152,7 +1170,7 @@ async fn consecutive_exclusive_handler_invocations_will_use_inbox() -> TestResul // Let's start the second invocation let actions = test_env - .apply(Command::Invoke(Box::new(ServiceInvocation { + .apply(records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id: second_invocation_id, invocation_target: invocation_target.clone(), ..ServiceInvocation::mock() @@ -1188,7 +1206,7 @@ async fn consecutive_exclusive_handler_invocations_will_use_inbox() -> TestResul // Send the End Effect to terminate the first invocation let actions = test_env - .apply(Command::InvokerEffect(Box::new(Effect { + .apply(records::InvokerEffect::new_test(Box::new(Effect { invocation_id: first_invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::End, @@ -1208,7 +1226,7 @@ async fn consecutive_exclusive_handler_invocations_will_use_inbox() -> TestResul ); let _ = test_env - .apply(Command::InvokerEffect(Box::new(Effect { + .apply(records::InvokerEffect::new_test(Box::new(Effect { invocation_id: second_invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::End, @@ -1252,7 +1270,7 @@ async fn deduplicate_requests_with_same_pp_rpc_request_id() -> TestResult { Box::new(si) }; let actions = test_env - .apply(Command::Invoke(service_invocation.clone())) + .apply(records::Invoke::new_test(service_invocation.clone())) .await; assert_that!( actions, @@ -1269,7 +1287,9 @@ async fn deduplicate_requests_with_same_pp_rpc_request_id() -> TestResult { // Applying this again won't generate Invoke action, // but will return same submit notification. - let actions = test_env.apply(Command::Invoke(service_invocation)).await; + let actions = test_env + .apply(records::Invoke::new_test(service_invocation)) + .await; assert_that!( actions, all!( diff --git a/crates/worker/src/partition/state_machine/tests/workflow.rs b/crates/worker/src/partition/state_machine/tests/workflow.rs index ec5df124f4..d7ed2b194c 100644 --- a/crates/worker/src/partition/state_machine/tests/workflow.rs +++ b/crates/worker/src/partition/state_machine/tests/workflow.rs @@ -8,9 +8,8 @@ // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0. -use super::*; +use std::time::Duration; -use crate::partition::state_machine::tests::matchers::actions::forward_purge_invocation_response; use restate_storage_api::invocation_status_table::CompletedInvocation; use restate_storage_api::service_status_table::ReadVirtualObjectStatusTable; use restate_types::errors::WORKFLOW_ALREADY_INVOKED_INVOCATION_ERROR; @@ -18,7 +17,9 @@ use restate_types::invocation::{ AttachInvocationRequest, IngressInvocationResponseSink, InvocationQuery, InvocationTarget, PurgeInvocationRequest, }; -use std::time::Duration; + +use super::*; +use crate::partition::state_machine::tests::matchers::actions::forward_purge_invocation_response; #[restate_core::test] async fn start_workflow_method() { @@ -31,7 +32,7 @@ async fn start_workflow_method() { // Send fresh invocation let actions = test_env - .apply(Command::Invoke(Box::new(ServiceInvocation { + .apply(records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id, invocation_target: invocation_target.clone(), completion_retention_duration: Duration::from_secs(60), @@ -61,7 +62,7 @@ async fn start_workflow_method() { // Sending another invocation won't re-execute let actions = test_env - .apply(Command::Invoke(Box::new(ServiceInvocation { + .apply(records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id, invocation_target: invocation_target.clone(), response_sink: Some(ServiceInvocationResponseSink::Ingress { @@ -92,7 +93,7 @@ async fn start_workflow_method() { let response_bytes = Bytes::from_static(b"123"); let actions = test_env .apply_multiple([ - Command::InvokerEffect(Box::new(Effect { + records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::JournalEntry { @@ -102,7 +103,7 @@ async fn start_workflow_method() { )), }, })), - Command::InvokerEffect(Box::new(Effect { + records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::End, @@ -150,7 +151,7 @@ async fn start_workflow_method() { // Sending a new request will not be completed because we don't support attach semantics let request_id_3 = PartitionProcessorRpcRequestId::default(); let actions = test_env - .apply(Command::Invoke(Box::new(ServiceInvocation { + .apply(records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id, invocation_target: invocation_target.clone(), response_sink: Some(ServiceInvocationResponseSink::Ingress { @@ -184,7 +185,7 @@ async fn attach_by_workflow_key() { // Send fresh invocation let actions = test_env - .apply(Command::Invoke(Box::new(ServiceInvocation { + .apply(records::Invoke::new_test(Box::new(ServiceInvocation { invocation_id, invocation_target: invocation_target.clone(), completion_retention_duration: Duration::from_secs(60), @@ -204,15 +205,17 @@ async fn attach_by_workflow_key() { // Sending another invocation won't re-execute let actions = test_env - .apply(Command::AttachInvocation(AttachInvocationRequest { - invocation_query: InvocationQuery::Workflow( - invocation_target.as_keyed_service_id().unwrap(), - ), - block_on_inflight: true, - response_sink: ServiceInvocationResponseSink::Ingress { - request_id: request_id_2, + .apply(records::AttachInvocation::new_test( + AttachInvocationRequest { + invocation_query: InvocationQuery::Workflow( + invocation_target.as_keyed_service_id().unwrap(), + ), + block_on_inflight: true, + response_sink: ServiceInvocationResponseSink::Ingress { + request_id: request_id_2, + }, }, - })) + )) .await; assert_that!( actions, @@ -229,7 +232,7 @@ async fn attach_by_workflow_key() { let response_bytes = Bytes::from_static(b"123"); let actions = test_env .apply_multiple([ - Command::InvokerEffect(Box::new(Effect { + records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::JournalEntry { @@ -239,7 +242,7 @@ async fn attach_by_workflow_key() { )), }, })), - Command::InvokerEffect(Box::new(Effect { + records::InvokerEffect::new_test(Box::new(Effect { invocation_id, invocation_epoch: 0, kind: InvokerEffectKind::End, @@ -285,15 +288,17 @@ async fn attach_by_workflow_key() { // Sending another attach will be completed immediately let actions = test_env - .apply(Command::AttachInvocation(AttachInvocationRequest { - invocation_query: InvocationQuery::Workflow( - invocation_target.as_keyed_service_id().unwrap(), - ), - block_on_inflight: true, - response_sink: ServiceInvocationResponseSink::Ingress { - request_id: request_id_3, + .apply(records::AttachInvocation::new_test( + AttachInvocationRequest { + invocation_query: InvocationQuery::Workflow( + invocation_target.as_keyed_service_id().unwrap(), + ), + block_on_inflight: true, + response_sink: ServiceInvocationResponseSink::Ingress { + request_id: request_id_3, + }, }, - })) + )) .await; assert_that!( actions, @@ -337,7 +342,7 @@ async fn purge_completed_workflow() { // Send timer fired command let request_id = PartitionProcessorRpcRequestId::new(); let actions = test_env - .apply(Command::PurgeInvocation(PurgeInvocationRequest { + .apply(records::PurgeInvocation::new_test(PurgeInvocationRequest { invocation_id, response_sink: Some(InvocationMutationResponseSink::Ingress( IngressInvocationResponseSink { request_id }, diff --git a/crates/worker/src/partition/state_machine/utils.rs b/crates/worker/src/partition/state_machine/utils.rs index e5e97aba96..7c3255d5d6 100644 --- a/crates/worker/src/partition/state_machine/utils.rs +++ b/crates/worker/src/partition/state_machine/utils.rs @@ -7,11 +7,11 @@ // As of the Change Date specified in that file, in accordance with // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0. +use tracing::{Level, Span, debug_span, event_enabled, trace_span}; use restate_types::invocation::InvocationEpoch; use restate_types::{identifiers::InvocationId, invocation::InvocationTarget}; -use restate_wal_protocol::Command; -use tracing::{Level, Span, debug_span, event_enabled, trace_span}; +use restate_wal_protocol::v2::RecordKind; pub(super) trait SpanExt { fn record_invocation_id(&self, id: &InvocationId); @@ -38,11 +38,11 @@ impl SpanExt for tracing::Span { } } -pub(super) fn state_machine_apply_command_span(is_leader: bool, cmd: &Command) -> Span { +pub(super) fn state_machine_apply_command_span(is_leader: bool, cmd: RecordKind) -> Span { let span = if is_leader { debug_span!( "apply_command", - otel.name = format!("apply-command: {}", cmd.name()), + otel.name = format!("apply-command: {}", cmd), restate.invocation.id = tracing::field::Empty, restate.invocation.epoch = tracing::field::Empty, restate.invocation.target = tracing::field::Empty, @@ -53,7 +53,7 @@ pub(super) fn state_machine_apply_command_span(is_leader: bool, cmd: &Command) - } else { trace_span!( "apply_command", - otel.name = format!("apply-command: {}", cmd.name()), + otel.name = format!("apply-command: {}", cmd), restate.invocation.id = tracing::field::Empty, restate.invocation.epoch = tracing::field::Empty, restate.invocation.target = tracing::field::Empty,