sui_bridge/
node.rs

1// Copyright (c) Mysten Labs, Inc.
2// SPDX-License-Identifier: Apache-2.0
3
4use crate::config::WatchdogConfig;
5use crate::crypto::BridgeAuthorityPublicKeyBytes;
6use crate::metered_eth_provider::MeteredEthHttpProvider;
7use crate::sui_bridge_watchdog::eth_bridge_status::EthBridgeStatus;
8use crate::sui_bridge_watchdog::eth_vault_balance::{EthereumVaultBalance, VaultAsset};
9use crate::sui_bridge_watchdog::metrics::WatchdogMetrics;
10use crate::sui_bridge_watchdog::sui_bridge_status::SuiBridgeStatus;
11use crate::sui_bridge_watchdog::total_supplies::TotalSupplies;
12use crate::sui_bridge_watchdog::{BridgeWatchDog, Observable};
13use crate::sui_client::SuiBridgeClient;
14use crate::types::BridgeCommittee;
15use crate::utils::{
16    get_committee_voting_power_by_name, get_eth_contract_addresses, get_validator_names_by_pub_keys,
17};
18use crate::{
19    action_executor::BridgeActionExecutor,
20    client::bridge_authority_aggregator::BridgeAuthorityAggregator,
21    config::{BridgeClientConfig, BridgeNodeConfig},
22    eth_syncer::EthSyncer,
23    events::init_all_struct_tags,
24    metrics::BridgeMetrics,
25    monitor::BridgeMonitor,
26    orchestrator::BridgeOrchestrator,
27    server::{BridgeNodePublicMetadata, handler::BridgeRequestHandler, run_server},
28    storage::BridgeOrchestratorTables,
29    sui_syncer::SuiSyncer,
30};
31use arc_swap::ArcSwap;
32use ethers::providers::Provider;
33use ethers::types::Address as EthAddress;
34use mysten_metrics::spawn_logged_monitored_task;
35use std::collections::BTreeMap;
36use std::{
37    collections::HashMap,
38    net::{IpAddr, Ipv4Addr, SocketAddr},
39    sync::Arc,
40    time::Duration,
41};
42use sui_types::{
43    Identifier,
44    bridge::{
45        BRIDGE_COMMITTEE_MODULE_NAME, BRIDGE_LIMITER_MODULE_NAME, BRIDGE_MODULE_NAME,
46        BRIDGE_TREASURY_MODULE_NAME,
47    },
48    event::EventID,
49};
50use tokio::task::JoinHandle;
51use tracing::info;
52
53pub async fn run_bridge_node(
54    config: BridgeNodeConfig,
55    metadata: BridgeNodePublicMetadata,
56    prometheus_registry: prometheus::Registry,
57) -> anyhow::Result<JoinHandle<()>> {
58    init_all_struct_tags();
59    let metrics = Arc::new(BridgeMetrics::new(&prometheus_registry));
60    let watchdog_config = config.watchdog_config.clone();
61    let (server_config, client_config) = config.validate(metrics.clone()).await?;
62    let sui_chain_identifier = server_config
63        .sui_client
64        .get_chain_identifier()
65        .await
66        .map_err(|e| anyhow::anyhow!("Failed to get sui chain identifier: {:?}", e))?;
67    let eth_chain_identifier = server_config
68        .eth_client
69        .get_chain_id()
70        .await
71        .map_err(|e| anyhow::anyhow!("Failed to get eth chain identifier: {:?}", e))?;
72    prometheus_registry
73        .register(mysten_metrics::bridge_uptime_metric(
74            "bridge",
75            metadata.version,
76            &sui_chain_identifier,
77            &eth_chain_identifier.to_string(),
78            client_config.is_some(),
79        ))
80        .unwrap();
81
82    let committee = Arc::new(
83        server_config
84            .sui_client
85            .get_bridge_committee()
86            .await
87            .expect("Failed to get committee"),
88    );
89    let mut handles = vec![];
90
91    // Start watchdog
92    let eth_provider = server_config.eth_client.provider();
93    let eth_bridge_proxy_address = server_config.eth_bridge_proxy_address;
94    let sui_client = server_config.sui_client.clone();
95    handles.push(spawn_logged_monitored_task!(start_watchdog(
96        watchdog_config,
97        &prometheus_registry,
98        eth_provider,
99        eth_bridge_proxy_address,
100        sui_client
101    )));
102
103    // Update voting right metrics
104    // Before reconfiguration happens we only set it once when the node starts
105    let sui_system = server_config
106        .sui_client
107        .jsonrpc_client()
108        .governance_api()
109        .get_latest_sui_system_state()
110        .await?;
111
112    // Start Client
113    if let Some(client_config) = client_config {
114        let committee_keys_to_names =
115            Arc::new(get_validator_names_by_pub_keys(&committee, &sui_system).await);
116        let client_components = start_client_components(
117            client_config,
118            committee.clone(),
119            committee_keys_to_names,
120            metrics.clone(),
121        )
122        .await?;
123        handles.extend(client_components);
124    }
125
126    let committee_name_mapping = get_committee_voting_power_by_name(&committee, &sui_system).await;
127    for (name, voting_power) in committee_name_mapping.into_iter() {
128        metrics
129            .current_bridge_voting_rights
130            .with_label_values(&[name.as_str()])
131            .set(voting_power as i64);
132    }
133
134    // Start Server
135    let socket_address = SocketAddr::new(
136        IpAddr::V4(Ipv4Addr::new(0, 0, 0, 0)),
137        server_config.server_listen_port,
138    );
139    Ok(run_server(
140        &socket_address,
141        BridgeRequestHandler::new(
142            server_config.key,
143            server_config.sui_client,
144            server_config.eth_client,
145            server_config.approved_governance_actions,
146        ),
147        metrics,
148        Arc::new(metadata),
149    ))
150}
151
152async fn start_watchdog(
153    watchdog_config: Option<WatchdogConfig>,
154    registry: &prometheus::Registry,
155    eth_provider: Arc<Provider<MeteredEthHttpProvider>>,
156    eth_bridge_proxy_address: EthAddress,
157    sui_client: Arc<SuiBridgeClient>,
158) {
159    let watchdog_metrics = WatchdogMetrics::new(registry);
160    let (
161        _committee_address,
162        _limiter_address,
163        vault_address,
164        _config_address,
165        weth_address,
166        usdt_address,
167        wbtc_address,
168        lbtc_address,
169    ) = get_eth_contract_addresses(eth_bridge_proxy_address, &eth_provider)
170        .await
171        .unwrap_or_else(|e| panic!("get_eth_contract_addresses should not fail: {}", e));
172
173    let eth_vault_balance = EthereumVaultBalance::new(
174        eth_provider.clone(),
175        vault_address,
176        weth_address,
177        VaultAsset::WETH,
178        watchdog_metrics.eth_vault_balance.clone(),
179    )
180    .await
181    .unwrap_or_else(|e| panic!("Failed to create eth vault balance: {}", e));
182
183    let usdt_vault_balance = EthereumVaultBalance::new(
184        eth_provider.clone(),
185        vault_address,
186        usdt_address,
187        VaultAsset::USDT,
188        watchdog_metrics.usdt_vault_balance.clone(),
189    )
190    .await
191    .unwrap_or_else(|e| panic!("Failed to create usdt vault balance: {}", e));
192
193    let wbtc_vault_balance = EthereumVaultBalance::new(
194        eth_provider.clone(),
195        vault_address,
196        wbtc_address,
197        VaultAsset::WBTC,
198        watchdog_metrics.wbtc_vault_balance.clone(),
199    )
200    .await
201    .unwrap_or_else(|e| panic!("Failed to create wbtc vault balance: {}", e));
202
203    let lbtc_vault_balance = if !lbtc_address.is_zero() {
204        Some(
205            EthereumVaultBalance::new(
206                eth_provider.clone(),
207                vault_address,
208                lbtc_address,
209                VaultAsset::LBTC,
210                watchdog_metrics.lbtc_vault_balance.clone(),
211            )
212            .await
213            .unwrap_or_else(|e| panic!("Failed to create lbtc vault balance: {}", e)),
214        )
215    } else {
216        None
217    };
218
219    let eth_bridge_status = EthBridgeStatus::new(
220        eth_provider,
221        eth_bridge_proxy_address,
222        watchdog_metrics.eth_bridge_paused.clone(),
223    );
224
225    let sui_bridge_status = SuiBridgeStatus::new(
226        sui_client.clone(),
227        watchdog_metrics.sui_bridge_paused.clone(),
228    );
229
230    let mut observables: Vec<Box<dyn Observable + Send + Sync>> = vec![
231        Box::new(eth_vault_balance),
232        Box::new(usdt_vault_balance),
233        Box::new(wbtc_vault_balance),
234        Box::new(eth_bridge_status),
235        Box::new(sui_bridge_status),
236    ];
237
238    // Add lbtc_vault_balance if it's available
239    if let Some(balance) = lbtc_vault_balance {
240        observables.push(Box::new(balance));
241    }
242
243    if let Some(watchdog_config) = watchdog_config
244        && !watchdog_config.total_supplies.is_empty()
245    {
246        let total_supplies = TotalSupplies::new(
247            Arc::new(sui_client.jsonrpc_client().clone()),
248            watchdog_config.total_supplies,
249            watchdog_metrics.total_supplies.clone(),
250        );
251        observables.push(Box::new(total_supplies));
252    }
253
254    BridgeWatchDog::new(observables).run().await
255}
256
257// TODO: is there a way to clean up the overrides after it's stored in DB?
258async fn start_client_components(
259    client_config: BridgeClientConfig,
260    committee: Arc<BridgeCommittee>,
261    committee_keys_to_names: Arc<BTreeMap<BridgeAuthorityPublicKeyBytes, String>>,
262    metrics: Arc<BridgeMetrics>,
263) -> anyhow::Result<Vec<JoinHandle<()>>> {
264    let store: std::sync::Arc<BridgeOrchestratorTables> =
265        BridgeOrchestratorTables::new(&client_config.db_path.join("client"));
266    let sui_modules_to_watch = get_sui_modules_to_watch(
267        &store,
268        client_config.sui_bridge_module_last_processed_event_id_override,
269    );
270    let eth_contracts_to_watch = get_eth_contracts_to_watch(
271        &store,
272        &client_config.eth_contracts,
273        client_config.eth_contracts_start_block_fallback,
274        client_config.eth_contracts_start_block_override,
275    );
276
277    let sui_client = client_config.sui_client.clone();
278
279    let mut all_handles = vec![];
280    let (task_handles, eth_events_rx, _) =
281        EthSyncer::new(client_config.eth_client.clone(), eth_contracts_to_watch)
282            .run(metrics.clone())
283            .await
284            .expect("Failed to start eth syncer");
285    all_handles.extend(task_handles);
286
287    let (task_handles, sui_events_rx) = SuiSyncer::new(
288        client_config.sui_client,
289        sui_modules_to_watch,
290        metrics.clone(),
291    )
292    .run(Duration::from_secs(2))
293    .await
294    .expect("Failed to start sui syncer");
295    all_handles.extend(task_handles);
296
297    let bridge_auth_agg = Arc::new(ArcSwap::from(Arc::new(BridgeAuthorityAggregator::new(
298        committee,
299        metrics.clone(),
300        committee_keys_to_names,
301    ))));
302    // TODO: should we use one query instead of two?
303    let sui_token_type_tags = sui_client.get_token_id_map().await.unwrap();
304    let is_bridge_paused = sui_client.is_bridge_paused().await.unwrap();
305
306    let (bridge_pause_tx, bridge_pause_rx) = tokio::sync::watch::channel(is_bridge_paused);
307
308    let (sui_monitor_tx, sui_monitor_rx) = mysten_metrics::metered_channel::channel(
309        10000,
310        &mysten_metrics::get_metrics()
311            .unwrap()
312            .channel_inflight
313            .with_label_values(&["sui_monitor_queue"]),
314    );
315    let (eth_monitor_tx, eth_monitor_rx) = mysten_metrics::metered_channel::channel(
316        10000,
317        &mysten_metrics::get_metrics()
318            .unwrap()
319            .channel_inflight
320            .with_label_values(&["eth_monitor_queue"]),
321    );
322
323    let sui_token_type_tags = Arc::new(ArcSwap::from(Arc::new(sui_token_type_tags)));
324    let bridge_action_executor = BridgeActionExecutor::new(
325        sui_client.clone(),
326        bridge_auth_agg.clone(),
327        store.clone(),
328        client_config.key,
329        client_config.sui_address,
330        client_config.gas_object_ref.0,
331        sui_token_type_tags.clone(),
332        bridge_pause_rx,
333        metrics.clone(),
334    )
335    .await;
336
337    let monitor = BridgeMonitor::new(
338        sui_client.clone(),
339        sui_monitor_rx,
340        eth_monitor_rx,
341        bridge_auth_agg.clone(),
342        bridge_pause_tx,
343        sui_token_type_tags,
344        metrics.clone(),
345    );
346    all_handles.push(spawn_logged_monitored_task!(monitor.run()));
347
348    let orchestrator = BridgeOrchestrator::new(
349        sui_client,
350        sui_events_rx,
351        eth_events_rx,
352        store.clone(),
353        sui_monitor_tx,
354        eth_monitor_tx,
355        metrics,
356    );
357
358    all_handles.extend(orchestrator.run(bridge_action_executor).await);
359    Ok(all_handles)
360}
361
362fn get_sui_modules_to_watch(
363    store: &std::sync::Arc<BridgeOrchestratorTables>,
364    sui_bridge_module_last_processed_event_id_override: Option<EventID>,
365) -> HashMap<Identifier, Option<EventID>> {
366    let sui_bridge_modules = vec![
367        BRIDGE_MODULE_NAME.to_owned(),
368        BRIDGE_COMMITTEE_MODULE_NAME.to_owned(),
369        BRIDGE_TREASURY_MODULE_NAME.to_owned(),
370        BRIDGE_LIMITER_MODULE_NAME.to_owned(),
371    ];
372    if let Some(cursor) = sui_bridge_module_last_processed_event_id_override {
373        info!("Overriding cursor for sui bridge modules to {:?}", cursor);
374        return HashMap::from_iter(
375            sui_bridge_modules
376                .iter()
377                .map(|module| (module.clone(), Some(cursor))),
378        );
379    }
380
381    let sui_bridge_module_stored_cursor = store
382        .get_sui_event_cursors(&sui_bridge_modules)
383        .expect("Failed to get eth sui event cursors from storage");
384    let mut sui_modules_to_watch = HashMap::new();
385    for (module_identifier, cursor) in sui_bridge_modules
386        .iter()
387        .zip(sui_bridge_module_stored_cursor)
388    {
389        if cursor.is_none() {
390            info!(
391                "No cursor found for sui bridge module {} in storage or config override, query start from the beginning.",
392                module_identifier
393            );
394        }
395        sui_modules_to_watch.insert(module_identifier.clone(), cursor);
396    }
397    sui_modules_to_watch
398}
399
400fn get_eth_contracts_to_watch(
401    store: &std::sync::Arc<BridgeOrchestratorTables>,
402    eth_contracts: &[EthAddress],
403    eth_contracts_start_block_fallback: u64,
404    eth_contracts_start_block_override: Option<u64>,
405) -> HashMap<EthAddress, u64> {
406    let stored_eth_cursors = store
407        .get_eth_event_cursors(eth_contracts)
408        .expect("Failed to get eth event cursors from storage");
409    let mut eth_contracts_to_watch = HashMap::new();
410    for (contract, stored_cursor) in eth_contracts.iter().zip(stored_eth_cursors) {
411        // start block precedence:
412        // eth_contracts_start_block_override > stored cursor > eth_contracts_start_block_fallback
413        match (eth_contracts_start_block_override, stored_cursor) {
414            (Some(override_), _) => {
415                eth_contracts_to_watch.insert(*contract, override_);
416                info!(
417                    "Overriding cursor for eth bridge contract {} to {}. Stored cursor: {:?}",
418                    contract, override_, stored_cursor
419                );
420            }
421            (None, Some(stored_cursor)) => {
422                // +1: The stored value is the last block that was processed, so we start from the next block.
423                eth_contracts_to_watch.insert(*contract, stored_cursor + 1);
424            }
425            (None, None) => {
426                // If no cursor is found, start from the fallback block.
427                eth_contracts_to_watch.insert(*contract, eth_contracts_start_block_fallback);
428            }
429        }
430    }
431    eth_contracts_to_watch
432}
433
434#[cfg(test)]
435mod tests {
436    use ethers::types::Address as EthAddress;
437    use prometheus::Registry;
438
439    use super::*;
440    use crate::config::BridgeNodeConfig;
441    use crate::config::EthConfig;
442    use crate::config::SuiConfig;
443    use crate::config::default_ed25519_key_pair;
444    use crate::e2e_tests::test_utils::BridgeTestCluster;
445    use crate::e2e_tests::test_utils::BridgeTestClusterBuilder;
446    use crate::utils::wait_for_server_to_be_up;
447    use fastcrypto::secp256k1::Secp256k1KeyPair;
448    use sui_config::local_ip_utils::get_available_port;
449    use sui_types::base_types::SuiAddress;
450    use sui_types::bridge::BridgeChainId;
451    use sui_types::crypto::EncodeDecodeBase64;
452    use sui_types::crypto::KeypairTraits;
453    use sui_types::crypto::SuiKeyPair;
454    use sui_types::crypto::get_key_pair;
455    use sui_types::digests::TransactionDigest;
456    use sui_types::event::EventID;
457    use tempfile::tempdir;
458
459    #[tokio::test]
460    async fn test_get_eth_contracts_to_watch() {
461        telemetry_subscribers::init_for_testing();
462        let temp_dir = tempfile::tempdir().unwrap();
463        let eth_contracts = vec![
464            EthAddress::from_low_u64_be(1),
465            EthAddress::from_low_u64_be(2),
466        ];
467        let store = BridgeOrchestratorTables::new(temp_dir.path());
468
469        // No override, no watermark found in DB, use fallback
470        let contracts = get_eth_contracts_to_watch(&store, &eth_contracts, 10, None);
471        assert_eq!(
472            contracts,
473            vec![(eth_contracts[0], 10), (eth_contracts[1], 10)]
474                .into_iter()
475                .collect::<HashMap<_, _>>()
476        );
477
478        // no watermark found in DB, use override
479        let contracts = get_eth_contracts_to_watch(&store, &eth_contracts, 10, Some(420));
480        assert_eq!(
481            contracts,
482            vec![(eth_contracts[0], 420), (eth_contracts[1], 420)]
483                .into_iter()
484                .collect::<HashMap<_, _>>()
485        );
486
487        store
488            .update_eth_event_cursor(eth_contracts[0], 100)
489            .unwrap();
490        store
491            .update_eth_event_cursor(eth_contracts[1], 102)
492            .unwrap();
493
494        // No override, found watermarks in DB, use +1
495        let contracts = get_eth_contracts_to_watch(&store, &eth_contracts, 10, None);
496        assert_eq!(
497            contracts,
498            vec![(eth_contracts[0], 101), (eth_contracts[1], 103)]
499                .into_iter()
500                .collect::<HashMap<_, _>>()
501        );
502
503        // use override
504        let contracts = get_eth_contracts_to_watch(&store, &eth_contracts, 10, Some(200));
505        assert_eq!(
506            contracts,
507            vec![(eth_contracts[0], 200), (eth_contracts[1], 200)]
508                .into_iter()
509                .collect::<HashMap<_, _>>()
510        );
511    }
512
513    #[tokio::test]
514    async fn test_get_sui_modules_to_watch() {
515        telemetry_subscribers::init_for_testing();
516        let temp_dir = tempfile::tempdir().unwrap();
517
518        let store = BridgeOrchestratorTables::new(temp_dir.path());
519        let bridge_module = BRIDGE_MODULE_NAME.to_owned();
520        let committee_module = BRIDGE_COMMITTEE_MODULE_NAME.to_owned();
521        let treasury_module = BRIDGE_TREASURY_MODULE_NAME.to_owned();
522        let limiter_module = BRIDGE_LIMITER_MODULE_NAME.to_owned();
523        // No override, no stored watermark, use None
524        let sui_modules_to_watch = get_sui_modules_to_watch(&store, None);
525        assert_eq!(
526            sui_modules_to_watch,
527            vec![
528                (bridge_module.clone(), None),
529                (committee_module.clone(), None),
530                (treasury_module.clone(), None),
531                (limiter_module.clone(), None)
532            ]
533            .into_iter()
534            .collect::<HashMap<_, _>>()
535        );
536
537        // no stored watermark, use override
538        let override_cursor = EventID {
539            tx_digest: TransactionDigest::random(),
540            event_seq: 42,
541        };
542        let sui_modules_to_watch = get_sui_modules_to_watch(&store, Some(override_cursor));
543        assert_eq!(
544            sui_modules_to_watch,
545            vec![
546                (bridge_module.clone(), Some(override_cursor)),
547                (committee_module.clone(), Some(override_cursor)),
548                (treasury_module.clone(), Some(override_cursor)),
549                (limiter_module.clone(), Some(override_cursor))
550            ]
551            .into_iter()
552            .collect::<HashMap<_, _>>()
553        );
554
555        // No override, found stored watermark for `bridge` module, use stored watermark for `bridge`
556        // and None for `committee`
557        let stored_cursor = EventID {
558            tx_digest: TransactionDigest::random(),
559            event_seq: 100,
560        };
561        store
562            .update_sui_event_cursor(bridge_module.clone(), stored_cursor)
563            .unwrap();
564        let sui_modules_to_watch = get_sui_modules_to_watch(&store, None);
565        assert_eq!(
566            sui_modules_to_watch,
567            vec![
568                (bridge_module.clone(), Some(stored_cursor)),
569                (committee_module.clone(), None),
570                (treasury_module.clone(), None),
571                (limiter_module.clone(), None)
572            ]
573            .into_iter()
574            .collect::<HashMap<_, _>>()
575        );
576
577        // found stored watermark, use override
578        let stored_cursor = EventID {
579            tx_digest: TransactionDigest::random(),
580            event_seq: 100,
581        };
582        store
583            .update_sui_event_cursor(committee_module.clone(), stored_cursor)
584            .unwrap();
585        let sui_modules_to_watch = get_sui_modules_to_watch(&store, Some(override_cursor));
586        assert_eq!(
587            sui_modules_to_watch,
588            vec![
589                (bridge_module.clone(), Some(override_cursor)),
590                (committee_module.clone(), Some(override_cursor)),
591                (treasury_module.clone(), Some(override_cursor)),
592                (limiter_module.clone(), Some(override_cursor))
593            ]
594            .into_iter()
595            .collect::<HashMap<_, _>>()
596        );
597    }
598
599    #[tokio::test(flavor = "multi_thread", worker_threads = 8)]
600    async fn test_starting_bridge_node() {
601        telemetry_subscribers::init_for_testing();
602        let bridge_test_cluster = setup().await;
603        let kp = bridge_test_cluster.bridge_authority_key(0);
604
605        // prepare node config (server only)
606        let tmp_dir = tempdir().unwrap().keep();
607        let authority_key_path = "test_starting_bridge_node_bridge_authority_key";
608        let server_listen_port = get_available_port("127.0.0.1");
609        let base64_encoded = kp.encode_base64();
610        std::fs::write(tmp_dir.join(authority_key_path), base64_encoded).unwrap();
611
612        let config = BridgeNodeConfig {
613            server_listen_port,
614            metrics_port: get_available_port("127.0.0.1"),
615            bridge_authority_key_path: tmp_dir.join(authority_key_path),
616            sui: SuiConfig {
617                sui_rpc_url: bridge_test_cluster.sui_rpc_url(),
618                sui_bridge_chain_id: BridgeChainId::SuiCustom as u8,
619                bridge_client_key_path: None,
620                bridge_client_gas_object: None,
621                sui_bridge_module_last_processed_event_id_override: None,
622            },
623            eth: EthConfig {
624                eth_rpc_url: bridge_test_cluster.eth_rpc_url(),
625                eth_bridge_proxy_address: bridge_test_cluster.sui_bridge_address(),
626                eth_bridge_chain_id: BridgeChainId::EthCustom as u8,
627                eth_contracts_start_block_fallback: None,
628                eth_contracts_start_block_override: None,
629            },
630            approved_governance_actions: vec![],
631            run_client: false,
632            db_path: None,
633            metrics_key_pair: default_ed25519_key_pair(),
634            metrics: None,
635            watchdog_config: None,
636        };
637        // Spawn bridge node in memory
638        let _handle = run_bridge_node(
639            config,
640            BridgeNodePublicMetadata::empty_for_testing(),
641            Registry::new(),
642        )
643        .await
644        .unwrap();
645
646        let server_url = format!("http://127.0.0.1:{}", server_listen_port);
647        // Now we expect to see the server to be up and running.
648        let res = wait_for_server_to_be_up(server_url, 5).await;
649        res.unwrap();
650    }
651
652    #[tokio::test(flavor = "multi_thread", worker_threads = 8)]
653    async fn test_starting_bridge_node_with_client() {
654        telemetry_subscribers::init_for_testing();
655        let bridge_test_cluster = setup().await;
656        let kp = bridge_test_cluster.bridge_authority_key(0);
657
658        // prepare node config (server + client)
659        let tmp_dir = tempdir().unwrap().keep();
660        let db_path = tmp_dir.join("test_starting_bridge_node_with_client_db");
661        let authority_key_path = "test_starting_bridge_node_with_client_bridge_authority_key";
662        let server_listen_port = get_available_port("127.0.0.1");
663
664        let base64_encoded = kp.encode_base64();
665        std::fs::write(tmp_dir.join(authority_key_path), base64_encoded).unwrap();
666
667        let client_sui_address = SuiAddress::from(kp.public());
668        let sender_address = bridge_test_cluster.sui_user_address();
669        // send some gas to this address
670        bridge_test_cluster
671            .test_cluster
672            .inner
673            .transfer_sui_must_exceed(sender_address, client_sui_address, 1000000000)
674            .await;
675
676        let config = BridgeNodeConfig {
677            server_listen_port,
678            metrics_port: get_available_port("127.0.0.1"),
679            bridge_authority_key_path: tmp_dir.join(authority_key_path),
680            sui: SuiConfig {
681                sui_rpc_url: bridge_test_cluster.sui_rpc_url(),
682                sui_bridge_chain_id: BridgeChainId::SuiCustom as u8,
683                bridge_client_key_path: None,
684                bridge_client_gas_object: None,
685                sui_bridge_module_last_processed_event_id_override: Some(EventID {
686                    tx_digest: TransactionDigest::random(),
687                    event_seq: 0,
688                }),
689            },
690            eth: EthConfig {
691                eth_rpc_url: bridge_test_cluster.eth_rpc_url(),
692                eth_bridge_proxy_address: bridge_test_cluster.sui_bridge_address(),
693                eth_bridge_chain_id: BridgeChainId::EthCustom as u8,
694                eth_contracts_start_block_fallback: Some(0),
695                eth_contracts_start_block_override: None,
696            },
697            approved_governance_actions: vec![],
698            run_client: true,
699            db_path: Some(db_path),
700            metrics_key_pair: default_ed25519_key_pair(),
701            metrics: None,
702            watchdog_config: None,
703        };
704        // Spawn bridge node in memory
705        let _handle = run_bridge_node(
706            config,
707            BridgeNodePublicMetadata::empty_for_testing(),
708            Registry::new(),
709        )
710        .await
711        .unwrap();
712
713        let server_url = format!("http://127.0.0.1:{}", server_listen_port);
714        // Now we expect to see the server to be up and running.
715        // client components are spawned earlier than server, so as long as the server is up,
716        // we know the client components are already running.
717        let res = wait_for_server_to_be_up(server_url, 5).await;
718        res.unwrap();
719    }
720
721    #[tokio::test(flavor = "multi_thread", worker_threads = 8)]
722    async fn test_starting_bridge_node_with_client_and_separate_client_key() {
723        telemetry_subscribers::init_for_testing();
724        let bridge_test_cluster = setup().await;
725        let kp = bridge_test_cluster.bridge_authority_key(0);
726
727        // prepare node config (server + client)
728        let tmp_dir = tempdir().unwrap().keep();
729        let db_path =
730            tmp_dir.join("test_starting_bridge_node_with_client_and_separate_client_key_db");
731        let authority_key_path =
732            "test_starting_bridge_node_with_client_and_separate_client_key_bridge_authority_key";
733        let server_listen_port = get_available_port("127.0.0.1");
734
735        // prepare bridge authority key
736        let base64_encoded = kp.encode_base64();
737        std::fs::write(tmp_dir.join(authority_key_path), base64_encoded).unwrap();
738
739        // prepare bridge client key
740        let (_, kp): (_, Secp256k1KeyPair) = get_key_pair();
741        let kp = SuiKeyPair::from(kp);
742        let client_key_path =
743            "test_starting_bridge_node_with_client_and_separate_client_key_bridge_client_key";
744        std::fs::write(tmp_dir.join(client_key_path), kp.encode_base64()).unwrap();
745        let client_sui_address = SuiAddress::from(&kp.public());
746        let sender_address = bridge_test_cluster.sui_user_address();
747        // send some gas to this address
748        let gas_obj = bridge_test_cluster
749            .test_cluster
750            .inner
751            .transfer_sui_must_exceed(sender_address, client_sui_address, 1000000000)
752            .await;
753
754        let config = BridgeNodeConfig {
755            server_listen_port,
756            metrics_port: get_available_port("127.0.0.1"),
757            bridge_authority_key_path: tmp_dir.join(authority_key_path),
758            sui: SuiConfig {
759                sui_rpc_url: bridge_test_cluster.sui_rpc_url(),
760                sui_bridge_chain_id: BridgeChainId::SuiCustom as u8,
761                bridge_client_key_path: Some(tmp_dir.join(client_key_path)),
762                bridge_client_gas_object: Some(gas_obj),
763                sui_bridge_module_last_processed_event_id_override: Some(EventID {
764                    tx_digest: TransactionDigest::random(),
765                    event_seq: 0,
766                }),
767            },
768            eth: EthConfig {
769                eth_rpc_url: bridge_test_cluster.eth_rpc_url(),
770                eth_bridge_proxy_address: bridge_test_cluster.sui_bridge_address(),
771                eth_bridge_chain_id: BridgeChainId::EthCustom as u8,
772                eth_contracts_start_block_fallback: Some(0),
773                eth_contracts_start_block_override: Some(0),
774            },
775            approved_governance_actions: vec![],
776            run_client: true,
777            db_path: Some(db_path),
778            metrics_key_pair: default_ed25519_key_pair(),
779            metrics: None,
780            watchdog_config: None,
781        };
782        // Spawn bridge node in memory
783        let _handle = run_bridge_node(
784            config,
785            BridgeNodePublicMetadata::empty_for_testing(),
786            Registry::new(),
787        )
788        .await
789        .unwrap();
790
791        let server_url = format!("http://127.0.0.1:{}", server_listen_port);
792        // Now we expect to see the server to be up and running.
793        // client components are spawned earlier than server, so as long as the server is up,
794        // we know the client components are already running.
795        let res = wait_for_server_to_be_up(server_url, 5).await;
796        res.unwrap();
797    }
798
799    async fn setup() -> BridgeTestCluster {
800        BridgeTestClusterBuilder::new()
801            .with_eth_env(true)
802            .with_bridge_cluster(false)
803            .with_num_validators(2)
804            .build()
805            .await
806    }
807}