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
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
/*
 * Copyright (c) Dell Inc., or its subsidiaries. All Rights Reserved.
 *
 * Licensed under the Apache License, Version 2.0 (the "License");
 * you may not use this file except in compliance with the License.
 * You may obtain a copy of the License at
 *
 *     http://www.apache.org/licenses/LICENSE-2.0
 */
#![allow(dead_code)]
#![allow(bare_trait_objects)]

use super::ControllerClient;
use super::ControllerError;
use crate::ResultRetry;
use async_trait::async_trait;
use im::HashMap as ImHashMap;
use jsonwebtoken::{encode, Algorithm, EncodingKey, Header};
use ordered_float::OrderedFloat;
use pravega_client_config::connection_type::{ConnectionType, MockType};
use pravega_client_retry::retry_result::RetryError;
use pravega_client_shared::*;
use pravega_connection_pool::connection_pool::ConnectionPool;
use pravega_wire_protocol::client_connection::{ClientConnection, ClientConnectionImpl};
use pravega_wire_protocol::commands::{CreateSegmentCommand, DeleteSegmentCommand, MergeSegmentsCommand};
use pravega_wire_protocol::connection_factory::{
    ConnectionFactory, ConnectionFactoryConfig, SegmentConnectionManager,
};
use pravega_wire_protocol::error::ClientConnectionError;
use pravega_wire_protocol::wire_commands::{Replies, Requests};
use serde::{Deserialize, Serialize};
use std::collections::HashSet;
use std::collections::{BTreeMap, HashMap};
use std::sync::atomic::{AtomicUsize, Ordering};
use std::time::Duration;
use std::time::SystemTime;
use tokio::sync::{RwLock, RwLockReadGuard};
use uuid::Uuid;

static ID_GENERATOR: AtomicUsize = AtomicUsize::new(0);

pub struct MockController {
    endpoint: PravegaNodeUri,
    pool: ConnectionPool<SegmentConnectionManager>,
    created_scopes: RwLock<HashMap<String, HashSet<ScopedStream>>>,
    created_streams: RwLock<HashMap<ScopedStream, StreamConfiguration>>,
    transactions: RwLock<HashMap<TxId, TransactionStatus>>,
}

impl MockController {
    pub fn new(endpoint: PravegaNodeUri) -> Self {
        let config = ConnectionFactoryConfig::new(ConnectionType::Mock(MockType::Happy));
        let cf = ConnectionFactory::create(config) as Box<dyn ConnectionFactory>;
        let manager = SegmentConnectionManager::new(cf, 10);
        let pool = ConnectionPool::new(manager);
        MockController {
            endpoint,
            pool,
            created_scopes: RwLock::new(HashMap::new()),
            created_streams: RwLock::new(HashMap::new()),
            transactions: RwLock::new(HashMap::new()),
        }
    }
}
#[async_trait]
impl ControllerClient for MockController {
    async fn create_scope(&self, scope: &Scope) -> Result<bool, RetryError<ControllerError>> {
        let scope_name = scope.name.clone();
        if self.created_scopes.read().await.contains_key(&scope_name) {
            return Ok(false);
        }

        self.created_scopes
            .write()
            .await
            .insert(scope_name, HashSet::new());
        Ok(true)
    }

    async fn check_scope_exists(&self, scope: &Scope) -> Result<bool, RetryError<ControllerError>> {
        let scope_name = scope.name.clone();
        if self.created_scopes.read().await.contains_key(&scope_name) {
            return Ok(true);
        }
        Ok(false)
    }

    async fn list_scopes(
        &self,
        _token: &CToken,
    ) -> Result<Option<(Vec<Scope>, CToken)>, RetryError<ControllerError>> {
        let map_guard = self.created_scopes.read().await;
        let scopes = map_guard.keys();
        let mut result = Vec::new();
        for scope in scopes {
            result.push(Scope::from(scope.clone()));
        }
        Ok(Some((result, CToken::from("mock_token"))))
    }

    async fn list_streams(
        &self,
        scope: &Scope,
        _token: &CToken,
    ) -> Result<Option<(Vec<ScopedStream>, CToken)>, RetryError<ControllerError>> {
        let map_guard = self.created_scopes.read().await;
        let streams_set = map_guard.get(&scope.name).ok_or(RetryError {
            error: ControllerError::OperationError {
                can_retry: false,
                operation: "listStreams".into(),
                error_msg: "Scope not exist".into(),
            },
            total_delay: Duration::from_millis(1),
            tries: 0,
        })?;
        let mut result = Vec::new();
        for stream in streams_set {
            result.push(stream.clone())
        }
        Ok(Some((result, CToken::from("mock_token"))))
    }

    async fn list_streams_for_tag(
        &self,
        scope: &Scope,
        tag: &str,
        _token: &CToken,
    ) -> Result<Option<(Vec<ScopedStream>, CToken)>, RetryError<ControllerError>> {
        let scope_gaurd = self.created_scopes.read().await;
        let stream_gaurd = self.created_streams.read().await;

        let streams_set = scope_gaurd.get(&scope.name).ok_or(RetryError {
            error: ControllerError::OperationError {
                can_retry: false,
                operation: "listStreams".into(),
                error_msg: "Scope not exist".into(),
            },
            total_delay: Duration::from_millis(1),
            tries: 0,
        })?;
        let mut result = Vec::new();
        for stream in streams_set {
            let cfg = stream_gaurd.get(stream).ok_or(RetryError {
                error: ControllerError::OperationError {
                    can_retry: false,
                    operation: "listStreamsForTag".into(),
                    error_msg: "Stream does not exist".into(),
                },
                total_delay: Duration::from_millis(1),
                tries: 0,
            })?;
            match &cfg.tags {
                None => {}
                Some(tag_list) => {
                    if tag_list.contains(&tag.to_string()) {
                        result.push(stream.clone())
                    }
                }
            };
        }
        Ok(Some((result, CToken::from("mock_token"))))
    }

    async fn delete_scope(&self, scope: &Scope) -> Result<bool, RetryError<ControllerError>> {
        let scope_name = scope.name.clone();
        if self.created_scopes.read().await.get(&scope_name).is_none() {
            return Ok(false);
        }

        if !self
            .created_scopes
            .read()
            .await
            .get(&scope_name)
            .unwrap()
            .is_empty()
        {
            Err(RetryError {
                error: ControllerError::OperationError {
                    can_retry: false,
                    operation: "DeleteScope".into(),
                    error_msg: "Scope not empty".into(),
                },
                total_delay: Duration::from_millis(1),
                tries: 0,
            })
        } else {
            self.created_scopes.write().await.remove(&scope_name);
            Ok(true)
        }
    }

    async fn create_stream(
        &self,
        stream_config: &StreamConfiguration,
    ) -> Result<bool, RetryError<ControllerError>> {
        let stream = stream_config.scoped_stream.clone();
        if self.created_streams.read().await.contains_key(&stream) {
            return Ok(false);
        }
        if self.created_scopes.read().await.get(&stream.scope.name).is_none() {
            return Err(RetryError {
                error: ControllerError::OperationError {
                    can_retry: false,
                    operation: "create stream".into(),
                    error_msg: "Scope does not exist.".into(),
                },
                total_delay: Duration::from_millis(1),
                tries: 0,
            });
        }
        self.created_streams
            .write()
            .await
            .insert(stream.clone(), stream_config.clone());
        self.created_scopes
            .write()
            .await
            .get_mut(&stream.scope.name)
            .unwrap()
            .insert(stream.clone());

        let read_guard = &self.created_streams.read().await;
        for segment in get_segments_for_stream(&stream, read_guard)? {
            let segment_name = segment.to_string();
            create_segment(segment_name, self, false).await?;
        }
        Ok(true)
    }

    async fn check_stream_exists(&self, stream: &ScopedStream) -> Result<bool, RetryError<ControllerError>> {
        if self.created_streams.read().await.contains_key(stream) {
            return Ok(true);
        }
        Ok(false)
    }

    async fn update_stream(
        &self,
        _stream_config: &StreamConfiguration,
    ) -> Result<bool, RetryError<ControllerError>> {
        Err(RetryError {
            error: ControllerError::OperationError {
                can_retry: false,
                operation: "update stream".into(),
                error_msg: "unsupported operation.".into(),
            },
            total_delay: Duration::from_millis(1),
            tries: 0,
        })
    }

    async fn get_stream_configuration(&self, _stream: &ScopedStream) -> ResultRetry<StreamConfiguration> {
        Err(RetryError {
            error: ControllerError::OperationError {
                can_retry: false,
                operation: "get stream configuration".into(),
                error_msg: "unsupported operation.".into(),
            },
            total_delay: Duration::from_millis(1),
            tries: 0,
        })
    }

    async fn get_stream_tags(&self, _stream: &ScopedStream) -> ResultRetry<Option<Vec<String>>> {
        Err(RetryError {
            error: ControllerError::OperationError {
                can_retry: false,
                operation: "get stream tags".into(),
                error_msg: "unsupported operation.".into(),
            },
            total_delay: Duration::from_millis(1),
            tries: 0,
        })
    }

    async fn truncate_stream(&self, _stream_cut: &StreamCut) -> Result<bool, RetryError<ControllerError>> {
        Err(RetryError {
            error: ControllerError::OperationError {
                can_retry: false,
                operation: "truncate stream".into(),
                error_msg: "unsupported operation.".into(),
            },
            total_delay: Duration::from_millis(1),
            tries: 0,
        })
    }

    async fn seal_stream(&self, _stream: &ScopedStream) -> Result<bool, RetryError<ControllerError>> {
        Err(RetryError {
            error: ControllerError::OperationError {
                can_retry: false,
                operation: "seal stream".into(),
                error_msg: "unsupported operation.".into(),
            },
            total_delay: Duration::from_millis(1),
            tries: 0,
        })
    }

    async fn delete_stream(&self, stream: &ScopedStream) -> Result<bool, RetryError<ControllerError>> {
        if self.created_streams.read().await.get(stream).is_none() {
            return Ok(false);
        }

        for segment in get_segments_for_stream(stream, &self.created_streams.read().await)? {
            let segment_name = segment.to_string();
            delete_segment(segment_name, self, false).await?;
        }

        self.created_streams.write().await.remove(stream);
        self.created_scopes
            .write()
            .await
            .get_mut(&stream.scope.name)
            .unwrap()
            .remove(stream);
        Ok(true)
    }

    async fn get_current_segments(
        &self,
        stream: &ScopedStream,
    ) -> Result<StreamSegments, RetryError<ControllerError>> {
        let segments_in_stream = get_segments_for_stream(stream, &self.created_streams.read().await)?;
        let mut segments = BTreeMap::new();
        let increment = 1.0 / segments_in_stream.len() as f64;
        for (number, segment) in segments_in_stream.into_iter().enumerate() {
            let segment_with_range = SegmentWithRange {
                scoped_segment: segment,
                min_key: OrderedFloat(number as f64 * increment),
                max_key: OrderedFloat((number + 1) as f64 * increment),
            };
            segments.insert(segment_with_range.max_key, segment_with_range);
        }

        Ok(StreamSegments {
            key_segment_map: segments.into(),
        })
    }

    async fn get_epoch_segments(
        &self,
        stream: &ScopedStream,
        _epoch: i32,
    ) -> Result<StreamSegments, RetryError<ControllerError>> {
        let segments_in_stream = get_segments_for_stream(stream, &self.created_streams.read().await)?;
        let mut segments = BTreeMap::new();
        let increment = 1.0 / segments_in_stream.len() as f64;
        for (number, segment) in segments_in_stream.into_iter().enumerate() {
            let segment_with_range = SegmentWithRange {
                scoped_segment: segment,
                min_key: OrderedFloat(number as f64 * increment),
                max_key: OrderedFloat((number + 1) as f64 * increment),
            };
            segments.insert(segment_with_range.max_key, segment_with_range);
        }

        Ok(StreamSegments {
            key_segment_map: segments.into(),
        })
    }

    async fn get_head_segments(
        &self,
        stream: &ScopedStream,
    ) -> ResultRetry<std::collections::HashMap<Segment, i64>> {
        let segments_in_stream: Vec<ScopedSegment> =
            get_segments_for_stream(stream, &self.created_streams.read().await)?;
        Ok(segments_in_stream
            .iter()
            .map(|t| (t.segment.clone(), 0i64))
            .collect())
    }

    async fn create_transaction(
        &self,
        stream: &ScopedStream,
        _lease: Duration,
    ) -> Result<TxnSegments, RetryError<ControllerError>> {
        let uuid = Uuid::new_v4().as_u128();
        let current_segments = self.get_current_segments(stream).await?;
        let mut guard = self.transactions.write().await;
        guard.insert(TxId(uuid), TransactionStatus::Open);

        Ok(TxnSegments {
            stream_segments: current_segments,
            tx_id: TxId(uuid),
        })
    }

    async fn ping_transaction(
        &self,
        _stream: &ScopedStream,
        tx_id: TxId,
        _lease: Duration,
    ) -> Result<PingStatus, RetryError<ControllerError>> {
        let guard = self.transactions.read().await;
        let status = guard.get(&tx_id).expect("get transaction status");
        match status {
            TransactionStatus::Committed => Ok(PingStatus::Committed),
            TransactionStatus::Aborted => Ok(PingStatus::Aborted),
            _ => Ok(PingStatus::Ok),
        }
    }

    async fn commit_transaction(
        &self,
        _stream: &ScopedStream,
        tx_id: TxId,
        _writer_id: WriterId,
        _time: Timestamp,
    ) -> Result<(), RetryError<ControllerError>> {
        let mut guard = self.transactions.write().await;
        guard.insert(tx_id, TransactionStatus::Committed);
        Ok(())
    }

    async fn abort_transaction(
        &self,
        _stream: &ScopedStream,
        tx_id: TxId,
    ) -> Result<(), RetryError<ControllerError>> {
        let mut guard = self.transactions.write().await;
        guard.insert(tx_id, TransactionStatus::Aborted);
        Ok(())
    }

    async fn check_transaction_status(
        &self,
        _stream: &ScopedStream,
        tx_id: TxId,
    ) -> Result<TransactionStatus, RetryError<ControllerError>> {
        let guard = self.transactions.read().await;
        let status = guard.get(&tx_id).expect("get transaction");
        Ok(status.clone())
    }

    async fn get_endpoint_for_segment(
        &self,
        _segment: &ScopedSegment,
    ) -> Result<PravegaNodeUri, RetryError<ControllerError>> {
        Ok(self.endpoint.clone())
    }

    async fn get_or_refresh_delegation_token_for(
        &self,
        _stream: ScopedStream,
    ) -> Result<String, RetryError<ControllerError>> {
        let now = SystemTime::now()
            .duration_since(SystemTime::UNIX_EPOCH)
            .expect("get unix time");
        let timeout = Duration::from_secs(5);
        let expiry_time = now.checked_add(timeout).expect("calculate expiry time");
        let claims = Claims {
            sub: "subject".to_string(),
            aud: "segmentstore".to_string(),
            iat: now.as_secs(),
            exp: expiry_time.as_secs(),
        };

        let header = Header {
            typ: Some("JWT".to_owned()),
            alg: Algorithm::HS256,
            ..Default::default()
        };

        let key = b"secret";
        let token = encode(&header, &claims, &EncodingKey::from_secret(key)).expect("encode to JWT token");
        Ok(token)
    }

    async fn get_successors(
        &self,
        _segment: &ScopedSegment,
    ) -> Result<StreamSegmentsWithPredecessors, RetryError<ControllerError>> {
        // empty hash map means the stream is sealed
        Ok(StreamSegmentsWithPredecessors {
            segment_with_predecessors: ImHashMap::new(),
            replacement_segments: ImHashMap::new(),
        })
    }

    async fn scale_stream(
        &self,
        _stream: &ScopedStream,
        _sealed_segments: &[Segment],
        _new_key_ranges: &[(f64, f64)],
    ) -> Result<(), RetryError<ControllerError>> {
        Err(RetryError {
            error: ControllerError::OperationError {
                can_retry: false, // do not retry.
                operation: "scale stream".into(),
                error_msg: "unsupported operation.".into(),
            },
            total_delay: Duration::from_millis(1),
            tries: 0,
        })
    }

    async fn check_scale(
        &self,
        _stream: &ScopedStream,
        _scale_epoch: i32,
    ) -> Result<bool, RetryError<ControllerError>> {
        Err(RetryError {
            error: ControllerError::OperationError {
                can_retry: false, // do not retry.
                operation: "check stream scale".into(),
                error_msg: "unsupported operation.".into(),
            },
            total_delay: Duration::from_millis(1),
            tries: 0,
        })
    }
}

fn get_segments_for_stream(
    stream: &ScopedStream,
    created_streams: &RwLockReadGuard<HashMap<ScopedStream, StreamConfiguration>>,
) -> Result<Vec<ScopedSegment>, RetryError<ControllerError>> {
    let stream_config = created_streams.get(stream);
    if stream_config.is_none() {
        return Err(RetryError {
            error: ControllerError::OperationError {
                can_retry: false, // do not retry.
                operation: "get segments for stream".into(),
                error_msg: "stream does not exist.".into(),
            },
            total_delay: Duration::from_millis(1),
            tries: 0,
        });
    }

    let scaling_policy = stream_config.unwrap().scaling.clone();

    if scaling_policy.scale_type != ScaleType::FixedNumSegments {
        return Err(RetryError {
            error: ControllerError::OperationError {
                can_retry: false, // do not retry.
                operation: "get segments for stream".into(),
                error_msg: "Dynamic scaling not supported with a mock controller.".into(),
            },
            total_delay: Duration::from_millis(1),
            tries: 0,
        });
    }
    let mut result = Vec::with_capacity(scaling_policy.min_num_segments as usize);
    for i in 0..scaling_policy.min_num_segments {
        result.push(ScopedSegment {
            scope: stream.scope.clone(),
            stream: stream.stream.clone(),
            segment: Segment::from(i.into()),
        });
    }

    Ok(result)
}

async fn create_segment(
    name: String,
    controller: &MockController,
    call_server: bool,
) -> Result<bool, RetryError<ControllerError>> {
    if !call_server {
        return Ok(true);
    }
    let scale_type = ScaleType::FixedNumSegments;
    let id = ID_GENERATOR.fetch_add(1, Ordering::SeqCst) as i64;
    let command = Requests::CreateSegment(CreateSegmentCommand {
        request_id: id,
        segment: name,
        target_rate: 0,
        scale_type: scale_type as u8,
        delegation_token: String::from(""),
    });
    let reply = send_request_over_connection(&command, controller).await;
    match reply {
        Ok(r) => {
            match r {
                Replies::WrongHost(_) => Err(ControllerError::OperationError {
                    can_retry: false, // do not retry.
                    operation: "create segment".into(),
                    error_msg: "Wrong host.".into(),
                }),
                Replies::SegmentCreated(_) => Ok(true),
                Replies::SegmentAlreadyExists(_) => Ok(false),
                Replies::AuthTokenCheckFailed(_) => Err(ControllerError::OperationError {
                    can_retry: false, // do not retry.
                    operation: "create segment".into(),
                    error_msg: "authToken check failed,".into(),
                }),
                _ => Err(ControllerError::OperationError {
                    can_retry: false, // do not retry.
                    operation: "create segment".into(),
                    error_msg: "Unsupported Command".into(),
                }),
            }
        }
        Err(_e) => Err(ControllerError::OperationError {
            can_retry: false, // do not retry.
            operation: "create segment".into(),
            error_msg: "Connection Error".into(),
        }),
    }
    .map_err(|e| RetryError {
        error: e,
        total_delay: Duration::from_millis(1),
        tries: 0,
    })
}

async fn delete_segment(
    name: String,
    controller: &MockController,
    call_server: bool,
) -> Result<bool, RetryError<ControllerError>> {
    if !call_server {
        return Ok(true);
    }
    let id = ID_GENERATOR.fetch_add(1, Ordering::SeqCst) as i64;
    let command = Requests::DeleteSegment(DeleteSegmentCommand {
        request_id: id,
        segment: name,
        delegation_token: String::from(""),
    });
    let reply = send_request_over_connection(&command, controller).await;
    match reply {
        Err(_e) => Err(ControllerError::OperationError {
            can_retry: false, // do not retry.
            operation: "delete segment".into(),
            error_msg: "Connection Error".into(),
        }),
        Ok(r) => {
            match r {
                Replies::WrongHost(_) => Err(ControllerError::OperationError {
                    can_retry: false, // do not retry.
                    operation: "delete segment".into(),
                    error_msg: "Wrong host.".into(),
                }),
                Replies::SegmentDeleted(_) => Ok(true),
                Replies::NoSuchSegment(_) => Ok(false),
                Replies::AuthTokenCheckFailed(_) => Err(ControllerError::OperationError {
                    can_retry: false, // do not retry.
                    operation: "delete segment".into(),
                    error_msg: "authToken check failed,".into(),
                }),
                _ => Err(ControllerError::OperationError {
                    can_retry: false, // do not retry.
                    operation: "delete segment".into(),
                    error_msg: "Unsupported Command.".into(),
                }),
            }
        }
    }
    .map_err(|e| RetryError {
        error: e,
        total_delay: Duration::from_millis(1),
        tries: 0,
    })
}

async fn commit_tx_segment(
    uuid: TxId,
    segment: ScopedSegment,
    controller: &MockController,
    call_server: bool,
) -> Result<(), RetryError<ControllerError>> {
    if !call_server {
        return Ok(());
    }
    let source_name = segment.scope.name.clone() + &uuid.to_string();
    let id = ID_GENERATOR.fetch_add(1, Ordering::SeqCst) as i64;
    let command = Requests::MergeSegments(MergeSegmentsCommand {
        request_id: id,
        target: segment.to_string(),
        source: source_name,
        delegation_token: String::from(""),
    });
    let reply = send_request_over_connection(&command, controller).await;
    match reply {
        Err(_e) => Err(ControllerError::OperationError {
            can_retry: false, // do not retry.
            operation: "commit tx segment".into(),
            error_msg: "Connection Error".into(),
        }),
        Ok(r) => {
            match r {
                Replies::SegmentsMerged(_) => Ok(()),
                Replies::WrongHost(_) => Err(ControllerError::OperationError {
                    can_retry: false, // do not retry.
                    operation: "commit tx segment".into(),
                    error_msg: "Wrong host.".into(),
                }),
                Replies::SegmentDeleted(_) => Err(ControllerError::OperationError {
                    can_retry: false, // do not retry.
                    operation: "commit tx segment".into(),
                    error_msg: "Transaction already aborted.".into(),
                }),
                Replies::AuthTokenCheckFailed(_) => Err(ControllerError::OperationError {
                    can_retry: false, // do not retry.
                    operation: "commit tx segment".into(),
                    error_msg: "authToken check failed,".into(),
                }),
                _ => Err(ControllerError::OperationError {
                    can_retry: false, // do not retry.
                    operation: "commit tx segment".into(),
                    error_msg: "Unsupported Command,".into(),
                }),
            }
        }
    }
    .map_err(|e| RetryError {
        error: e,
        total_delay: Duration::from_millis(1),
        tries: 0,
    })
}

async fn abort_tx_segment(
    uuid: TxId,
    segment: ScopedSegment,
    controller: &MockController,
    call_server: bool,
) -> Result<(), RetryError<ControllerError>> {
    if !call_server {
        return Ok(());
    }
    let transaction_name = segment.scope.name.clone() + &uuid.to_string();
    let id = ID_GENERATOR.fetch_add(1, Ordering::SeqCst) as i64;
    let command = Requests::DeleteSegment(DeleteSegmentCommand {
        request_id: id,
        segment: transaction_name,
        delegation_token: String::from(""),
    });
    let reply = send_request_over_connection(&command, controller).await;
    match reply {
        Err(_e) => Err(ControllerError::OperationError {
            can_retry: false, // do not retry.
            operation: "abort tx segment".into(),
            error_msg: "Connection Error".into(),
        }),
        Ok(r) => {
            match r {
                Replies::SegmentsMerged(_) => Err(ControllerError::OperationError {
                    can_retry: false, // do not retry.
                    operation: "abort tx segment".into(),
                    error_msg: "Transaction already committed.".into(),
                }),
                Replies::WrongHost(_) => Err(ControllerError::OperationError {
                    can_retry: false, // do not retry.
                    operation: "abort tx segment".into(),
                    error_msg: "Wrong host.".into(),
                }),
                Replies::AuthTokenCheckFailed(_) => Err(ControllerError::OperationError {
                    can_retry: false, // do not retry.
                    operation: "abort tx segment".into(),
                    error_msg: "authToken check failed,".into(),
                }),
                Replies::SegmentDeleted(_) => Ok(()),
                _ => Err(ControllerError::OperationError {
                    can_retry: false, // do not retry.
                    operation: "abort tx segment".into(),
                    error_msg: "Unsupported Command,".into(),
                }),
            }
        }
    }
    .map_err(|e| RetryError {
        error: e,
        total_delay: Duration::from_millis(1),
        tries: 0,
    })
}

async fn create_tx_segment(
    uuid: TxId,
    segment: ScopedSegment,
    controller: &MockController,
    call_server: bool,
) -> Result<(), RetryError<ControllerError>> {
    if !call_server {
        return Ok(());
    }
    let transaction_name = segment.scope.name.clone() + &uuid.to_string();
    let scale_type = ScaleType::FixedNumSegments;
    let id = ID_GENERATOR.fetch_add(1, Ordering::SeqCst) as i64;
    let command = Requests::CreateSegment(CreateSegmentCommand {
        request_id: id,
        segment: transaction_name,
        target_rate: 0,
        scale_type: scale_type as u8,
        delegation_token: String::from(""),
    });
    let reply = send_request_over_connection(&command, controller).await;
    match reply {
        Err(_e) => Err(ControllerError::OperationError {
            can_retry: false, // do not retry.
            operation: "abort tx segment".into(),
            error_msg: "Connection Error".into(),
        }),
        Ok(r) => {
            match r {
                Replies::SegmentCreated(_) => Ok(()),
                Replies::WrongHost(_) => Err(ControllerError::OperationError {
                    can_retry: false, // do not retry.
                    operation: "create tx segment".into(),
                    error_msg: "Wrong host.".into(),
                }),
                Replies::AuthTokenCheckFailed(_) => Err(ControllerError::OperationError {
                    can_retry: false, // do not retry.
                    operation: "create tx segment".into(),
                    error_msg: "authToken check failed,".into(),
                }),
                _ => Err(ControllerError::OperationError {
                    can_retry: false, // do not retry.
                    operation: "create tx segment".into(),
                    error_msg: "Unsupported Command,".into(),
                }),
            }
        }
    }
    .map_err(|e| RetryError {
        error: e,
        total_delay: Duration::from_millis(1),
        tries: 0,
    })
}

async fn send_request_over_connection(
    command: &Requests,
    controller: &MockController,
) -> Result<Replies, ClientConnectionError> {
    let pooled_connection = controller
        .pool
        .get_connection(controller.endpoint.clone())
        .await
        .expect("get connection from pool");
    let mut connection = ClientConnectionImpl {
        connection: pooled_connection,
    };
    connection.write(command).await?;
    connection.read().await
}

#[derive(Debug, Serialize, Deserialize)]
struct Claims {
    sub: String,
    aud: String,
    iat: u64,
    exp: u64,
}