-
Notifications
You must be signed in to change notification settings - Fork 26
Expand file tree
/
Copy pathraft_repl_dev.cpp
More file actions
2423 lines (2154 loc) · 119 KB
/
raft_repl_dev.cpp
File metadata and controls
2423 lines (2154 loc) · 119 KB
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
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
#include <flatbuffers/idl.h>
#include <flatbuffers/minireflect.h>
#include <folly/executors/InlineExecutor.h>
#include <iomgr/iomgr_flip.hpp>
#include <boost/lexical_cast.hpp>
#include <boost/uuid/nil_generator.hpp>
#include <sisl/fds/buffer.hpp>
#include <sisl/grpc/generic_service.hpp>
#include <sisl/grpc/rpc_client.hpp>
#include <sisl/fds/vector_pool.hpp>
#include <homestore/blkdata_service.hpp>
#include <homestore/logstore_service.hpp>
#include <homestore/superblk_handler.hpp>
#include "common/homestore_assert.hpp"
#include "common/homestore_config.hpp"
#include "common/homestore_utils.hpp"
#include "replication/service/raft_repl_service.h"
#include "replication/repl_dev/raft_repl_dev.h"
#include "device/chunk.h"
#include "device/device.h"
#include "push_data_rpc_generated.h"
#include "fetch_data_rpc_generated.h"
namespace homestore {
std::atomic< uint64_t > RaftReplDev::s_next_group_ordinal{1};
RaftReplDev::RaftReplDev(RaftReplService& svc, superblk< raft_repl_dev_superblk >&& rd_sb, bool load_existing) :
m_repl_svc{svc},
m_msg_mgr{svc.msg_manager()},
m_group_id{rd_sb->group_id},
m_my_repl_id{svc.get_my_repl_uuid()},
m_raft_server_id{nuraft_mesg::to_server_id(m_my_repl_id)},
m_rd_sb{std::move(rd_sb)},
m_metrics{fmt::format("{}_{}", group_id_str(), m_raft_server_id).c_str()} {
m_state_machine = std::make_shared< RaftStateMachine >(*this);
if (load_existing) {
m_data_journal = std::make_shared< ReplLogStore >(
*this, *m_state_machine, m_rd_sb->logdev_id, m_rd_sb->logstore_id,
[this](logstore_seq_num_t lsn, log_buffer buf, void* key) { on_log_found(lsn, buf, key); },
[this](std::shared_ptr< HomeLogStore > hs, logstore_seq_num_t lsn) {
m_log_store_replay_done = true;
set_log_store_last_durable_lsn(hs->tail_lsn());
});
m_next_dsn = m_rd_sb->last_applied_dsn + 1;
m_commit_upto_lsn = m_rd_sb->durable_commit_lsn;
m_last_flushed_commit_lsn = m_commit_upto_lsn;
m_compact_lsn = m_rd_sb->compact_lsn;
m_rdev_name = m_rd_sb->rdev_name;
// Its ok not to do compare exchange, because loading is always single threaded as of now
if (m_rd_sb->group_ordinal >= s_next_group_ordinal.load()) {
s_next_group_ordinal.store(m_rd_sb->group_ordinal + 1);
}
if (m_rd_sb->is_timeline_consistent) {
logstore_service()
.open_log_store(m_rd_sb->logdev_id, m_rd_sb->free_blks_journal_id, false)
.thenValue([this](auto log_store) {
m_free_blks_journal = std::move(log_store);
m_rd_sb->free_blks_journal_id = m_free_blks_journal->get_store_id();
});
}
} else {
m_data_journal = std::make_shared< ReplLogStore >(*this, *m_state_machine);
m_rd_sb->logdev_id = m_data_journal->logdev_id();
m_rd_sb->logstore_id = m_data_journal->logstore_id();
m_rd_sb->last_applied_dsn = 0;
m_rd_sb->destroy_pending = 0x0;
m_rd_sb->last_snapshot_lsn = 0;
m_rd_sb->group_ordinal = s_next_group_ordinal.fetch_add(1);
m_rdev_name = fmt::format("rdev{}", m_rd_sb->group_ordinal);
m_rd_sb->set_rdev_name(m_rdev_name);
if (m_rd_sb->is_timeline_consistent) {
m_free_blks_journal = logstore_service().create_new_log_store(m_rd_sb->logdev_id, false /* append_mode */);
m_rd_sb->free_blks_journal_id = m_free_blks_journal->get_store_id();
}
m_rd_sb.write();
bind_data_service();
}
m_identify_str = m_rdev_name + ":" + group_id_str();
RD_LOGI(NO_TRACE_ID,
"Started {} RaftReplDev group_id={}, replica_id={}, raft_server_id={} commited_lsn={}, "
"compact_lsn={}, checkpoint_lsn:{}, next_dsn={} "
"log_dev={} log_store={}",
(load_existing ? "Existing" : "New"), group_id_str(), my_replica_id_str(), m_raft_server_id,
m_commit_upto_lsn.load(), m_compact_lsn.load(), m_rd_sb->checkpoint_lsn, m_next_dsn.load(),
m_rd_sb->logdev_id, m_rd_sb->logstore_id);
}
bool RaftReplDev::bind_data_service() {
RD_LOGI(NO_TRACE_ID, "Starting data channel, group_id={}, replica_id={}", group_id_str(), my_replica_id_str());
bool success = false;
#ifdef _PRERELEASE
success =
m_msg_mgr.bind_data_service_request(PUSH_DATA, m_group_id, [this](intrusive< sisl::GenericRpcData >& rpc_data) {
if (iomgr_flip::instance()->delay_flip("slow_down_data_channel", [this, rpc_data]() mutable {
RD_LOGI(NO_TRACE_ID, "Resuming after slow down data channel flip");
on_push_data_received(rpc_data);
})) {
RD_LOGI(NO_TRACE_ID, "Slow down data channel flip is enabled, scheduling to call later");
} else {
on_push_data_received(rpc_data);
}
});
#else
success =
m_msg_mgr.bind_data_service_request(PUSH_DATA, m_group_id, bind_this(RaftReplDev::on_push_data_received, 1));
#endif
if (!success) {
RD_LOGE(NO_TRACE_ID, "Failed to bind data service request for PUSH_DATA");
return false;
}
success =
m_msg_mgr.bind_data_service_request(FETCH_DATA, m_group_id, bind_this(RaftReplDev::on_fetch_data_received, 1));
if (!success) {
RD_LOGE(NO_TRACE_ID, "Failed to bind data service request for FETCH_DATA");
return false;
}
return true;
}
bool RaftReplDev::join_group() {
bind_data_service();
auto raft_result =
m_msg_mgr.join_group(m_group_id, "homestore_replication",
std::dynamic_pointer_cast< nuraft_mesg::mesg_state_mgr >(shared_from_this()));
if (!raft_result) {
HS_DBG_ASSERT(false, "Unable to join the group_id={} with error={}", group_id_str(), raft_result.error());
return false;
}
return true;
}
// All the steps in the implementation should be idempotent and retryable.
AsyncReplResult<> RaftReplDev::start_replace_member(uuid_t task_id, const replica_member_info& member_out,
const replica_member_info& member_in, uint32_t commit_quorum,
uint64_t trace_id) {
if (is_stopping()) {
RD_LOGI(trace_id, "repl dev is being shutdown!");
return make_async_error<>(ReplServiceError::STOPPING);
}
if (get_stage() != repl_dev_stage_t::ACTIVE) {
RD_LOGE(trace_id, "repl dev is not ready, stage={}", static_cast< int >(get_stage()));
return make_async_error<>(ReplServiceError::UNREADY_STATE);
}
incr_pending_request_num();
RD_LOGI(trace_id, "Start replace member, task_id={}, member_out={} member_in={}", boost::uuids::to_string(task_id),
boost::uuids::to_string(member_out.id), boost::uuids::to_string(member_in.id));
// Step1, validate request
// TODO support rollback, this could happen when the first task failed, and we want to launch a new task to
// remediate it. Need to rollback the first task. And for the same task, it's reentrant and idempotent.
if (!m_rd_sb->replace_member_task.task_id.is_nil() && m_rd_sb->replace_member_task.task_id != task_id) {
RD_LOGE(trace_id, "Step1. Replace member, task_id={} is not the same as existing task_id={}",
boost::uuids::to_string(task_id), boost::uuids::to_string(m_rd_sb->replace_member_task.task_id));
decr_pending_request_num();
return make_async_error<>(ReplServiceError::REPLACE_MEMBER_TASK_MISMATCH);
}
auto out_srv_cfg = raft_server()->get_config()->get_server(nuraft_mesg::to_server_id(member_out.id));
if (!out_srv_cfg) {
auto in_srv_cfg = raft_server()->get_config()->get_server(nuraft_mesg::to_server_id(member_in.id));
if (in_srv_cfg) {
RD_LOGI(trace_id,
"Step1. Replace member, the intent has already been fulfilled, ignore it, task_id={}, "
"member_out={} member_in={}",
boost::uuids::to_string(task_id), boost::uuids::to_string(member_out.id),
boost::uuids::to_string(member_in.id));
decr_pending_request_num();
return make_async_success<>();
}
RD_LOGE(trace_id, "Step1. Replace member invalid parameter, out member is not found, task_id={}",
boost::uuids::to_string(task_id));
decr_pending_request_num();
return make_async_error<>(ReplServiceError::SERVER_NOT_FOUND);
}
if (m_my_repl_id != get_leader_id()) {
decr_pending_request_num();
return make_async_error<>(ReplServiceError::NOT_LEADER);
}
// Check if leader itself is requested to move out.
if (m_my_repl_id == member_out.id) {
// immediate=false successor=-1, nuraft will choose an alive peer with highest priority as successor, and wait
// until the successor finishes the catch-up of the latest log, and then resign. Return NOT_LEADER and let
// client retry.
raft_server()->yield_leadership(false /* immediate */, -1 /* successor */);
RD_LOGI(trace_id, "Step1. Replace member, leader is the member_out so yield leadership, task_id={}",
boost::uuids::to_string(task_id));
decr_pending_request_num();
return make_async_error<>(ReplServiceError::NOT_LEADER);
}
// quorum safety check. TODO currently only consider lsn, need to check last response time.
auto active_peers = get_active_peers();
// active_peers doesn't include leader itself.
auto active_num = active_peers.size() + 1;
for (const auto& p : active_peers) {
active_num = p == member_out.id ? active_num - 1 : active_num;
active_num = p == member_in.id ? active_num - 1 : active_num;
}
RD_LOGD(trace_id,
"Step1. Replace member, quorum safety check, active_peers={}, active_peers_exclude_out/in_member={}, "
"commit_quorum={}, task_id={}",
active_peers.size(), active_num, commit_quorum, boost::uuids::to_string(task_id));
// commit_quorum=0 means actual commit quorum is the majority. In this case, active normal member count should be
// >= majority. To be more specific, if we have S1(leader), S2, S3(out), S4(in), we don't allow
// replace_member(S3, S4) if S2 is down or laggy. Needs to recover S2 first or retry with commit_quorum=1.
auto quorum = get_quorum_for_commit();
if (active_num < quorum && commit_quorum == 0) {
RD_LOGE(trace_id,
"Step1. Replace member, quorum safety check failed, active_peers={}, "
"active_peers_exclude_out/in_member={}, required_quorum={}, commit_quorum={}, task_id={}",
active_peers.size(), active_num, quorum, commit_quorum, boost::uuids::to_string(task_id));
decr_pending_request_num();
return make_async_error<>(ReplServiceError::QUORUM_NOT_MET);
}
if (commit_quorum >= 1) {
// Two members are down and leader cant form the quorum. Reduce the quorum size.
reset_quorum_size(commit_quorum, trace_id);
}
// Step 2: Handle out member.
#ifdef _PRERELEASE
if (iomgr_flip::instance()->test_flip("replace_member_set_learner_failure")) {
RD_LOGE(trace_id, "Simulating set member to learner failure");
return make_async_error(ReplServiceError::FAILED);
}
#endif
RD_LOGI(trace_id, "Step2. Replace member, flip out member to learner, task_id={}",
boost::uuids::to_string(task_id));
auto learner_ret = do_flip_learner(member_out, true, true, trace_id);
if (learner_ret != ReplServiceError::OK) {
RD_LOGE(trace_id, "Step2. Replace member, failed to flip out member to learner {}, task_id={}", learner_ret,
boost::uuids::to_string(task_id));
reset_quorum_size(0, trace_id);
decr_pending_request_num();
return make_async_error(std::move(learner_ret));
}
RD_LOGI(trace_id, "Step2. Replace member, flip out member to learner and set priority to 0, task_id={}",
boost::uuids::to_string(task_id));
// Step 3. Append log entry to mark the old member is out and new member is added.
RD_LOGI(trace_id, "Step3. Replace member, propose to raft for HS_CTRL_START_REPLACE req, group_id={}, task_id={}",
boost::uuids::to_string(task_id), group_id_str());
auto rreq = repl_req_ptr_t(new repl_req_ctx{});
replace_member_ctx ctx;
ctx.task_id = task_id;
ctx.replica_out = member_out;
ctx.replica_in = member_in;
sisl::blob header(r_cast< uint8_t* >(&ctx), sizeof(replace_member_ctx));
rreq->init(repl_key{.server_id = server_id(),
.term = raft_server()->get_term(),
.dsn = m_next_dsn.fetch_add(1),
.traceID = trace_id},
journal_type_t::HS_CTRL_START_REPLACE, true, header, sisl::blob{}, 0, m_listener);
auto err = m_state_machine->propose_to_raft(std::move(rreq));
if (err != ReplServiceError::OK) {
RD_LOGE(trace_id,
"Step3. Replace member, propose to raft for HS_CTRL_START_REPLACE req failed, task_id={}, err={}",
boost::uuids::to_string(task_id), err);
reset_quorum_size(0, trace_id);
decr_pending_request_num();
return make_async_error<>(std::move(err));
}
// Step 4. Add the new member, new member will inherit the priority of the out member.
#ifdef _PRERELEASE
if (iomgr_flip::instance()->test_flip("replace_member_add_member_failure")) {
RD_LOGE(trace_id, "Simulating add member failure");
return make_async_error(ReplServiceError::FAILED);
}
#endif
RD_LOGI(trace_id, "Step4. Replace member, propose to raft to add new member, group_id={}, task_id={}",
group_id_str(), boost::uuids::to_string(task_id));
replica_member_info member_to_add = member_in;
member_to_add.priority = out_srv_cfg.get()->get_priority();
auto ret = do_add_member(member_to_add, trace_id);
if (ret != ReplServiceError::OK) {
RD_LOGE(trace_id, "Step4. Replace member, add member failed, err={}, task_id={}", ret,
boost::uuids::to_string(task_id));
reset_quorum_size(0, trace_id);
decr_pending_request_num();
return make_async_error<>(std::move(ret));
}
RD_LOGI(trace_id, "Step4. Replace member, proposed to raft to add member, task_id={}, member={}",
boost::uuids::to_string(task_id), boost::uuids::to_string(member_in.id));
reset_quorum_size(0, trace_id);
decr_pending_request_num();
return make_async_success<>();
}
AsyncReplResult<> RaftReplDev::complete_replace_member(uuid_t task_id, const replica_member_info& member_out,
const replica_member_info& member_in, uint32_t commit_quorum,
uint64_t trace_id) {
if (is_stopping()) {
RD_LOGI(trace_id, "repl dev is being shutdown!");
return make_async_error<>(ReplServiceError::STOPPING);
}
if (get_stage() != repl_dev_stage_t::ACTIVE) {
RD_LOGE(trace_id, "repl dev is not ready, stage={}", static_cast< int >(get_stage()));
return make_async_error<>(ReplServiceError::UNREADY_STATE);
}
incr_pending_request_num();
RD_LOGI(trace_id, "Complete replace member, task_id={}, member_out={}, member_in={}",
boost::uuids::to_string(task_id), boost::uuids::to_string(member_out.id),
boost::uuids::to_string(member_in.id));
if (commit_quorum >= 1) {
// Two members are down and leader cant form the quorum. Reduce the quorum size.
reset_quorum_size(commit_quorum, trace_id);
}
// Step 5: Remove member
RD_LOGI(trace_id, "Step5. Replace member, remove old member, task_id={}, member={}",
boost::uuids::to_string(task_id), boost::uuids::to_string(member_out.id));
#ifdef _PRERELEASE
if (iomgr_flip::instance()->test_flip("replace_member_remove_member_failure")) {
RD_LOGE(trace_id, "Simulating remove member failure");
return make_async_error(ReplServiceError::FAILED);
}
#endif
auto ret = do_remove_member(member_out, trace_id);
if (ret != ReplServiceError::OK) {
RD_LOGE(trace_id, "Step5. Replace member, failed to remove member, task_id={}, member={}, err={}",
boost::uuids::to_string(task_id), boost::uuids::to_string(member_out.id), ret);
reset_quorum_size(0, trace_id);
decr_pending_request_num();
return make_async_error<>(std::move(ret));
}
RD_LOGI(trace_id, "Step5. Replace member, proposed to raft to remove member, task_id={}, member={}",
boost::uuids::to_string(task_id), boost::uuids::to_string(member_out.id));
auto timeout = HS_DYNAMIC_CONFIG(consensus.wait_for_config_change_ms);
// TODO Move wait logic to nuraft_mesg
if (!wait_and_check(
[&]() {
auto srv_conf = raft_server()->get_srv_config(nuraft_mesg::to_server_id(member_out.id));
if (srv_conf) {
RD_LOGD(trace_id, "out member still exists in raft group, member={}",
boost::uuids::to_string(member_out.id));
return false;
}
return true;
},
timeout)) {
RD_LOGD(trace_id,
"Step5. Replace member, wait for old member removed timed out, cancel the request, timeout: {}",
timeout);
// If the member_out is down, leader will force remove it after
// leave_timeout=leave_limit_(default=5)*heart_beat_interval_, it's better for client to retry it.
return make_async_error<>(ReplServiceError::RETRY_REQUEST);
}
RD_LOGD(trace_id, "Step5. Replace member, old member is removed, task_id={}, member={}",
boost::uuids::to_string(task_id), boost::uuids::to_string(member_out.id));
// Step 2. Append log entry to complete replace member
RD_LOGI(trace_id,
"Step6. Replace member, propose to raft for HS_CTRL_COMPLETE_REPLACE req, group_id={}, task_id={}",
boost::uuids::to_string(task_id), group_id_str());
auto rreq = repl_req_ptr_t(new repl_req_ctx{});
replace_member_ctx ctx;
ctx.task_id = task_id;
ctx.replica_out = member_out;
ctx.replica_in = member_in;
sisl::blob header(r_cast< uint8_t* >(&ctx), sizeof(replace_member_ctx));
rreq->init(repl_key{.server_id = server_id(),
.term = raft_server()->get_term(),
.dsn = m_next_dsn.fetch_add(1),
.traceID = trace_id},
journal_type_t::HS_CTRL_COMPLETE_REPLACE, true, header, sisl::blob{}, 0, m_listener);
auto err = m_state_machine->propose_to_raft(std::move(rreq));
if (err != ReplServiceError::OK) {
RD_LOGE(trace_id,
"Step6. Replace member, propose to raft for HS_CTRL_COMPLETE_REPLACE req failed , task_id={}, err={}",
boost::uuids::to_string(task_id), err);
reset_quorum_size(0, trace_id);
decr_pending_request_num();
return make_async_error<>(std::move(err));
}
reset_quorum_size(0, trace_id);
decr_pending_request_num();
RD_LOGI(trace_id, "Complete replace member done, group_id={}, task_id={}, member_out={} member_in={}",
group_id_str(), boost::uuids::to_string(task_id), boost::uuids::to_string(member_out.id),
boost::uuids::to_string(member_in.id));
return make_async_success<>();
}
ReplaceMemberStatus RaftReplDev::get_replace_member_status(uuid_t task_id, const replica_member_info& member_out,
const replica_member_info& member_in,
const std::vector< replica_member_info >& others,
uint64_t trace_id) {
if (is_stopping()) {
RD_LOGI(trace_id, "repl dev is being shutdown!");
return ReplaceMemberStatus::UNKNOWN;
}
incr_pending_request_num();
if (!m_repl_svc_ctx || !is_leader()) {
decr_pending_request_num();
return ReplaceMemberStatus::NOT_LEADER;
}
auto peers = get_replication_status();
peer_info out_peer_info;
bool found_out = false;
bool found_in = false;
for (auto p : peers) {
if (p.id_ == member_out.id) {
out_peer_info = p;
found_out = true;
} else if (p.id_ == member_in.id) {
found_in = true;
}
}
bool intent_completed = !found_out && found_in;
if (m_rd_sb->replace_member_task.task_id.is_nil()) {
if (intent_completed) {
// If caller doesn't give others, won't check it.
bool others_match = others.size() == 0 || others.size() + 1 == peers.size();
auto detail = std::string{};
for (const auto& other : others) {
if (!raft_server()->get_srv_config(nuraft_mesg::to_server_id(other.id))) {
others_match = false;
detail = fmt::format("member {} is not found in raft group", boost::uuids::to_string(other.id));
break;
}
}
if (!others_match) {
RD_LOGE(trace_id,
"get_replace_member_status failed, other membership mismatch, task_id={}, detail={}, "
"others.size={}, "
"all_peers.size={}",
boost::uuids::to_string(task_id), detail, others.size(), peers.size());
decr_pending_request_num();
return ReplaceMemberStatus::UNKNOWN;
}
decr_pending_request_num();
return ReplaceMemberStatus::COMPLETED;
}
decr_pending_request_num();
return ReplaceMemberStatus::TASK_NOT_FOUND;
}
if (m_rd_sb->replace_member_task.task_id != task_id) {
RD_LOGE(trace_id, "get_replace_member_status failed, task_id mismatch, persisted={}, received={}",
boost::uuids::to_string(m_rd_sb->replace_member_task.task_id), boost::uuids::to_string(task_id));
decr_pending_request_num();
return ReplaceMemberStatus::TASK_ID_MISMATCH;
}
// If the first attempt to remove out_member fails because out_member is down or leader crashes between Step5(remove
// member) and Step6(HS_CTRL_COMPLETE_REPLACE mesg). Replace member intent might be already fulfilled but
// replace_member_task sb still exists. In this case, we honor task sb, return IN_PROGRESS, and wait for reaper
// thread to trigger complete_replace_member again to cleanup the sb.
if (intent_completed) {
RD_LOGI(trace_id,
"Member replacement fulfilled, but task still exists, wait for reaper thread to retry "
"complete_replace_member. task_id={}, out_member={}, in_member={}",
boost::uuids::to_string(m_rd_sb->replace_member_task.task_id), boost::uuids::to_string(member_out.id),
boost::uuids::to_string(member_in.id));
}
RD_LOGD(trace_id, "Member replacement is in progress. task_id={}, out_member={}, in_member={}",
boost::uuids::to_string(task_id), boost::uuids::to_string(member_out.id),
boost::uuids::to_string(member_in.id));
decr_pending_request_num();
return ReplaceMemberStatus::IN_PROGRESS;
}
ReplServiceError RaftReplDev::do_add_member(const replica_member_info& member, uint64_t trace_id) {
if (m_my_repl_id != get_leader_id()) {
RD_LOGI(trace_id, "Member to add failed, not leader");
return ReplServiceError::BAD_REQUEST;
}
auto ret = retry_when_config_changing(
[&] {
auto srv_config = nuraft::srv_config(nuraft_mesg::to_server_id(member.id), 0,
boost::uuids::to_string(member.id), "", false, member.priority);
auto add_ret = m_msg_mgr.add_member(m_group_id, srv_config)
.via(&folly::InlineExecutor::instance())
.thenValue([this, member, trace_id](auto&& e) -> nuraft::cmd_result_code {
return e.hasError() ? e.error() : nuraft::cmd_result_code::OK;
});
return add_ret.value();
},
trace_id);
if (ret == nuraft::cmd_result_code::SERVER_ALREADY_EXISTS) {
RD_LOGW(trace_id, "Ignoring error returned from nuraft add_member, member={}, err={}",
boost::uuids::to_string(member.id), ret);
} else if (ret == nuraft::cmd_result_code::CANCELLED) {
// nuraft mesg will return cancelled if the change is not commited after waiting for
// raft_leader_change_timeout_ms(default 3200).
RD_LOGE(trace_id, "Add member failed, member={}, err={}", boost::uuids::to_string(member.id), ret);
return ReplServiceError::CANCELLED;
} else if (ret != nuraft::cmd_result_code::OK) {
// It's ok to retry this request as the request
// replace member is idempotent.
RD_LOGE(trace_id, "Add member failed, member={}, err={}", boost::uuids::to_string(member.id), ret);
return ReplServiceError::RETRY_REQUEST;
}
RD_LOGI(trace_id, "Proposed to raft to add member, member={}", boost::uuids::to_string(member.id));
return ReplServiceError::OK;
}
ReplServiceError RaftReplDev::do_remove_member(const replica_member_info& member, uint64_t trace_id) {
// The member should not be the leader.
if (m_my_repl_id == member.id && m_my_repl_id == get_leader_id()) {
// If leader is the member requested to move out, then give up leadership and return error.
// Client will retry replace_member request to the new leader.
raft_server()->yield_leadership(false /* immediate */, -1 /* successor */);
RD_LOGI(trace_id, "Member to remove is the leader so yield leadership");
return ReplServiceError::NOT_LEADER;
}
auto ret = retry_when_config_changing(
[&] {
auto rem_ret = m_msg_mgr.rem_member(m_group_id, member.id)
.via(&folly::InlineExecutor::instance())
.thenValue([this, member, trace_id](auto&& e) -> nuraft::cmd_result_code {
return e.hasError() ? e.error() : nuraft::cmd_result_code::OK;
});
return rem_ret.value();
},
trace_id);
if (ret == nuraft::cmd_result_code::SERVER_NOT_FOUND) {
RD_LOGW(trace_id, "Remove member not found in group error, ignoring, member={}",
boost::uuids::to_string(member.id));
} else if (ret != nuraft::cmd_result_code::OK) {
// Its ok to retry this request as the request
// of replace member is idempotent.
RD_LOGE(trace_id, "Replace member failed to remove member, member={}, err={}",
boost::uuids::to_string(member.id), ret);
return ReplServiceError::RETRY_REQUEST;
}
RD_LOGI(trace_id, "Proposed to raft to remove member, member={}", boost::uuids::to_string(member.id));
return ReplServiceError::OK;
}
AsyncReplResult<> RaftReplDev::flip_learner_flag(const replica_member_info& member, bool target, uint32_t commit_quorum,
bool wait_and_verify, uint64_t trace_id) {
RD_LOGI(trace_id, "Flip learner flag to {}, member={}", target, boost::uuids::to_string(member.id));
if (is_stopping()) {
RD_LOGI(trace_id, "repl dev is being shutdown!");
return make_async_error<>(ReplServiceError::STOPPING);
}
incr_pending_request_num();
if (commit_quorum >= 1) {
// Two members are down and leader cant form the quorum. Reduce the quorum size.
reset_quorum_size(commit_quorum, trace_id);
}
auto ret = do_flip_learner(member, target, wait_and_verify, trace_id);
if (ret != ReplServiceError::OK) {
RD_LOGE(trace_id, "Flip learner flag failed {}, member={}", ret, boost::uuids::to_string(member.id));
reset_quorum_size(0, trace_id);
decr_pending_request_num();
return make_async_error<>(std::move(ret));
}
RD_LOGI(trace_id, "Learner flag has been set to {}, member={}", target, boost::uuids::to_string(member.id));
return make_async_success<>();
}
ReplServiceError RaftReplDev::do_flip_learner(const replica_member_info& member, bool target, bool wait_and_verify,
uint64_t trace_id) {
// 1. Prerequisite check
if (m_my_repl_id != get_leader_id()) {
RD_LOGI(trace_id, "flip learner flag failed, not leader");
return ReplServiceError::NOT_LEADER;
}
// 2. Flip learner
RD_LOGI(trace_id, "flip learner flag to {}, member={}", target, boost::uuids::to_string(member.id));
auto srv_cfg = raft_server()->get_config()->get_server(nuraft_mesg::to_server_id(member.id));
if (!srv_cfg) {
RD_LOGE(trace_id, "invalid parameter, member is not found, member={}", boost::uuids::to_string(member.id));
return ReplServiceError::SERVER_NOT_FOUND;
}
if (srv_cfg->is_learner() != target) {
auto ret = retry_when_config_changing(
[&] {
auto learner_ret = raft_server()->flip_learner_flag(nuraft_mesg::to_server_id(member.id), target);
return learner_ret->get_result_code();
},
trace_id);
if (ret != nuraft::cmd_result_code::OK) {
RD_LOGE(trace_id, "Propose to raft to flip learner failed, err: {}", ret);
return ReplServiceError::RETRY_REQUEST;
}
} else {
RD_LOGD(trace_id, "learner flag has already been set to {}, skip, member={}", target,
boost::uuids::to_string(member.id));
}
// 3. Verification
if (wait_and_verify) {
auto timeout = HS_DYNAMIC_CONFIG(consensus.wait_for_config_change_ms);
if (!wait_and_check(
[&]() {
auto srv_conf = raft_server()->get_srv_config(nuraft_mesg::to_server_id(member.id));
return srv_conf->is_learner();
},
timeout)) {
RD_LOGD(trace_id, "Wait for flipping learner timed out, please retry, timeout: {}", timeout);
return ReplServiceError::RETRY_REQUEST;
}
}
return ReplServiceError::OK;
}
nuraft::cmd_result_code RaftReplDev::retry_when_config_changing(const std::function< nuraft::cmd_result_code() >& func,
uint64_t trace_id) {
auto ret = nuraft::cmd_result_code::OK;
int32_t retries = HS_DYNAMIC_CONFIG(consensus.config_changing_error_retries);
for (auto i = 0; i < retries; i++) {
ret = func();
if (ret == nuraft::cmd_result_code::CONFIG_CHANGING) {
RD_LOGW(trace_id, "Propose to raft failed due to config_changing, attempt: {}", i);
std::this_thread::sleep_for(std::chrono::milliseconds(500));
continue;
}
break;
}
return ret;
}
bool RaftReplDev::wait_and_check(const std::function< bool() >& check_func, uint32_t timeout_ms, uint32_t interval_ms) {
auto times = timeout_ms / interval_ms;
if (times == 0) { times = 1; }
for (auto i = 0; i < static_cast< int32_t >(times); i++) {
if (check_func()) { return true; }
std::this_thread::sleep_for(std::chrono::milliseconds(100));
}
return false;
}
ReplServiceError RaftReplDev::set_priority(const replica_id_t& member, int32_t priority, uint64_t trace_id) {
auto priority_ret = raft_server()->set_priority(nuraft_mesg::to_server_id(member), priority);
// Set_priority should be handled by leader, but if the intent is to set the leader's priority to 0, it returns
// BROADCAST. In this case return NOT_LEADER to let client retry new leader.
// If there is an uncommited_config, nuraft set_priority will honor this uncommited config and generate new
// config based on it and won't have config_changing error.
if (priority_ret != nuraft::raft_server::PrioritySetResult::SET) {
RD_LOGE(trace_id, "Propose to raft to set priority failed, result: {}",
priority_ret == nuraft::raft_server::PrioritySetResult::BROADCAST ? "BROADCAST" : "IGNORED");
return ReplServiceError::NOT_LEADER;
}
return ReplServiceError::OK;
}
void RaftReplDev::reset_quorum_size(uint32_t commit_quorum, uint64_t trace_id) {
RD_LOGI(trace_id, "Reset raft quorum size={}", commit_quorum);
nuraft::raft_params params = raft_server()->get_current_params();
params.with_custom_commit_quorum_size(commit_quorum);
params.with_custom_election_quorum_size(commit_quorum);
raft_server()->update_params(params);
}
folly::SemiFuture< ReplServiceError > RaftReplDev::destroy_group() {
// Set the intent to destroy the group
m_stage.update([](auto* stage) { *stage = repl_dev_stage_t::DESTROYING; });
// Propose to the group to destroy
auto rreq = repl_req_ptr_t(new repl_req_ctx{});
// if we have a rreq {originator=1, term=1, dsn=0, lsn=7} in follower and a baseline resync is triggerd before the
// rreq is committed in the follower, then the on_commit of the rreq will not be called and as a result this rreq
// will become a garbage rreq in this follower. now if we trigger a destroy_group, a new rreq {originator=1, term=1,
// dsn=0} will created in the follower since the default dsn of a repl_key is 0.after the log of this rreq is
// appended to log store and get a new lsn, if we link the new lsn to the old rreq (rreq is identified by
// {originator, term, dsn}) which has alread have a lsn, then a assert will be throw out. pls refer to
// repl_req_ctx::set_lsn
// here, we set the dsn to a new one , which is definitely unique in the follower, so that the new rreq will not
// have a conflict with the old rreq.
auto err = init_req_ctx(rreq,
repl_key{.server_id = server_id(),
.term = raft_server()->get_term(),
.dsn = m_next_dsn.fetch_add(1),
.traceID = std::numeric_limits< uint64_t >::max()},
journal_type_t::HS_CTRL_DESTROY, true, sisl::blob{}, sisl::blob{}, 0, m_listener);
if (err != ReplServiceError::OK) {
// Failed to initialize the repl_req_ctx for replace member.
LOGERROR("Failed to initialize repl_req_ctx for destorying group, error={}", err);
return folly::makeSemiFuture< ReplServiceError >(std::move(err));
}
err = m_state_machine->propose_to_raft(std::move(rreq));
if (err != ReplServiceError::OK) {
m_stage.update([](auto* stage) { *stage = repl_dev_stage_t::ACTIVE; });
return folly::makeSemiFuture< ReplServiceError >(std::move(err));
LOGERROR("RaftReplDev::destroy_group failed {}", err);
}
LOGINFO("Raft repl dev destroy_group={}", group_id_str());
return m_destroy_promise.getSemiFuture();
}
void RaftReplDev::use_config(json_superblk raft_config_sb) { m_raft_config_sb = std::move(raft_config_sb); }
void RaftReplDev::on_create_snapshot(nuraft::snapshot& s, nuraft::async_result< bool >::handler_type& when_done) {
RD_LOGD(NO_TRACE_ID, "create_snapshot last_idx={}/term={}", s.get_last_log_idx(), s.get_last_log_term());
auto snp_ctx = std::make_shared< nuraft_snapshot_context >(s);
auto result = m_listener->create_snapshot(snp_ctx).get();
auto null_except = std::shared_ptr< std::exception >();
HS_REL_ASSERT(result.hasError() == false, "Not expecting creating snapshot to return false. ");
auto ret_val{true};
if (when_done) { when_done(ret_val, null_except); }
}
// 1 before repl_dev.stop() is called, the upper layer should make sure that there is no pending request. so graceful
// shutdown can consider when stopping repl_dev, there is no pending request.
// 2 before the log is appended to log store, repl_dev will guarantee the corresponding data is persisted on disk. so
// even if we do not care about this when stop, it will be ok, since log will replayed after restart.
// we do not have shutdown for async_alloc_write according to the two points above.
void RaftReplDev::async_alloc_write(sisl::blob const& header, sisl::blob const& key, sisl::sg_list const& data,
repl_req_ptr_t rreq, bool part_of_batch, trace_id_t tid) {
if (!rreq) { auto rreq = repl_req_ptr_t(new repl_req_ctx{}); }
{
auto const guard = m_stage.access();
if (auto const stage = *guard.get(); stage != repl_dev_stage_t::ACTIVE) {
RD_LOGW(tid, "Raft channel: Not ready to accept writes, stage={}", enum_name(stage));
handle_error(rreq,
(stage == repl_dev_stage_t::INIT) ? ReplServiceError::SERVER_IS_JOINING
: (stage == repl_dev_stage_t::UNREADY) ? ReplServiceError::UNREADY_STATE
: ReplServiceError::SERVER_IS_LEAVING);
return;
}
}
auto status = init_req_ctx(rreq,
repl_key{.server_id = server_id(),
.term = raft_server()->get_term(),
.dsn = m_next_dsn.fetch_add(1),
.traceID = tid},
data.size ? journal_type_t::HS_DATA_LINKED : journal_type_t::HS_DATA_INLINED,
true /* is_proposer */, header, key, data.size, m_listener);
if (status != ReplServiceError::OK) {
RD_LOGI(tid, "Initializing rreq failed error={}, failing this req", status);
handle_error(rreq, status);
return;
}
RD_LOGD(tid, "repl_key [{}], header size [{}] bytes, user_key size [{}] bytes, data size [{}] bytes", rreq->rkey(),
header.size(), key.size(), data.size);
// Add the request to the repl_dev_rreq map, it will be accessed throughout the life cycle of this request
auto const [it, happened] = m_repl_key_req_map.emplace(rreq->rkey(), rreq);
RD_DBG_ASSERT(happened, "Duplicate repl_key={} found in the map", rreq->rkey().to_string());
// If it is header only entry, directly propose to the raft
if (rreq->has_linked_data()) {
if (rreq->is_proposer() && rreq->has_state(repl_req_state_t::DATA_COMMITTED)) {
RD_LOGE(tid, "data blks has already been allocated and committed, failing this req");
handle_error(rreq, ReplServiceError::DATA_DUPLICATED);
return;
}
#ifdef _PRERELEASE
if (iomgr_flip::instance()->test_flip("disable_leader_push_data")) {
RD_LOGD(tid, "Simulating push data failure, so that all the follower will have to fetch data");
} else
push_data_to_all_followers(rreq, data);
#else
push_data_to_all_followers(rreq, data);
#endif
COUNTER_INCREMENT(m_metrics, total_write_cnt, 1);
COUNTER_INCREMENT(m_metrics, outstanding_data_write_cnt, 1);
auto const data_write_start_time = Clock::now();
// Write the data
data_service()
.async_write(data, rreq->local_blkid())
.thenValue([this, rreq, data_write_start_time](auto&& err) {
// update outstanding no matter error or not;
COUNTER_DECREMENT(m_metrics, outstanding_data_write_cnt, 1);
if (err) {
HS_DBG_ASSERT(false, "Error in writing data, err_code={}", err.value());
handle_error(rreq, ReplServiceError::DRIVE_WRITE_ERROR);
} else {
// update metrics for originated rreq;
const auto write_num_pieces = rreq->local_blkid().num_pieces();
HISTOGRAM_OBSERVE(m_metrics, rreq_pieces_per_write, write_num_pieces);
HISTOGRAM_OBSERVE(m_metrics, rreq_data_write_latency_us,
get_elapsed_time_us(data_write_start_time));
HISTOGRAM_OBSERVE(m_metrics, rreq_total_data_write_latency_us,
get_elapsed_time_us(rreq->created_time()));
auto raft_status = m_state_machine->propose_to_raft(rreq);
if (raft_status != ReplServiceError::OK) { handle_error(rreq, raft_status); }
}
});
} else {
RD_LOGT(tid, "Skipping data channel send since value size is 0");
rreq->add_state(repl_req_state_t::DATA_WRITTEN);
auto raft_status = m_state_machine->propose_to_raft(rreq);
if (raft_status != ReplServiceError::OK) { handle_error(rreq, raft_status); }
}
}
void RaftReplDev::push_data_to_all_followers(repl_req_ptr_t rreq, sisl::sg_list const& data) {
auto& builder = rreq->create_fb_builder();
// Prepare the rpc request packet with all repl_reqs details
builder.FinishSizePrefixed(CreatePushDataRequest(
builder, rreq->traceID(), server_id(), rreq->term(), rreq->dsn(),
builder.CreateVector(rreq->header().cbytes(), rreq->header().size()),
builder.CreateVector(rreq->key().cbytes(), rreq->key().size()), data.size, get_time_since_epoch_ms()));
rreq->m_pkts = sisl::io_blob::sg_list_to_ioblob_list(data);
rreq->m_pkts.insert(rreq->m_pkts.begin(), sisl::io_blob{builder.GetBufferPointer(), builder.GetSize(), false});
/*RD_LOGI("Data Channel: Pushing data to all followers: rreq=[{}] data=[{}]", rreq->to_string(),
flatbuffers::FlatBufferToString(builder.GetBufferPointer() + sizeof(flatbuffers::uoffset_t),
PushDataRequestTypeTable()));*/
auto peers = get_active_peers();
auto calls = std::vector< nuraft_mesg::NullAsyncResult >();
for (auto peer : peers) {
RD_LOGD(rreq->traceID(), "Data Channel: Pushing data to follower {}, rreq=[{}]", peer, rreq->to_string());
calls.push_back(group_msg_service()
->data_service_request_unidirectional(peer, PUSH_DATA, rreq->m_pkts)
.via(&folly::InlineExecutor::instance()));
}
folly::collectAllUnsafe(calls).thenValue([this, rreq](auto&& v_res) {
for (auto const& res : v_res) {
if (sisl_likely(res.value())) {
auto r = res.value();
if (r.hasError()) {
// Just logging PushData error, no action is needed as follower can try by fetchData.
RD_LOGI(rreq->traceID(), "Data Channel: Error in pushing data to all followers: rreq=[{}] error={}",
rreq->to_string(), r.error());
}
}
}
RD_LOGD(rreq->traceID(), "Data Channel: Data push completed for rreq=[{}]", rreq->to_compact_string());
// Release the buffer which holds the packets
rreq->release_fb_builder();
rreq->m_pkts.clear();
});
}
void RaftReplDev::on_push_data_received(intrusive< sisl::GenericRpcData >& rpc_data) {
auto const push_data_rcv_time = Clock::now();
auto const& incoming_buf = rpc_data->request_blob();
if (!incoming_buf.cbytes()) {
RD_LOGW(NO_TRACE_ID, "Data Channel: PushData received with empty buffer, ignoring this call");
rpc_data->send_response();
return;
}
auto const fb_size =
flatbuffers::ReadScalar< flatbuffers::uoffset_t >(incoming_buf.cbytes()) + sizeof(flatbuffers::uoffset_t);
auto push_req = GetSizePrefixedPushDataRequest(incoming_buf.cbytes());
if (fb_size + push_req->data_size() != incoming_buf.size()) {
RD_LOGW(NO_TRACE_ID,
"Data Channel: PushData received with size mismatch, header size {}, data size {}, received size {}",
fb_size, push_req->data_size(), incoming_buf.size());
rpc_data->send_response();
return;
}
sisl::blob header = sisl::blob{push_req->user_header()->Data(), push_req->user_header()->size()};
sisl::blob key = sisl::blob{push_req->user_key()->Data(), push_req->user_key()->size()};
repl_key rkey{.server_id = push_req->issuer_replica_id(),
.term = push_req->raft_term(),
.dsn = push_req->dsn(),
.traceID = push_req->trace_id()};
auto const req_orig_time_ms = push_req->time_ms();
RD_LOGD(rkey.traceID, "Data Channel: PushData received: time diff={} ms.", get_elapsed_time_ms(req_orig_time_ms));
#ifdef _PRERELEASE
if (iomgr_flip::instance()->test_flip("drop_push_data_request")) {
RD_LOGI(rkey.traceID,
"Data Channel: Flip is enabled, skip on_push_data_received to simulate fetch remote data, "
"server_id={}, term={}, dsn={}",
push_req->issuer_replica_id(), push_req->raft_term(), push_req->dsn());
rpc_data->send_response();
return;
}
#endif
auto rreq = applier_create_req(rkey, journal_type_t::HS_DATA_LINKED, header, key, push_req->data_size(),
true /* is_data_channel */);
if (rreq == nullptr) {
RD_LOGE(rkey.traceID,
"Data Channel: Creating rreq on applier has failed, will ignore the push and let Raft channel send "
"trigger a fetch explicitly if needed. rkey={}",
rkey.to_string());
rpc_data->send_response();
return;
}
if (!rreq->save_pushed_data(rpc_data, incoming_buf.cbytes() + fb_size, push_req->data_size())) {
RD_LOGT(rkey.traceID, "Data Channel: Data already received for rreq=[{}], ignoring this data",
rreq->to_string());
rpc_data->send_response();
return;
}
COUNTER_INCREMENT(m_metrics, total_write_cnt, 1);
COUNTER_INCREMENT(m_metrics, outstanding_data_write_cnt, 1);
// Schedule a write and upon completion, mark the data as written.
data_service()
.async_write(r_cast< const char* >(rreq->data()), push_req->data_size(), rreq->local_blkid())
.thenValue([this, rreq, push_data_rcv_time](auto&& err) {
// update outstanding no matter error or not;
COUNTER_DECREMENT(m_metrics, outstanding_data_write_cnt, 1);
if (err) {
COUNTER_INCREMENT(m_metrics, write_err_cnt, 1);
RD_DBG_ASSERT(false, "Error in writing data, error_code={}", err.value());
handle_error(rreq, ReplServiceError::DRIVE_WRITE_ERROR);
} else {
rreq->release_data();
rreq->add_state(repl_req_state_t::DATA_WRITTEN);
rreq->m_data_written_promise.setValue();
// if rreq create time is earlier than push_data receive time, that means the rreq was created by raft
// channel log. Otherwise set to zero as rreq is created by data channel.
const auto data_log_diff_us =
push_data_rcv_time.time_since_epoch().count() > rreq->created_time().time_since_epoch().count()
? get_elapsed_time_us(rreq->created_time(), push_data_rcv_time)
: 0;
auto const data_write_latency = get_elapsed_time_us(push_data_rcv_time);
auto const total_data_write_latency = get_elapsed_time_us(rreq->created_time());
auto const write_num_pieces = rreq->local_blkid().num_pieces();
HISTOGRAM_OBSERVE(m_metrics, rreq_pieces_per_write, write_num_pieces);
HISTOGRAM_OBSERVE(m_metrics, rreq_push_data_latency_us, data_write_latency);
HISTOGRAM_OBSERVE(m_metrics, rreq_total_data_write_latency_us, total_data_write_latency);
RD_LOGD(rreq->traceID(),
"Data Channel: Data write completed for rreq=[{}], time_diff_data_log_us={}, "
"data_write_latency_us={}, total_data_write_latency_us(rreq creation to write complete)={}, "
"local_blkid.num_pieces={}",
rreq->to_compact_string(), data_log_diff_us, data_write_latency, total_data_write_latency,
write_num_pieces);
}
});
}
repl_req_ptr_t RaftReplDev::applier_create_req(repl_key const& rkey, journal_type_t code, sisl::blob const& user_header,
sisl::blob const& key, uint32_t data_size, bool is_data_channel,
int64_t lsn) {
if (is_data_channel) RD_DBG_ASSERT(-1 == lsn, "lsn from data channel should always be -1 , got lsn {}", lsn);
auto const [it, happened] = m_repl_key_req_map.try_emplace(rkey, repl_req_ptr_t(new repl_req_ctx()));
RD_DBG_ASSERT((it != m_repl_key_req_map.end()), "Unexpected error in map_repl_key_to_req");
auto rreq = it->second;
if (!happened) {
// We already have the entry in the map, reset its start time to prevent it from being incorrectly gc during
// use.
rreq->set_created_time();
// Check if we are already allocated the blk by previous caller, in that case we need to return the req.
if (rreq->has_state(repl_req_state_t::BLK_ALLOCATED)) {
// Do validation if we have the correct mapping
// RD_REL_ASSERT(blob_equals(user_header, rreq->header), "User header mismatch for repl_key={}",
// rkey.to_string());
// RD_REL_ASSERT(blob_equals(user_key, rreq->key), "User key mismatch for repl_key={}", rkey.to_string());
RD_LOGT(rkey.traceID, "Repl_key=[{}] already received ", rkey.to_string());
return rreq;
}
}
// rreq->init will allocate the block if it has linked data.
auto status =
init_req_ctx(rreq, rkey, code, m_raft_server_id == rkey.server_id, user_header, key, data_size, m_listener);
if (status != ReplServiceError::OK) {
RD_LOGD(rkey.traceID, "For Repl_key=[{}] alloc hints returned error={}, failing this req", rkey.to_string(),
status);
if (status == ReplServiceError::NO_SPACE_LEFT && !is_data_channel && !rreq->is_proposer()) {
RD_LOGD(rkey.traceID, "Repl_key=[{}] got no_space_left error on follower as lsn={}", rkey.to_string(), lsn);
m_listener->on_no_space_left(lsn, user_header);
} else {
RD_LOGD(
rkey.traceID,
"For Repl_key=[{}] alloc hints returned error={}, failing this req, data_channl: {}, is_proposer: {} ",
rkey.to_string(), status, is_data_channel, rreq->is_proposer());
}
// Do not call handle_error here, because handle_error is for rreq which needs to be terminated. This one can be