fedimint_server/consensus/
api.rs

1//! Implements the client API through which users interact with the federation
2use std::cmp::Ordering;
3use std::collections::{BTreeMap, HashMap};
4use std::path::{Path, PathBuf};
5use std::sync::Arc;
6
7use anyhow::{anyhow, Result};
8use async_trait::async_trait;
9use bitcoin::hashes::sha256;
10use fedimint_aead::{encrypt, get_encryption_key, random_salt};
11use fedimint_api_client::api::{
12    FederationStatus, GuardianConfigBackup, PeerConnectionStatus, PeerStatus, StatusResponse,
13};
14use fedimint_core::admin_client::ServerStatus;
15use fedimint_core::backup::{ClientBackupKey, ClientBackupSnapshot};
16use fedimint_core::config::{ClientConfig, JsonClientConfig};
17use fedimint_core::core::backup::{SignedBackupRequest, BACKUP_REQUEST_MAX_PAYLOAD_SIZE_BYTES};
18use fedimint_core::core::{DynOutputOutcome, ModuleInstanceId};
19use fedimint_core::db::{
20    Committable, Database, DatabaseTransaction, IDatabaseTransactionOpsCoreTyped,
21};
22use fedimint_core::endpoint_constants::{
23    API_ANNOUNCEMENTS_ENDPOINT, AUDIT_ENDPOINT, AUTH_ENDPOINT, AWAIT_OUTPUT_OUTCOME_ENDPOINT,
24    AWAIT_SESSION_OUTCOME_ENDPOINT, AWAIT_SIGNED_SESSION_OUTCOME_ENDPOINT,
25    AWAIT_TRANSACTION_ENDPOINT, BACKUP_ENDPOINT, CLIENT_CONFIG_ENDPOINT,
26    CLIENT_CONFIG_JSON_ENDPOINT, FEDERATION_ID_ENDPOINT, FEDIMINTD_VERSION_ENDPOINT,
27    GUARDIAN_CONFIG_BACKUP_ENDPOINT, INVITE_CODE_ENDPOINT, RECOVER_ENDPOINT,
28    SERVER_CONFIG_CONSENSUS_HASH_ENDPOINT, SESSION_COUNT_ENDPOINT, SESSION_STATUS_ENDPOINT,
29    SHUTDOWN_ENDPOINT, SIGN_API_ANNOUNCEMENT_ENDPOINT, STATUS_ENDPOINT,
30    SUBMIT_API_ANNOUNCEMENT_ENDPOINT, SUBMIT_TRANSACTION_ENDPOINT, VERSION_ENDPOINT,
31};
32use fedimint_core::epoch::ConsensusItem;
33use fedimint_core::module::audit::{Audit, AuditSummary};
34use fedimint_core::module::registry::ServerModuleRegistry;
35use fedimint_core::module::{
36    api_endpoint, ApiEndpoint, ApiEndpointContext, ApiError, ApiRequestErased, ApiVersion,
37    SerdeModuleEncoding, SupportedApiVersionsSummary,
38};
39use fedimint_core::net::api_announcement::{
40    ApiAnnouncement, SignedApiAnnouncement, SignedApiAnnouncementSubmission,
41};
42use fedimint_core::secp256k1::{PublicKey, SECP256K1};
43use fedimint_core::server::DynServerModule;
44use fedimint_core::session_outcome::{SessionOutcome, SessionStatus, SignedSessionOutcome};
45use fedimint_core::transaction::{
46    SerdeTransaction, Transaction, TransactionError, TransactionSubmissionOutcome,
47};
48use fedimint_core::util::SafeUrl;
49use fedimint_core::{secp256k1, OutPoint, PeerId, TransactionId};
50use fedimint_logging::LOG_NET_API;
51use futures::StreamExt;
52use tokio::sync::{watch, RwLock};
53use tracing::{debug, info};
54
55use crate::config::io::{
56    CONSENSUS_CONFIG, ENCRYPTED_EXT, JSON_EXT, LOCAL_CONFIG, PRIVATE_CONFIG, SALT_FILE,
57};
58use crate::config::ServerConfig;
59use crate::consensus::db::{AcceptedItemPrefix, AcceptedTransactionKey, SignedSessionOutcomeKey};
60use crate::consensus::engine::get_finished_session_count_static;
61use crate::consensus::transaction::process_transaction_with_dbtx;
62use crate::fedimint_core::encoding::Encodable;
63use crate::metrics::{BACKUP_WRITE_SIZE_BYTES, STORED_BACKUPS_COUNT};
64use crate::net::api::announcement::{ApiAnnouncementKey, ApiAnnouncementPrefix};
65use crate::net::api::{check_auth, ApiResult, GuardianAuthToken, HasApiContext};
66
67#[derive(Clone)]
68pub struct ConsensusApi {
69    /// Our server configuration
70    pub cfg: ServerConfig,
71    /// Database for serving the API
72    pub db: Database,
73    /// Modules registered with the federation
74    pub modules: ServerModuleRegistry,
75    /// Cached client config
76    pub client_cfg: ClientConfig,
77
78    pub force_api_secret: Option<String>,
79    /// For sending API events to consensus such as transactions
80    pub submission_sender: async_channel::Sender<ConsensusItem>,
81    pub shutdown_receiver: watch::Receiver<Option<u64>>,
82    pub shutdown_sender: watch::Sender<Option<u64>>,
83    pub connection_status_channels: Arc<RwLock<BTreeMap<PeerId, PeerConnectionStatus>>>,
84    pub last_ci_by_peer: Arc<RwLock<BTreeMap<PeerId, u64>>>,
85    pub supported_api_versions: SupportedApiVersionsSummary,
86    pub code_version_str: String,
87}
88
89impl ConsensusApi {
90    pub fn api_versions_summary(&self) -> &SupportedApiVersionsSummary {
91        &self.supported_api_versions
92    }
93
94    pub fn get_active_api_secret(&self) -> Option<String> {
95        // TODO: In the future, we might want to fetch it from the DB, so it's possible
96        // to customize from the UX
97        self.force_api_secret.clone()
98    }
99
100    // we want to return an error if and only if the submitted transaction is
101    // invalid and will be rejected if we were to submit it to consensus
102    pub async fn submit_transaction(
103        &self,
104        transaction: Transaction,
105    ) -> Result<TransactionId, TransactionError> {
106        let txid = transaction.tx_hash();
107
108        debug!(target: LOG_NET_API, %txid, "Received a submitted transaction");
109
110        // Create read-only DB tx so that the read state is consistent
111        let mut dbtx = self.db.begin_transaction_nc().await;
112        // we already processed the transaction before
113        if dbtx
114            .get_value(&AcceptedTransactionKey(txid))
115            .await
116            .is_some()
117        {
118            debug!(target: LOG_NET_API, %txid, "Transaction already accepted");
119            return Ok(txid);
120        }
121
122        // We ignore any writes, as we only verify if the transaction is valid here
123        dbtx.ignore_uncommitted();
124
125        process_transaction_with_dbtx(self.modules.clone(), &mut dbtx, &transaction).await?;
126
127        self.submission_sender
128            .send(ConsensusItem::Transaction(transaction))
129            .await
130            .ok();
131
132        Ok(txid)
133    }
134
135    pub async fn await_transaction(
136        &self,
137        txid: TransactionId,
138    ) -> (Vec<ModuleInstanceId>, DatabaseTransaction<'_, Committable>) {
139        self.db
140            .wait_key_check(&AcceptedTransactionKey(txid), std::convert::identity)
141            .await
142    }
143
144    pub async fn await_output_outcome(
145        &self,
146        outpoint: OutPoint,
147    ) -> Result<SerdeModuleEncoding<DynOutputOutcome>> {
148        let (module_ids, mut dbtx) = self.await_transaction(outpoint.txid).await;
149
150        let module_id = module_ids
151            .into_iter()
152            .nth(outpoint.out_idx as usize)
153            .ok_or(anyhow!("Outpoint index out of bounds {:?}", outpoint))?;
154
155        let outcome = self
156            .modules
157            .get_expect(module_id)
158            .output_status(
159                &mut dbtx.to_ref_with_prefix_module_id(module_id).0.into_nc(),
160                outpoint,
161                module_id,
162            )
163            .await
164            .expect("The transaction is accepted");
165
166        Ok((&outcome).into())
167    }
168
169    pub async fn session_count(&self) -> u64 {
170        get_finished_session_count_static(&mut self.db.begin_transaction_nc().await).await
171    }
172
173    pub async fn await_signed_session_outcome(&self, index: u64) -> SignedSessionOutcome {
174        self.db
175            .wait_key_check(&SignedSessionOutcomeKey(index), std::convert::identity)
176            .await
177            .0
178    }
179
180    pub async fn session_status(&self, session_index: u64) -> SessionStatus {
181        let mut dbtx = self.db.begin_transaction_nc().await;
182
183        match session_index.cmp(&get_finished_session_count_static(&mut dbtx).await) {
184            Ordering::Greater => SessionStatus::Initial,
185            Ordering::Equal => SessionStatus::Pending(
186                dbtx.find_by_prefix(&AcceptedItemPrefix)
187                    .await
188                    .map(|entry| entry.1)
189                    .collect()
190                    .await,
191            ),
192            Ordering::Less => SessionStatus::Complete(
193                dbtx.get_value(&SignedSessionOutcomeKey(session_index))
194                    .await
195                    .expect("There are no gaps in session outcomes")
196                    .session_outcome,
197            ),
198        }
199    }
200
201    pub async fn get_federation_status(&self) -> ApiResult<FederationStatus> {
202        let peers_connection_status = self.connection_status_channels.read().await.clone();
203        let last_ci_by_peer = self.last_ci_by_peer.read().await.clone();
204        let session_count = self.session_count().await;
205        let scheduled_shutdown = self.shutdown_receiver.borrow().to_owned();
206
207        let status_by_peer = peers_connection_status
208            .into_iter()
209            .map(|(peer, connection_status)| {
210                let last_contribution = last_ci_by_peer.get(&peer).copied();
211                let flagged = last_contribution.unwrap_or(0) + 1 < session_count;
212
213                let consensus_status = PeerStatus {
214                    last_contribution,
215                    connection_status,
216                    flagged,
217                };
218
219                (peer, consensus_status)
220            })
221            .collect::<HashMap<PeerId, PeerStatus>>();
222
223        let peers_flagged = status_by_peer
224            .values()
225            .filter(|status| status.flagged)
226            .count() as u64;
227
228        let peers_online = status_by_peer
229            .values()
230            .filter(|status| status.connection_status == PeerConnectionStatus::Connected)
231            .count() as u64;
232
233        let peers_offline = status_by_peer
234            .values()
235            .filter(|status| status.connection_status == PeerConnectionStatus::Disconnected)
236            .count() as u64;
237
238        Ok(FederationStatus {
239            session_count,
240            status_by_peer,
241            peers_online,
242            peers_offline,
243            peers_flagged,
244            scheduled_shutdown,
245        })
246    }
247
248    fn shutdown(&self, index: Option<u64>) {
249        self.shutdown_sender.send_replace(index);
250    }
251
252    async fn get_federation_audit(&self, _auth: &GuardianAuthToken) -> ApiResult<AuditSummary> {
253        let mut dbtx = self.db.begin_transaction_nc().await;
254        // Writes are related to compacting audit keys, which we can safely ignore
255        // within an API request since the compaction will happen when constructing an
256        // audit in the consensus server
257        dbtx.ignore_uncommitted();
258
259        let mut audit = Audit::default();
260        let mut module_instance_id_to_kind: HashMap<ModuleInstanceId, String> = HashMap::new();
261        for (module_instance_id, kind, module) in self.modules.iter_modules() {
262            module_instance_id_to_kind.insert(module_instance_id, kind.as_str().to_string());
263            module
264                .audit(
265                    &mut dbtx.to_ref_with_prefix_module_id(module_instance_id).0,
266                    &mut audit,
267                    module_instance_id,
268                )
269                .await;
270        }
271        Ok(AuditSummary::from_audit(
272            &audit,
273            &module_instance_id_to_kind,
274        ))
275    }
276
277    /// Uses the in-memory config to write a config backup tar archive that
278    /// guardians can download. Private keys are encrypted with the guardian
279    /// password, so it should be safe to store anywhere, this also means the
280    /// backup is useless without the password.
281    fn get_guardian_config_backup(
282        &self,
283        password: &str,
284        _auth: &GuardianAuthToken,
285    ) -> GuardianConfigBackup {
286        let mut tar_archive_builder = tar::Builder::new(Vec::new());
287
288        let mut append = |name: &Path, data: &[u8]| {
289            let mut header = tar::Header::new_gnu();
290            header.set_path(name).expect("Error setting path");
291            header.set_size(data.len() as u64);
292            header.set_mode(0o644);
293            header.set_cksum();
294            tar_archive_builder
295                .append(&header, data)
296                .expect("Error adding data to tar archive");
297        };
298
299        append(
300            &PathBuf::from(LOCAL_CONFIG).with_extension(JSON_EXT),
301            &serde_json::to_vec(&self.cfg.local).expect("Error encoding local config"),
302        );
303
304        append(
305            &PathBuf::from(CONSENSUS_CONFIG).with_extension(JSON_EXT),
306            &serde_json::to_vec(&self.cfg.consensus).expect("Error encoding consensus config"),
307        );
308
309        // Note that the encrypted config returned here uses a different salt than the
310        // on-disk version. While this may be confusing it shouldn't be a problem since
311        // the content and encryption key are the same. It's unpractical to read the
312        // on-disk version here since the server/api aren't aware of the config dir and
313        // ideally we can keep it that way.
314        let encryption_salt = random_salt();
315        append(&PathBuf::from(SALT_FILE), encryption_salt.as_bytes());
316
317        let private_config_bytes =
318            serde_json::to_vec(&self.cfg.private).expect("Error encoding private config");
319        let encryption_key = get_encryption_key(password, &encryption_salt)
320            .expect("Generating key from password failed");
321        let private_config_encrypted =
322            hex::encode(encrypt(private_config_bytes, &encryption_key).expect("Encryption failed"));
323        append(
324            &PathBuf::from(PRIVATE_CONFIG).with_extension(ENCRYPTED_EXT),
325            private_config_encrypted.as_bytes(),
326        );
327
328        let tar_archive_bytes = tar_archive_builder
329            .into_inner()
330            .expect("Error building tar archive");
331
332        GuardianConfigBackup { tar_archive_bytes }
333    }
334
335    async fn handle_backup_request<'s, 'dbtx, 'a>(
336        &'s self,
337        dbtx: &'dbtx mut DatabaseTransaction<'a>,
338        request: SignedBackupRequest,
339    ) -> Result<(), ApiError> {
340        let request = request
341            .verify_valid(SECP256K1)
342            .map_err(|_| ApiError::bad_request("invalid request".into()))?;
343
344        if request.payload.len() > BACKUP_REQUEST_MAX_PAYLOAD_SIZE_BYTES {
345            return Err(ApiError::bad_request("snapshot too large".into()));
346        }
347        debug!(target: LOG_NET_API, id = %request.id, len = request.payload.len(), "Received client backup request");
348        if let Some(prev) = dbtx.get_value(&ClientBackupKey(request.id)).await {
349            if request.timestamp <= prev.timestamp {
350                debug!(id = %request.id, len = request.payload.len(), "Received client backup request with old timestamp - ignoring");
351                return Err(ApiError::bad_request("timestamp too small".into()));
352            }
353        }
354
355        info!(target: LOG_NET_API, id = %request.id, len = request.payload.len(), "Storing new client backup");
356        let overwritten = dbtx
357            .insert_entry(
358                &ClientBackupKey(request.id),
359                &ClientBackupSnapshot {
360                    timestamp: request.timestamp,
361                    data: request.payload.clone(),
362                },
363            )
364            .await
365            .is_some();
366        BACKUP_WRITE_SIZE_BYTES.observe(request.payload.len() as f64);
367        if !overwritten {
368            dbtx.on_commit(|| STORED_BACKUPS_COUNT.inc());
369        }
370
371        Ok(())
372    }
373
374    async fn handle_recover_request(
375        &self,
376        dbtx: &mut DatabaseTransaction<'_>,
377        id: PublicKey,
378    ) -> Option<ClientBackupSnapshot> {
379        dbtx.get_value(&ClientBackupKey(id)).await
380    }
381
382    /// List API URL announcements from all peers we have received them from (at
383    /// least ourselves)
384    async fn api_announcements(&self) -> BTreeMap<PeerId, SignedApiAnnouncement> {
385        self.db
386            .begin_transaction_nc()
387            .await
388            .find_by_prefix(&ApiAnnouncementPrefix)
389            .await
390            .map(|(announcement_key, announcement)| (announcement_key.0, announcement))
391            .collect()
392            .await
393    }
394
395    /// Returns the tagged fedimintd version currently running
396    fn fedimintd_version(&self) -> String {
397        self.code_version_str.clone()
398    }
399
400    /// Add an API URL announcement from a peer to our database to be returned
401    /// by [`ConsensusApi::api_announcements`].
402    async fn submit_api_announcement(
403        &self,
404        peer_id: PeerId,
405        announcement: SignedApiAnnouncement,
406    ) -> Result<(), ApiError> {
407        let Some(peer_key) = self.cfg.consensus.broadcast_public_keys.get(&peer_id) else {
408            return Err(ApiError::bad_request("Peer not in federation".into()));
409        };
410
411        if !announcement.verify(SECP256K1, peer_key) {
412            return Err(ApiError::bad_request("Invalid signature".into()));
413        }
414
415        let mut dbtx = self.db.begin_transaction().await;
416
417        if let Some(existing_announcement) = dbtx.get_value(&ApiAnnouncementKey(peer_id)).await {
418            // If the current announcement is semantically identical to the new one (except
419            // for potentially having a different, valid signature) we return ok to allow
420            // the caller to stop submitting the value if they are in a retry loop.
421            if existing_announcement.api_announcement == announcement.api_announcement {
422                return Ok(());
423            }
424
425            // We only accept announcements with a nonce higher than the current one to
426            // avoid replay attacks.
427            if existing_announcement.api_announcement.nonce >= announcement.api_announcement.nonce {
428                return Err(ApiError::bad_request(
429                    "Outdated or redundant announcement".into(),
430                ));
431            }
432        }
433
434        dbtx.insert_entry(&ApiAnnouncementKey(peer_id), &announcement)
435            .await;
436        dbtx.commit_tx().await;
437        Ok(())
438    }
439
440    async fn sign_api_announcement(&self, new_url: SafeUrl) -> SignedApiAnnouncement {
441        self.db
442            .autocommit(
443                |dbtx, _| {
444                    let new_url_inner = new_url.clone();
445                    Box::pin(async move {
446                        let new_nonce = dbtx
447                            .get_value(&ApiAnnouncementKey(self.cfg.local.identity))
448                            .await
449                            .map_or(0, |a| a.api_announcement.nonce + 1);
450                        let announcement = ApiAnnouncement {
451                            api_url: new_url_inner,
452                            nonce: new_nonce,
453                        };
454                        let ctx = secp256k1::Secp256k1::new();
455                        let signed_announcement = announcement
456                            .sign(&ctx, &self.cfg.private.broadcast_secret_key.keypair(&ctx));
457
458                        dbtx.insert_entry(
459                            &ApiAnnouncementKey(self.cfg.local.identity),
460                            &signed_announcement,
461                        )
462                        .await;
463
464                        Result::<_, ()>::Ok(signed_announcement)
465                    })
466                },
467                None,
468            )
469            .await
470            .expect("Will not terminate on error")
471    }
472}
473
474#[async_trait]
475impl HasApiContext<ConsensusApi> for ConsensusApi {
476    async fn context(
477        &self,
478        request: &ApiRequestErased,
479        id: Option<ModuleInstanceId>,
480    ) -> (&ConsensusApi, ApiEndpointContext<'_>) {
481        let mut db = self.db.clone();
482        let mut dbtx = self.db.begin_transaction().await;
483        if let Some(id) = id {
484            db = self.db.with_prefix_module_id(id).0;
485            dbtx = dbtx.with_prefix_module_id(id).0;
486        }
487        (
488            self,
489            ApiEndpointContext::new(
490                db,
491                dbtx,
492                request.auth == Some(self.cfg.private.api_auth.clone()),
493                request.auth.clone(),
494            ),
495        )
496    }
497}
498
499#[async_trait]
500impl HasApiContext<DynServerModule> for ConsensusApi {
501    async fn context(
502        &self,
503        request: &ApiRequestErased,
504        id: Option<ModuleInstanceId>,
505    ) -> (&DynServerModule, ApiEndpointContext<'_>) {
506        let (_, context): (&ConsensusApi, _) = self.context(request, id).await;
507        (
508            self.modules.get_expect(id.expect("required module id")),
509            context,
510        )
511    }
512}
513
514pub fn server_endpoints() -> Vec<ApiEndpoint<ConsensusApi>> {
515    vec![
516        api_endpoint! {
517            VERSION_ENDPOINT,
518            ApiVersion::new(0, 0),
519            async |fedimint: &ConsensusApi, _context, _v: ()| -> SupportedApiVersionsSummary {
520                Ok(fedimint.api_versions_summary().to_owned())
521            }
522        },
523        api_endpoint! {
524            SUBMIT_TRANSACTION_ENDPOINT,
525            ApiVersion::new(0, 0),
526            async |fedimint: &ConsensusApi, _context, transaction: SerdeTransaction| -> SerdeModuleEncoding<TransactionSubmissionOutcome> {
527                let transaction = transaction
528                    .try_into_inner(&fedimint.modules.decoder_registry())
529                    .map_err(|e| ApiError::bad_request(e.to_string()))?;
530
531                // we return an inner error if and only if the submitted transaction is
532                // invalid and will be rejected if we were to submit it to consensus
533                Ok((&TransactionSubmissionOutcome(fedimint.submit_transaction(transaction).await)).into())
534            }
535        },
536        api_endpoint! {
537            AWAIT_TRANSACTION_ENDPOINT,
538            ApiVersion::new(0, 0),
539            async |fedimint: &ConsensusApi, _context, tx_hash: TransactionId| -> TransactionId {
540                debug!(transaction = %tx_hash, "Received request");
541
542                fedimint.await_transaction(tx_hash).await;
543
544                debug!(transaction = %tx_hash, "Sending outcome");
545
546                Ok(tx_hash)
547            }
548        },
549        api_endpoint! {
550            AWAIT_OUTPUT_OUTCOME_ENDPOINT,
551            ApiVersion::new(0, 0),
552            async |fedimint: &ConsensusApi, _context, outpoint: OutPoint| -> SerdeModuleEncoding<DynOutputOutcome> {
553                let outcome = fedimint
554                    .await_output_outcome(outpoint)
555                    .await
556                    .map_err(|e| ApiError::bad_request(e.to_string()))?;
557
558                Ok(outcome)
559            }
560        },
561        api_endpoint! {
562            INVITE_CODE_ENDPOINT,
563            ApiVersion::new(0, 0),
564            async |fedimint: &ConsensusApi, _context,  _v: ()| -> String {
565                Ok(fedimint.cfg.get_invite_code(fedimint.get_active_api_secret()).to_string())
566            }
567        },
568        api_endpoint! {
569            FEDERATION_ID_ENDPOINT,
570            ApiVersion::new(0, 2),
571            async |fedimint: &ConsensusApi, _context,  _v: ()| -> String {
572                Ok(fedimint.cfg.calculate_federation_id().to_string())
573            }
574        },
575        api_endpoint! {
576            CLIENT_CONFIG_ENDPOINT,
577            ApiVersion::new(0, 0),
578            async |fedimint: &ConsensusApi, _context, _v: ()| -> ClientConfig {
579                Ok(fedimint.client_cfg.clone())
580            }
581        },
582        // Helper endpoint for Admin UI that can't parse consensus encoding
583        api_endpoint! {
584            CLIENT_CONFIG_JSON_ENDPOINT,
585            ApiVersion::new(0, 0),
586            async |fedimint: &ConsensusApi, _context, _v: ()| -> JsonClientConfig {
587                Ok(fedimint.client_cfg.to_json())
588            }
589        },
590        api_endpoint! {
591            SERVER_CONFIG_CONSENSUS_HASH_ENDPOINT,
592            ApiVersion::new(0, 0),
593            async |fedimint: &ConsensusApi, _context, _v: ()| -> sha256::Hash {
594                Ok(fedimint.cfg.consensus.consensus_hash())
595            }
596        },
597        api_endpoint! {
598            STATUS_ENDPOINT,
599            ApiVersion::new(0, 0),
600            async |fedimint: &ConsensusApi, _context, _v: ()| -> StatusResponse {
601                Ok(StatusResponse {
602                    server: ServerStatus::ConsensusRunning,
603                    federation: Some(fedimint.get_federation_status().await?)
604                })
605            }
606        },
607        api_endpoint! {
608            SESSION_COUNT_ENDPOINT,
609            ApiVersion::new(0, 0),
610            async |fedimint: &ConsensusApi, _context, _v: ()| -> u64 {
611                Ok(fedimint.session_count().await)
612            }
613        },
614        api_endpoint! {
615            AWAIT_SESSION_OUTCOME_ENDPOINT,
616            ApiVersion::new(0, 0),
617            async |fedimint: &ConsensusApi, _context, index: u64| -> SerdeModuleEncoding<SessionOutcome> {
618                Ok((&fedimint.await_signed_session_outcome(index).await.session_outcome).into())
619            }
620        },
621        api_endpoint! {
622            AWAIT_SIGNED_SESSION_OUTCOME_ENDPOINT,
623            ApiVersion::new(0, 0),
624            async |fedimint: &ConsensusApi, _context, index: u64| -> SerdeModuleEncoding<SignedSessionOutcome> {
625                Ok((&fedimint.await_signed_session_outcome(index).await).into())
626            }
627        },
628        api_endpoint! {
629            SESSION_STATUS_ENDPOINT,
630            ApiVersion::new(0, 1),
631            async |fedimint: &ConsensusApi, _context, index: u64| -> SerdeModuleEncoding<SessionStatus> {
632                Ok((&fedimint.session_status(index).await).into())
633            }
634        },
635        api_endpoint! {
636            SHUTDOWN_ENDPOINT,
637            ApiVersion::new(0, 3),
638            async |fedimint: &ConsensusApi, context, index: Option<u64>| -> () {
639                check_auth(context)?;
640                fedimint.shutdown(index);
641                Ok(())
642            }
643        },
644        api_endpoint! {
645            AUDIT_ENDPOINT,
646            ApiVersion::new(0, 0),
647            async |fedimint: &ConsensusApi, context, _v: ()| -> AuditSummary {
648                let auth = check_auth(context)?;
649                Ok(fedimint.get_federation_audit(&auth).await?)
650            }
651        },
652        api_endpoint! {
653            GUARDIAN_CONFIG_BACKUP_ENDPOINT,
654            ApiVersion::new(0, 2),
655            async |fedimint: &ConsensusApi, context, _v: ()| -> GuardianConfigBackup {
656                let auth = check_auth(context)?;
657                let password = context.request_auth().expect("Auth was checked before").0;
658                Ok(fedimint.get_guardian_config_backup(&password, &auth))
659            }
660        },
661        api_endpoint! {
662            BACKUP_ENDPOINT,
663            ApiVersion::new(0, 0),
664            async |fedimint: &ConsensusApi, context, request: SignedBackupRequest| -> () {
665                fedimint
666                    .handle_backup_request(&mut context.dbtx().into_nc(), request).await?;
667                Ok(())
668
669            }
670        },
671        api_endpoint! {
672            RECOVER_ENDPOINT,
673            ApiVersion::new(0, 0),
674            async |fedimint: &ConsensusApi, context, id: PublicKey| -> Option<ClientBackupSnapshot> {
675                Ok(fedimint
676                    .handle_recover_request(&mut context.dbtx().into_nc(), id).await)
677            }
678        },
679        api_endpoint! {
680            AUTH_ENDPOINT,
681            ApiVersion::new(0, 0),
682            async |_fedimint: &ConsensusApi, context, _v: ()| -> () {
683                check_auth(context)?;
684                Ok(())
685            }
686        },
687        api_endpoint! {
688            API_ANNOUNCEMENTS_ENDPOINT,
689            ApiVersion::new(0, 3),
690            async |fedimint: &ConsensusApi, _context, _v: ()| -> BTreeMap<PeerId, SignedApiAnnouncement> {
691                Ok(fedimint.api_announcements().await)
692            }
693        },
694        api_endpoint! {
695            SUBMIT_API_ANNOUNCEMENT_ENDPOINT,
696            ApiVersion::new(0, 3),
697            async |fedimint: &ConsensusApi, _context, submission: SignedApiAnnouncementSubmission| -> () {
698                fedimint.submit_api_announcement(submission.peer_id, submission.signed_api_announcement).await
699            }
700        },
701        api_endpoint! {
702            SIGN_API_ANNOUNCEMENT_ENDPOINT,
703            ApiVersion::new(0, 3),
704            async |fedimint: &ConsensusApi, context, new_url: SafeUrl| -> SignedApiAnnouncement {
705                check_auth(context)?;
706                Ok(fedimint.sign_api_announcement(new_url).await)
707            }
708        },
709        api_endpoint! {
710            FEDIMINTD_VERSION_ENDPOINT,
711            ApiVersion::new(0, 4),
712            async |fedimint: &ConsensusApi, _context, _v: ()| -> String {
713                Ok(fedimint.fedimintd_version())
714            }
715        },
716    ]
717}