#![cfg_attr(web, feature(trait_upcasting))]
#![deny(clippy::large_futures)]
mod applications;
pub mod committee;
mod execution;
mod execution_state_actor;
mod graphql;
mod policy;
mod resources;
#[cfg(with_revm)]
pub mod revm;
mod runtime;
pub mod system;
#[cfg(with_testing)]
pub mod test_utils;
mod transaction_tracker;
mod util;
mod wasm;
use std::{any::Any, fmt, str::FromStr, sync::Arc};
use async_graphql::SimpleObject;
use async_trait::async_trait;
use committee::Epoch;
use custom_debug_derive::Debug;
use dashmap::DashMap;
use derive_more::Display;
#[cfg(web)]
use js_sys::wasm_bindgen::JsValue;
use linera_base::{
abi::Abi,
crypto::{BcsHashable, CryptoHash},
data_types::{
Amount, ApplicationPermissions, ArithmeticError, Blob, BlockHeight, DecompressionError,
Resources, SendMessageRequest, Timestamp, UserApplicationDescription,
},
doc_scalar, hex_debug, http,
identifiers::{
Account, AccountOwner, ApplicationId, BlobId, BytecodeId, ChainId, ChannelName,
Destination, EventId, GenericApplicationId, MessageId, Owner, StreamName,
UserApplicationId,
},
ownership::ChainOwnership,
task,
};
use linera_views::{batch::Batch, views::ViewError};
use serde::{Deserialize, Serialize};
use system::OpenChainConfig;
use thiserror::Error;
#[cfg(with_testing)]
pub use crate::applications::ApplicationRegistry;
#[cfg(with_revm)]
use crate::revm::EvmExecutionError;
use crate::runtime::ContractSyncRuntime;
#[cfg(all(with_testing, with_wasm_runtime))]
pub use crate::wasm::test as wasm_test;
#[cfg(with_wasm_runtime)]
pub use crate::wasm::{
BaseRuntimeApi, ContractEntrypoints, ContractRuntimeApi, RuntimeApiData, ServiceEntrypoints,
ServiceRuntimeApi, WasmContractModule, WasmExecutionError, WasmServiceModule,
};
pub use crate::{
applications::ApplicationRegistryView,
execution::{ExecutionStateView, ServiceRuntimeEndpoint},
execution_state_actor::ExecutionRequest,
policy::ResourceControlPolicy,
resources::{ResourceController, ResourceTracker},
runtime::{
ContractSyncRuntimeHandle, ServiceRuntimeRequest, ServiceSyncRuntime,
ServiceSyncRuntimeHandle,
},
system::{
SystemExecutionError, SystemExecutionStateView, SystemMessage, SystemOperation,
SystemQuery, SystemResponse,
},
transaction_tracker::{TransactionOutcome, TransactionTracker},
};
const MAX_EVENT_KEY_LEN: usize = 64;
const MAX_STREAM_NAME_LEN: usize = 64;
#[derive(Clone)]
pub struct UserContractCode(Box<dyn UserContractModule>);
#[derive(Clone)]
pub struct UserServiceCode(Box<dyn UserServiceModule>);
pub type UserContractInstance = Box<dyn UserContract>;
pub type UserServiceInstance = Box<dyn UserService>;
pub trait UserContractModule: dyn_clone::DynClone + Any + task::Post + Send + Sync {
fn instantiate(
&self,
runtime: ContractSyncRuntimeHandle,
) -> Result<UserContractInstance, ExecutionError>;
}
impl<T: UserContractModule + Send + Sync + 'static> From<T> for UserContractCode {
fn from(module: T) -> Self {
Self(Box::new(module))
}
}
dyn_clone::clone_trait_object!(UserContractModule);
pub trait UserServiceModule: dyn_clone::DynClone + Any + task::Post + Send + Sync {
fn instantiate(
&self,
runtime: ServiceSyncRuntimeHandle,
) -> Result<UserServiceInstance, ExecutionError>;
}
impl<T: UserServiceModule + Send + Sync + 'static> From<T> for UserServiceCode {
fn from(module: T) -> Self {
Self(Box::new(module))
}
}
dyn_clone::clone_trait_object!(UserServiceModule);
impl UserServiceCode {
fn instantiate(
&self,
runtime: ServiceSyncRuntimeHandle,
) -> Result<UserServiceInstance, ExecutionError> {
self.0.instantiate(runtime)
}
}
impl UserContractCode {
fn instantiate(
&self,
runtime: ContractSyncRuntimeHandle,
) -> Result<UserContractInstance, ExecutionError> {
self.0.instantiate(runtime)
}
}
#[cfg(web)]
const _: () = {
impl From<UserContractCode> for JsValue {
fn from(code: UserContractCode) -> JsValue {
let module: WasmContractModule = *(code.0 as Box<dyn Any>)
.downcast()
.expect("we only support Wasm modules on the Web for now");
module.into()
}
}
impl From<UserServiceCode> for JsValue {
fn from(code: UserServiceCode) -> JsValue {
let module: WasmServiceModule = *(code.0 as Box<dyn Any>)
.downcast()
.expect("we only support Wasm modules on the Web for now");
module.into()
}
}
impl TryFrom<JsValue> for UserContractCode {
type Error = JsValue;
fn try_from(value: JsValue) -> Result<Self, JsValue> {
WasmContractModule::try_from(value).map(Into::into)
}
}
impl TryFrom<JsValue> for UserServiceCode {
type Error = JsValue;
fn try_from(value: JsValue) -> Result<Self, JsValue> {
WasmServiceModule::try_from(value).map(Into::into)
}
}
};
#[derive(Error, Debug)]
pub enum ExecutionError {
#[error(transparent)]
ViewError(ViewError),
#[error(transparent)]
ArithmeticError(#[from] ArithmeticError),
#[error(transparent)]
SystemError(SystemExecutionError),
#[error("User application reported an error: {0}")]
UserError(String),
#[cfg(with_wasm_runtime)]
#[error(transparent)]
WasmError(#[from] WasmExecutionError),
#[cfg(with_revm)]
#[error(transparent)]
EvmError(#[from] EvmExecutionError),
#[error(transparent)]
DecompressionError(#[from] DecompressionError),
#[error("The given promise is invalid or was polled once already")]
InvalidPromise,
#[error("Attempted to perform a reentrant call to application {0}")]
ReentrantCall(UserApplicationId),
#[error(
"Application {caller_id} attempted to perform a cross-application to {callee_id} call \
from `finalize`"
)]
CrossApplicationCallInFinalize {
caller_id: Box<UserApplicationId>,
callee_id: Box<UserApplicationId>,
},
#[error("Attempt to write to storage from a contract")]
ServiceWriteAttempt,
#[error("Failed to load bytecode from storage {0:?}")]
ApplicationBytecodeNotFound(Box<UserApplicationDescription>),
#[error("Unsupported dynamic application load: {0:?}")]
UnsupportedDynamicApplicationLoad(Box<UserApplicationId>),
#[error("Excessive number of bytes read from storage")]
ExcessiveRead,
#[error("Excessive number of bytes written to storage")]
ExcessiveWrite,
#[error("Block execution required too much fuel")]
MaximumFuelExceeded,
#[error("Serialized size of the executed block exceeds limit")]
ExecutedBlockTooLarge,
#[error("Runtime failed to respond to application")]
MissingRuntimeResponse,
#[error("Bytecode ID {0:?} is invalid")]
InvalidBytecodeId(BytecodeId),
#[error("Owner is None")]
OwnerIsNone,
#[error("Application is not authorized to perform system operations on this chain: {0:}")]
UnauthorizedApplication(UserApplicationId),
#[error("Failed to make network reqwest: {0}")]
ReqwestError(#[from] reqwest::Error),
#[error("Encountered I/O error: {0}")]
IoError(#[from] std::io::Error),
#[error("More recorded oracle responses than expected")]
UnexpectedOracleResponse,
#[error("Invalid JSON: {0}")]
JsonError(#[from] serde_json::Error),
#[error(transparent)]
BcsError(#[from] bcs::Error),
#[error("Recorded response for oracle query has the wrong type")]
OracleResponseMismatch,
#[error("Assertion failed: local time {local_time} is not earlier than {timestamp}")]
AssertBefore {
timestamp: Timestamp,
local_time: Timestamp,
},
#[error("Event keys can be at most {MAX_EVENT_KEY_LEN} bytes.")]
EventKeyTooLong,
#[error("Stream names can be at most {MAX_STREAM_NAME_LEN} bytes.")]
StreamNameTooLong,
#[error("Blob exceeds size limit")]
BlobTooLarge,
#[error("Bytecode exceeds size limit")]
BytecodeTooLarge,
#[error("Unstable oracles are disabled on this network.")]
UnstableOracle,
#[error("Failed to send contract code to worker thread: {0:?}")]
ContractModuleSend(#[from] linera_base::task::SendError<UserContractCode>),
#[error("Failed to send service code to worker thread: {0:?}")]
ServiceModuleSend(#[from] linera_base::task::SendError<UserServiceCode>),
#[error("Blobs not found: {0:?}")]
BlobsNotFound(Vec<BlobId>),
#[error("Invalid HTTP header name used for HTTP request")]
InvalidHeaderName(#[from] reqwest::header::InvalidHeaderName),
#[error("Invalid HTTP header value used for HTTP request")]
InvalidHeaderValue(#[from] reqwest::header::InvalidHeaderValue),
}
impl From<ViewError> for ExecutionError {
fn from(error: ViewError) -> Self {
match error {
ViewError::BlobsNotFound(blob_ids) => ExecutionError::BlobsNotFound(blob_ids),
error => ExecutionError::ViewError(error),
}
}
}
impl From<SystemExecutionError> for ExecutionError {
fn from(error: SystemExecutionError) -> Self {
match error {
SystemExecutionError::BlobsNotFound(blob_ids) => {
ExecutionError::BlobsNotFound(blob_ids)
}
error => ExecutionError::SystemError(error),
}
}
}
pub trait UserContract {
fn instantiate(
&mut self,
context: OperationContext,
argument: Vec<u8>,
) -> Result<(), ExecutionError>;
fn execute_operation(
&mut self,
context: OperationContext,
operation: Vec<u8>,
) -> Result<Vec<u8>, ExecutionError>;
fn execute_message(
&mut self,
context: MessageContext,
message: Vec<u8>,
) -> Result<(), ExecutionError>;
fn finalize(&mut self, context: FinalizeContext) -> Result<(), ExecutionError>;
}
pub trait UserService {
fn handle_query(
&mut self,
context: QueryContext,
argument: Vec<u8>,
) -> Result<Vec<u8>, ExecutionError>;
}
#[derive(Default)]
pub struct ApplicationCallOutcome {
pub value: Vec<u8>,
pub execution_outcome: RawExecutionOutcome<Vec<u8>>,
}
impl ApplicationCallOutcome {
pub fn with_message(mut self, message: RawOutgoingMessage<Vec<u8>>) -> Self {
self.execution_outcome.messages.push(message);
self
}
}
#[derive(Clone, Copy, Default)]
pub struct ExecutionRuntimeConfig {}
#[cfg_attr(not(web), async_trait)]
#[cfg_attr(web, async_trait(?Send))]
pub trait ExecutionRuntimeContext {
fn chain_id(&self) -> ChainId;
fn execution_runtime_config(&self) -> ExecutionRuntimeConfig;
fn user_contracts(&self) -> &Arc<DashMap<UserApplicationId, UserContractCode>>;
fn user_services(&self) -> &Arc<DashMap<UserApplicationId, UserServiceCode>>;
async fn get_user_contract(
&self,
description: &UserApplicationDescription,
) -> Result<UserContractCode, ExecutionError>;
async fn get_user_service(
&self,
description: &UserApplicationDescription,
) -> Result<UserServiceCode, ExecutionError>;
async fn get_blob(&self, blob_id: BlobId) -> Result<Blob, ViewError>;
async fn get_event(&self, event_id: EventId) -> Result<Vec<u8>, ViewError>;
async fn contains_blob(&self, blob_id: BlobId) -> Result<bool, ViewError>;
#[cfg(with_testing)]
async fn add_blobs(
&self,
blobs: impl IntoIterator<Item = Blob> + Send,
) -> Result<(), ViewError>;
#[cfg(with_testing)]
async fn add_events(
&self,
events: impl IntoIterator<Item = (EventId, Vec<u8>)> + Send,
) -> Result<(), ViewError>;
}
#[derive(Clone, Copy, Debug)]
pub struct OperationContext {
pub chain_id: ChainId,
#[debug(skip_if = Option::is_none)]
pub authenticated_signer: Option<Owner>,
#[debug(skip_if = Option::is_none)]
pub authenticated_caller_id: Option<UserApplicationId>,
pub height: BlockHeight,
pub round: Option<u32>,
#[debug(skip_if = Option::is_none)]
pub index: Option<u32>,
}
#[derive(Clone, Copy, Debug)]
pub struct MessageContext {
pub chain_id: ChainId,
pub is_bouncing: bool,
#[debug(skip_if = Option::is_none)]
pub authenticated_signer: Option<Owner>,
#[debug(skip_if = Option::is_none)]
pub refund_grant_to: Option<Account>,
pub height: BlockHeight,
pub round: Option<u32>,
pub certificate_hash: CryptoHash,
pub message_id: MessageId,
}
#[derive(Clone, Copy, Debug)]
pub struct FinalizeContext {
pub chain_id: ChainId,
#[debug(skip_if = Option::is_none)]
pub authenticated_signer: Option<Owner>,
pub height: BlockHeight,
pub round: Option<u32>,
}
#[derive(Clone, Copy, Debug, Eq, PartialEq)]
pub struct QueryContext {
pub chain_id: ChainId,
pub next_block_height: BlockHeight,
pub local_time: Timestamp,
}
pub trait BaseRuntime {
type Read: fmt::Debug + Send + Sync;
type ContainsKey: fmt::Debug + Send + Sync;
type ContainsKeys: fmt::Debug + Send + Sync;
type ReadMultiValuesBytes: fmt::Debug + Send + Sync;
type ReadValueBytes: fmt::Debug + Send + Sync;
type FindKeysByPrefix: fmt::Debug + Send + Sync;
type FindKeyValuesByPrefix: fmt::Debug + Send + Sync;
fn chain_id(&mut self) -> Result<ChainId, ExecutionError>;
fn block_height(&mut self) -> Result<BlockHeight, ExecutionError>;
fn application_id(&mut self) -> Result<UserApplicationId, ExecutionError>;
fn application_creator_chain_id(&mut self) -> Result<ChainId, ExecutionError>;
fn application_parameters(&mut self) -> Result<Vec<u8>, ExecutionError>;
fn read_system_timestamp(&mut self) -> Result<Timestamp, ExecutionError>;
fn read_chain_balance(&mut self) -> Result<Amount, ExecutionError>;
fn read_owner_balance(&mut self, owner: AccountOwner) -> Result<Amount, ExecutionError>;
fn read_owner_balances(&mut self) -> Result<Vec<(AccountOwner, Amount)>, ExecutionError>;
fn read_balance_owners(&mut self) -> Result<Vec<AccountOwner>, ExecutionError>;
fn chain_ownership(&mut self) -> Result<ChainOwnership, ExecutionError>;
#[cfg(feature = "test")]
fn contains_key(&mut self, key: Vec<u8>) -> Result<bool, ExecutionError> {
let promise = self.contains_key_new(key)?;
self.contains_key_wait(&promise)
}
fn contains_key_new(&mut self, key: Vec<u8>) -> Result<Self::ContainsKey, ExecutionError>;
fn contains_key_wait(&mut self, promise: &Self::ContainsKey) -> Result<bool, ExecutionError>;
#[cfg(feature = "test")]
fn contains_keys(&mut self, keys: Vec<Vec<u8>>) -> Result<Vec<bool>, ExecutionError> {
let promise = self.contains_keys_new(keys)?;
self.contains_keys_wait(&promise)
}
fn contains_keys_new(
&mut self,
keys: Vec<Vec<u8>>,
) -> Result<Self::ContainsKeys, ExecutionError>;
fn contains_keys_wait(
&mut self,
promise: &Self::ContainsKeys,
) -> Result<Vec<bool>, ExecutionError>;
#[cfg(feature = "test")]
fn read_multi_values_bytes(
&mut self,
keys: Vec<Vec<u8>>,
) -> Result<Vec<Option<Vec<u8>>>, ExecutionError> {
let promise = self.read_multi_values_bytes_new(keys)?;
self.read_multi_values_bytes_wait(&promise)
}
fn read_multi_values_bytes_new(
&mut self,
keys: Vec<Vec<u8>>,
) -> Result<Self::ReadMultiValuesBytes, ExecutionError>;
fn read_multi_values_bytes_wait(
&mut self,
promise: &Self::ReadMultiValuesBytes,
) -> Result<Vec<Option<Vec<u8>>>, ExecutionError>;
#[cfg(feature = "test")]
fn read_value_bytes(&mut self, key: Vec<u8>) -> Result<Option<Vec<u8>>, ExecutionError> {
let promise = self.read_value_bytes_new(key)?;
self.read_value_bytes_wait(&promise)
}
fn read_value_bytes_new(
&mut self,
key: Vec<u8>,
) -> Result<Self::ReadValueBytes, ExecutionError>;
fn read_value_bytes_wait(
&mut self,
promise: &Self::ReadValueBytes,
) -> Result<Option<Vec<u8>>, ExecutionError>;
fn find_keys_by_prefix_new(
&mut self,
key_prefix: Vec<u8>,
) -> Result<Self::FindKeysByPrefix, ExecutionError>;
fn find_keys_by_prefix_wait(
&mut self,
promise: &Self::FindKeysByPrefix,
) -> Result<Vec<Vec<u8>>, ExecutionError>;
#[cfg(feature = "test")]
#[expect(clippy::type_complexity)]
fn find_key_values_by_prefix(
&mut self,
key_prefix: Vec<u8>,
) -> Result<Vec<(Vec<u8>, Vec<u8>)>, ExecutionError> {
let promise = self.find_key_values_by_prefix_new(key_prefix)?;
self.find_key_values_by_prefix_wait(&promise)
}
fn find_key_values_by_prefix_new(
&mut self,
key_prefix: Vec<u8>,
) -> Result<Self::FindKeyValuesByPrefix, ExecutionError>;
#[expect(clippy::type_complexity)]
fn find_key_values_by_prefix_wait(
&mut self,
promise: &Self::FindKeyValuesByPrefix,
) -> Result<Vec<(Vec<u8>, Vec<u8>)>, ExecutionError>;
fn perform_http_request(
&mut self,
request: http::Request,
) -> Result<http::Response, ExecutionError>;
fn assert_before(&mut self, timestamp: Timestamp) -> Result<(), ExecutionError>;
fn read_data_blob(&mut self, hash: &CryptoHash) -> Result<Vec<u8>, ExecutionError>;
fn assert_data_blob_exists(&mut self, hash: &CryptoHash) -> Result<(), ExecutionError>;
}
pub trait ServiceRuntime: BaseRuntime {
fn try_query_application(
&mut self,
queried_id: UserApplicationId,
argument: Vec<u8>,
) -> Result<Vec<u8>, ExecutionError>;
fn fetch_url(&mut self, url: &str) -> Result<Vec<u8>, ExecutionError>;
fn schedule_operation(&mut self, operation: Vec<u8>) -> Result<(), ExecutionError>;
}
pub trait ContractRuntime: BaseRuntime {
fn authenticated_signer(&mut self) -> Result<Option<Owner>, ExecutionError>;
fn message_id(&mut self) -> Result<Option<MessageId>, ExecutionError>;
fn message_is_bouncing(&mut self) -> Result<Option<bool>, ExecutionError>;
fn authenticated_caller_id(&mut self) -> Result<Option<UserApplicationId>, ExecutionError>;
fn remaining_fuel(&mut self) -> Result<u64, ExecutionError>;
fn consume_fuel(&mut self, fuel: u64) -> Result<(), ExecutionError>;
fn send_message(&mut self, message: SendMessageRequest<Vec<u8>>) -> Result<(), ExecutionError>;
fn subscribe(&mut self, chain: ChainId, channel: ChannelName) -> Result<(), ExecutionError>;
fn unsubscribe(&mut self, chain: ChainId, channel: ChannelName) -> Result<(), ExecutionError>;
fn transfer(
&mut self,
source: Option<AccountOwner>,
destination: Account,
amount: Amount,
) -> Result<(), ExecutionError>;
fn claim(
&mut self,
source: Account,
destination: Account,
amount: Amount,
) -> Result<(), ExecutionError>;
fn try_call_application(
&mut self,
authenticated: bool,
callee_id: UserApplicationId,
argument: Vec<u8>,
) -> Result<Vec<u8>, ExecutionError>;
fn emit(
&mut self,
name: StreamName,
key: Vec<u8>,
value: Vec<u8>,
) -> Result<(), ExecutionError>;
fn query_service(
&mut self,
application_id: ApplicationId,
query: Vec<u8>,
) -> Result<Vec<u8>, ExecutionError>;
fn open_chain(
&mut self,
ownership: ChainOwnership,
application_permissions: ApplicationPermissions,
balance: Amount,
) -> Result<(MessageId, ChainId), ExecutionError>;
fn close_chain(&mut self) -> Result<(), ExecutionError>;
fn change_application_permissions(
&mut self,
application_permissions: ApplicationPermissions,
) -> Result<(), ExecutionError>;
fn create_application(
&mut self,
bytecode_id: BytecodeId,
parameters: Vec<u8>,
argument: Vec<u8>,
required_application_ids: Vec<UserApplicationId>,
) -> Result<UserApplicationId, ExecutionError>;
fn validation_round(&mut self) -> Result<Option<u32>, ExecutionError>;
fn write_batch(&mut self, batch: Batch) -> Result<(), ExecutionError>;
}
#[derive(Debug, PartialEq, Eq, Hash, Clone, Serialize, Deserialize)]
pub enum Operation {
System(SystemOperation),
User {
application_id: UserApplicationId,
#[serde(with = "serde_bytes")]
#[debug(with = "hex_debug")]
bytes: Vec<u8>,
},
}
impl<'de> BcsHashable<'de> for Operation {}
#[derive(Debug, PartialEq, Eq, Hash, Clone, Serialize, Deserialize)]
pub enum Message {
System(SystemMessage),
User {
application_id: UserApplicationId,
#[serde(with = "serde_bytes")]
#[debug(with = "hex_debug")]
bytes: Vec<u8>,
},
}
#[derive(Debug, PartialEq, Eq, Hash, Clone, Serialize, Deserialize)]
pub enum Query {
System(SystemQuery),
User {
application_id: UserApplicationId,
#[serde(with = "serde_bytes")]
#[debug(with = "hex_debug")]
bytes: Vec<u8>,
},
}
#[derive(Debug, PartialEq, Eq, Hash, Clone, Serialize, Deserialize)]
pub struct QueryOutcome<Response = QueryResponse> {
pub response: Response,
pub operations: Vec<Operation>,
}
impl From<QueryOutcome<SystemResponse>> for QueryOutcome {
fn from(system_outcome: QueryOutcome<SystemResponse>) -> Self {
let QueryOutcome {
response,
operations,
} = system_outcome;
QueryOutcome {
response: QueryResponse::System(response),
operations,
}
}
}
impl From<QueryOutcome<Vec<u8>>> for QueryOutcome {
fn from(user_service_outcome: QueryOutcome<Vec<u8>>) -> Self {
let QueryOutcome {
response,
operations,
} = user_service_outcome;
QueryOutcome {
response: QueryResponse::User(response),
operations,
}
}
}
#[derive(Debug, PartialEq, Eq, Hash, Clone, Serialize, Deserialize)]
pub enum QueryResponse {
System(SystemResponse),
User(
#[serde(with = "serde_bytes")]
#[debug(with = "hex_debug")]
Vec<u8>,
),
}
#[derive(Clone, Debug)]
#[cfg_attr(with_testing, derive(Eq, PartialEq))]
pub struct RawOutgoingMessage<Message, Grant = Resources> {
pub destination: Destination,
pub authenticated: bool,
pub grant: Grant,
pub kind: MessageKind,
pub message: Message,
}
impl<Message> From<SendMessageRequest<Message>> for RawOutgoingMessage<Message, Resources> {
fn from(request: SendMessageRequest<Message>) -> Self {
let SendMessageRequest {
destination,
authenticated,
grant,
is_tracked,
message,
} = request;
let kind = if is_tracked {
MessageKind::Tracked
} else {
MessageKind::Simple
};
RawOutgoingMessage {
destination,
authenticated,
grant,
kind,
message,
}
}
}
#[derive(Debug, PartialEq, Eq, Hash, Clone, Serialize, Deserialize, Copy)]
pub enum MessageKind {
Simple,
Protected,
Tracked,
Bouncing,
}
#[derive(Debug)]
#[cfg_attr(with_testing, derive(Eq, PartialEq))]
pub struct RawExecutionOutcome<Message, Grant = Resources> {
pub authenticated_signer: Option<Owner>,
pub refund_grant_to: Option<Account>,
pub messages: Vec<RawOutgoingMessage<Message, Grant>>,
}
#[derive(
Eq, PartialEq, Ord, PartialOrd, Debug, Clone, Hash, Serialize, Deserialize, SimpleObject,
)]
pub struct ChannelSubscription {
pub chain_id: ChainId,
pub name: ChannelName,
}
#[derive(Debug)]
#[cfg_attr(with_testing, derive(Eq, PartialEq))]
#[expect(clippy::large_enum_variant)]
pub enum ExecutionOutcome {
System(RawExecutionOutcome<SystemMessage, Amount>),
User(UserApplicationId, RawExecutionOutcome<Vec<u8>, Amount>),
}
impl ExecutionOutcome {
pub fn application_id(&self) -> GenericApplicationId {
match self {
ExecutionOutcome::System(_) => GenericApplicationId::System,
ExecutionOutcome::User(app_id, _) => GenericApplicationId::User(*app_id),
}
}
pub fn message_count(&self) -> usize {
match self {
ExecutionOutcome::System(outcome) => outcome.messages.len(),
ExecutionOutcome::User(_, outcome) => outcome.messages.len(),
}
}
}
impl<Message, Grant> RawExecutionOutcome<Message, Grant> {
pub fn with_authenticated_signer(mut self, authenticated_signer: Option<Owner>) -> Self {
self.authenticated_signer = authenticated_signer;
self
}
pub fn with_refund_grant_to(mut self, refund_grant_to: Option<Account>) -> Self {
self.refund_grant_to = refund_grant_to;
self
}
pub fn with_message(mut self, message: RawOutgoingMessage<Message, Grant>) -> Self {
self.messages.push(message);
self
}
}
impl<Message, Grant> Default for RawExecutionOutcome<Message, Grant> {
fn default() -> Self {
Self {
authenticated_signer: None,
refund_grant_to: None,
messages: Vec::new(),
}
}
}
impl<Message> RawOutgoingMessage<Message, Resources> {
pub fn into_priced(
self,
policy: &ResourceControlPolicy,
) -> Result<RawOutgoingMessage<Message, Amount>, ArithmeticError> {
let RawOutgoingMessage {
destination,
authenticated,
grant,
kind,
message,
} = self;
Ok(RawOutgoingMessage {
destination,
authenticated,
grant: policy.total_price(&grant)?,
kind,
message,
})
}
}
impl<Message> RawExecutionOutcome<Message, Resources> {
pub fn into_priced(
self,
policy: &ResourceControlPolicy,
) -> Result<RawExecutionOutcome<Message, Amount>, ArithmeticError> {
let RawExecutionOutcome {
authenticated_signer,
refund_grant_to,
messages,
} = self;
let messages = messages
.into_iter()
.map(|message| message.into_priced(policy))
.collect::<Result<_, _>>()?;
Ok(RawExecutionOutcome {
authenticated_signer,
refund_grant_to,
messages,
})
}
}
impl OperationContext {
fn refund_grant_to(&self) -> Option<Account> {
Some(Account {
chain_id: self.chain_id,
owner: self.authenticated_signer.map(AccountOwner::User),
})
}
fn next_message_id(&self, next_message_index: u32) -> MessageId {
MessageId {
chain_id: self.chain_id,
height: self.height,
index: next_message_index,
}
}
}
#[cfg(with_testing)]
#[derive(Clone)]
pub struct TestExecutionRuntimeContext {
chain_id: ChainId,
execution_runtime_config: ExecutionRuntimeConfig,
user_contracts: Arc<DashMap<UserApplicationId, UserContractCode>>,
user_services: Arc<DashMap<UserApplicationId, UserServiceCode>>,
blobs: Arc<DashMap<BlobId, Blob>>,
events: Arc<DashMap<EventId, Vec<u8>>>,
}
#[cfg(with_testing)]
impl TestExecutionRuntimeContext {
pub fn new(chain_id: ChainId, execution_runtime_config: ExecutionRuntimeConfig) -> Self {
Self {
chain_id,
execution_runtime_config,
user_contracts: Arc::default(),
user_services: Arc::default(),
blobs: Arc::default(),
events: Arc::default(),
}
}
}
#[cfg(with_testing)]
#[cfg_attr(not(web), async_trait)]
#[cfg_attr(web, async_trait(?Send))]
impl ExecutionRuntimeContext for TestExecutionRuntimeContext {
fn chain_id(&self) -> ChainId {
self.chain_id
}
fn execution_runtime_config(&self) -> ExecutionRuntimeConfig {
self.execution_runtime_config
}
fn user_contracts(&self) -> &Arc<DashMap<UserApplicationId, UserContractCode>> {
&self.user_contracts
}
fn user_services(&self) -> &Arc<DashMap<UserApplicationId, UserServiceCode>> {
&self.user_services
}
async fn get_user_contract(
&self,
description: &UserApplicationDescription,
) -> Result<UserContractCode, ExecutionError> {
let application_id = description.into();
Ok(self
.user_contracts()
.get(&application_id)
.ok_or_else(|| {
ExecutionError::ApplicationBytecodeNotFound(Box::new(description.clone()))
})?
.clone())
}
async fn get_user_service(
&self,
description: &UserApplicationDescription,
) -> Result<UserServiceCode, ExecutionError> {
let application_id = description.into();
Ok(self
.user_services()
.get(&application_id)
.ok_or_else(|| {
ExecutionError::ApplicationBytecodeNotFound(Box::new(description.clone()))
})?
.clone())
}
async fn get_blob(&self, blob_id: BlobId) -> Result<Blob, ViewError> {
Ok(self
.blobs
.get(&blob_id)
.ok_or_else(|| ViewError::BlobsNotFound(vec![blob_id]))?
.clone())
}
async fn get_event(&self, event_id: EventId) -> Result<Vec<u8>, ViewError> {
Ok(self
.events
.get(&event_id)
.ok_or_else(|| ViewError::EventsNotFound(vec![event_id]))?
.clone())
}
async fn contains_blob(&self, blob_id: BlobId) -> Result<bool, ViewError> {
Ok(self.blobs.contains_key(&blob_id))
}
#[cfg(with_testing)]
async fn add_blobs(
&self,
blobs: impl IntoIterator<Item = Blob> + Send,
) -> Result<(), ViewError> {
for blob in blobs {
self.blobs.insert(blob.id(), blob);
}
Ok(())
}
#[cfg(with_testing)]
async fn add_events(
&self,
events: impl IntoIterator<Item = (EventId, Vec<u8>)> + Send,
) -> Result<(), ViewError> {
for (event_id, bytes) in events {
self.events.insert(event_id, bytes);
}
Ok(())
}
}
impl From<SystemOperation> for Operation {
fn from(operation: SystemOperation) -> Self {
Operation::System(operation)
}
}
impl Operation {
pub fn system(operation: SystemOperation) -> Self {
Operation::System(operation)
}
#[cfg(with_testing)]
pub fn user<A: Abi>(
application_id: UserApplicationId<A>,
operation: &A::Operation,
) -> Result<Self, bcs::Error> {
Self::user_without_abi(application_id.forget_abi(), operation)
}
#[cfg(with_testing)]
pub fn user_without_abi(
application_id: UserApplicationId<()>,
operation: &impl Serialize,
) -> Result<Self, bcs::Error> {
Ok(Operation::User {
application_id,
bytes: bcs::to_bytes(&operation)?,
})
}
pub fn application_id(&self) -> GenericApplicationId {
match self {
Self::System(_) => GenericApplicationId::System,
Self::User { application_id, .. } => GenericApplicationId::User(*application_id),
}
}
}
impl From<SystemMessage> for Message {
fn from(message: SystemMessage) -> Self {
Message::System(message)
}
}
impl Message {
pub fn system(message: SystemMessage) -> Self {
Message::System(message)
}
pub fn user<A, M: Serialize>(
application_id: UserApplicationId<A>,
message: &M,
) -> Result<Self, bcs::Error> {
let application_id = application_id.forget_abi();
let bytes = bcs::to_bytes(&message)?;
Ok(Message::User {
application_id,
bytes,
})
}
pub fn application_id(&self) -> GenericApplicationId {
match self {
Self::System(_) => GenericApplicationId::System,
Self::User { application_id, .. } => GenericApplicationId::User(*application_id),
}
}
pub fn goes_to_inbox(&self) -> bool {
!matches!(
self,
Message::System(SystemMessage::Subscribe { .. } | SystemMessage::Unsubscribe { .. })
)
}
pub fn matches_subscribe(&self) -> Option<(&ChainId, &ChannelSubscription)> {
match self {
Message::System(SystemMessage::Subscribe { id, subscription }) => {
Some((id, subscription))
}
_ => None,
}
}
pub fn matches_unsubscribe(&self) -> Option<(&ChainId, &ChannelSubscription)> {
match self {
Message::System(SystemMessage::Unsubscribe { id, subscription }) => {
Some((id, subscription))
}
_ => None,
}
}
pub fn matches_open_chain(&self) -> Option<&OpenChainConfig> {
match self {
Message::System(SystemMessage::OpenChain(config)) => Some(config),
_ => None,
}
}
}
impl From<SystemQuery> for Query {
fn from(query: SystemQuery) -> Self {
Query::System(query)
}
}
impl Query {
pub fn system(query: SystemQuery) -> Self {
Query::System(query)
}
pub fn user<A: Abi>(
application_id: UserApplicationId<A>,
query: &A::Query,
) -> Result<Self, serde_json::Error> {
Self::user_without_abi(application_id.forget_abi(), query)
}
pub fn user_without_abi(
application_id: UserApplicationId<()>,
query: &impl Serialize,
) -> Result<Self, serde_json::Error> {
Ok(Query::User {
application_id,
bytes: serde_json::to_vec(&query)?,
})
}
pub fn application_id(&self) -> GenericApplicationId {
match self {
Self::System(_) => GenericApplicationId::System,
Self::User { application_id, .. } => GenericApplicationId::User(*application_id),
}
}
}
impl From<SystemResponse> for QueryResponse {
fn from(response: SystemResponse) -> Self {
QueryResponse::System(response)
}
}
impl From<Vec<u8>> for QueryResponse {
fn from(response: Vec<u8>) -> Self {
QueryResponse::User(response)
}
}
#[derive(Eq, PartialEq, Debug, Hash, Clone, Serialize, Deserialize)]
pub struct BlobState {
pub last_used_by: CryptoHash,
pub chain_id: ChainId,
pub block_height: BlockHeight,
pub epoch: Epoch,
}
#[derive(Clone, Copy, Display)]
#[cfg_attr(with_wasm_runtime, derive(Debug, Default))]
pub enum WasmRuntime {
#[cfg(with_wasmer)]
#[default]
#[display("wasmer")]
Wasmer,
#[cfg(with_wasmtime)]
#[cfg_attr(not(with_wasmer), default)]
#[display("wasmtime")]
Wasmtime,
#[cfg(with_wasmer)]
WasmerWithSanitizer,
#[cfg(with_wasmtime)]
WasmtimeWithSanitizer,
}
#[derive(Clone, Copy, Display)]
#[cfg_attr(with_revm, derive(Debug, Default))]
pub enum EvmRuntime {
#[cfg(with_revm)]
#[default]
#[display("revm")]
Revm,
}
pub trait WithWasmDefault {
fn with_wasm_default(self) -> Self;
}
impl WasmRuntime {
pub fn needs_sanitizer(self) -> bool {
match self {
#[cfg(with_wasmer)]
WasmRuntime::WasmerWithSanitizer => true,
#[cfg(with_wasmtime)]
WasmRuntime::WasmtimeWithSanitizer => true,
#[cfg(with_wasm_runtime)]
_ => false,
}
}
}
impl WithWasmDefault for Option<WasmRuntime> {
fn with_wasm_default(self) -> Self {
#[cfg(with_wasm_runtime)]
{
Some(self.unwrap_or_default())
}
#[cfg(not(with_wasm_runtime))]
{
None
}
}
}
impl FromStr for WasmRuntime {
type Err = InvalidWasmRuntime;
fn from_str(string: &str) -> Result<Self, Self::Err> {
match string {
#[cfg(with_wasmer)]
"wasmer" => Ok(WasmRuntime::Wasmer),
#[cfg(with_wasmtime)]
"wasmtime" => Ok(WasmRuntime::Wasmtime),
unknown => Err(InvalidWasmRuntime(unknown.to_owned())),
}
}
}
#[derive(Clone, Debug, Error)]
#[error("{0:?} is not a valid WebAssembly runtime")]
pub struct InvalidWasmRuntime(String);
doc_scalar!(Operation, "An operation to be executed in a block");
doc_scalar!(
Message,
"A message to be sent and possibly executed in the receiver's block."
);
doc_scalar!(MessageKind, "The kind of outgoing message being sent");