-
Notifications
You must be signed in to change notification settings - Fork 3.6k
/
StorageProxy.java
3150 lines (2784 loc) · 140 KB
/
StorageProxy.java
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.
*/
package org.apache.cassandra.service;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.Future;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Function;
import java.util.stream.Collectors;
import com.google.common.base.Preconditions;
import com.google.common.cache.CacheLoader;
import com.google.common.collect.Iterables;
import com.google.common.util.concurrent.Uninterruptibles;
import org.apache.cassandra.service.paxos.Ballot;
import org.apache.cassandra.service.paxos.Commit;
import org.apache.cassandra.service.paxos.ContentionStrategy;
import org.apache.cassandra.service.paxos.Paxos;
import org.apache.cassandra.service.paxos.PaxosState;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.cassandra.batchlog.Batch;
import org.apache.cassandra.batchlog.BatchlogManager;
import org.apache.cassandra.concurrent.DebuggableTask.RunnableDebuggableTask;
import org.apache.cassandra.concurrent.Stage;
import org.apache.cassandra.config.CassandraRelevantProperties;
import org.apache.cassandra.config.Config;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.ConsistencyLevel;
import org.apache.cassandra.db.CounterMutation;
import org.apache.cassandra.db.DecoratedKey;
import org.apache.cassandra.db.IMutation;
import org.apache.cassandra.db.Keyspace;
import org.apache.cassandra.db.MessageParams;
import org.apache.cassandra.db.Mutation;
import org.apache.cassandra.db.PartitionRangeReadCommand;
import org.apache.cassandra.db.ReadCommand;
import org.apache.cassandra.db.ReadExecutionController;
import org.apache.cassandra.db.ReadResponse;
import org.apache.cassandra.db.RejectException;
import org.apache.cassandra.db.SinglePartitionReadCommand;
import org.apache.cassandra.db.TruncateRequest;
import org.apache.cassandra.db.WriteType;
import org.apache.cassandra.db.filter.TombstoneOverwhelmingException;
import org.apache.cassandra.db.partitions.FilteredPartition;
import org.apache.cassandra.db.partitions.PartitionIterator;
import org.apache.cassandra.db.partitions.PartitionIterators;
import org.apache.cassandra.db.partitions.PartitionUpdate;
import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
import org.apache.cassandra.db.rows.RowIterator;
import org.apache.cassandra.db.view.ViewUtils;
import org.apache.cassandra.dht.Token;
import org.apache.cassandra.exceptions.CasWriteTimeoutException;
import org.apache.cassandra.exceptions.CasWriteUnknownResultException;
import org.apache.cassandra.exceptions.InvalidRequestException;
import org.apache.cassandra.exceptions.IsBootstrappingException;
import org.apache.cassandra.exceptions.OverloadedException;
import org.apache.cassandra.exceptions.ReadAbortException;
import org.apache.cassandra.exceptions.ReadFailureException;
import org.apache.cassandra.exceptions.ReadTimeoutException;
import org.apache.cassandra.exceptions.RequestFailureException;
import org.apache.cassandra.exceptions.RequestFailureReason;
import org.apache.cassandra.exceptions.RequestTimeoutException;
import org.apache.cassandra.exceptions.UnavailableException;
import org.apache.cassandra.exceptions.WriteFailureException;
import org.apache.cassandra.exceptions.WriteTimeoutException;
import org.apache.cassandra.gms.FailureDetector;
import org.apache.cassandra.gms.Gossiper;
import org.apache.cassandra.hints.Hint;
import org.apache.cassandra.hints.HintsService;
import org.apache.cassandra.locator.AbstractReplicationStrategy;
import org.apache.cassandra.locator.EndpointsForToken;
import org.apache.cassandra.locator.IEndpointSnitch;
import org.apache.cassandra.locator.InetAddressAndPort;
import org.apache.cassandra.locator.Replica;
import org.apache.cassandra.locator.ReplicaLayout;
import org.apache.cassandra.locator.ReplicaPlan;
import org.apache.cassandra.locator.ReplicaPlans;
import org.apache.cassandra.locator.Replicas;
import org.apache.cassandra.metrics.CASClientRequestMetrics;
import org.apache.cassandra.metrics.DenylistMetrics;
import org.apache.cassandra.metrics.ReadRepairMetrics;
import org.apache.cassandra.metrics.StorageMetrics;
import org.apache.cassandra.net.ForwardingInfo;
import org.apache.cassandra.net.Message;
import org.apache.cassandra.net.MessageFlag;
import org.apache.cassandra.net.MessagingService;
import org.apache.cassandra.net.RequestCallback;
import org.apache.cassandra.net.Verb;
import org.apache.cassandra.schema.PartitionDenylist;
import org.apache.cassandra.schema.Schema;
import org.apache.cassandra.schema.SchemaConstants;
import org.apache.cassandra.schema.TableId;
import org.apache.cassandra.schema.TableMetadata;
import org.apache.cassandra.service.paxos.v1.PrepareCallback;
import org.apache.cassandra.service.paxos.v1.ProposeCallback;
import org.apache.cassandra.service.reads.AbstractReadExecutor;
import org.apache.cassandra.service.reads.ReadCallback;
import org.apache.cassandra.service.reads.range.RangeCommands;
import org.apache.cassandra.service.reads.repair.ReadRepair;
import org.apache.cassandra.tracing.Tracing;
import org.apache.cassandra.triggers.TriggerExecutor;
import org.apache.cassandra.utils.Clock;
import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.MBeanWrapper;
import org.apache.cassandra.utils.MonotonicClock;
import org.apache.cassandra.utils.NoSpamLogger;
import org.apache.cassandra.utils.Pair;
import org.apache.cassandra.utils.TimeUUID;
import org.apache.cassandra.utils.concurrent.CountDownLatch;
import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static java.util.concurrent.TimeUnit.NANOSECONDS;
import static com.google.common.collect.Iterables.concat;
import static org.apache.commons.lang3.StringUtils.join;
import static org.apache.cassandra.db.ConsistencyLevel.SERIAL;
import static org.apache.cassandra.metrics.ClientRequestsMetricsHolder.casReadMetrics;
import static org.apache.cassandra.metrics.ClientRequestsMetricsHolder.casWriteMetrics;
import static org.apache.cassandra.metrics.ClientRequestsMetricsHolder.readMetrics;
import static org.apache.cassandra.metrics.ClientRequestsMetricsHolder.readMetricsForLevel;
import static org.apache.cassandra.metrics.ClientRequestsMetricsHolder.viewWriteMetrics;
import static org.apache.cassandra.metrics.ClientRequestsMetricsHolder.writeMetrics;
import static org.apache.cassandra.metrics.ClientRequestsMetricsHolder.writeMetricsForLevel;
import static org.apache.cassandra.net.Message.out;
import static org.apache.cassandra.net.NoPayload.noPayload;
import static org.apache.cassandra.net.Verb.BATCH_STORE_REQ;
import static org.apache.cassandra.net.Verb.MUTATION_REQ;
import static org.apache.cassandra.net.Verb.PAXOS_COMMIT_REQ;
import static org.apache.cassandra.net.Verb.PAXOS_PREPARE_REQ;
import static org.apache.cassandra.net.Verb.PAXOS_PROPOSE_REQ;
import static org.apache.cassandra.net.Verb.SCHEMA_VERSION_REQ;
import static org.apache.cassandra.net.Verb.TRUNCATE_REQ;
import static org.apache.cassandra.service.BatchlogResponseHandler.BatchlogCleanup;
import static org.apache.cassandra.service.paxos.Ballot.Flag.GLOBAL;
import static org.apache.cassandra.service.paxos.Ballot.Flag.LOCAL;
import static org.apache.cassandra.service.paxos.BallotGenerator.Global.nextBallot;
import static org.apache.cassandra.service.paxos.v1.PrepareVerbHandler.doPrepare;
import static org.apache.cassandra.service.paxos.v1.ProposeVerbHandler.doPropose;
import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis;
import static org.apache.cassandra.utils.Clock.Global.nanoTime;
import static org.apache.cassandra.utils.TimeUUID.Generator.nextTimeUUID;
import static org.apache.cassandra.utils.concurrent.CountDownLatch.newCountDownLatch;
public class StorageProxy implements StorageProxyMBean
{
public static final String MBEAN_NAME = "org.apache.cassandra.db:type=StorageProxy";
private static final Logger logger = LoggerFactory.getLogger(StorageProxy.class);
public static final String UNREACHABLE = "UNREACHABLE";
private static final int FAILURE_LOGGING_INTERVAL_SECONDS = CassandraRelevantProperties.FAILURE_LOGGING_INTERVAL_SECONDS.getInt();
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<InetAddressAndPort, AtomicInteger> hintsInProgress = new CacheLoader<InetAddressAndPort, AtomicInteger>()
{
public AtomicInteger load(InetAddressAndPort inetAddress)
{
return new AtomicInteger(0);
}
};
private static final DenylistMetrics denylistMetrics = new DenylistMetrics();
private static final PartitionDenylist partitionDenylist = new PartitionDenylist();
private volatile long logBlockingReadRepairAttemptsUntilNanos = Long.MIN_VALUE;
private StorageProxy()
{
}
static
{
MBeanWrapper.instance.registerMBean(instance, MBEAN_NAME);
HintsService.instance.registerMBean();
standardWritePerformer = (mutation, targets, responseHandler, localDataCenter) ->
{
assert mutation instanceof Mutation;
sendToHintedReplicas((Mutation) mutation, targets, responseHandler, localDataCenter, Stage.MUTATION);
};
/*
* 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 = (mutation, targets, responseHandler, localDataCenter) ->
{
EndpointsForToken selected = targets.contacts().withoutSelf();
Replicas.temporaryAssertFull(selected); // TODO CASSANDRA-14548
counterWriteTask(mutation, targets.withContacts(selected), responseHandler, localDataCenter).run();
};
counterWriteOnCoordinatorPerformer = (mutation, targets, responseHandler, localDataCenter) ->
{
EndpointsForToken selected = targets.contacts().withoutSelf();
Replicas.temporaryAssertFull(selected); // TODO CASSANDRA-14548
Stage.COUNTER_MUTATION.executor()
.execute(counterWriteTask(mutation, targets.withContacts(selected), responseHandler, localDataCenter));
};
ReadRepairMetrics.init();
if (!Paxos.isLinearizable())
{
logger.warn("This node was started with paxos variant {}. SERIAL (and LOCAL_SERIAL) reads coordinated by this node " +
"will not offer linearizability (see CASSANDRA-12126 for details on what this means) with " +
"respect to other SERIAL operations. Please note that with this variant, SERIAL reads will be " +
"slower than QUORUM reads, yet offer no additional guarantees. This flag should only be used in " +
"the restricted case of upgrading from a pre-CASSANDRA-12126 version, and only if you " +
"understand the tradeoff.", Paxos.getPaxosVariant());
}
}
/**
* 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 respond, 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 RowIterator cas(String keyspaceName,
String cfName,
DecoratedKey key,
CASRequest request,
ConsistencyLevel consistencyForPaxos,
ConsistencyLevel consistencyForCommit,
ClientState clientState,
int nowInSeconds,
long queryStartNanoTime)
throws UnavailableException, IsBootstrappingException, RequestFailureException, RequestTimeoutException, InvalidRequestException, CasWriteUnknownResultException
{
if (DatabaseDescriptor.getPartitionDenylistEnabled() && DatabaseDescriptor.getDenylistWritesEnabled() && !partitionDenylist.isKeyPermitted(keyspaceName, cfName, key.getKey()))
{
denylistMetrics.incrementWritesRejected();
throw new InvalidRequestException(String.format("Unable to CAS write to denylisted partition [0x%s] in %s/%s",
key.toString(), keyspaceName, cfName));
}
return Paxos.useV2()
? Paxos.cas(key, request, consistencyForPaxos, consistencyForCommit, clientState)
: legacyCas(keyspaceName, cfName, key, request, consistencyForPaxos, consistencyForCommit, clientState, nowInSeconds, queryStartNanoTime);
}
public static RowIterator legacyCas(String keyspaceName,
String cfName,
DecoratedKey key,
CASRequest request,
ConsistencyLevel consistencyForPaxos,
ConsistencyLevel consistencyForCommit,
ClientState clientState,
int nowInSeconds,
long queryStartNanoTime)
throws UnavailableException, IsBootstrappingException, RequestFailureException, RequestTimeoutException, InvalidRequestException
{
final long startTimeForMetrics = nanoTime();
try
{
TableMetadata metadata = Schema.instance.validateTable(keyspaceName, cfName);
Function<Ballot, Pair<PartitionUpdate, RowIterator>> updateProposer = ballot ->
{
// read the current values and check they validate the conditions
Tracing.trace("Reading existing values for CAS precondition");
SinglePartitionReadCommand readCommand = (SinglePartitionReadCommand) request.readCommand(nowInSeconds);
ConsistencyLevel readConsistency = consistencyForPaxos == ConsistencyLevel.LOCAL_SERIAL ? ConsistencyLevel.LOCAL_QUORUM : ConsistencyLevel.QUORUM;
FilteredPartition current;
try (RowIterator rowIter = readOne(readCommand, readConsistency, queryStartNanoTime))
{
current = FilteredPartition.create(rowIter);
}
if (!request.appliesTo(current))
{
Tracing.trace("CAS precondition does not match current values {}", current);
casWriteMetrics.conditionNotMet.inc();
return Pair.create(PartitionUpdate.emptyUpdate(metadata, key), current.rowIterator());
}
// Create the desired updates
PartitionUpdate updates = request.makeUpdates(current, clientState, ballot);
long size = updates.dataSize();
casWriteMetrics.mutationSize.update(size);
writeMetricsForLevel(consistencyForPaxos).mutationSize.update(size);
// 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(updates);
return Pair.create(updates, null);
};
return doPaxos(metadata,
key,
consistencyForPaxos,
consistencyForCommit,
consistencyForCommit,
queryStartNanoTime,
casWriteMetrics,
updateProposer);
}
catch (CasWriteUnknownResultException e)
{
casWriteMetrics.unknownResult.mark();
throw e;
}
catch (CasWriteTimeoutException wte)
{
casWriteMetrics.timeouts.mark();
writeMetricsForLevel(consistencyForPaxos).timeouts.mark();
throw new CasWriteTimeoutException(wte.writeType, wte.consistency, wte.received, wte.blockFor, wte.contentions);
}
catch (ReadTimeoutException e)
{
casWriteMetrics.timeouts.mark();
writeMetricsForLevel(consistencyForPaxos).timeouts.mark();
throw e;
}
catch (ReadAbortException e)
{
casWriteMetrics.markAbort(e);
writeMetricsForLevel(consistencyForPaxos).markAbort(e);
throw e;
}
catch (WriteFailureException | ReadFailureException e)
{
casWriteMetrics.failures.mark();
writeMetricsForLevel(consistencyForPaxos).failures.mark();
throw e;
}
catch (UnavailableException e)
{
casWriteMetrics.unavailables.mark();
writeMetricsForLevel(consistencyForPaxos).unavailables.mark();
throw e;
}
finally
{
final long latency = nanoTime() - startTimeForMetrics;
casWriteMetrics.addNano(latency);
writeMetricsForLevel(consistencyForPaxos).addNano(latency);
}
}
private static void recordCasContention(TableMetadata table,
DecoratedKey key,
CASClientRequestMetrics casMetrics,
int contentions)
{
if (contentions == 0)
return;
casMetrics.contention.update(contentions);
Keyspace.open(table.keyspace)
.getColumnFamilyStore(table.name)
.metric
.topCasPartitionContention
.addSample(key.getKey(), contentions);
}
/**
* Performs the Paxos rounds for a given proposal, retrying when preempted until the timeout.
*
* <p>The main 'configurable' of this method is the {@code createUpdateProposal} method: it is called by the method
* once a ballot has been successfully 'prepared' to generate the update to 'propose' (and commit if the proposal is
* successful). That method also generates the result that the whole method will return. Note that due to retrying,
* this method may be called multiple times and does not have to return the same results.
*
* @param metadata the table to update with Paxos.
* @param key the partition updated.
* @param consistencyForPaxos the serial consistency of the operation (either {@link ConsistencyLevel#SERIAL} or
* {@link ConsistencyLevel#LOCAL_SERIAL}).
* @param consistencyForReplayCommits the consistency for the commit phase of "replayed" in-progress operations.
* @param consistencyForCommit the consistency for the commit phase of _this_ operation update.
* @param queryStartNanoTime the nano time for the start of the query this is part of. This is the base time for
* timeouts.
* @param casMetrics the metrics to update for this operation.
* @param createUpdateProposal method called after a successful 'prepare' phase to obtain 1) the actual update of
* this operation and 2) the result that the whole method should return. This can return {@code null} in the
* special where, after having "prepared" (and thus potentially replayed in-progress upgdates), we don't want
* to propose anything (the whole method then return {@code null}).
* @return the second element of the pair returned by {@code createUpdateProposal} (for the last call of that method
* if that method is called multiple times due to retries).
*/
private static RowIterator doPaxos(TableMetadata metadata,
DecoratedKey key,
ConsistencyLevel consistencyForPaxos,
ConsistencyLevel consistencyForReplayCommits,
ConsistencyLevel consistencyForCommit,
long queryStartNanoTime,
CASClientRequestMetrics casMetrics,
Function<Ballot, Pair<PartitionUpdate, RowIterator>> createUpdateProposal)
throws UnavailableException, IsBootstrappingException, RequestFailureException, RequestTimeoutException, InvalidRequestException
{
int contentions = 0;
Keyspace keyspace = Keyspace.open(metadata.keyspace);
AbstractReplicationStrategy latestRs = keyspace.getReplicationStrategy();
try
{
consistencyForPaxos.validateForCas();
consistencyForReplayCommits.validateForCasCommit(latestRs);
consistencyForCommit.validateForCasCommit(latestRs);
long timeoutNanos = DatabaseDescriptor.getCasContentionTimeout(NANOSECONDS);
while (nanoTime() - queryStartNanoTime < timeoutNanos)
{
// for simplicity, we'll do a single liveness check at the start of each attempt
ReplicaPlan.ForPaxosWrite replicaPlan = ReplicaPlans.forPaxos(keyspace, key, consistencyForPaxos);
latestRs = replicaPlan.replicationStrategy();
PaxosBallotAndContention pair = beginAndRepairPaxos(queryStartNanoTime,
key,
metadata,
replicaPlan,
consistencyForPaxos,
consistencyForReplayCommits,
casMetrics);
final Ballot ballot = pair.ballot;
contentions += pair.contentions;
Pair<PartitionUpdate, RowIterator> proposalPair = createUpdateProposal.apply(ballot);
// See method javadoc: null here is code for "stop here and return null".
if (proposalPair == null)
return null;
Commit proposal = Commit.newProposal(ballot, proposalPair.left);
Tracing.trace("CAS precondition is met; proposing client-requested updates for {}", ballot);
if (proposePaxos(proposal, replicaPlan, true, queryStartNanoTime))
{
// We skip committing accepted updates when they are empty. This is an optimization which works
// because we also skip replaying those same empty update in beginAndRepairPaxos (see the longer
// comment there). As empty update are somewhat common (serial reads and non-applying CAS propose
// them), this is worth bothering.
if (!proposal.update.isEmpty())
commitPaxos(proposal, consistencyForCommit, true, queryStartNanoTime);
RowIterator result = proposalPair.right;
if (result != null)
Tracing.trace("CAS did not apply");
else
Tracing.trace("CAS applied successfully");
return result;
}
Tracing.trace("Paxos proposal not accepted (pre-empted by a higher ballot)");
contentions++;
Uninterruptibles.sleepUninterruptibly(ThreadLocalRandom.current().nextInt(100), TimeUnit.MILLISECONDS);
// continue to retry
}
}
catch (CasWriteTimeoutException e)
{
// Might be thrown by beginRepairAndPaxos. In that case, any contention that happened within the method and
// led up to the timeout was not accounted in our local 'contentions' variable and we add it now so it the
// contention recorded in the finally is correct.
contentions += e.contentions;
throw e;
}
catch (WriteTimeoutException e)
{
// Might be thrown by proposePaxos or commitPaxos
throw new CasWriteTimeoutException(e.writeType, e.consistency, e.received, e.blockFor, contentions);
}
finally
{
recordCasContention(metadata, key, casMetrics, contentions);
}
throw new CasWriteTimeoutException(WriteType.CAS, consistencyForPaxos, 0, consistencyForPaxos.blockFor(latestRs), contentions);
}
/**
* 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 PaxosBallotAndContention beginAndRepairPaxos(long queryStartNanoTime,
DecoratedKey key,
TableMetadata metadata,
ReplicaPlan.ForPaxosWrite paxosPlan,
ConsistencyLevel consistencyForPaxos,
ConsistencyLevel consistencyForCommit,
CASClientRequestMetrics casMetrics)
throws WriteTimeoutException, WriteFailureException
{
long timeoutNanos = DatabaseDescriptor.getCasContentionTimeout(NANOSECONDS);
PrepareCallback summary = null;
int contentions = 0;
while (nanoTime() - queryStartNanoTime < timeoutNanos)
{
// We want a timestamp that is guaranteed to be unique for that node (so that the ballot is globally unique), but if we've got a prepare rejected
// already we also want to make sure we pick a timestamp that has a chance to be promised, i.e. one that is greater that the most recently known
// in progress (#5667). Lastly, we don't want to use a timestamp that is older than the last one assigned by ClientState or operations may appear
// out-of-order (#7801).
long minTimestampMicrosToUse = summary == null ? Long.MIN_VALUE : 1 + summary.mostRecentInProgressCommit.ballot.unixMicros();
// Note that ballotMicros is not guaranteed to be unique if two proposal are being handled concurrently by the same coordinator. But we still
// need ballots to be unique for each proposal so we have to use getRandomTimeUUIDFromMicros.
Ballot ballot = nextBallot(minTimestampMicrosToUse, consistencyForPaxos == SERIAL ? GLOBAL : LOCAL);
// prepare
try
{
Tracing.trace("Preparing {}", ballot);
Commit toPrepare = Commit.newPrepare(key, metadata, ballot);
summary = preparePaxos(toPrepare, paxosPlan, queryStartNanoTime);
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), MILLISECONDS);
continue;
}
Commit inProgress = summary.mostRecentInProgressCommit;
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.
// One special case we make is for update that are empty (which are proposed by serial reads and
// non-applying CAS). While we could handle those as any other updates, we can optimize this somewhat by
// neither committing those empty updates, nor replaying in-progress ones. The reasoning is this: as the
// update is empty, we have nothing to apply to storage in the commit phase, so the only reason to commit
// would be to update the MRC. However, if we skip replaying those empty updates, then we don't need to
// update the MRC for following updates to make progress (that is, if we didn't had the empty update skip
// below _but_ skipped updating the MRC on empty updates, then we'd be stuck always proposing that same
// empty update). And the reason skipping that replay is safe is that when an operation tries to propose
// an empty value, there can be only 2 cases:
// 1) the propose succeed, meaning a quorum of nodes accept it, in which case we are guaranteed no earlier
// pending operation can ever be replayed (which is what we want to guarantee with the empty update).
// 2) the propose does not succeed. But then the operation proposing the empty update will not succeed
// either (it will retry or ultimately timeout), and we're actually ok if earlier pending operation gets
// replayed in that case.
// Tl;dr, it is safe to skip committing empty updates _as long as_ we also skip replying them below. And
// doing is more efficient, so we do so.
if (!inProgress.update.isEmpty() && inProgress.isAfter(mostRecent))
{
Tracing.trace("Finishing incomplete paxos round {}", inProgress);
casMetrics.unfinishedCommit.inc();
Commit refreshedInProgress = Commit.newProposal(ballot, inProgress.update);
if (proposePaxos(refreshedInProgress, paxosPlan, false, queryStartNanoTime))
{
commitPaxos(refreshedInProgress, consistencyForCommit, false, queryStartNanoTime);
}
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), 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<InetAddressAndPort> 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;
}
return new PaxosBallotAndContention(ballot, contentions);
}
catch (WriteTimeoutException e)
{
// We're still doing preparation for the paxos rounds, so we want to use the CAS (see CASSANDRA-8672)
throw new CasWriteTimeoutException(WriteType.CAS, e.consistency, e.received, e.blockFor, contentions);
}
}
throw new CasWriteTimeoutException(WriteType.CAS, consistencyForPaxos, 0, consistencyForPaxos.blockFor(paxosPlan.replicationStrategy()), contentions);
}
/**
* Unlike commitPaxos, this does not wait for replies
*/
private static void sendCommit(Commit commit, Iterable<InetAddressAndPort> replicas)
{
Message<Commit> message = Message.out(PAXOS_COMMIT_REQ, commit);
for (InetAddressAndPort target : replicas)
MessagingService.instance().send(message, target);
}
private static PrepareCallback preparePaxos(Commit toPrepare, ReplicaPlan.ForPaxosWrite replicaPlan, long queryStartNanoTime)
throws WriteTimeoutException
{
PrepareCallback callback = new PrepareCallback(toPrepare.update.partitionKey(), toPrepare.update.metadata(), replicaPlan.requiredParticipants(), replicaPlan.consistencyLevel(), queryStartNanoTime);
Message<Commit> message = Message.out(PAXOS_PREPARE_REQ, toPrepare);
boolean hasLocalRequest = false;
for (Replica replica: replicaPlan.contacts())
{
if (replica.isSelf())
{
hasLocalRequest = true;
PAXOS_PREPARE_REQ.stage.execute(() -> {
try
{
callback.onResponse(message.responseWith(doPrepare(toPrepare)));
}
catch (Exception ex)
{
logger.error("Failed paxos prepare locally", ex);
}
});
}
else
{
MessagingService.instance().sendWithCallback(message, replica.endpoint(), callback);
}
}
if (hasLocalRequest)
writeMetrics.localRequests.mark();
else
writeMetrics.remoteRequests.mark();
callback.await();
return callback;
}
/**
* Propose the {@param proposal} accoding to the {@param replicaPlan}.
* When {@param backoffIfPartial} is true, the proposer backs off when seeing the proposal being accepted by some but not a quorum.
* The result of the cooresponding CAS in uncertain as the accepted proposal may or may not be spread to other nodes in later rounds.
*/
private static boolean proposePaxos(Commit proposal, ReplicaPlan.ForPaxosWrite replicaPlan, boolean backoffIfPartial, long queryStartNanoTime)
throws WriteTimeoutException, CasWriteUnknownResultException
{
ProposeCallback callback = new ProposeCallback(replicaPlan.contacts().size(), replicaPlan.requiredParticipants(), !backoffIfPartial, replicaPlan.consistencyLevel(), queryStartNanoTime);
Message<Commit> message = Message.out(PAXOS_PROPOSE_REQ, proposal);
for (Replica replica : replicaPlan.contacts())
{
if (replica.isSelf())
{
PAXOS_PROPOSE_REQ.stage.execute(() -> {
try
{
Message<Boolean> response = message.responseWith(doPropose(proposal));
callback.onResponse(response);
}
catch (Exception ex)
{
logger.error("Failed paxos propose locally", ex);
}
});
}
else
{
MessagingService.instance().sendWithCallback(message, replica.endpoint(), callback);
}
}
callback.await();
if (callback.isSuccessful())
return true;
if (backoffIfPartial && !callback.isFullyRefused())
throw new CasWriteUnknownResultException(replicaPlan.consistencyLevel(), callback.getAcceptCount(), replicaPlan.requiredParticipants());
return false;
}
private static void commitPaxos(Commit proposal, ConsistencyLevel consistencyLevel, boolean allowHints, long queryStartNanoTime) throws WriteTimeoutException
{
boolean shouldBlock = consistencyLevel != ConsistencyLevel.ANY;
Keyspace keyspace = Keyspace.open(proposal.update.metadata().keyspace);
Token tk = proposal.update.partitionKey().getToken();
AbstractWriteResponseHandler<Commit> responseHandler = null;
// NOTE: this ReplicaPlan is a lie, this usage of ReplicaPlan could do with being clarified - the selected() collection is essentially (I think) never used
ReplicaPlan.ForWrite replicaPlan = ReplicaPlans.forWrite(keyspace, consistencyLevel, tk, ReplicaPlans.writeAll);
if (shouldBlock)
{
AbstractReplicationStrategy rs = replicaPlan.replicationStrategy();
responseHandler = rs.getWriteResponseHandler(replicaPlan, null, WriteType.SIMPLE, proposal::makeMutation, queryStartNanoTime);
}
Message<Commit> message = Message.outWithFlag(PAXOS_COMMIT_REQ, proposal, MessageFlag.CALL_BACK_ON_FAILURE);
for (Replica replica : replicaPlan.liveAndDown())
{
InetAddressAndPort destination = replica.endpoint();
checkHintOverload(replica);
if (replicaPlan.isAlive(replica))
{
if (shouldBlock)
{
if (replica.isSelf())
commitPaxosLocal(replica, message, responseHandler);
else
MessagingService.instance().sendWriteWithCallback(message, replica, responseHandler);
}
else
{
MessagingService.instance().send(message, destination);
}
}
else
{
if (responseHandler != null)
{
responseHandler.expired();
}
if (allowHints && shouldHint(replica))
{
submitHint(proposal.makeMutation(), replica, null);
}
}
}
if (shouldBlock)
responseHandler.get();
}
/**
* Commit a PAXOS task locally, and if the task times out rather then submitting a real hint
* submit a fake one that executes immediately on the mutation stage, but generates the necessary backpressure
* signal for hints
*/
private static void commitPaxosLocal(Replica localReplica, final Message<Commit> message, final AbstractWriteResponseHandler<?> responseHandler)
{
PAXOS_COMMIT_REQ.stage.maybeExecuteImmediately(new LocalMutationRunnable(localReplica)
{
public void runMayThrow()
{
try
{
PaxosState.commitDirect(message.payload);
if (responseHandler != null)
responseHandler.onResponse(null);
}
catch (Exception ex)
{
if (!(ex instanceof WriteTimeoutException))
logger.error("Failed to apply paxos commit locally : ", ex);
responseHandler.onFailure(FBUtilities.getBroadcastAddressAndPort(), RequestFailureReason.forException(ex));
}
}
@Override
public String description()
{
return "Paxos " + message.payload.toString();
}
@Override
protected Verb verb()
{
return PAXOS_COMMIT_REQ;
}
});
}
/**
* 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 consistencyLevel the consistency level for the operation
* @param queryStartNanoTime the value of nanoTime() when the query started to be processed
*/
public static void mutate(List<? extends IMutation> mutations, ConsistencyLevel consistencyLevel, long queryStartNanoTime)
throws UnavailableException, OverloadedException, WriteTimeoutException, WriteFailureException
{
Tracing.trace("Determining replicas for mutation");
final String localDataCenter = DatabaseDescriptor.getEndpointSnitch().getLocalDatacenter();
long startTime = nanoTime();
List<AbstractWriteResponseHandler<IMutation>> responseHandlers = new ArrayList<>(mutations.size());
WriteType plainWriteType = mutations.size() <= 1 ? WriteType.SIMPLE : WriteType.UNLOGGED_BATCH;
try
{
for (IMutation mutation : mutations)
{
if (mutation instanceof CounterMutation)
responseHandlers.add(mutateCounter((CounterMutation)mutation, localDataCenter, queryStartNanoTime));
else
responseHandlers.add(performWrite(mutation, consistencyLevel, localDataCenter, standardWritePerformer, null, plainWriteType, queryStartNanoTime));
}
// upgrade to full quorum any failed cheap quorums
for (int i = 0 ; i < mutations.size() ; ++i)
{
if (!(mutations.get(i) instanceof CounterMutation)) // at the moment, only non-counter writes support cheap quorums
responseHandlers.get(i).maybeTryAdditionalReplicas(mutations.get(i), standardWritePerformer, localDataCenter);
}
// wait for writes. throws TimeoutException if necessary
for (AbstractWriteResponseHandler<IMutation> responseHandler : responseHandlers)
responseHandler.get();
}
catch (WriteTimeoutException|WriteFailureException ex)
{
if (consistencyLevel == ConsistencyLevel.ANY)
{
hintMutations(mutations);
}
else
{
if (ex instanceof WriteFailureException)
{
writeMetrics.failures.mark();
writeMetricsForLevel(consistencyLevel).failures.mark();
WriteFailureException fe = (WriteFailureException)ex;
Tracing.trace("Write failure; received {} of {} required replies, failed {} requests",
fe.received, fe.blockFor, fe.failureReasonByEndpoint.size());
}
else
{
writeMetrics.timeouts.mark();
writeMetricsForLevel(consistencyLevel).timeouts.mark();
WriteTimeoutException te = (WriteTimeoutException)ex;
Tracing.trace("Write timeout; received {} of {} required replies", te.received, te.blockFor);
}
throw ex;
}
}
catch (UnavailableException e)
{
writeMetrics.unavailables.mark();
writeMetricsForLevel(consistencyLevel).unavailables.mark();
Tracing.trace("Unavailable");
throw e;
}
catch (OverloadedException e)
{
writeMetrics.unavailables.mark();
writeMetricsForLevel(consistencyLevel).unavailables.mark();
Tracing.trace("Overloaded");
throw e;
}
finally
{
long latency = nanoTime() - startTime;
writeMetrics.addNano(latency);
writeMetricsForLevel(consistencyLevel).addNano(latency);
updateCoordinatorWriteLatencyTableMetric(mutations, latency);
}
}
/**
* Hint all the mutations (except counters, which can't be safely retried). This means
* we'll re-hint any successful ones; doesn't seem worth it to track individual success
* just for this unusual case.
*
* Only used for CL.ANY
*
* @param mutations the mutations that require hints
*/
private static void hintMutations(Collection<? extends IMutation> mutations)
{
for (IMutation mutation : mutations)
if (!(mutation instanceof CounterMutation))
hintMutation((Mutation) mutation);
Tracing.trace("Wrote hints to satisfy CL.ANY after no replicas acknowledged the write");
}
private static void hintMutation(Mutation mutation)
{
String keyspaceName = mutation.getKeyspaceName();
Token token = mutation.key().getToken();
// local writes can timeout, but cannot be dropped (see LocalMutationRunnable and CASSANDRA-6510),
// so there is no need to hint or retry.
EndpointsForToken replicasToHint = ReplicaLayout.forTokenWriteLiveAndDown(Keyspace.open(keyspaceName), token)
.all()
.filter(StorageProxy::shouldHint);
submitHint(mutation, replicasToHint, null);
}
public boolean appliesLocally(Mutation mutation)
{
String keyspaceName = mutation.getKeyspaceName();
Token token = mutation.key().getToken();
InetAddressAndPort local = FBUtilities.getBroadcastAddressAndPort();
return ReplicaLayout.forTokenWriteLiveAndDown(Keyspace.open(keyspaceName), token)
.all().endpoints().contains(local);
}
/**
* Use this method to have these Mutations applied
* across all replicas.
*
* @param mutations the mutations to be applied across the replicas
* @param writeCommitLog if commitlog should be written
* @param baseComplete time from epoch in ms that the local base mutation was(or will be) completed
* @param queryStartNanoTime the value of nanoTime() when the query started to be processed
*/
public static void mutateMV(ByteBuffer dataKey, Collection<Mutation> mutations, boolean writeCommitLog, AtomicLong baseComplete, long queryStartNanoTime)
throws UnavailableException, OverloadedException, WriteTimeoutException
{