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