-
Notifications
You must be signed in to change notification settings - Fork 3.6k
/
CassandraDaemon.java
953 lines (821 loc) · 35.7 KB
/
CassandraDaemon.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
/*
* 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.IOException;
import java.lang.management.ManagementFactory;
import java.lang.management.MemoryPoolMXBean;
import java.net.InetAddress;
import java.net.URL;
import java.net.UnknownHostException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.TimeUnit;
import java.util.stream.Stream;
import javax.management.ObjectName;
import javax.management.StandardMBean;
import javax.management.remote.JMXConnectorServer;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ImmutableList;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.addthis.metrics3.reporter.config.ReporterConfig;
import com.codahale.metrics.Meter;
import com.codahale.metrics.MetricRegistryListener;
import com.codahale.metrics.SharedMetricRegistries;
import com.codahale.metrics.jvm.BufferPoolMetricSet;
import com.codahale.metrics.jvm.FileDescriptorRatioGauge;
import com.codahale.metrics.jvm.GarbageCollectorMetricSet;
import com.codahale.metrics.jvm.MemoryUsageGaugeSet;
import org.apache.cassandra.audit.AuditLogManager;
import org.apache.cassandra.auth.AuthCacheService;
import org.apache.cassandra.concurrent.ScheduledExecutors;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.cql3.QueryProcessor;
import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.Keyspace;
import org.apache.cassandra.db.SizeEstimatesRecorder;
import org.apache.cassandra.db.SystemKeyspace;
import org.apache.cassandra.db.SystemKeyspaceMigrator41;
import org.apache.cassandra.db.commitlog.CommitLog;
import org.apache.cassandra.db.virtual.SystemViewsKeyspace;
import org.apache.cassandra.db.virtual.VirtualKeyspaceRegistry;
import org.apache.cassandra.db.virtual.VirtualSchemaKeyspace;
import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.exceptions.StartupException;
import org.apache.cassandra.gms.Gossiper;
import org.apache.cassandra.io.util.File;
import org.apache.cassandra.io.util.FileUtils;
import org.apache.cassandra.locator.InetAddressAndPort;
import org.apache.cassandra.metrics.CassandraMetricsRegistry;
import org.apache.cassandra.metrics.DefaultNameFactory;
import org.apache.cassandra.net.StartupClusterConnectivityChecker;
import org.apache.cassandra.schema.Schema;
import org.apache.cassandra.schema.SchemaConstants;
import org.apache.cassandra.schema.TableMetadata;
import org.apache.cassandra.security.ThreadAwareSecurityManager;
import org.apache.cassandra.streaming.StreamManager;
import org.apache.cassandra.service.paxos.PaxosState;
import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.JMXServerUtils;
import org.apache.cassandra.utils.JVMStabilityInspector;
import org.apache.cassandra.utils.MBeanWrapper;
import org.apache.cassandra.utils.Mx4jTool;
import org.apache.cassandra.utils.NativeLibrary;
import org.apache.cassandra.utils.concurrent.Future;
import org.apache.cassandra.utils.concurrent.FutureCombiner;
import org.apache.cassandra.utils.logging.LoggingSupportFactory;
import org.apache.cassandra.utils.logging.VirtualTableAppender;
import static java.util.concurrent.TimeUnit.NANOSECONDS;
import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_FOREGROUND;
import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_JMX_LOCAL_PORT;
import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_JMX_REMOTE_PORT;
import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_PID_FILE;
import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_PORT;
import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_CLASS_PATH;
import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_RMI_SERVER_RANDOM_ID;
import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_VERSION;
import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_VM_NAME;
import static org.apache.cassandra.config.CassandraRelevantProperties.METRICS_REPORTER_CONFIG_FILE;
import static org.apache.cassandra.config.CassandraRelevantProperties.SIZE_RECORDER_INTERVAL;
import static org.apache.cassandra.config.CassandraRelevantProperties.START_NATIVE_TRANSPORT;
/**
* The <code>CassandraDaemon</code> is an abstraction for a Cassandra daemon
* service, which defines not only a way to activate and deactivate it, but also
* hooks into its lifecycle methods (see {@link #setup()}, {@link #start()},
* {@link #stop()} and {@link #setup()}).
*/
public class CassandraDaemon
{
public static final String MBEAN_NAME = "org.apache.cassandra.db:type=NativeAccess";
private static final Logger logger;
@VisibleForTesting
public static CassandraDaemon getInstanceForTesting()
{
return instance;
}
static {
// Need to register metrics before instrumented appender is created(first access to LoggerFactory).
SharedMetricRegistries.getOrCreate("logback-metrics").addListener(new MetricRegistryListener.Base()
{
@Override
public void onMeterAdded(String metricName, Meter meter)
{
// Given metricName consists of appender name in logback.xml + "." + metric name.
// We first separate appender name
int separator = metricName.lastIndexOf('.');
String appenderName = metricName.substring(0, separator);
String metric = metricName.substring(separator + 1); // remove "."
ObjectName name = DefaultNameFactory.createMetricName(appenderName, metric, null).getMBeanName();
CassandraMetricsRegistry.Metrics.registerMBean(meter, name);
}
});
logger = LoggerFactory.getLogger(CassandraDaemon.class);
}
private void maybeInitJmx()
{
// If the standard com.sun.management.jmxremote.port property has been set
// then the JVM agent will have already started up a default JMX connector
// server. This behaviour is deprecated, but some clients may be relying
// on it, so log a warning and skip setting up the server with the settings
// as configured in cassandra-env.(sh|ps1)
// See: CASSANDRA-11540 & CASSANDRA-11725
if (COM_SUN_MANAGEMENT_JMXREMOTE_PORT.isPresent())
{
logger.warn("JMX settings in cassandra-env.sh have been bypassed as the JMX connector server is " +
"already initialized. Please refer to cassandra-env.(sh|ps1) for JMX configuration info");
return;
}
JAVA_RMI_SERVER_RANDOM_ID.setBoolean(true);
// If a remote port has been specified then use that to set up a JMX
// connector server which can be accessed remotely. Otherwise, look
// for the local port property and create a server which is bound
// only to the loopback address. Auth options are applied to both
// remote and local-only servers, but currently SSL is only
// available for remote.
// If neither is remote nor local port is set in cassandra-env.(sh|ps)
// then JMX is effectively disabled.
boolean localOnly = false;
String jmxPort = CASSANDRA_JMX_REMOTE_PORT.getString();
if (jmxPort == null)
{
localOnly = true;
jmxPort = CASSANDRA_JMX_LOCAL_PORT.getString();
}
if (jmxPort == null)
return;
try
{
jmxServer = JMXServerUtils.createJMXServer(Integer.parseInt(jmxPort), localOnly);
if (jmxServer == null)
return;
}
catch (IOException e)
{
exitOrFail(1, e.getMessage(), e.getCause());
}
}
@VisibleForTesting
public static Runnable SPECULATION_THRESHOLD_UPDATER =
() ->
{
try
{
Keyspace.allExisting().forEach(k -> k.getColumnFamilyStores().forEach(ColumnFamilyStore::updateSpeculationThreshold));
}
catch (Throwable t)
{
logger.warn("Failed to update speculative retry thresholds.", t);
JVMStabilityInspector.inspectThrowable(t);
}
};
static final CassandraDaemon instance = new CassandraDaemon();
private volatile NativeTransportService nativeTransportService;
private JMXConnectorServer jmxServer;
private final boolean runManaged;
protected final StartupChecks startupChecks;
private boolean setupCompleted;
public CassandraDaemon()
{
this(false);
}
public CassandraDaemon(boolean runManaged)
{
this.runManaged = runManaged;
this.startupChecks = new StartupChecks().withDefaultTests().withTest(new FileSystemOwnershipCheck());
this.setupCompleted = false;
}
/**
* This is a hook for concrete daemons to initialize themselves suitably.
*
* Subclasses should override this to finish the job (listening on ports, etc.)
*/
protected void setup()
{
FileUtils.setFSErrorHandler(new DefaultFSErrorHandler());
// Since CASSANDRA-14793 the local system keyspaces data are not dispatched across the data directories
// anymore to reduce the risks in case of disk failures. By consequence, the system need to ensure in case of
// upgrade that the old data files have been migrated to the new directories before we start deleting
// snapshots and upgrading system tables.
try
{
migrateSystemDataIfNeeded();
}
catch (IOException e)
{
exitOrFail(StartupException.ERR_WRONG_DISK_STATE, e.getMessage(), e);
}
maybeInitJmx();
Mx4jTool.maybeLoad();
ThreadAwareSecurityManager.install();
logSystemInfo(logger);
NativeLibrary.tryMlockall();
CommitLog.instance.start();
runStartupChecks();
try
{
SystemKeyspace.snapshotOnVersionChange();
}
catch (IOException e)
{
exitOrFail(StartupException.ERR_WRONG_DISK_STATE, e.getMessage(), e.getCause());
}
// We need to persist this as soon as possible after startup checks.
// This should be the first write to SystemKeyspace (CASSANDRA-11742)
SystemKeyspace.persistLocalMetadata();
Thread.setDefaultUncaughtExceptionHandler(JVMStabilityInspector::uncaughtException);
SystemKeyspaceMigrator41.migrate();
// Populate token metadata before flushing, for token-aware sstable partitioning (#6696)
StorageService.instance.populateTokenMetadata();
try
{
// load schema from disk
Schema.instance.loadFromDisk();
}
catch (Exception e)
{
logger.error("Error while loading schema: ", e);
throw e;
}
setupVirtualKeyspaces();
try
{
scrubDataDirectories();
}
catch (StartupException e)
{
exitOrFail(e.returnCode, e.getMessage(), e.getCause());
}
Keyspace.setInitialized();
// initialize keyspaces
for (String keyspaceName : Schema.instance.getKeyspaces())
{
if (logger.isDebugEnabled())
logger.debug("opening keyspace {}", keyspaceName);
// disable auto compaction until gossip settles since disk boundaries may be affected by ring layout
for (ColumnFamilyStore cfs : Keyspace.open(keyspaceName).getColumnFamilyStores())
{
for (ColumnFamilyStore store : cfs.concatWithIndexes())
{
store.disableAutoCompaction();
}
}
}
try
{
loadRowAndKeyCacheAsync().get();
}
catch (Throwable t)
{
JVMStabilityInspector.inspectThrowable(t);
logger.warn("Error loading key or row cache", t);
}
try
{
GCInspector.register();
}
catch (Throwable t)
{
JVMStabilityInspector.inspectThrowable(t);
logger.warn("Unable to start GCInspector (currently only supported on the Sun JVM)");
}
// Replay any CommitLogSegments found on disk
PaxosState.initializeTrackers();
// replay the log if necessary
try
{
CommitLog.instance.recoverSegmentsOnDisk();
}
catch (IOException e)
{
throw new RuntimeException(e);
}
// Re-populate token metadata after commit log recover (new peers might be loaded onto system keyspace #10293)
StorageService.instance.populateTokenMetadata();
try
{
PaxosState.maybeRebuildUncommittedState();
}
catch (IOException e)
{
throw new RuntimeException(e);
}
// Clean up system.size_estimates entries left lying around from missed keyspace drops (CASSANDRA-14905)
StorageService.instance.cleanupSizeEstimates();
// schedule periodic dumps of table size estimates into SystemKeyspace.SIZE_ESTIMATES_CF
// set cassandra.size_recorder_interval to 0 to disable
int sizeRecorderInterval = SIZE_RECORDER_INTERVAL.getInt();
if (sizeRecorderInterval > 0)
ScheduledExecutors.optionalTasks.scheduleWithFixedDelay(SizeEstimatesRecorder.instance, 30, sizeRecorderInterval, TimeUnit.SECONDS);
ActiveRepairService.instance.start();
StreamManager.instance.start();
// Prepared statements
QueryProcessor.instance.preloadPreparedStatements();
// Metrics
String metricsReporterConfigFile = METRICS_REPORTER_CONFIG_FILE.getString();
if (metricsReporterConfigFile != null)
{
logger.info("Trying to load metrics-reporter-config from file: {}", metricsReporterConfigFile);
try
{
// enable metrics provided by metrics-jvm.jar
CassandraMetricsRegistry.Metrics.register("jvm.buffers", new BufferPoolMetricSet(ManagementFactory.getPlatformMBeanServer()));
CassandraMetricsRegistry.Metrics.register("jvm.gc", new GarbageCollectorMetricSet());
CassandraMetricsRegistry.Metrics.register("jvm.memory", new MemoryUsageGaugeSet());
CassandraMetricsRegistry.Metrics.register("jvm.fd.usage", new FileDescriptorRatioGauge());
// initialize metrics-reporter-config from yaml file
URL resource = CassandraDaemon.class.getClassLoader().getResource(metricsReporterConfigFile);
if (resource == null)
{
logger.warn("Failed to load metrics-reporter-config, file does not exist: {}", metricsReporterConfigFile);
}
else
{
String reportFileLocation = resource.getFile();
ReporterConfig.loadFromFile(reportFileLocation).enableAll(CassandraMetricsRegistry.Metrics);
}
}
catch (Exception e)
{
logger.warn("Failed to load metrics-reporter-config, metric sinks will not be activated", e);
}
}
// start server internals
StorageService.instance.registerDaemon(this);
try
{
StorageService.instance.initServer();
}
catch (ConfigurationException e)
{
System.err.println(e.getMessage() + "\nFatal configuration error; unable to start server. See log for stacktrace.");
exitOrFail(1, "Fatal configuration error", e);
}
// Because we are writing to the system_distributed keyspace, this should happen after that is created, which
// happens in StorageService.instance.initServer()
Runnable viewRebuild = () -> {
for (Keyspace keyspace : Keyspace.all())
{
keyspace.viewManager.buildAllViews();
}
logger.debug("Completed submission of build tasks for any materialized views defined at startup");
};
ScheduledExecutors.optionalTasks.schedule(viewRebuild, StorageService.RING_DELAY_MILLIS, TimeUnit.MILLISECONDS);
if (!FBUtilities.getBroadcastAddressAndPort().equals(InetAddressAndPort.getLoopbackAddress()))
Gossiper.waitToSettle();
StorageService.instance.doAuthSetup(false);
// re-enable auto-compaction after gossip is settled, so correct disk boundaries are used
for (Keyspace keyspace : Keyspace.all())
{
for (ColumnFamilyStore cfs : keyspace.getColumnFamilyStores())
{
for (final ColumnFamilyStore store : cfs.concatWithIndexes())
{
store.reload(); //reload CFs in case there was a change of disk boundaries
if (store.getCompactionStrategyManager().shouldBeEnabled())
{
if (DatabaseDescriptor.getAutocompactionOnStartupEnabled())
{
store.enableAutoCompaction();
}
else
{
logger.info("Not enabling compaction for {}.{}; autocompaction_on_startup_enabled is set to false", store.getKeyspaceName(), store.name);
}
}
}
}
}
AuditLogManager.instance.initialize();
// schedule periodic background compaction task submission. this is simply a backstop against compactions stalling
// due to scheduling errors or race conditions
ScheduledExecutors.optionalTasks.scheduleWithFixedDelay(ColumnFamilyStore.getBackgroundCompactionTaskSubmitter(), 5, 1, TimeUnit.MINUTES);
// schedule periodic recomputation of speculative retry thresholds
ScheduledExecutors.optionalTasks.scheduleWithFixedDelay(SPECULATION_THRESHOLD_UPDATER,
DatabaseDescriptor.getReadRpcTimeout(NANOSECONDS),
DatabaseDescriptor.getReadRpcTimeout(NANOSECONDS),
NANOSECONDS);
initializeClientTransports();
// Ensure you've registered all caches during startup you want pre-warmed before this call -> be wary of adding
// init below this mark before completeSetup().
if (DatabaseDescriptor.getAuthCacheWarmingEnabled())
AuthCacheService.instance.warmCaches();
else
logger.info("Prewarming of auth caches is disabled");
PaxosState.startAutoRepairs();
completeSetup();
}
public void runStartupChecks()
{
try
{
startupChecks.verify(DatabaseDescriptor.getStartupChecksOptions());
}
catch (StartupException e)
{
exitOrFail(e.returnCode, e.getMessage(), e.getCause());
}
}
/**
* Checks if the data of the local system keyspaces need to be migrated to a different location.
*
* @throws IOException
*/
public void migrateSystemDataIfNeeded() throws IOException
{
// If there is only one directory and no system keyspace directory has been specified we do not need to do
// anything. If it is not the case we want to try to migrate the data.
if (!DatabaseDescriptor.useSpecificLocationForLocalSystemData()
&& DatabaseDescriptor.getNonLocalSystemKeyspacesDataFileLocations().length <= 1)
return;
// We can face several cases:
// 1) The system data are spread accross the data file locations and need to be moved to
// the first data location (upgrade to 4.0)
// 2) The system data are spread accross the data file locations and need to be moved to
// the system keyspace location configured by the user (upgrade to 4.0)
// 3) The system data are stored in the first data location and need to be moved to
// the system keyspace location configured by the user (system_data_file_directory has been configured)
Path target = File.getPath(DatabaseDescriptor.getLocalSystemKeyspacesDataFileLocations()[0]);
String[] nonLocalSystemKeyspacesFileLocations = DatabaseDescriptor.getNonLocalSystemKeyspacesDataFileLocations();
String[] sources = DatabaseDescriptor.useSpecificLocationForLocalSystemData() ? nonLocalSystemKeyspacesFileLocations
: Arrays.copyOfRange(nonLocalSystemKeyspacesFileLocations,
1,
nonLocalSystemKeyspacesFileLocations.length);
for (String source : sources)
{
Path dataFileLocation = File.getPath(source);
if (!Files.exists(dataFileLocation))
continue;
try (Stream<Path> locationChildren = Files.list(dataFileLocation))
{
Path[] keyspaceDirectories = locationChildren.filter(p -> SchemaConstants.isLocalSystemKeyspace(p.getFileName().toString()))
.toArray(Path[]::new);
for (Path keyspaceDirectory : keyspaceDirectories)
{
try (Stream<Path> keyspaceChildren = Files.list(keyspaceDirectory))
{
Path[] tableDirectories = keyspaceChildren.filter(Files::isDirectory)
.filter(p -> !SystemKeyspace.TABLES_SPLIT_ACROSS_MULTIPLE_DISKS
.contains(p.getFileName()
.toString()))
.toArray(Path[]::new);
for (Path tableDirectory : tableDirectories)
{
FileUtils.moveRecursively(tableDirectory,
target.resolve(dataFileLocation.relativize(tableDirectory)));
}
if (!SchemaConstants.SYSTEM_KEYSPACE_NAME.equals(keyspaceDirectory.getFileName().toString()))
{
FileUtils.deleteDirectoryIfEmpty(keyspaceDirectory);
}
}
}
}
}
}
public void setupVirtualKeyspaces()
{
VirtualKeyspaceRegistry.instance.register(VirtualSchemaKeyspace.instance);
VirtualKeyspaceRegistry.instance.register(SystemViewsKeyspace.instance);
// flush log messages to system_views.system_logs virtual table as there were messages already logged
// before that virtual table was instantiated
LoggingSupportFactory.getLoggingSupport()
.getAppender(VirtualTableAppender.class, VirtualTableAppender.APPENDER_NAME)
.ifPresent(appender -> ((VirtualTableAppender) appender).flushBuffer());
}
public void scrubDataDirectories() throws StartupException
{
// clean up debris in the rest of the keyspaces
for (String keyspaceName : Schema.instance.getKeyspaces())
{
// Skip system as we've already cleaned it
if (keyspaceName.equals(SchemaConstants.SYSTEM_KEYSPACE_NAME))
continue;
for (TableMetadata cfm : Schema.instance.getTablesAndViews(keyspaceName))
{
ColumnFamilyStore.scrubDataDirectories(cfm);
}
}
}
public synchronized void initializeClientTransports()
{
// Native transport
if (nativeTransportService == null)
nativeTransportService = new NativeTransportService();
}
/*
* Asynchronously load the row and key cache in one off threads and return a compound future of the result.
* Error handling is pushed into the cache load since cache loads are allowed to fail and are handled by logging.
*/
private Future<?> loadRowAndKeyCacheAsync()
{
final Future<Integer> keyCacheLoad = CacheService.instance.keyCache.loadSavedAsync();
final Future<Integer> rowCacheLoad = CacheService.instance.rowCache.loadSavedAsync();
@SuppressWarnings("unchecked")
Future<List<Integer>> retval = FutureCombiner.allOf(ImmutableList.of(keyCacheLoad, rowCacheLoad));
return retval;
}
@VisibleForTesting
public void completeSetup()
{
setupCompleted = true;
}
public boolean setupCompleted()
{
return setupCompleted;
}
public static void logSystemInfo(Logger logger)
{
if (logger.isInfoEnabled())
{
try
{
logger.info("Hostname: {}", InetAddress.getLocalHost().getHostName() + ":" + DatabaseDescriptor.getStoragePort() + ":" + DatabaseDescriptor.getSSLStoragePort());
}
catch (UnknownHostException e1)
{
logger.info("Could not resolve local host");
}
logger.info("JVM vendor/version: {}/{}", JAVA_VM_NAME.getString(), JAVA_VERSION.getString());
logger.info("Heap size: {}/{}",
FBUtilities.prettyPrintMemory(Runtime.getRuntime().totalMemory()),
FBUtilities.prettyPrintMemory(Runtime.getRuntime().maxMemory()));
for(MemoryPoolMXBean pool: ManagementFactory.getMemoryPoolMXBeans())
logger.info("{} {}: {}", pool.getName(), pool.getType(), pool.getPeakUsage());
logger.info("Classpath: {}", JAVA_CLASS_PATH.getString());
logger.info("JVM Arguments: {}", ManagementFactory.getRuntimeMXBean().getInputArguments());
}
}
/**
* Initialize the Cassandra Daemon based on the given <a
* href="http://commons.apache.org/daemon/jsvc.html">Commons
* Daemon</a>-specific arguments. To clarify, this is a hook for JSVC.
*
* @param arguments
* the arguments passed in from JSVC
* @throws IOException
*/
public void init(String[] arguments) throws IOException
{
setup();
}
/**
* Start the Cassandra Daemon, assuming that it has already been
* initialized via {@link #init(String[])}
*
* Hook for JSVC
*/
public void start()
{
StartupClusterConnectivityChecker connectivityChecker = StartupClusterConnectivityChecker.create(DatabaseDescriptor.getBlockForPeersTimeoutInSeconds(),
DatabaseDescriptor.getBlockForPeersInRemoteDatacenters());
connectivityChecker.execute(Gossiper.instance.getEndpoints(), DatabaseDescriptor.getEndpointSnitch()::getDatacenter);
// check to see if transports may start else return without starting. This is needed when in survey mode or
// when bootstrap has not completed.
try
{
validateTransportsCanStart();
}
catch (IllegalStateException isx)
{
// If there are any errors, we just log and return in this case
logger.warn(isx.getMessage());
return;
}
startClientTransports();
}
private void startClientTransports()
{
String nativeFlag = START_NATIVE_TRANSPORT.getString();
if (START_NATIVE_TRANSPORT.getBoolean() || (nativeFlag == null && DatabaseDescriptor.startNativeTransport()))
{
startNativeTransport();
StorageService.instance.setRpcReady(true);
}
else
logger.info("Not starting native transport as requested. Use JMX (StorageService->startNativeTransport()) or nodetool (enablebinary) to start it");
}
/**
* Stop the daemon, ideally in an idempotent manner.
*
* Hook for JSVC / Procrun
*/
public void stop()
{
// On linux, this doesn't entirely shut down Cassandra, just the RPC server.
// jsvc takes care of taking the rest down
logger.info("Cassandra shutting down...");
destroyClientTransports();
StorageService.instance.setRpcReady(false);
if (jmxServer != null)
{
try
{
jmxServer.stop();
}
catch (IOException e)
{
logger.error("Error shutting down local JMX server: ", e);
}
}
}
@VisibleForTesting
public void destroyClientTransports()
{
stopNativeTransport();
if (nativeTransportService != null)
nativeTransportService.destroy();
}
/**
* Clean up all resources obtained during the lifetime of the daemon. This
* is a hook for JSVC.
*/
public void destroy()
{}
/**
* A convenience method to initialize and start the daemon in one shot.
*/
public void activate()
{
// Do not put any references to DatabaseDescriptor above the forceStaticInitialization call.
try
{
applyConfig();
registerNativeAccess();
setup();
String pidFile = CASSANDRA_PID_FILE.getString();
if (pidFile != null)
{
new File(pidFile).deleteOnExit();
}
if (CASSANDRA_FOREGROUND.getString() == null)
{
System.out.close();
System.err.close();
}
start();
logger.info("Startup complete");
}
catch (Throwable e)
{
boolean logStackTrace =
e instanceof ConfigurationException ? ((ConfigurationException)e).logStackTrace : true;
System.out.println("Exception (" + e.getClass().getName() + ") encountered during startup: " + e.getMessage());
if (logStackTrace)
{
if (runManaged)
logger.error("Exception encountered during startup", e);
// try to warn user on stdout too, if we haven't already detached
e.printStackTrace();
exitOrFail(3, "Exception encountered during startup", e);
}
else
{
if (runManaged)
logger.error("Exception encountered during startup: {}", e.getMessage());
// try to warn user on stdout too, if we haven't already detached
System.err.println(e.getMessage());
exitOrFail(3, "Exception encountered during startup: " + e.getMessage());
}
}
}
@VisibleForTesting
public static void registerNativeAccess() throws javax.management.NotCompliantMBeanException
{
MBeanWrapper.instance.registerMBean(new StandardMBean(new NativeAccess(), NativeAccessMBean.class), MBEAN_NAME, MBeanWrapper.OnException.LOG);
}
public void applyConfig()
{
DatabaseDescriptor.daemonInitialization();
}
public void validateTransportsCanStart()
{
// We only start transports if bootstrap has completed and we're not in survey mode, OR if we are in
// survey mode and streaming has completed but we're not using auth.
// OR if we have not joined the ring yet.
if (StorageService.instance.hasJoined())
{
if (StorageService.instance.isSurveyMode())
{
if (StorageService.instance.isBootstrapMode() || DatabaseDescriptor.getAuthenticator().requireAuthentication())
{
throw new IllegalStateException("Not starting client transports in write_survey mode as it's bootstrapping or " +
"auth is enabled");
}
}
else
{
if (!SystemKeyspace.bootstrapComplete())
{
throw new IllegalStateException("Node is not yet bootstrapped completely. Use nodetool to check bootstrap" +
" state and resume. For more, see `nodetool help bootstrap`");
}
}
}
}
public void startNativeTransport()
{
validateTransportsCanStart();
if (nativeTransportService == null)
throw new IllegalStateException("setup() must be called first for CassandraDaemon");
nativeTransportService.start();
}
public void stopNativeTransport()
{
if (nativeTransportService != null)
nativeTransportService.stop();
}
public boolean isNativeTransportRunning()
{
return nativeTransportService != null && nativeTransportService.isRunning();
}
/**
* A convenience method to stop and destroy the daemon in one shot.
*/
public void deactivate()
{
stop();
destroy();
// completely shut down cassandra
if(!runManaged)
{
System.exit(0);
}
}
public static void stop(String[] args)
{
instance.deactivate();
}
public static void main(String[] args)
{
instance.activate();
}
public void clearConnectionHistory()
{
nativeTransportService.clearConnectionHistory();
}
private void exitOrFail(int code, String message)
{
exitOrFail(code, message, null);
}
private void exitOrFail(int code, String message, Throwable cause)
{
if (runManaged)
{
RuntimeException t = cause!=null ? new RuntimeException(message, cause) : new RuntimeException(message);
throw t;
}
else
{
logger.error(message, cause);
System.exit(code);
}
}
static class NativeAccess implements NativeAccessMBean
{
public boolean isAvailable()
{
return NativeLibrary.isAvailable();
}
public boolean isMemoryLockable()
{
return NativeLibrary.jnaMemoryLockable();
}
}
public interface Server
{
/**
* Start the server.
* This method shoud be able to restart a server stopped through stop().
* Should throw a RuntimeException if the server cannot be started
*/
public void start();
/**
* Stop the server.
* This method should be able to stop server started through start().
* Should throw a RuntimeException if the server cannot be stopped
*/
public void stop();
/**
* Returns whether the server is currently running.
*/
public boolean isRunning();
public void clearConnectionHistory();
}
}