-
Notifications
You must be signed in to change notification settings - Fork 3.7k
/
Copy pathStorageService.java
5917 lines (5135 loc) · 242 KB
/
StorageService.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.io.*;
import java.net.InetAddress;
import java.net.UnknownHostException;
import java.nio.ByteBuffer;
import java.nio.file.Paths;
import java.util.*;
import java.util.Map.Entry;
import java.util.concurrent.*;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.regex.MatchResult;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import java.util.stream.StreamSupport;
import javax.annotation.Nullable;
import javax.management.*;
import javax.management.openmbean.CompositeData;
import javax.management.openmbean.OpenDataException;
import javax.management.openmbean.TabularData;
import javax.management.openmbean.TabularDataSupport;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.google.common.base.Predicates;
import com.google.common.collect.*;
import com.google.common.util.concurrent.*;
import org.apache.cassandra.config.ParameterizedClass;
import org.apache.cassandra.dht.RangeStreamer.FetchReplica;
import org.apache.cassandra.fql.FullQueryLogger;
import org.apache.cassandra.fql.FullQueryLoggerOptions;
import org.apache.cassandra.fql.FullQueryLoggerOptionsCompositeData;
import org.apache.cassandra.locator.ReplicaCollection.Builder.Conflict;
import org.apache.commons.lang3.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.cassandra.audit.AuditLogManager;
import org.apache.cassandra.audit.AuditLogOptions;
import org.apache.cassandra.auth.AuthKeyspace;
import org.apache.cassandra.auth.AuthSchemaChangeListener;
import org.apache.cassandra.batchlog.BatchlogManager;
import org.apache.cassandra.concurrent.ExecutorLocals;
import org.apache.cassandra.concurrent.NamedThreadFactory;
import org.apache.cassandra.concurrent.ScheduledExecutors;
import org.apache.cassandra.concurrent.Stage;
import org.apache.cassandra.config.Config;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.cql3.QueryProcessor;
import org.apache.cassandra.db.*;
import org.apache.cassandra.db.commitlog.CommitLog;
import org.apache.cassandra.db.compaction.CompactionManager;
import org.apache.cassandra.db.compaction.Verifier;
import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.db.virtual.VirtualKeyspaceRegistry;
import org.apache.cassandra.dht.*;
import org.apache.cassandra.dht.Range;
import org.apache.cassandra.dht.Token.TokenFactory;
import org.apache.cassandra.exceptions.*;
import org.apache.cassandra.gms.*;
import org.apache.cassandra.hints.HintsService;
import org.apache.cassandra.io.sstable.SSTableLoader;
import org.apache.cassandra.io.util.FileUtils;
import org.apache.cassandra.locator.*;
import org.apache.cassandra.metrics.StorageMetrics;
import org.apache.cassandra.net.*;
import org.apache.cassandra.repair.*;
import org.apache.cassandra.repair.messages.RepairOption;
import org.apache.cassandra.schema.CompactionParams.TombstoneOption;
import org.apache.cassandra.schema.KeyspaceMetadata;
import org.apache.cassandra.schema.MigrationCoordinator;
import org.apache.cassandra.schema.MigrationManager;
import org.apache.cassandra.schema.ReplicationParams;
import org.apache.cassandra.schema.Schema;
import org.apache.cassandra.schema.SchemaConstants;
import org.apache.cassandra.schema.TableMetadata;
import org.apache.cassandra.schema.TableMetadataRef;
import org.apache.cassandra.schema.ViewMetadata;
import org.apache.cassandra.streaming.*;
import org.apache.cassandra.tracing.TraceKeyspace;
import org.apache.cassandra.transport.ClientResourceLimits;
import org.apache.cassandra.transport.ProtocolVersion;
import org.apache.cassandra.utils.*;
import org.apache.cassandra.utils.logging.LoggingSupportFactory;
import org.apache.cassandra.utils.progress.ProgressEvent;
import org.apache.cassandra.utils.progress.ProgressEventType;
import org.apache.cassandra.utils.progress.ProgressListener;
import org.apache.cassandra.utils.progress.jmx.JMXBroadcastExecutor;
import org.apache.cassandra.utils.progress.jmx.JMXProgressSupport;
import static com.google.common.collect.Iterables.transform;
import static com.google.common.collect.Iterables.tryFind;
import static java.util.Arrays.asList;
import static java.util.Arrays.stream;
import static java.util.concurrent.TimeUnit.MINUTES;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static java.util.concurrent.TimeUnit.NANOSECONDS;
import static java.util.stream.Collectors.toList;
import static java.util.stream.Collectors.toMap;
import static org.apache.cassandra.config.CassandraRelevantProperties.BOOTSTRAP_SCHEMA_DELAY_MS;
import static org.apache.cassandra.config.CassandraRelevantProperties.BOOTSTRAP_SKIP_SCHEMA_CHECK;
import static org.apache.cassandra.config.CassandraRelevantProperties.REPLACEMENT_ALLOW_EMPTY;
import static org.apache.cassandra.index.SecondaryIndexManager.getIndexName;
import static org.apache.cassandra.index.SecondaryIndexManager.isIndexColumnFamily;
import static org.apache.cassandra.net.NoPayload.noPayload;
import static org.apache.cassandra.net.Verb.REPLICATION_DONE_REQ;
import static org.apache.cassandra.schema.MigrationManager.evolveSystemKeyspace;
/**
* This abstraction contains the token/identifier of this node
* on the identifier space. This token gets gossiped around.
* This class will also maintain histograms of the load information
* of other nodes in the cluster.
*/
public class StorageService extends NotificationBroadcasterSupport implements IEndpointStateChangeSubscriber, StorageServiceMBean
{
private static final Logger logger = LoggerFactory.getLogger(StorageService.class);
public static final int INDEFINITE = -1;
public static final int RING_DELAY = getRingDelay(); // delay after which we assume ring has stablized
public static final int SCHEMA_DELAY_MILLIS = getSchemaDelay();
private static final boolean REQUIRE_SCHEMAS = !BOOTSTRAP_SKIP_SCHEMA_CHECK.getBoolean();
private final JMXProgressSupport progressSupport = new JMXProgressSupport(this);
private static int getRingDelay()
{
String newdelay = System.getProperty("cassandra.ring_delay_ms");
if (newdelay != null)
{
logger.info("Overriding RING_DELAY to {}ms", newdelay);
return Integer.parseInt(newdelay);
}
else
{
return 30 * 1000;
}
}
private static int getSchemaDelay()
{
String newdelay = BOOTSTRAP_SCHEMA_DELAY_MS.getString();
if (newdelay != null)
{
logger.info("Overriding SCHEMA_DELAY_MILLIS to {}ms", newdelay);
return Integer.parseInt(newdelay);
}
else
{
return 30 * 1000;
}
}
/* This abstraction maintains the token/endpoint metadata information */
private TokenMetadata tokenMetadata = new TokenMetadata();
public volatile VersionedValue.VersionedValueFactory valueFactory = new VersionedValue.VersionedValueFactory(tokenMetadata.partitioner);
private Thread drainOnShutdown = null;
private volatile boolean isShutdown = false;
private final List<Runnable> preShutdownHooks = new ArrayList<>();
private final List<Runnable> postShutdownHooks = new ArrayList<>();
public static final StorageService instance = new StorageService();
@Deprecated
public boolean isInShutdownHook()
{
return isShutdown();
}
public boolean isShutdown()
{
return isShutdown;
}
/**
* for in-jvm dtest use - forces isShutdown to be set to whatever passed in.
*/
@VisibleForTesting
public void setIsShutdownUnsafeForTests(boolean isShutdown)
{
this.isShutdown = isShutdown;
}
public RangesAtEndpoint getLocalReplicas(String keyspaceName)
{
return Keyspace.open(keyspaceName).getReplicationStrategy()
.getAddressReplicas(FBUtilities.getBroadcastAddressAndPort());
}
public List<Range<Token>> getLocalAndPendingRanges(String ks)
{
InetAddressAndPort broadcastAddress = FBUtilities.getBroadcastAddressAndPort();
Keyspace keyspace = Keyspace.open(ks);
List<Range<Token>> ranges = new ArrayList<>();
for (Replica r : keyspace.getReplicationStrategy().getAddressReplicas(broadcastAddress))
ranges.add(r.range());
for (Replica r : getTokenMetadata().getPendingRanges(ks, broadcastAddress))
ranges.add(r.range());
return ranges;
}
public Collection<Range<Token>> getPrimaryRanges(String keyspace)
{
return getPrimaryRangesForEndpoint(keyspace, FBUtilities.getBroadcastAddressAndPort());
}
public Collection<Range<Token>> getPrimaryRangesWithinDC(String keyspace)
{
return getPrimaryRangeForEndpointWithinDC(keyspace, FBUtilities.getBroadcastAddressAndPort());
}
private final Set<InetAddressAndPort> replicatingNodes = Sets.newConcurrentHashSet();
private CassandraDaemon daemon;
private InetAddressAndPort removingNode;
/* Are we starting this node in bootstrap mode? */
private volatile boolean isBootstrapMode;
/* we bootstrap but do NOT join the ring unless told to do so */
private boolean isSurveyMode = Boolean.parseBoolean(System.getProperty
("cassandra.write_survey", "false"));
/* true if node is rebuilding and receiving data */
private final AtomicBoolean isRebuilding = new AtomicBoolean();
private final AtomicBoolean isDecommissioning = new AtomicBoolean();
private volatile boolean initialized = false;
private volatile boolean joined = false;
private volatile boolean gossipActive = false;
private final AtomicBoolean authSetupCalled = new AtomicBoolean(false);
private volatile boolean authSetupComplete = false;
/* the probability for tracing any particular request, 0 disables tracing and 1 enables for all */
private double traceProbability = 0.0;
private static enum Mode { STARTING, NORMAL, JOINING, LEAVING, DECOMMISSIONED, MOVING, DRAINING, DRAINED }
private volatile Mode operationMode = Mode.STARTING;
/* Used for tracking drain progress */
private volatile int totalCFs, remainingCFs;
private static final AtomicInteger nextRepairCommand = new AtomicInteger();
private final List<IEndpointLifecycleSubscriber> lifecycleSubscribers = new CopyOnWriteArrayList<>();
private final String jmxObjectName;
private Collection<Token> bootstrapTokens = null;
// true when keeping strict consistency while bootstrapping
public static final boolean useStrictConsistency = Boolean.parseBoolean(System.getProperty("cassandra.consistent.rangemovement", "true"));
private static final boolean allowSimultaneousMoves = Boolean.parseBoolean(System.getProperty("cassandra.consistent.simultaneousmoves.allow","false"));
private static final boolean joinRing = Boolean.parseBoolean(System.getProperty("cassandra.join_ring", "true"));
private boolean replacing;
private final StreamStateStore streamStateStore = new StreamStateStore();
public boolean isSurveyMode()
{
return isSurveyMode;
}
public boolean hasJoined()
{
return joined;
}
/**
* This method updates the local token on disk
*/
public void setTokens(Collection<Token> tokens)
{
assert tokens != null && !tokens.isEmpty() : "Node needs at least one token.";
if (logger.isDebugEnabled())
logger.debug("Setting tokens to {}", tokens);
SystemKeyspace.updateTokens(tokens);
Collection<Token> localTokens = getLocalTokens();
setGossipTokens(localTokens);
tokenMetadata.updateNormalTokens(tokens, FBUtilities.getBroadcastAddressAndPort());
setMode(Mode.NORMAL, false);
}
public void setGossipTokens(Collection<Token> tokens)
{
List<Pair<ApplicationState, VersionedValue>> states = new ArrayList<Pair<ApplicationState, VersionedValue>>();
states.add(Pair.create(ApplicationState.TOKENS, valueFactory.tokens(tokens)));
states.add(Pair.create(ApplicationState.STATUS_WITH_PORT, valueFactory.normal(tokens)));
states.add(Pair.create(ApplicationState.STATUS, valueFactory.normal(tokens)));
Gossiper.instance.addLocalApplicationStates(states);
}
public StorageService()
{
// use dedicated executor for handling JMX notifications
super(JMXBroadcastExecutor.executor);
jmxObjectName = "org.apache.cassandra.db:type=StorageService";
MBeanWrapper.instance.registerMBean(this, jmxObjectName);
MBeanWrapper.instance.registerMBean(StreamManager.instance, StreamManager.OBJECT_NAME);
}
public void registerDaemon(CassandraDaemon daemon)
{
this.daemon = daemon;
}
public void register(IEndpointLifecycleSubscriber subscriber)
{
lifecycleSubscribers.add(subscriber);
}
public void unregister(IEndpointLifecycleSubscriber subscriber)
{
lifecycleSubscribers.remove(subscriber);
}
// should only be called via JMX
public void stopGossiping()
{
if (gossipActive)
{
if (!isNormal())
throw new IllegalStateException("Unable to stop gossip because the node is not in the normal state. Try to stop the node instead.");
logger.warn("Stopping gossip by operator request");
if (isNativeTransportRunning())
{
logger.warn("Disabling gossip while native transport is still active is unsafe");
}
Gossiper.instance.stop();
gossipActive = false;
}
}
// should only be called via JMX
public synchronized void startGossiping()
{
if (!gossipActive)
{
checkServiceAllowedToStart("gossip");
logger.warn("Starting gossip by operator request");
Collection<Token> tokens = SystemKeyspace.getSavedTokens();
boolean validTokens = tokens != null && !tokens.isEmpty();
// shouldn't be called before these are set if we intend to join the ring/are in the process of doing so
if (joined || joinRing)
assert validTokens : "Cannot start gossiping for a node intended to join without valid tokens";
if (validTokens)
setGossipTokens(tokens);
Gossiper.instance.forceNewerGeneration();
Gossiper.instance.start((int) (System.currentTimeMillis() / 1000));
gossipActive = true;
}
}
// should only be called via JMX
public boolean isGossipRunning()
{
return Gossiper.instance.isEnabled();
}
public synchronized void startNativeTransport()
{
checkServiceAllowedToStart("native transport");
if (daemon == null)
{
throw new IllegalStateException("No configured daemon");
}
try
{
daemon.startNativeTransport();
}
catch (Exception e)
{
throw new RuntimeException("Error starting native transport: " + e.getMessage());
}
}
public void stopNativeTransport()
{
if (daemon == null)
{
throw new IllegalStateException("No configured daemon");
}
daemon.stopNativeTransport();
}
public boolean isNativeTransportRunning()
{
if (daemon == null)
{
return false;
}
return daemon.isNativeTransportRunning();
}
@Override
public void enableNativeTransportOldProtocolVersions()
{
DatabaseDescriptor.setNativeTransportAllowOlderProtocols(true);
}
@Override
public void disableNativeTransportOldProtocolVersions()
{
DatabaseDescriptor.setNativeTransportAllowOlderProtocols(false);
}
public void stopTransports()
{
if (isNativeTransportRunning())
{
logger.error("Stopping native transport");
stopNativeTransport();
}
if (isGossipActive())
{
logger.error("Stopping gossiper");
stopGossiping();
}
}
/**
* Set the Gossip flag RPC_READY to false and then
* shutdown the client services (thrift and CQL).
*
* Note that other nodes will do this for us when
* they get the Gossip shutdown message, so even if
* we don't get time to broadcast this, it is not a problem.
*
* See {@link Gossiper#markAsShutdown(InetAddressAndPort)}
*/
private void shutdownClientServers()
{
setRpcReady(false);
stopNativeTransport();
}
public void stopClient()
{
Gossiper.instance.unregister(this);
Gossiper.instance.stop();
MessagingService.instance().shutdown();
// give it a second so that task accepted before the MessagingService shutdown gets submitted to the stage (to avoid RejectedExecutionException)
Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
Stage.shutdownNow();
}
public boolean isInitialized()
{
return initialized;
}
public boolean isGossipActive()
{
return gossipActive;
}
public boolean isDaemonSetupCompleted()
{
return daemon == null
? false
: daemon.setupCompleted();
}
public void stopDaemon()
{
if (daemon == null)
throw new IllegalStateException("No configured daemon");
daemon.deactivate();
}
private synchronized UUID prepareForReplacement() throws ConfigurationException
{
if (SystemKeyspace.bootstrapComplete())
throw new RuntimeException("Cannot replace address with a node that is already bootstrapped");
if (!joinRing)
throw new ConfigurationException("Cannot set both join_ring=false and attempt to replace a node");
if (!DatabaseDescriptor.isAutoBootstrap() && !Boolean.getBoolean("cassandra.allow_unsafe_replace"))
throw new RuntimeException("Replacing a node without bootstrapping risks invalidating consistency " +
"guarantees as the expected data may not be present until repair is run. " +
"To perform this operation, please restart with " +
"-Dcassandra.allow_unsafe_replace=true");
InetAddressAndPort replaceAddress = DatabaseDescriptor.getReplaceAddress();
logger.info("Gathering node replacement information for {}", replaceAddress);
Map<InetAddressAndPort, EndpointState> epStates = Gossiper.instance.doShadowRound();
// as we've completed the shadow round of gossip, we should be able to find the node we're replacing
EndpointState state = epStates.get(replaceAddress);
if (state == null)
throw new RuntimeException(String.format("Cannot replace_address %s because it doesn't exist in gossip", replaceAddress));
validateEndpointSnitch(epStates.values().iterator());
try
{
VersionedValue tokensVersionedValue = state.getApplicationState(ApplicationState.TOKENS);
if (tokensVersionedValue == null)
throw new RuntimeException(String.format("Could not find tokens for %s to replace", replaceAddress));
Collection<Token> tokens = TokenSerializer.deserialize(tokenMetadata.partitioner, new DataInputStream(new ByteArrayInputStream(tokensVersionedValue.toBytes())));
bootstrapTokens = validateReplacementBootstrapTokens(tokenMetadata, replaceAddress, tokens);
if (state.isEmptyWithoutStatus() && REPLACEMENT_ALLOW_EMPTY.getBoolean())
{
logger.warn("Gossip state not present for replacing node {}. Adding temporary entry to continue.", replaceAddress);
// When replacing a node, we take ownership of all its tokens.
// If that node is currently down and not present in the gossip info
// of any other live peers, then we will not be able to take ownership
// of its tokens during bootstrap as they have no way of being propagated
// to this node's TokenMetadata. TM is loaded at startup (in which case
// it will be/ empty for a new replacement node) and only updated with
// tokens for an endpoint during normal state propagation (which will not
// occur if no peers have gossip state for it).
// However, the presence of host id and tokens in the system tables implies
// that the node managed to complete bootstrap at some point in the past.
// Peers may include this information loaded directly from system tables
// in a GossipDigestAck *only if* the GossipDigestSyn was sent as part of a
// shadow round (otherwise, a GossipDigestAck contains only state about peers
// learned via gossip).
// It is safe to do this here as since we completed a shadow round we know
// that :
// * replaceAddress successfully bootstrapped at some point and owned these
// tokens
// * we know that no other node currently owns these tokens
// * we are going to completely take over replaceAddress's ownership of
// these tokens.
tokenMetadata.updateNormalTokens(bootstrapTokens, replaceAddress);
UUID hostId = Gossiper.instance.getHostId(replaceAddress, epStates);
if (hostId != null)
tokenMetadata.updateHostId(hostId, replaceAddress);
// If we were only able to learn about the node being replaced through the
// shadow gossip round (i.e. there is no state in gossip across the cluster
// about it, perhaps because the entire cluster has been bounced since it went
// down), then we're safe to proceed with the replacement. In this case, there
// will be no local endpoint state as we discard the results of the shadow
// round after preparing replacement info. We inject a minimal EndpointState
// to keep FailureDetector::isAlive and Gossiper::compareEndpointStartup from
// failing later in the replacement, as they both expect the replaced node to
// be fully present in gossip.
// Otherwise, if the replaced node is present in gossip, we need check that
// it is not in fact live.
// We choose to not include the EndpointState provided during the shadow round
// as its possible to include more state than is desired, so by creating a
// new empty endpoint without that information we can control what is in our
// local gossip state
Gossiper.instance.initializeUnreachableNodeUnsafe(replaceAddress);
}
}
catch (IOException e)
{
throw new RuntimeException(e);
}
UUID localHostId = SystemKeyspace.getLocalHostId();
if (isReplacingSameAddress())
{
localHostId = Gossiper.instance.getHostId(replaceAddress, epStates);
SystemKeyspace.setLocalHostId(localHostId); // use the replacee's host Id as our own so we receive hints, etc
}
return localHostId;
}
private static Collection<Token> validateReplacementBootstrapTokens(TokenMetadata tokenMetadata,
InetAddressAndPort replaceAddress,
Collection<Token> bootstrapTokens)
{
Map<Token, InetAddressAndPort> conflicts = new HashMap<>();
for (Token token : bootstrapTokens)
{
InetAddressAndPort conflict = tokenMetadata.getEndpoint(token);
if (null != conflict && !conflict.equals(replaceAddress))
conflicts.put(token, tokenMetadata.getEndpoint(token));
}
if (!conflicts.isEmpty())
{
String error = String.format("Conflicting token ownership information detected between " +
"gossip and current ring view during proposed replacement " +
"of %s. Some tokens identified in gossip for the node being " +
"replaced are currently owned by other peers: %s",
replaceAddress,
conflicts.entrySet()
.stream()
.map(e -> e.getKey() + "(" + e.getValue() + ")" )
.collect(Collectors.joining(",")));
throw new RuntimeException(error);
}
return bootstrapTokens;
}
private synchronized void checkForEndpointCollision(UUID localHostId, Set<InetAddressAndPort> peers) throws ConfigurationException
{
if (Boolean.getBoolean("cassandra.allow_unsafe_join"))
{
logger.warn("Skipping endpoint collision check as cassandra.allow_unsafe_join=true");
return;
}
logger.debug("Starting shadow gossip round to check for endpoint collision");
Map<InetAddressAndPort, EndpointState> epStates = Gossiper.instance.doShadowRound(peers);
if (epStates.isEmpty() && DatabaseDescriptor.getSeeds().contains(FBUtilities.getBroadcastAddressAndPort()))
logger.info("Unable to gossip with any peers but continuing anyway since node is in its own seed list");
// If bootstrapping, check whether any previously known status for the endpoint makes it unsafe to do so.
// If not bootstrapping, compare the host id for this endpoint learned from gossip (if any) with the local
// one, which was either read from system.local or generated at startup. If a learned id is present &
// doesn't match the local, then the node needs replacing
if (!Gossiper.instance.isSafeForStartup(FBUtilities.getBroadcastAddressAndPort(), localHostId, shouldBootstrap(), epStates))
{
throw new RuntimeException(String.format("A node with address %s already exists, cancelling join. " +
"Use cassandra.replace_address if you want to replace this node.",
FBUtilities.getBroadcastAddressAndPort()));
}
validateEndpointSnitch(epStates.values().iterator());
if (shouldBootstrap() && useStrictConsistency && !allowSimultaneousMoves())
{
for (Map.Entry<InetAddressAndPort, EndpointState> entry : epStates.entrySet())
{
// ignore local node or empty status
if (entry.getKey().equals(FBUtilities.getBroadcastAddressAndPort()) || (entry.getValue().getApplicationState(ApplicationState.STATUS_WITH_PORT) == null & entry.getValue().getApplicationState(ApplicationState.STATUS) == null))
continue;
VersionedValue value = entry.getValue().getApplicationState(ApplicationState.STATUS_WITH_PORT);
if (value == null)
{
value = entry.getValue().getApplicationState(ApplicationState.STATUS);
}
String[] pieces = splitValue(value);
assert (pieces.length > 0);
String state = pieces[0];
if (state.equals(VersionedValue.STATUS_BOOTSTRAPPING) || state.equals(VersionedValue.STATUS_LEAVING) || state.equals(VersionedValue.STATUS_MOVING))
throw new UnsupportedOperationException("Other bootstrapping/leaving/moving nodes detected, cannot bootstrap while cassandra.consistent.rangemovement is true");
}
}
}
private static void validateEndpointSnitch(Iterator<EndpointState> endpointStates)
{
Set<String> datacenters = new HashSet<>();
Set<String> racks = new HashSet<>();
while (endpointStates.hasNext())
{
EndpointState state = endpointStates.next();
VersionedValue val = state.getApplicationState(ApplicationState.DC);
if (val != null)
datacenters.add(val.value);
val = state.getApplicationState(ApplicationState.RACK);
if (val != null)
racks.add(val.value);
}
IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
if (!snitch.validate(datacenters, racks))
{
throw new IllegalStateException();
}
}
private boolean allowSimultaneousMoves()
{
return allowSimultaneousMoves && DatabaseDescriptor.getNumTokens() == 1;
}
// for testing only
public void unsafeInitialize() throws ConfigurationException
{
initialized = true;
gossipActive = true;
Gossiper.instance.register(this);
Gossiper.instance.start((int) (System.currentTimeMillis() / 1000)); // needed for node-ring gathering.
Gossiper.instance.addLocalApplicationState(ApplicationState.NET_VERSION, valueFactory.networkVersion());
MessagingService.instance().listen();
}
public synchronized void initServer() throws ConfigurationException
{
initServer(RING_DELAY);
}
public synchronized void initServer(int delay) throws ConfigurationException
{
logger.info("Cassandra version: {}", FBUtilities.getReleaseVersionString());
logger.info("CQL version: {}", QueryProcessor.CQL_VERSION);
logger.info("Native protocol supported versions: {} (default: {})",
StringUtils.join(ProtocolVersion.supportedVersions(), ", "), ProtocolVersion.CURRENT);
try
{
// Ensure StorageProxy is initialized on start-up; see CASSANDRA-3797.
Class.forName("org.apache.cassandra.service.StorageProxy");
// also IndexSummaryManager, which is otherwise unreferenced
Class.forName("org.apache.cassandra.io.sstable.IndexSummaryManager");
}
catch (ClassNotFoundException e)
{
throw new AssertionError(e);
}
if (Boolean.parseBoolean(System.getProperty("cassandra.load_ring_state", "true")))
{
logger.info("Loading persisted ring state");
populatePeerTokenMetadata();
for (InetAddressAndPort endpoint : tokenMetadata.getAllEndpoints())
Gossiper.runInGossipStageBlocking(() -> Gossiper.instance.addSavedEndpoint(endpoint));
}
// daemon threads, like our executors', continue to run while shutdown hooks are invoked
drainOnShutdown = NamedThreadFactory.createThread(new WrappedRunnable()
{
@Override
public void runMayThrow() throws InterruptedException, ExecutionException, IOException
{
drain(true);
if (FBUtilities.isWindows)
WindowsTimer.endTimerPeriod(DatabaseDescriptor.getWindowsTimerInterval());
LoggingSupportFactory.getLoggingSupport().onShutdown();
}
}, "StorageServiceShutdownHook");
Runtime.getRuntime().addShutdownHook(drainOnShutdown);
replacing = isReplacing();
if (!Boolean.parseBoolean(System.getProperty("cassandra.start_gossip", "true")))
{
logger.info("Not starting gossip as requested.");
initialized = true;
return;
}
prepareToJoin();
// Has to be called after the host id has potentially changed in prepareToJoin().
try
{
CacheService.instance.counterCache.loadSavedAsync().get();
}
catch (Throwable t)
{
JVMStabilityInspector.inspectThrowable(t);
logger.warn("Error loading counter cache", t);
}
if (joinRing)
{
joinTokenRing(delay);
}
else
{
Collection<Token> tokens = SystemKeyspace.getSavedTokens();
if (!tokens.isEmpty())
{
tokenMetadata.updateNormalTokens(tokens, FBUtilities.getBroadcastAddressAndPort());
// order is important here, the gossiper can fire in between adding these two states. It's ok to send TOKENS without STATUS, but *not* vice versa.
List<Pair<ApplicationState, VersionedValue>> states = new ArrayList<Pair<ApplicationState, VersionedValue>>();
states.add(Pair.create(ApplicationState.TOKENS, valueFactory.tokens(tokens)));
states.add(Pair.create(ApplicationState.STATUS_WITH_PORT, valueFactory.hibernate(true)));
states.add(Pair.create(ApplicationState.STATUS, valueFactory.hibernate(true)));
Gossiper.instance.addLocalApplicationStates(states);
}
doAuthSetup(true);
logger.info("Not joining ring as requested. Use JMX (StorageService->joinRing()) to initiate ring joining");
}
initialized = true;
}
public void populateTokenMetadata()
{
if (Boolean.parseBoolean(System.getProperty("cassandra.load_ring_state", "true")))
{
populatePeerTokenMetadata();
// if we have not completed bootstrapping, we should not add ourselves as a normal token
if (!shouldBootstrap())
tokenMetadata.updateNormalTokens(SystemKeyspace.getSavedTokens(), FBUtilities.getBroadcastAddressAndPort());
logger.info("Token metadata: {}", tokenMetadata);
}
}
private void populatePeerTokenMetadata()
{
logger.info("Populating token metadata from system tables");
Multimap<InetAddressAndPort, Token> loadedTokens = SystemKeyspace.loadTokens();
// entry has been mistakenly added, delete it
if (loadedTokens.containsKey(FBUtilities.getBroadcastAddressAndPort()))
SystemKeyspace.removeEndpoint(FBUtilities.getBroadcastAddressAndPort());
Map<InetAddressAndPort, UUID> loadedHostIds = SystemKeyspace.loadHostIds();
for (InetAddressAndPort ep : loadedTokens.keySet())
{
tokenMetadata.updateNormalTokens(loadedTokens.get(ep), ep);
if (loadedHostIds.containsKey(ep))
tokenMetadata.updateHostId(loadedHostIds.get(ep), ep);
}
}
private boolean isReplacing()
{
if (System.getProperty("cassandra.replace_address_first_boot", null) != null && SystemKeyspace.bootstrapComplete())
{
logger.info("Replace address on first boot requested; this node is already bootstrapped");
return false;
}
return DatabaseDescriptor.getReplaceAddress() != null;
}
/**
* In the event of forceful termination we need to remove the shutdown hook to prevent hanging (OOM for instance)
*/
public void removeShutdownHook()
{
if (drainOnShutdown != null)
Runtime.getRuntime().removeShutdownHook(drainOnShutdown);
if (FBUtilities.isWindows)
WindowsTimer.endTimerPeriod(DatabaseDescriptor.getWindowsTimerInterval());
}
private boolean shouldBootstrap()
{
return DatabaseDescriptor.isAutoBootstrap() && !SystemKeyspace.bootstrapComplete() && !isSeed();
}
public static boolean isSeed()
{
return DatabaseDescriptor.getSeeds().contains(FBUtilities.getBroadcastAddressAndPort());
}
private void prepareToJoin() throws ConfigurationException
{
MigrationCoordinator.instance.start();
if (!joined)
{
Map<ApplicationState, VersionedValue> appStates = new EnumMap<>(ApplicationState.class);
if (SystemKeyspace.wasDecommissioned())
{
if (Boolean.getBoolean("cassandra.override_decommission"))
{
logger.warn("This node was decommissioned, but overriding by operator request.");
SystemKeyspace.setBootstrapState(SystemKeyspace.BootstrapState.COMPLETED);
}
else
throw new ConfigurationException("This node was decommissioned and will not rejoin the ring unless cassandra.override_decommission=true has been set, or all existing data is removed and the node is bootstrapped again");
}
if (DatabaseDescriptor.getReplaceTokens().size() > 0 || DatabaseDescriptor.getReplaceNode() != null)
throw new RuntimeException("Replace method removed; use cassandra.replace_address instead");
MessagingService.instance().listen();
UUID localHostId = SystemKeyspace.getLocalHostId();
if (replacing)
{
localHostId = prepareForReplacement();
appStates.put(ApplicationState.TOKENS, valueFactory.tokens(bootstrapTokens));
if (!DatabaseDescriptor.isAutoBootstrap())
{
// Will not do replace procedure, persist the tokens we're taking over locally
// so that they don't get clobbered with auto generated ones in joinTokenRing
SystemKeyspace.updateTokens(bootstrapTokens);
}
else if (isReplacingSameAddress())
{
//only go into hibernate state if replacing the same address (CASSANDRA-8523)
logger.warn("Writes will not be forwarded to this node during replacement because it has the same address as " +
"the node to be replaced ({}). If the previous node has been down for longer than max_hint_window_in_ms, " +
"repair must be run after the replacement process in order to make this node consistent.",
DatabaseDescriptor.getReplaceAddress());
appStates.put(ApplicationState.STATUS_WITH_PORT, valueFactory.hibernate(true));
appStates.put(ApplicationState.STATUS, valueFactory.hibernate(true));
}
}
else
{
checkForEndpointCollision(localHostId, SystemKeyspace.loadHostIds().keySet());
if (SystemKeyspace.bootstrapComplete())
{
Preconditions.checkState(!Config.isClientMode());
// tokens are only ever saved to system.local after bootstrap has completed and we're joining the ring,
// or when token update operations (move, decom) are completed
Collection<Token> savedTokens = SystemKeyspace.getSavedTokens();
if (!savedTokens.isEmpty())
appStates.put(ApplicationState.TOKENS, valueFactory.tokens(savedTokens));
}
}
// have to start the gossip service before we can see any info on other nodes. this is necessary
// for bootstrap to get the load info it needs.
// (we won't be part of the storage ring though until we add a counterId to our state, below.)
// Seed the host ID-to-endpoint map with our own ID.
getTokenMetadata().updateHostId(localHostId, FBUtilities.getBroadcastAddressAndPort());
appStates.put(ApplicationState.NET_VERSION, valueFactory.networkVersion());
appStates.put(ApplicationState.HOST_ID, valueFactory.hostId(localHostId));
appStates.put(ApplicationState.NATIVE_ADDRESS_AND_PORT, valueFactory.nativeaddressAndPort(FBUtilities.getBroadcastNativeAddressAndPort()));
appStates.put(ApplicationState.RPC_ADDRESS, valueFactory.rpcaddress(FBUtilities.getJustBroadcastNativeAddress()));
appStates.put(ApplicationState.RELEASE_VERSION, valueFactory.releaseVersion());
logger.info("Starting up server gossip");
Gossiper.instance.register(this);
Gossiper.instance.start(SystemKeyspace.incrementAndGetGeneration(), appStates); // needed for node-ring gathering.
gossipActive = true;
// gossip snitch infos (local DC and rack)
gossipSnitchInfo();
// gossip Schema.emptyVersion forcing immediate check for schema updates (see MigrationManager#maybeScheduleSchemaPull)
Schema.instance.updateVersionAndAnnounce(); // Ensure we know our own actual Schema UUID in preparation for updates
LoadBroadcaster.instance.startBroadcasting();
HintsService.instance.startDispatch();
BatchlogManager.instance.start();
}
}
public void waitForSchema(long delay)
{
// first sleep the delay to make sure we see all our peers
for (long i = 0; i < delay; i += 1000)
{
// if we see schema, we can proceed to the next check directly
if (!Schema.instance.isEmpty())
{
logger.debug("current schema version: {}", Schema.instance.getVersion());
break;
}
Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
}
boolean schemasReceived = MigrationCoordinator.instance.awaitSchemaRequests(SCHEMA_DELAY_MILLIS);
if (schemasReceived)
return;
logger.warn(String.format("There are nodes in the cluster with a different schema version than us we did not merged schemas from, " +
"our version : (%s), outstanding versions -> endpoints : %s",
Schema.instance.getVersion(),
MigrationCoordinator.instance.outstandingVersions()));
if (REQUIRE_SCHEMAS)
throw new RuntimeException("Didn't receive schemas for all known versions within the timeout");
}
private void joinTokenRing(long schemaTimeoutMillis) throws ConfigurationException
{
joinTokenRing(!isSurveyMode, shouldBootstrap(), schemaTimeoutMillis, INDEFINITE);
}
@VisibleForTesting
public void joinTokenRing(boolean finishJoiningRing,
boolean shouldBootstrap,
long schemaTimeoutMillis,