-
Notifications
You must be signed in to change notification settings - Fork 1.2k
/
storage_proxy.cc
3052 lines (2703 loc) · 137 KB
/
storage_proxy.cc
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
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
/*
* Copyright 2015 Cloudius Systems
*
* Modified by Cloudius Systems
*/
#include "db/consistency_level.hh"
#include "db/commitlog/commitlog.hh"
#include "db/serializer.hh"
#include "storage_proxy.hh"
#include "unimplemented.hh"
#include "frozen_mutation.hh"
#include "query_result_merger.hh"
#include "core/do_with.hh"
#include "message/messaging_service.hh"
#include "gms/failure_detector.hh"
#include "gms/gossiper.hh"
#include "db/serializer.hh"
#include "storage_service.hh"
#include "core/future-util.hh"
#include "db/read_repair_decision.hh"
#include "db/config.hh"
#include <boost/range/algorithm_ext/push_back.hpp>
#include <boost/range/adaptor/transformed.hpp>
#include <boost/iterator/counting_iterator.hpp>
#include <boost/range/adaptor/filtered.hpp>
#include <boost/range/algorithm/count_if.hpp>
#include <boost/range/algorithm/find.hpp>
#include <boost/range/algorithm/find_if.hpp>
#include <boost/range/algorithm/remove_if.hpp>
#include <boost/range/algorithm/heap_algorithm.hpp>
#include <boost/range/numeric.hpp>
#include <boost/range/algorithm/sort.hpp>
namespace service {
distributed<service::storage_proxy> _the_storage_proxy;
struct mutation_write_timeout_error : public std::exception {
size_t total_block_for;
size_t acks;
mutation_write_timeout_error(size_t tbf, size_t acks_) : total_block_for(tbf), acks(acks_) {}
virtual const char* what() const noexcept {
return "Mutation write timeout";
}
};
struct overloaded_exception : public std::exception {
size_t hints_in_progress;
overloaded_exception(size_t c) : hints_in_progress(c) {}
virtual const char* what() const noexcept {
return "Too many in flight hints";
}
};
static inline bool is_me(gms::inet_address from) {
return from == utils::fb_utilities::get_broadcast_address();
}
class abstract_write_response_handler {
protected:
semaphore _ready; // available when cl is achieved
db::consistency_level _cl;
keyspace& _ks;
lw_shared_ptr<const frozen_mutation> _mutation;
std::unordered_set<gms::inet_address> _targets; // who we sent this mutation to
size_t _pending_endpoints; // how many endpoints in bootstrap state there is
size_t _cl_acks = 0;
virtual size_t total_block_for() {
// original comment from cassandra:
// during bootstrap, include pending endpoints in the count
// or we may fail the consistency level guarantees (see #833, #8058)
return db::block_for(_ks, _cl) + _pending_endpoints;
}
virtual void signal(gms::inet_address from) {
signal();
}
public:
abstract_write_response_handler(keyspace& ks, db::consistency_level cl, lw_shared_ptr<const frozen_mutation> mutation, std::unordered_set<gms::inet_address> targets, size_t pending_endpoints) :
_ready(0), _cl(cl), _ks(ks), _mutation(std::move(mutation)), _targets(targets), _pending_endpoints(pending_endpoints) {}
virtual ~abstract_write_response_handler() {};
void signal(size_t nr = 1) {
_cl_acks += nr;
_ready.signal(nr);
}
// return true on last ack
bool response(gms::inet_address from) {
signal(from);
auto it = _targets.find(from);
assert(it != _targets.end());
_targets.erase(it);
return _targets.size() == 0;
}
future<> wait() {
return _ready.wait(total_block_for());
}
const std::unordered_set<gms::inet_address>& get_targets() {
return _targets;
}
lw_shared_ptr<const frozen_mutation> get_mutation() {
return _mutation;
}
friend storage_proxy;
};
class datacenter_write_response_handler : public abstract_write_response_handler {
void signal(gms::inet_address from) override {
if (is_me(from) || db::is_local(from)) {
abstract_write_response_handler::signal();
}
}
public:
datacenter_write_response_handler(keyspace& ks, db::consistency_level cl, lw_shared_ptr<const frozen_mutation> mutation, std::unordered_set<gms::inet_address> targets, size_t pending_endpoints) :
abstract_write_response_handler(ks, cl, std::move(mutation), targets, pending_endpoints) {}
};
class write_response_handler : public abstract_write_response_handler {
public:
write_response_handler(keyspace& ks, db::consistency_level cl, lw_shared_ptr<const frozen_mutation> mutation, std::unordered_set<gms::inet_address> targets, size_t pending_endpoints) :
abstract_write_response_handler(ks, cl, std::move(mutation), targets, pending_endpoints) {}
};
class datacenter_sync_write_response_handler : public abstract_write_response_handler {
std::unordered_map<sstring, size_t> _dc_responses;
void signal(gms::inet_address from) override {
auto& snitch_ptr = locator::i_endpoint_snitch::get_local_snitch_ptr();
sstring data_center = snitch_ptr->get_datacenter(from);
auto dc_resp = _dc_responses.find(data_center);
if (dc_resp->second > 0) {
--dc_resp->second;
abstract_write_response_handler::signal();
}
}
public:
datacenter_sync_write_response_handler(keyspace& ks, db::consistency_level cl, lw_shared_ptr<const frozen_mutation> mutation, std::unordered_set<gms::inet_address> targets, size_t pending_endpoints) :
abstract_write_response_handler(ks, cl, std::move(mutation), targets, pending_endpoints) {
auto& snitch_ptr = locator::i_endpoint_snitch::get_local_snitch_ptr();
for (auto& target : targets) {
auto dc = snitch_ptr->get_datacenter(target);
if (_dc_responses.find(dc) == _dc_responses.end()) {
_dc_responses.emplace(dc, db::local_quorum_for(ks, dc));
}
}
}
};
storage_proxy::storage_proxy::response_id_type storage_proxy::register_response_handler(std::unique_ptr<abstract_write_response_handler>&& h) {
auto id = _next_response_id++;
auto e = _response_handlers.emplace(id, rh_entry(std::move(h), [this, id] {
auto& e = _response_handlers.find(id)->second;
auto block_for = e.handler->total_block_for();
auto left_for_cl = block_for - e.handler->_cl_acks;
if (left_for_cl <= 0 || e.handler->_cl == db::consistency_level::ANY) {
// we are here because either cl was achieved, but targets left in the handler are not
// responding, so a hint should be written for them, or cl == any in which case
// hints are counted towards consistency, so we need to write hints and count how much was written
e.handler->signal(hint_to_dead_endpoints(e.handler->get_mutation(), e.handler->get_targets()));
// Tracing.trace("Wrote hint to satisfy CL.ANY after no replicas acknowledged the write");
// check cl status after hints are written (can change for cl == any)
left_for_cl = block_for - e.handler->_cl_acks;
}
if (left_for_cl > 0) {
// timeout happened before cl was achieved, throw exception
e.handler->_ready.broken(mutation_write_timeout_error(block_for, e.handler->_cl_acks));
} else {
remove_response_handler(id);
}
}));
assert(e.second);
return id;
}
void storage_proxy::remove_response_handler(storage_proxy::storage_proxy::response_id_type id) {
_response_handlers.erase(id);
}
void storage_proxy::got_response(storage_proxy::storage_proxy::response_id_type id, gms::inet_address from) {
auto it = _response_handlers.find(id);
if (it != _response_handlers.end()) {
if (it->second.handler->response(from)) {
remove_response_handler(id); // last one, remove entry. Will cancel expiration timer too.
}
}
}
future<> storage_proxy::response_wait(storage_proxy::response_id_type id) {
auto& e = _response_handlers.find(id)->second;
e.expire_timer.arm(std::chrono::milliseconds(_db.local().get_config().write_request_timeout_in_ms()));
return e.handler->wait();
}
abstract_write_response_handler& storage_proxy::get_write_response_handler(storage_proxy::response_id_type id) {
return *_response_handlers.find(id)->second.handler;
}
storage_proxy::response_id_type storage_proxy::create_write_response_handler(keyspace& ks, db::consistency_level cl, frozen_mutation&& mutation, std::unordered_set<gms::inet_address> targets, std::vector<gms::inet_address>& pending_endpoints)
{
std::unique_ptr<abstract_write_response_handler> h;
auto& rs = ks.get_replication_strategy();
size_t pending_count = pending_endpoints.size();
auto m = make_lw_shared<const frozen_mutation>(std::move(mutation));
// for now make is simple
if (db::is_datacenter_local(cl)) {
pending_count = std::count_if(pending_endpoints.begin(), pending_endpoints.end(), db::is_local);
h = std::make_unique<datacenter_write_response_handler>(ks, cl, std::move(m), std::move(targets), pending_count);
} else if (cl == db::consistency_level::EACH_QUORUM &&
rs.get_type() == locator::replication_strategy_type::network_topology){
h = std::make_unique<datacenter_sync_write_response_handler>(ks, cl, std::move(m), std::move(targets), pending_count);
} else {
h = std::make_unique<write_response_handler>(ks, cl, std::move(m), std::move(targets), pending_count);
}
return register_response_handler(std::move(h));
}
storage_proxy::~storage_proxy() {}
storage_proxy::storage_proxy(distributed<database>& db) : _db(db) {
init_messaging_service();
}
storage_proxy::rh_entry::rh_entry(std::unique_ptr<abstract_write_response_handler>&& h, std::function<void()>&& cb) : handler(std::move(h)), expire_timer(std::move(cb)) {}
#if 0
public static final String MBEAN_NAME = "org.apache.cassandra.db:type=StorageProxy";
private static final Logger logger = LoggerFactory.getLogger(StorageProxy.class);
static final boolean OPTIMIZE_LOCAL_REQUESTS = true; // set to false to test messagingservice path on single node
public static final String UNREACHABLE = "UNREACHABLE";
private static final WritePerformer standardWritePerformer;
private static final WritePerformer counterWritePerformer;
private static final WritePerformer counterWriteOnCoordinatorPerformer;
public static final StorageProxy instance = new StorageProxy();
private static volatile int maxHintsInProgress = 128 * FBUtilities.getAvailableProcessors();
private static final CacheLoader<InetAddress, AtomicInteger> hintsInProgress = new CacheLoader<InetAddress, AtomicInteger>()
{
public AtomicInteger load(InetAddress inetAddress)
{
return new AtomicInteger(0);
}
};
private static final ClientRequestMetrics readMetrics = new ClientRequestMetrics("Read");
private static final ClientRequestMetrics rangeMetrics = new ClientRequestMetrics("RangeSlice");
private static final ClientRequestMetrics writeMetrics = new ClientRequestMetrics("Write");
private static final CASClientRequestMetrics casWriteMetrics = new CASClientRequestMetrics("CASWrite");
private static final CASClientRequestMetrics casReadMetrics = new CASClientRequestMetrics("CASRead");
private static final double CONCURRENT_SUBREQUESTS_MARGIN = 0.10;
private StorageProxy() {}
static
{
MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
try
{
mbs.registerMBean(instance, new ObjectName(MBEAN_NAME));
}
catch (Exception e)
{
throw new RuntimeException(e);
}
standardWritePerformer = new WritePerformer()
{
public void apply(IMutation mutation,
Iterable<InetAddress> targets,
AbstractWriteResponseHandler responseHandler,
String localDataCenter,
ConsistencyLevel consistency_level)
throws OverloadedException
{
assert mutation instanceof Mutation;
sendToHintedEndpoints((Mutation) mutation, targets, responseHandler, localDataCenter);
}
};
/*
* We execute counter writes in 2 places: either directly in the coordinator node if it is a replica, or
* in CounterMutationVerbHandler on a replica othewise. The write must be executed on the COUNTER_MUTATION stage
* but on the latter case, the verb handler already run on the COUNTER_MUTATION stage, so we must not execute the
* underlying on the stage otherwise we risk a deadlock. Hence two different performer.
*/
counterWritePerformer = new WritePerformer()
{
public void apply(IMutation mutation,
Iterable<InetAddress> targets,
AbstractWriteResponseHandler responseHandler,
String localDataCenter,
ConsistencyLevel consistencyLevel)
{
counterWriteTask(mutation, targets, responseHandler, localDataCenter).run();
}
};
counterWriteOnCoordinatorPerformer = new WritePerformer()
{
public void apply(IMutation mutation,
Iterable<InetAddress> targets,
AbstractWriteResponseHandler responseHandler,
String localDataCenter,
ConsistencyLevel consistencyLevel)
{
StageManager.getStage(Stage.COUNTER_MUTATION)
.execute(counterWriteTask(mutation, targets, responseHandler, localDataCenter));
}
};
}
/**
* Apply @param updates if and only if the current values in the row for @param key
* match the provided @param conditions. The algorithm is "raw" Paxos: that is, Paxos
* minus leader election -- any node in the cluster may propose changes for any row,
* which (that is, the row) is the unit of values being proposed, not single columns.
*
* The Paxos cohort is only the replicas for the given key, not the entire cluster.
* So we expect performance to be reasonable, but CAS is still intended to be used
* "when you really need it," not for all your updates.
*
* There are three phases to Paxos:
* 1. Prepare: the coordinator generates a ballot (timeUUID in our case) and asks replicas to (a) promise
* not to accept updates from older ballots and (b) tell us about the most recent update it has already
* accepted.
* 2. Accept: if a majority of replicas reply, the coordinator asks replicas to accept the value of the
* highest proposal ballot it heard about, or a new value if no in-progress proposals were reported.
* 3. Commit (Learn): if a majority of replicas acknowledge the accept request, we can commit the new
* value.
*
* Commit procedure is not covered in "Paxos Made Simple," and only briefly mentioned in "Paxos Made Live,"
* so here is our approach:
* 3a. The coordinator sends a commit message to all replicas with the ballot and value.
* 3b. Because of 1-2, this will be the highest-seen commit ballot. The replicas will note that,
* and send it with subsequent promise replies. This allows us to discard acceptance records
* for successfully committed replicas, without allowing incomplete proposals to commit erroneously
* later on.
*
* Note that since we are performing a CAS rather than a simple update, we perform a read (of committed
* values) between the prepare and accept phases. This gives us a slightly longer window for another
* coordinator to come along and trump our own promise with a newer one but is otherwise safe.
*
* @param keyspaceName the keyspace for the CAS
* @param cfName the column family for the CAS
* @param key the row key for the row to CAS
* @param request the conditions for the CAS to apply as well as the update to perform if the conditions hold.
* @param consistencyForPaxos the consistency for the paxos prepare and propose round. This can only be either SERIAL or LOCAL_SERIAL.
* @param consistencyForCommit the consistency for write done during the commit phase. This can be anything, except SERIAL or LOCAL_SERIAL.
*
* @return null if the operation succeeds in updating the row, or the current values corresponding to conditions.
* (since, if the CAS doesn't succeed, it means the current value do not match the conditions).
*/
public static ColumnFamily cas(String keyspaceName,
String cfName,
ByteBuffer key,
CASRequest request,
ConsistencyLevel consistencyForPaxos,
ConsistencyLevel consistencyForCommit,
ClientState state)
throws UnavailableException, IsBootstrappingException, ReadTimeoutException, WriteTimeoutException, InvalidRequestException
{
final long start = System.nanoTime();
int contentions = 0;
try
{
consistencyForPaxos.validateForCas();
consistencyForCommit.validateForCasCommit(keyspaceName);
CFMetaData metadata = Schema.instance.getCFMetaData(keyspaceName, cfName);
long timeout = TimeUnit.MILLISECONDS.toNanos(DatabaseDescriptor.getCasContentionTimeout());
while (System.nanoTime() - start < timeout)
{
// for simplicity, we'll do a single liveness check at the start of each attempt
Pair<List<InetAddress>, Integer> p = getPaxosParticipants(keyspaceName, key, consistencyForPaxos);
List<InetAddress> liveEndpoints = p.left;
int requiredParticipants = p.right;
final Pair<UUID, Integer> pair = beginAndRepairPaxos(start, key, metadata, liveEndpoints, requiredParticipants, consistencyForPaxos, consistencyForCommit, true, state);
final UUID ballot = pair.left;
contentions += pair.right;
// read the current values and check they validate the conditions
Tracing.trace("Reading existing values for CAS precondition");
long timestamp = System.currentTimeMillis();
ReadCommand readCommand = ReadCommand.create(keyspaceName, key, cfName, timestamp, request.readFilter());
List<Row> rows = read(Arrays.asList(readCommand), consistencyForPaxos == ConsistencyLevel.LOCAL_SERIAL ? ConsistencyLevel.LOCAL_QUORUM : ConsistencyLevel.QUORUM);
ColumnFamily current = rows.get(0).cf;
if (!request.appliesTo(current))
{
Tracing.trace("CAS precondition does not match current values {}", current);
// We should not return null as this means success
casWriteMetrics.conditionNotMet.inc();
return current == null ? ArrayBackedSortedColumns.factory.create(metadata) : current;
}
// finish the paxos round w/ the desired updates
// TODO turn null updates into delete?
ColumnFamily updates = request.makeUpdates(current);
// Apply triggers to cas updates. A consideration here is that
// triggers emit Mutations, and so a given trigger implementation
// may generate mutations for partitions other than the one this
// paxos round is scoped for. In this case, TriggerExecutor will
// validate that the generated mutations are targetted at the same
// partition as the initial updates and reject (via an
// InvalidRequestException) any which aren't.
updates = TriggerExecutor.instance.execute(key, updates);
Commit proposal = Commit.newProposal(key, ballot, updates);
Tracing.trace("CAS precondition is met; proposing client-requested updates for {}", ballot);
if (proposePaxos(proposal, liveEndpoints, requiredParticipants, true, consistencyForPaxos))
{
commitPaxos(proposal, consistencyForCommit);
Tracing.trace("CAS successful");
return null;
}
Tracing.trace("Paxos proposal not accepted (pre-empted by a higher ballot)");
contentions++;
Uninterruptibles.sleepUninterruptibly(ThreadLocalRandom.current().nextInt(100), TimeUnit.MILLISECONDS);
// continue to retry
}
throw new WriteTimeoutException(WriteType.CAS, consistencyForPaxos, 0, consistencyForPaxos.blockFor(Keyspace.open(keyspaceName)));
}
catch (WriteTimeoutException|ReadTimeoutException e)
{
casWriteMetrics.timeouts.mark();
throw e;
}
catch(UnavailableException e)
{
casWriteMetrics.unavailables.mark();
throw e;
}
finally
{
if(contentions > 0)
casWriteMetrics.contention.update(contentions);
casWriteMetrics.addNano(System.nanoTime() - start);
}
}
private static Predicate<InetAddress> sameDCPredicateFor(final String dc)
{
final IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
return new Predicate<InetAddress>()
{
public boolean apply(InetAddress host)
{
return dc.equals(snitch.getDatacenter(host));
}
};
}
private static Pair<List<InetAddress>, Integer> getPaxosParticipants(String keyspaceName, ByteBuffer key, ConsistencyLevel consistencyForPaxos) throws UnavailableException
{
Token tk = StorageService.getPartitioner().getToken(key);
List<InetAddress> naturalEndpoints = StorageService.instance.getNaturalEndpoints(keyspaceName, tk);
Collection<InetAddress> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, keyspaceName);
if (consistencyForPaxos == ConsistencyLevel.LOCAL_SERIAL)
{
// Restrict naturalEndpoints and pendingEndpoints to node in the local DC only
String localDc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddress());
Predicate<InetAddress> isLocalDc = sameDCPredicateFor(localDc);
naturalEndpoints = ImmutableList.copyOf(Iterables.filter(naturalEndpoints, isLocalDc));
pendingEndpoints = ImmutableList.copyOf(Iterables.filter(pendingEndpoints, isLocalDc));
}
int participants = pendingEndpoints.size() + naturalEndpoints.size();
int requiredParticipants = participants + 1 / 2; // See CASSANDRA-833
List<InetAddress> liveEndpoints = ImmutableList.copyOf(Iterables.filter(Iterables.concat(naturalEndpoints, pendingEndpoints), IAsyncCallback.isAlive));
if (liveEndpoints.size() < requiredParticipants)
throw new UnavailableException(consistencyForPaxos, requiredParticipants, liveEndpoints.size());
// We cannot allow CAS operations with 2 or more pending endpoints, see #8346.
// Note that we fake an impossible number of required nodes in the unavailable exception
// to nail home the point that it's an impossible operation no matter how many nodes are live.
if (pendingEndpoints.size() > 1)
throw new UnavailableException(String.format("Cannot perform LWT operation as there is more than one (%d) pending range movement", pendingEndpoints.size()),
consistencyForPaxos,
participants + 1,
liveEndpoints.size());
return Pair.create(liveEndpoints, requiredParticipants);
}
/**
* begin a Paxos session by sending a prepare request and completing any in-progress requests seen in the replies
*
* @return the Paxos ballot promised by the replicas if no in-progress requests were seen and a quorum of
* nodes have seen the mostRecentCommit. Otherwise, return null.
*/
private static Pair<UUID, Integer> beginAndRepairPaxos(long start,
ByteBuffer key,
CFMetaData metadata,
List<InetAddress> liveEndpoints,
int requiredParticipants,
ConsistencyLevel consistencyForPaxos,
ConsistencyLevel consistencyForCommit,
final boolean isWrite,
ClientState state)
throws WriteTimeoutException
{
long timeout = TimeUnit.MILLISECONDS.toNanos(DatabaseDescriptor.getCasContentionTimeout());
PrepareCallback summary = null;
int contentions = 0;
while (System.nanoTime() - start < timeout)
{
// We don't want to use a timestamp that is older than the last one assigned by the ClientState or operations
// may appear out-of-order (#7801). But note that state.getTimestamp() is in microseconds while the ballot
// timestamp is only in milliseconds
long currentTime = (state.getTimestamp() / 1000) + 1;
long ballotMillis = summary == null
? currentTime
: Math.max(currentTime, 1 + UUIDGen.unixTimestamp(summary.mostRecentInProgressCommit.ballot));
UUID ballot = UUIDGen.getTimeUUID(ballotMillis);
// prepare
Tracing.trace("Preparing {}", ballot);
Commit toPrepare = Commit.newPrepare(key, metadata, ballot);
summary = preparePaxos(toPrepare, liveEndpoints, requiredParticipants, consistencyForPaxos);
if (!summary.promised)
{
Tracing.trace("Some replicas have already promised a higher ballot than ours; aborting");
contentions++;
// sleep a random amount to give the other proposer a chance to finish
Uninterruptibles.sleepUninterruptibly(ThreadLocalRandom.current().nextInt(100), TimeUnit.MILLISECONDS);
continue;
}
Commit inProgress = summary.mostRecentInProgressCommitWithUpdate;
Commit mostRecent = summary.mostRecentCommit;
// If we have an in-progress ballot greater than the MRC we know, then it's an in-progress round that
// needs to be completed, so do it.
if (!inProgress.update.isEmpty() && inProgress.isAfter(mostRecent))
{
Tracing.trace("Finishing incomplete paxos round {}", inProgress);
if(isWrite)
casWriteMetrics.unfinishedCommit.inc();
else
casReadMetrics.unfinishedCommit.inc();
Commit refreshedInProgress = Commit.newProposal(inProgress.key, ballot, inProgress.update);
if (proposePaxos(refreshedInProgress, liveEndpoints, requiredParticipants, false, consistencyForPaxos))
{
commitPaxos(refreshedInProgress, consistencyForCommit);
}
else
{
Tracing.trace("Some replicas have already promised a higher ballot than ours; aborting");
// sleep a random amount to give the other proposer a chance to finish
contentions++;
Uninterruptibles.sleepUninterruptibly(ThreadLocalRandom.current().nextInt(100), TimeUnit.MILLISECONDS);
}
continue;
}
// To be able to propose our value on a new round, we need a quorum of replica to have learn the previous one. Why is explained at:
// https://issues.apache.org/jira/browse/CASSANDRA-5062?focusedCommentId=13619810&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13619810)
// Since we waited for quorum nodes, if some of them haven't seen the last commit (which may just be a timing issue, but may also
// mean we lost messages), we pro-actively "repair" those nodes, and retry.
Iterable<InetAddress> missingMRC = summary.replicasMissingMostRecentCommit();
if (Iterables.size(missingMRC) > 0)
{
Tracing.trace("Repairing replicas that missed the most recent commit");
sendCommit(mostRecent, missingMRC);
// TODO: provided commits don't invalid the prepare we just did above (which they don't), we could just wait
// for all the missingMRC to acknowledge this commit and then move on with proposing our value. But that means
// adding the ability to have commitPaxos block, which is exactly CASSANDRA-5442 will do. So once we have that
// latter ticket, we can pass CL.ALL to the commit above and remove the 'continue'.
continue;
}
// We might commit this ballot and we want to ensure operations starting after this CAS succeed will be assigned
// a timestamp greater that the one of this ballot, so operation order is preserved (#7801)
state.updateLastTimestamp(ballotMillis * 1000);
return Pair.create(ballot, contentions);
}
throw new WriteTimeoutException(WriteType.CAS, consistencyForPaxos, 0, consistencyForPaxos.blockFor(Keyspace.open(metadata.ksName)));
}
/**
* Unlike commitPaxos, this does not wait for replies
*/
private static void sendCommit(Commit commit, Iterable<InetAddress> replicas)
{
MessageOut<Commit> message = new MessageOut<Commit>(MessagingService.Verb.PAXOS_COMMIT, commit, Commit.serializer);
for (InetAddress target : replicas)
MessagingService.instance().sendOneWay(message, target);
}
private static PrepareCallback preparePaxos(Commit toPrepare, List<InetAddress> endpoints, int requiredParticipants, ConsistencyLevel consistencyForPaxos)
throws WriteTimeoutException
{
PrepareCallback callback = new PrepareCallback(toPrepare.key, toPrepare.update.metadata(), requiredParticipants, consistencyForPaxos);
MessageOut<Commit> message = new MessageOut<Commit>(MessagingService.Verb.PAXOS_PREPARE, toPrepare, Commit.serializer);
for (InetAddress target : endpoints)
MessagingService.instance().sendRR(message, target, callback);
callback.await();
return callback;
}
private static boolean proposePaxos(Commit proposal, List<InetAddress> endpoints, int requiredParticipants, boolean timeoutIfPartial, ConsistencyLevel consistencyLevel)
throws WriteTimeoutException
{
ProposeCallback callback = new ProposeCallback(endpoints.size(), requiredParticipants, !timeoutIfPartial, consistencyLevel);
MessageOut<Commit> message = new MessageOut<Commit>(MessagingService.Verb.PAXOS_PROPOSE, proposal, Commit.serializer);
for (InetAddress target : endpoints)
MessagingService.instance().sendRR(message, target, callback);
callback.await();
if (callback.isSuccessful())
return true;
if (timeoutIfPartial && !callback.isFullyRefused())
throw new WriteTimeoutException(WriteType.CAS, consistencyLevel, callback.getAcceptCount(), requiredParticipants);
return false;
}
private static void commitPaxos(Commit proposal, ConsistencyLevel consistencyLevel) throws WriteTimeoutException
{
boolean shouldBlock = consistencyLevel != ConsistencyLevel.ANY;
Keyspace keyspace = Keyspace.open(proposal.update.metadata().ksName);
Token tk = StorageService.getPartitioner().getToken(proposal.key);
List<InetAddress> naturalEndpoints = StorageService.instance.getNaturalEndpoints(keyspace.getName(), tk);
Collection<InetAddress> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, keyspace.getName());
AbstractWriteResponseHandler responseHandler = null;
if (shouldBlock)
{
AbstractReplicationStrategy rs = keyspace.getReplicationStrategy();
responseHandler = rs.getWriteResponseHandler(naturalEndpoints, pendingEndpoints, consistencyLevel, null, WriteType.SIMPLE);
}
MessageOut<Commit> message = new MessageOut<Commit>(MessagingService.Verb.PAXOS_COMMIT, proposal, Commit.serializer);
for (InetAddress destination : Iterables.concat(naturalEndpoints, pendingEndpoints))
{
if (FailureDetector.instance.isAlive(destination))
{
if (shouldBlock)
MessagingService.instance().sendRR(message, destination, responseHandler);
else
MessagingService.instance().sendOneWay(message, destination);
}
}
if (shouldBlock)
responseHandler.get();
}
#endif
future<>
storage_proxy::mutate_locally(const mutation& m) {
auto shard = _db.local().shard_of(m);
return _db.invoke_on(shard, [m = freeze(m)] (database& db) -> future<> {
return db.apply(m);
});
}
future<>
storage_proxy::mutate_locally(const frozen_mutation& m) {
auto shard = _db.local().shard_of(m);
return _db.invoke_on(shard, [&m] (database& db) -> future<> {
return db.apply(m);
});
}
future<>
storage_proxy::mutate_locally(std::vector<mutation> mutations) {
auto pmut = make_lw_shared(std::move(mutations));
return parallel_for_each(pmut->begin(), pmut->end(), [this, pmut] (const mutation& m) {
return mutate_locally(m);
}).finally([pmut]{});
}
/**
* Use this method to have these Mutations applied
* across all replicas. This method will take care
* of the possibility of a replica being down and hint
* the data across to some other replica.
*
* @param mutations the mutations to be applied across the replicas
* @param consistency_level the consistency level for the operation
*/
future<>
storage_proxy::mutate(std::vector<mutation> mutations, db::consistency_level cl) {
auto have_cl = make_lw_shared<semaphore>(0);
auto local_addr = utils::fb_utilities::get_broadcast_address();
auto& snitch_ptr = locator::i_endpoint_snitch::get_local_snitch_ptr();
sstring local_dc = snitch_ptr->get_datacenter(local_addr);
for (auto& m : mutations) {
try {
keyspace& ks = _db.local().find_keyspace(m.schema()->ks_name());
auto& rs = ks.get_replication_strategy();
std::vector<gms::inet_address> natural_endpoints = rs.get_natural_endpoints(m.token());
std::vector<gms::inet_address> pending_endpoints = get_local_storage_service().get_token_metadata().pending_endpoints_for(m.token(), ks);
auto all = boost::range::join(natural_endpoints, pending_endpoints);
if (std::find_if(all.begin(), all.end(), std::bind1st(std::mem_fn(&storage_proxy::cannot_hint), this)) != all.end()) {
// avoid OOMing due to excess hints. we need to do this check even for "live" nodes, since we can
// still generate hints for those if it's overloaded or simply dead but not yet known-to-be-dead.
// The idea is that if we have over maxHintsInProgress hints in flight, this is probably due to
// a small number of nodes causing problems, so we should avoid shutting down writes completely to
// healthy nodes. Any node with no hintsInProgress is considered healthy.
throw overloaded_exception(_total_hints_in_progress);
}
// filter live endpoints from dead ones
std::unordered_set<gms::inet_address> live_endpoints;
std::vector<gms::inet_address> dead_endpoints;
live_endpoints.reserve(all.size());
dead_endpoints.reserve(all.size());
std::partition_copy(all.begin(), all.end(), std::inserter(live_endpoints, live_endpoints.begin()), std::back_inserter(dead_endpoints),
std::bind1st(std::mem_fn(&gms::failure_detector::is_alive), &gms::get_local_failure_detector()));
db::assure_sufficient_live_nodes(cl, ks, live_endpoints);
storage_proxy::response_id_type response_id = create_write_response_handler(ks, cl, freeze(m), std::move(live_endpoints), pending_endpoints);
// it is better to send first and hint afterwards to reduce latency
// but request may complete before hint_to_dead_endpoints() is called and
// response_id handler will be removed, so we will have to do hint with separate
// frozen_mutation copy, or manage handler live time differently.
size_t hints = hint_to_dead_endpoints(get_write_response_handler(response_id).get_mutation(), dead_endpoints);
if (cl == db::consistency_level::ANY) {
// for cl==ANY hints are counted towards consistency
get_write_response_handler(response_id).signal(hints);
}
// call before send_to_live_endpoints() for the same reason as above
auto f = response_wait(response_id);
send_to_live_endpoints(response_id, local_dc);
f.then_wrapped([this, have_cl, response_id, cl] (future<>&& f) mutable {
try {
f.get();
have_cl->signal();
return;
} catch(mutation_write_timeout_error& ex) {
// timeout
// writeMetrics.timeouts.mark();
// ClientRequestMetrics.writeTimeouts.inc();
// Tracing.trace("Write timeout; received {} of {} required replies", ex.received, ex.blockFor);
have_cl->broken(ex);
} catch(...) {
have_cl->broken(std::current_exception());
}
remove_response_handler(response_id); // cancel expire_timer, so no hint will happen
});
} catch (no_such_keyspace& ex) {
return make_exception_future<>(std::current_exception());
} catch(db::unavailable_exception& ex) {
// writeMetrics.unavailables.mark();
// ClientRequestMetrics.writeUnavailables.inc();
// Tracing.trace("Unavailable");
return make_exception_future<>(std::current_exception());
} catch(overloaded_exception& ex) {
// ClientRequestMetrics.writeUnavailables.inc();
// Tracing.trace("Overloaded");
return make_exception_future<>(std::current_exception());
}
}
return have_cl->wait(mutations.size());
}
future<>
storage_proxy::mutate_with_triggers(std::vector<mutation> mutations, db::consistency_level cl,
bool should_mutate_atomically) {
warn(unimplemented::cause::TRIGGERS);
#if 0
Collection<Mutation> augmented = TriggerExecutor.instance.execute(mutations);
if (augmented != null) {
return mutate_atomically(augmented, consistencyLevel);
} else {
#endif
if (should_mutate_atomically) {
return mutate_atomically(std::move(mutations), cl);
}
return mutate(std::move(mutations), cl);
#if 0
}
#endif
}
/**
* See mutate. Adds additional steps before and after writing a batch.
* Before writing the batch (but after doing availability check against the FD for the row replicas):
* write the entire batch to a batchlog elsewhere in the cluster.
* After: remove the batchlog entry (after writing hints for the batch rows, if necessary).
*
* @param mutations the Mutations to be applied across the replicas
* @param consistency_level the consistency level for the operation
*/
future<>
storage_proxy::mutate_atomically(std::vector<mutation> mutations, db::consistency_level cl) {
fail(unimplemented::cause::LWT);
#if 0
Tracing.trace("Determining replicas for atomic batch");
long startTime = System.nanoTime();
List<WriteResponseHandlerWrapper> wrappers = new ArrayList<WriteResponseHandlerWrapper>(mutations.size());
String localDataCenter = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddress());
try
{
// add a handler for each mutation - includes checking availability, but doesn't initiate any writes, yet
for (Mutation mutation : mutations)
{
WriteResponseHandlerWrapper wrapper = wrapResponseHandler(mutation, consistency_level, WriteType.BATCH);
// exit early if we can't fulfill the CL at this time.
wrapper.handler.assureSufficientLiveNodes();
wrappers.add(wrapper);
}
// write to the batchlog
Collection<InetAddress> batchlogEndpoints = getBatchlogEndpoints(localDataCenter, consistency_level);
UUID batchUUID = UUIDGen.getTimeUUID();
syncWriteToBatchlog(mutations, batchlogEndpoints, batchUUID);
// now actually perform the writes and wait for them to complete
syncWriteBatchedMutations(wrappers, localDataCenter);
// remove the batchlog entries asynchronously
asyncRemoveFromBatchlog(batchlogEndpoints, batchUUID);
}
catch (UnavailableException e)
{
writeMetrics.unavailables.mark();
ClientRequestMetrics.writeUnavailables.inc();
Tracing.trace("Unavailable");
throw e;
}
catch (WriteTimeoutException e)
{
writeMetrics.timeouts.mark();
ClientRequestMetrics.writeTimeouts.inc();
Tracing.trace("Write timeout; received {} of {} required replies", e.received, e.blockFor);
throw e;
}
finally
{
writeMetrics.addNano(System.nanoTime() - startTime);
}
#endif
}
#if 0
private static void syncWriteToBatchlog(Collection<Mutation> mutations, Collection<InetAddress> endpoints, UUID uuid)
throws WriteTimeoutException
{
AbstractWriteResponseHandler handler = new WriteResponseHandler(endpoints,
Collections.<InetAddress>emptyList(),
ConsistencyLevel.ONE,
Keyspace.open(SystemKeyspace.NAME),
null,
WriteType.BATCH_LOG);
MessageOut<Mutation> message = BatchlogManager.getBatchlogMutationFor(mutations, uuid, MessagingService.current_version)
.createMessage();
for (InetAddress target : endpoints)
{
int targetVersion = MessagingService.instance().getVersion(target);
if (target.equals(FBUtilities.getBroadcastAddress()) && OPTIMIZE_LOCAL_REQUESTS)
{
insertLocal(message.payload, handler);
}
else if (targetVersion == MessagingService.current_version)
{
MessagingService.instance().sendRR(message, target, handler, false);
}
else
{
MessagingService.instance().sendRR(BatchlogManager.getBatchlogMutationFor(mutations, uuid, targetVersion)
.createMessage(),
target,
handler,
false);
}
}
handler.get();
}
private static void asyncRemoveFromBatchlog(Collection<InetAddress> endpoints, UUID uuid)
{
AbstractWriteResponseHandler handler = new WriteResponseHandler(endpoints,
Collections.<InetAddress>emptyList(),
ConsistencyLevel.ANY,
Keyspace.open(SystemKeyspace.NAME),
null,
WriteType.SIMPLE);
Mutation mutation = new Mutation(SystemKeyspace.NAME, UUIDType.instance.decompose(uuid));
mutation.delete(SystemKeyspace.BATCHLOG, FBUtilities.timestampMicros());
MessageOut<Mutation> message = mutation.createMessage();
for (InetAddress target : endpoints)
{
if (target.equals(FBUtilities.getBroadcastAddress()) && OPTIMIZE_LOCAL_REQUESTS)
insertLocal(message.payload, handler);
else
MessagingService.instance().sendRR(message, target, handler, false);
}
}
private static void syncWriteBatchedMutations(List<WriteResponseHandlerWrapper> wrappers, String localDataCenter)
throws WriteTimeoutException, OverloadedException
{
for (WriteResponseHandlerWrapper wrapper : wrappers)
{
Iterable<InetAddress> endpoints = Iterables.concat(wrapper.handler.naturalEndpoints, wrapper.handler.pendingEndpoints);
sendToHintedEndpoints(wrapper.mutation, endpoints, wrapper.handler, localDataCenter);
}
for (WriteResponseHandlerWrapper wrapper : wrappers)
wrapper.handler.get();
}
/**
* Perform the write of a mutation given a WritePerformer.
* Gather the list of write endpoints, apply locally and/or forward the mutation to
* said write endpoint (deletaged to the actual WritePerformer) and wait for the
* responses based on consistency level.
*
* @param mutation the mutation to be applied
* @param consistency_level the consistency level for the write operation
* @param performer the WritePerformer in charge of appliying the mutation
* given the list of write endpoints (either standardWritePerformer for
* standard writes or counterWritePerformer for counter writes).
* @param callback an optional callback to be run if and when the write is
* successful.
*/
public static AbstractWriteResponseHandler performWrite(IMutation mutation,
ConsistencyLevel consistency_level,
String localDataCenter,
WritePerformer performer,
Runnable callback,
WriteType writeType)
throws UnavailableException, OverloadedException
{
String keyspaceName = mutation.getKeyspaceName();
AbstractReplicationStrategy rs = Keyspace.open(keyspaceName).getReplicationStrategy();
Token tk = StorageService.getPartitioner().getToken(mutation.key());
List<InetAddress> naturalEndpoints = StorageService.instance.getNaturalEndpoints(keyspaceName, tk);
Collection<InetAddress> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, keyspaceName);
AbstractWriteResponseHandler responseHandler = rs.getWriteResponseHandler(naturalEndpoints, pendingEndpoints, consistency_level, callback, writeType);
// exit early if we can't fulfill the CL at this time
responseHandler.assureSufficientLiveNodes();
performer.apply(mutation, Iterables.concat(naturalEndpoints, pendingEndpoints), responseHandler, localDataCenter, consistency_level);
return responseHandler;
}
// same as above except does not initiate writes (but does perform availability checks).
private static WriteResponseHandlerWrapper wrapResponseHandler(Mutation mutation, ConsistencyLevel consistency_level, WriteType writeType)
{
AbstractReplicationStrategy rs = Keyspace.open(mutation.getKeyspaceName()).getReplicationStrategy();
String keyspaceName = mutation.getKeyspaceName();
Token tk = StorageService.getPartitioner().getToken(mutation.key());
List<InetAddress> naturalEndpoints = StorageService.instance.getNaturalEndpoints(keyspaceName, tk);
Collection<InetAddress> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, keyspaceName);
AbstractWriteResponseHandler responseHandler = rs.getWriteResponseHandler(naturalEndpoints, pendingEndpoints, consistency_level, null, writeType);
return new WriteResponseHandlerWrapper(responseHandler, mutation);
}