d-engine-server 0.2.3

Production-ready Raft consensus engine server and runtime
Documentation
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
//! Zero-overhead KV client for embedded d-engine.
//!
//! [`EmbeddedClient`] provides direct access to Raft state machine
//! without gRPC serialization or network traversal.
//!
//! # Performance
//! - **10-20x faster** than gRPC (localhost)
//! - **<0.1ms latency** per operation
//! - Zero serialization overhead
//!
//! # Usage
//! ```rust,ignore
//! let engine = EmbeddedEngine::start().await?;
//! let client = engine.client();
//! client.put(b"key", b"value").await?;
//! ```

use std::time::Duration;

#[cfg(feature = "watch")]
use std::sync::Arc;

use bytes::Bytes;
use d_engine_core::MaybeCloneOneshot;
use d_engine_core::RaftEvent;
use d_engine_core::RaftOneshot;
use d_engine_core::client::{ClientApi, ClientApiError, ClientApiResult};
use d_engine_proto::client::ClientReadRequest;
use d_engine_proto::client::ClientWriteRequest;
use d_engine_proto::client::ReadConsistencyPolicy;
use d_engine_proto::client::WriteCommand;
use d_engine_proto::common::LeaderHint;
use d_engine_proto::error::ErrorCode;
use tokio::sync::mpsc;

#[cfg(feature = "watch")]
use d_engine_core::watch::WatchRegistry;

// ============================================================================
// Error helpers - simplify ClientApiError construction for embedded client
// ============================================================================

fn channel_closed_error() -> ClientApiError {
    ClientApiError::Network {
        code: ErrorCode::ConnectionTimeout,
        message: "Channel closed, node may be shutting down".to_string(),
        retry_after_ms: None,
        leader_hint: None,
    }
}

fn timeout_error(duration: Duration) -> ClientApiError {
    ClientApiError::Network {
        code: ErrorCode::ConnectionTimeout,
        message: format!("Operation timed out after {duration:?}"),
        retry_after_ms: Some(1000),
        leader_hint: None,
    }
}

fn not_leader_error(
    leader_id: Option<String>,
    leader_address: Option<String>,
) -> ClientApiError {
    let message = match (&leader_address, &leader_id) {
        (Some(addr), _) => format!("Not leader, try leader at: {addr}"),
        (None, Some(id)) => format!("Not leader, leader_id: {id}"),
        (None, None) => "Not leader".to_string(),
    };

    let leader_hint = match (&leader_id, &leader_address) {
        (Some(id_str), Some(addr)) => id_str.parse::<u32>().ok().map(|id| LeaderHint {
            leader_id: id,
            address: addr.clone(),
        }),
        _ => None,
    };

    ClientApiError::Network {
        code: ErrorCode::NotLeader,
        message,
        retry_after_ms: Some(100),
        leader_hint,
    }
}

fn server_error(msg: String) -> ClientApiError {
    ClientApiError::Business {
        code: ErrorCode::Uncategorized,
        message: msg,
        required_action: None,
    }
}

/// Zero-overhead KV client for embedded mode.
///
/// Directly calls Raft core without gRPC overhead.
#[derive(Clone)]
pub struct EmbeddedClient {
    event_tx: mpsc::Sender<RaftEvent>,
    cmd_tx: mpsc::UnboundedSender<d_engine_core::ClientCmd>,
    client_id: u32,
    timeout: Duration,
    #[cfg(feature = "watch")]
    watch_registry: Option<Arc<WatchRegistry>>,
}

impl EmbeddedClient {
    /// Internal constructor (used by EmbeddedEngine)
    pub(crate) fn new_internal(
        event_tx: mpsc::Sender<RaftEvent>,
        cmd_tx: mpsc::UnboundedSender<d_engine_core::ClientCmd>,
        client_id: u32,
        timeout: Duration,
    ) -> Self {
        Self {
            event_tx,
            cmd_tx,
            client_id,
            timeout,
            #[cfg(feature = "watch")]
            watch_registry: None,
        }
    }

    /// Set watch registry for watch operations
    #[cfg(feature = "watch")]
    pub(crate) fn with_watch_registry(
        mut self,
        registry: Arc<WatchRegistry>,
    ) -> Self {
        self.watch_registry = Some(registry);
        self
    }

    /// Map ErrorCode and ErrorMetadata to ClientApiError
    fn map_error_response(
        error_code: i32,
        metadata: Option<d_engine_proto::error::ErrorMetadata>,
    ) -> ClientApiError {
        match ErrorCode::try_from(error_code) {
            Ok(ErrorCode::NotLeader) => {
                let (leader_id, leader_address) = if let Some(meta) = metadata {
                    (meta.leader_id, meta.leader_address)
                } else {
                    (None, None)
                };
                not_leader_error(leader_id, leader_address)
            }
            _ => server_error(format!("Error code: {error_code}")),
        }
    }

    /// Store a key-value pair with strong consistency.
    pub async fn put(
        &self,
        key: impl AsRef<[u8]>,
        value: impl AsRef<[u8]>,
    ) -> ClientApiResult<()> {
        let command = WriteCommand::insert(
            Bytes::copy_from_slice(key.as_ref()),
            Bytes::copy_from_slice(value.as_ref()),
        );

        let request = ClientWriteRequest {
            client_id: self.client_id,
            command: Some(command),
        };

        let (resp_tx, resp_rx) = MaybeCloneOneshot::new();

        self.cmd_tx
            .send(d_engine_core::ClientCmd::Propose(request, resp_tx))
            .map_err(|_| channel_closed_error())?;

        let result = tokio::time::timeout(self.timeout, resp_rx)
            .await
            .map_err(|_| timeout_error(self.timeout))?
            .map_err(|_| channel_closed_error())?;

        let response =
            result.map_err(|status| server_error(format!("RPC error: {}", status.message())))?;

        if response.error != ErrorCode::Success as i32 {
            return Err(Self::map_error_response(response.error, response.metadata));
        }

        Ok(())
    }

    /// Strongly consistent read (linearizable).
    ///
    /// Guarantees reading the latest committed value by querying the Leader.
    /// Use for critical reads where staleness is unacceptable.
    ///
    /// # Performance
    /// - Latency: ~1-5ms (network RTT to Leader)
    /// - Throughput: Limited by Leader capacity
    ///
    /// # Raft Protocol
    /// Implements linearizable read per Raft §8.
    ///
    /// # Example
    /// ```ignore
    /// let client = engine.client();
    /// let value = client.get_linearizable(b"critical-config").await?;
    /// ```
    pub async fn get_linearizable(
        &self,
        key: impl AsRef<[u8]>,
    ) -> ClientApiResult<Option<Bytes>> {
        self.get_with_consistency(key, ReadConsistencyPolicy::LinearizableRead).await
    }

    /// Eventually consistent read (stale OK).
    ///
    /// Reads from local state machine without Leader coordination.
    /// Fast but may return stale data if replication is lagging.
    ///
    /// # Performance
    /// - Latency: ~0.1ms (local memory access)
    /// - Throughput: High (no Leader bottleneck)
    ///
    /// # Use Cases
    /// - Read-heavy workloads
    /// - Analytics/reporting (staleness acceptable)
    /// - Caching scenarios
    ///
    /// # Example
    /// ```ignore
    /// let client = engine.client();
    /// let cached_value = client.get_eventual(b"user-preference").await?;
    /// ```
    pub async fn get_eventual(
        &self,
        key: impl AsRef<[u8]>,
    ) -> ClientApiResult<Option<Bytes>> {
        self.get_with_consistency(key, ReadConsistencyPolicy::EventualConsistency).await
    }

    /// Advanced: Read with explicit consistency policy.
    ///
    /// For fine-grained control over read consistency vs performance trade-off.
    ///
    /// # Consistency Policies
    /// - `LinearizableRead`: Read from Leader (strong consistency, may be slower)
    /// - `EventualConsistency`: Read from local node (fast, may be stale)
    /// - `LeaseRead`: Optimized Leader read using lease mechanism
    ///
    /// # Example
    /// ```ignore
    /// use d_engine_proto::client::ReadConsistencyPolicy;
    ///
    /// let value = client.get_with_consistency(
    ///     b"key",
    ///     ReadConsistencyPolicy::LeaseRead,
    /// ).await?;
    /// ```
    pub async fn get_with_consistency(
        &self,
        key: impl AsRef<[u8]>,
        consistency: ReadConsistencyPolicy,
    ) -> ClientApiResult<Option<Bytes>> {
        let request = ClientReadRequest {
            client_id: self.client_id,
            keys: vec![Bytes::copy_from_slice(key.as_ref())],
            consistency_policy: Some(consistency as i32),
        };

        let (resp_tx, resp_rx) = MaybeCloneOneshot::new();

        self.cmd_tx
            .send(d_engine_core::ClientCmd::Read(request, resp_tx))
            .map_err(|_| channel_closed_error())?;

        let result = tokio::time::timeout(self.timeout, resp_rx)
            .await
            .map_err(|_| timeout_error(self.timeout))?
            .map_err(|_| channel_closed_error())?;

        let response =
            result.map_err(|status| server_error(format!("RPC error: {}", status.message())))?;

        if response.error != ErrorCode::Success as i32 {
            return Err(Self::map_error_response(response.error, response.metadata));
        }

        match response.success_result {
            Some(d_engine_proto::client::client_response::SuccessResult::ReadData(
                read_results,
            )) => {
                // If results list is empty, key doesn't exist
                // Otherwise, return the value (even if empty bytes)
                Ok(read_results.results.first().map(|r| r.value.clone()))
            }
            _ => Ok(None),
        }
    }

    /// Get multiple keys with linearizable consistency.
    ///
    /// Reads multiple keys from the Leader with strong consistency guarantee.
    ///
    /// # Example
    /// ```ignore
    /// let keys = vec![Bytes::from("key1"), Bytes::from("key2")];
    /// let values = client.get_multi_linearizable(&keys).await?;
    /// ```
    pub async fn get_multi_linearizable(
        &self,
        keys: &[Bytes],
    ) -> ClientApiResult<Vec<Option<Bytes>>> {
        self.get_multi_with_consistency(keys, ReadConsistencyPolicy::LinearizableRead)
            .await
    }

    /// Get multiple keys with eventual consistency.
    ///
    /// Reads multiple keys from local state machine (fast, may be stale).
    ///
    /// # Example
    /// ```ignore
    /// let keys = vec![Bytes::from("key1"), Bytes::from("key2")];
    /// let values = client.get_multi_eventual(&keys).await?;
    /// ```
    pub async fn get_multi_eventual(
        &self,
        keys: &[Bytes],
    ) -> ClientApiResult<Vec<Option<Bytes>>> {
        self.get_multi_with_consistency(keys, ReadConsistencyPolicy::EventualConsistency)
            .await
    }

    /// Advanced: Get multiple keys with explicit consistency policy.
    pub async fn get_multi_with_consistency(
        &self,
        keys: &[Bytes],
        consistency: ReadConsistencyPolicy,
    ) -> ClientApiResult<Vec<Option<Bytes>>> {
        let request = ClientReadRequest {
            client_id: self.client_id,
            keys: keys.to_vec(),
            consistency_policy: Some(consistency as i32),
        };

        let (resp_tx, resp_rx) = MaybeCloneOneshot::new();

        self.cmd_tx
            .send(d_engine_core::ClientCmd::Read(request, resp_tx))
            .map_err(|_| channel_closed_error())?;

        let result = tokio::time::timeout(self.timeout, resp_rx)
            .await
            .map_err(|_| timeout_error(self.timeout))?
            .map_err(|_| channel_closed_error())?;

        let response =
            result.map_err(|status| server_error(format!("RPC error: {}", status.message())))?;

        if response.error != ErrorCode::Success as i32 {
            return Err(Self::map_error_response(response.error, response.metadata));
        }

        match response.success_result {
            Some(d_engine_proto::client::client_response::SuccessResult::ReadData(
                read_results,
            )) => {
                // Reconstruct result vector in requested key order.
                // Server only returns results for keys that exist, so we must
                // map by key to preserve positional correspondence with input.
                let results_by_key: std::collections::HashMap<_, _> =
                    read_results.results.into_iter().map(|r| (r.key, r.value)).collect();

                Ok(keys.iter().map(|k| results_by_key.get(k).cloned()).collect())
            }
            _ => Ok(vec![None; keys.len()]),
        }
    }

    /// Delete a key-value pair.
    pub async fn delete(
        &self,
        key: impl AsRef<[u8]>,
    ) -> ClientApiResult<()> {
        let command = WriteCommand::delete(Bytes::copy_from_slice(key.as_ref()));

        let request = ClientWriteRequest {
            client_id: self.client_id,
            command: Some(command),
        };

        let (resp_tx, resp_rx) = MaybeCloneOneshot::new();

        self.cmd_tx
            .send(d_engine_core::ClientCmd::Propose(request, resp_tx))
            .map_err(|_| channel_closed_error())?;

        let result = tokio::time::timeout(self.timeout, resp_rx)
            .await
            .map_err(|_| timeout_error(self.timeout))?
            .map_err(|_| channel_closed_error())?;

        let response =
            result.map_err(|status| server_error(format!("RPC error: {}", status.message())))?;

        if response.error != ErrorCode::Success as i32 {
            return Err(Self::map_error_response(response.error, response.metadata));
        }

        Ok(())
    }

    /// Returns the client ID assigned to this local client
    pub fn client_id(&self) -> u32 {
        self.client_id
    }

    /// Returns the configured timeout duration for operations
    pub fn timeout(&self) -> Duration {
        self.timeout
    }

    /// Returns the node ID for testing purposes
    pub fn node_id(&self) -> u32 {
        self.client_id
    }

    /// Watch for changes to a specific key
    ///
    /// Returns a `WatcherHandle` that yields watch events when the key's value changes.
    /// The stream will continue until explicitly dropped or a connection error occurs.
    ///
    /// # Arguments
    ///
    /// * `key` - The key to watch
    ///
    /// # Returns
    ///
    /// A `WatcherHandle` that can be used to receive watch events
    ///
    /// # Errors
    ///
    /// Returns error if watch feature is not enabled or WatchRegistry not initialized
    ///
    /// # Example
    ///
    /// ```ignore
    /// let client = engine.client();
    /// let mut watcher = client.watch(b"config/timeout").await?;
    /// while let Some(event) = watcher.next().await {
    ///     println!("Value changed: {:?}", event);
    /// }
    /// ```
    #[cfg(feature = "watch")]
    pub fn watch(
        &self,
        key: impl AsRef<[u8]>,
    ) -> ClientApiResult<d_engine_core::watch::WatcherHandle> {
        let registry = self.watch_registry.as_ref().ok_or_else(|| ClientApiError::Business {
            code: ErrorCode::Uncategorized,
            message: "Watch feature disabled (WatchRegistry not initialized)".to_string(),
            required_action: None,
        })?;

        let key_bytes = Bytes::copy_from_slice(key.as_ref());
        Ok(registry.register(key_bytes))
    }

    /// Internal helper: Get cluster membership via ClusterConf event
    async fn get_cluster_membership(
        &self
    ) -> ClientApiResult<d_engine_proto::server::cluster::ClusterMembership> {
        let request = d_engine_proto::server::cluster::MetadataRequest {};

        let (resp_tx, resp_rx) = MaybeCloneOneshot::new();

        self.event_tx
            .send(RaftEvent::ClusterConf(request, resp_tx))
            .await
            .map_err(|_| channel_closed_error())?;

        let result = tokio::time::timeout(self.timeout, resp_rx)
            .await
            .map_err(|_| timeout_error(self.timeout))?
            .map_err(|_| channel_closed_error())?;

        result.map_err(|status| server_error(format!("ClusterConf error: {}", status.message())))
    }
}

impl std::fmt::Debug for EmbeddedClient {
    fn fmt(
        &self,
        f: &mut std::fmt::Formatter<'_>,
    ) -> std::fmt::Result {
        f.debug_struct("EmbeddedClient")
            .field("client_id", &self.client_id)
            .field("timeout", &self.timeout)
            .finish()
    }
}

// Implement ClientApi trait
#[async_trait::async_trait]
impl ClientApi for EmbeddedClient {
    async fn put(
        &self,
        key: impl AsRef<[u8]> + Send,
        value: impl AsRef<[u8]> + Send,
    ) -> ClientApiResult<()> {
        self.put(key, value).await
    }

    async fn put_with_ttl(
        &self,
        key: impl AsRef<[u8]> + Send,
        value: impl AsRef<[u8]> + Send,
        ttl_secs: u64,
    ) -> ClientApiResult<()> {
        let command = WriteCommand::insert_with_ttl(
            Bytes::copy_from_slice(key.as_ref()),
            Bytes::copy_from_slice(value.as_ref()),
            ttl_secs,
        );

        let request = ClientWriteRequest {
            client_id: self.client_id,
            command: Some(command),
        };

        let (resp_tx, resp_rx) = MaybeCloneOneshot::new();

        self.cmd_tx
            .send(d_engine_core::ClientCmd::Propose(request, resp_tx))
            .map_err(|_| channel_closed_error())?;

        let result = tokio::time::timeout(self.timeout, resp_rx)
            .await
            .map_err(|_| timeout_error(self.timeout))?
            .map_err(|_| channel_closed_error())?;

        let response =
            result.map_err(|status| server_error(format!("RPC error: {}", status.message())))?;

        if response.error != ErrorCode::Success as i32 {
            return Err(Self::map_error_response(response.error, response.metadata));
        }

        Ok(())
    }

    async fn get(
        &self,
        key: impl AsRef<[u8]> + Send,
    ) -> ClientApiResult<Option<Bytes>> {
        self.get_linearizable(key).await
    }

    async fn get_multi(
        &self,
        keys: &[Bytes],
    ) -> ClientApiResult<Vec<Option<Bytes>>> {
        self.get_multi_linearizable(keys).await
    }

    async fn delete(
        &self,
        key: impl AsRef<[u8]> + Send,
    ) -> ClientApiResult<()> {
        self.delete(key).await
    }

    async fn compare_and_swap(
        &self,
        key: impl AsRef<[u8]> + Send,
        expected_value: Option<impl AsRef<[u8]> + Send>,
        new_value: impl AsRef<[u8]> + Send,
    ) -> ClientApiResult<bool> {
        let command = WriteCommand::compare_and_swap(
            Bytes::copy_from_slice(key.as_ref()),
            expected_value.map(|v| Bytes::copy_from_slice(v.as_ref())),
            Bytes::copy_from_slice(new_value.as_ref()),
        );

        let request = ClientWriteRequest {
            client_id: self.client_id,
            command: Some(command),
        };

        let (resp_tx, resp_rx) = MaybeCloneOneshot::new();

        self.cmd_tx
            .send(d_engine_core::ClientCmd::Propose(request, resp_tx))
            .map_err(|_| channel_closed_error())?;

        let result = tokio::time::timeout(self.timeout, resp_rx)
            .await
            .map_err(|_| timeout_error(self.timeout))?
            .map_err(|_| channel_closed_error())?;

        let response =
            result.map_err(|status| server_error(format!("RPC error: {}", status.message())))?;

        if response.error != ErrorCode::Success as i32 {
            return Err(Self::map_error_response(response.error, response.metadata));
        }

        match response.success_result {
            Some(d_engine_proto::client::client_response::SuccessResult::WriteResult(result)) => {
                Ok(result.succeeded)
            }
            _ => Err(server_error("Invalid CAS response".to_string())),
        }
    }

    async fn list_members(
        &self
    ) -> ClientApiResult<Vec<d_engine_proto::server::cluster::NodeMeta>> {
        let cluster_membership = self.get_cluster_membership().await?;
        Ok(cluster_membership.nodes)
    }

    async fn get_leader_id(&self) -> ClientApiResult<Option<u32>> {
        let cluster_membership = self.get_cluster_membership().await?;
        Ok(cluster_membership.current_leader_id)
    }

    async fn get_multi_with_policy(
        &self,
        keys: &[Bytes],
        consistency_policy: Option<ReadConsistencyPolicy>,
    ) -> ClientApiResult<Vec<Option<Bytes>>> {
        self.get_multi_with_consistency(
            keys,
            consistency_policy.unwrap_or(ReadConsistencyPolicy::LinearizableRead),
        )
        .await
    }

    async fn get_linearizable(
        &self,
        key: impl AsRef<[u8]> + Send,
    ) -> ClientApiResult<Option<Bytes>> {
        self.get_linearizable(key).await
    }

    async fn get_lease(
        &self,
        key: impl AsRef<[u8]> + Send,
    ) -> ClientApiResult<Option<Bytes>> {
        self.get_with_consistency(key, ReadConsistencyPolicy::LeaseRead).await
    }

    async fn get_eventual(
        &self,
        key: impl AsRef<[u8]> + Send,
    ) -> ClientApiResult<Option<Bytes>> {
        self.get_eventual(key).await
    }
}