forked from apache/cassandra
-
Notifications
You must be signed in to change notification settings - Fork 1
/
ColumnFamilyStore.java
2646 lines (2320 loc) · 108 KB
/
ColumnFamilyStore.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.db;
import java.io.File;
import java.io.IOException;
import java.io.PrintStream;
import java.lang.management.ManagementFactory;
import java.nio.ByteBuffer;
import java.nio.file.Files;
import java.util.*;
import java.util.concurrent.*;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.regex.Pattern;
import javax.management.*;
import javax.management.openmbean.*;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.*;
import com.google.common.base.Throwables;
import com.google.common.collect.*;
import com.google.common.util.concurrent.*;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.clearspring.analytics.stream.Counter;
import org.apache.cassandra.cache.*;
import org.apache.cassandra.concurrent.*;
import org.apache.cassandra.config.*;
import org.apache.cassandra.db.commitlog.CommitLog;
import org.apache.cassandra.db.commitlog.CommitLogPosition;
import org.apache.cassandra.db.compaction.*;
import org.apache.cassandra.db.filter.ClusteringIndexFilter;
import org.apache.cassandra.db.filter.DataLimits;
import org.apache.cassandra.db.view.TableViews;
import org.apache.cassandra.db.lifecycle.*;
import org.apache.cassandra.db.partitions.CachedPartition;
import org.apache.cassandra.db.partitions.PartitionUpdate;
import org.apache.cassandra.db.rows.CellPath;
import org.apache.cassandra.dht.*;
import org.apache.cassandra.dht.Range;
import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.exceptions.StartupException;
import org.apache.cassandra.index.SecondaryIndexManager;
import org.apache.cassandra.index.internal.CassandraIndex;
import org.apache.cassandra.index.transactions.UpdateTransaction;
import org.apache.cassandra.io.FSError;
import org.apache.cassandra.io.FSWriteError;
import org.apache.cassandra.io.sstable.Component;
import org.apache.cassandra.io.sstable.CorruptSSTableException;
import org.apache.cassandra.io.sstable.Descriptor;
import org.apache.cassandra.io.sstable.SSTableMultiWriter;
import org.apache.cassandra.io.sstable.format.*;
import org.apache.cassandra.io.sstable.format.big.BigFormat;
import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
import org.apache.cassandra.io.util.FileUtils;
import org.apache.cassandra.metrics.TableMetrics;
import org.apache.cassandra.metrics.TableMetrics.Sampler;
import org.apache.cassandra.schema.*;
import org.apache.cassandra.schema.CompactionParams.TombstoneOption;
import org.apache.cassandra.service.CacheService;
import org.apache.cassandra.service.StorageService;
import org.apache.cassandra.utils.*;
import org.apache.cassandra.utils.TopKSampler.SamplerResult;
import org.apache.cassandra.utils.concurrent.OpOrder;
import org.apache.cassandra.utils.concurrent.Refs;
import org.apache.cassandra.utils.memory.MemtableAllocator;
import org.json.simple.JSONArray;
import org.json.simple.JSONObject;
import static org.apache.cassandra.utils.Throwables.maybeFail;
public class ColumnFamilyStore implements ColumnFamilyStoreMBean
{
// The directories which will be searched for sstables on cfs instantiation.
private static volatile Directories.DataDirectory[] initialDirectories = Directories.dataDirectories;
/**
* A hook to add additional directories to initialDirectories.
* Any additional directories should be added prior to ColumnFamilyStore instantiation on startup
*
* Since the directories used by a given table are determined by the compaction strategy,
* it's possible for sstables to be written to directories specified outside of cassandra.yaml.
* By adding additional directories to initialDirectories, sstables in these extra locations are
* made discoverable on sstable instantiation.
*/
public static synchronized void addInitialDirectories(Directories.DataDirectory[] newDirectories)
{
assert newDirectories != null;
Set<Directories.DataDirectory> existing = Sets.newHashSet(initialDirectories);
List<Directories.DataDirectory> replacementList = Lists.newArrayList(initialDirectories);
for (Directories.DataDirectory directory: newDirectories)
{
if (!existing.contains(directory))
{
replacementList.add(directory);
}
}
Directories.DataDirectory[] replacementArray = new Directories.DataDirectory[replacementList.size()];
replacementList.toArray(replacementArray);
initialDirectories = replacementArray;
}
public static Directories.DataDirectory[] getInitialDirectories()
{
Directories.DataDirectory[] src = initialDirectories;
return Arrays.copyOf(src, src.length);
}
private static final Logger logger = LoggerFactory.getLogger(ColumnFamilyStore.class);
/*
We keep a pool of threads for each data directory, size of each pool is memtable_flush_writers.
When flushing we start a Flush runnable in the flushExecutor. Flush calculates how to split the
memtable ranges over the existing data directories and creates a FlushRunnable for each of the directories.
The FlushRunnables are executed in the perDiskflushExecutors and the Flush will block until all FlushRunnables
are finished. By having flushExecutor size the same size as each of the perDiskflushExecutors we make sure we can
have that many flushes going at the same time.
*/
private static final ExecutorService flushExecutor = new JMXEnabledThreadPoolExecutor(DatabaseDescriptor.getFlushWriters(),
StageManager.KEEPALIVE,
TimeUnit.SECONDS,
new LinkedBlockingQueue<Runnable>(),
new NamedThreadFactory("MemtableFlushWriter"),
"internal");
private static final ExecutorService [] perDiskflushExecutors = new ExecutorService[DatabaseDescriptor.getAllDataFileLocations().length];
static
{
for (int i = 0; i < DatabaseDescriptor.getAllDataFileLocations().length; i++)
{
perDiskflushExecutors[i] = new JMXEnabledThreadPoolExecutor(DatabaseDescriptor.getFlushWriters(),
StageManager.KEEPALIVE,
TimeUnit.SECONDS,
new LinkedBlockingQueue<Runnable>(),
new NamedThreadFactory("PerDiskMemtableFlushWriter_"+i),
"internal");
}
}
// post-flush executor is single threaded to provide guarantee that any flush Future on a CF will never return until prior flushes have completed
private static final ExecutorService postFlushExecutor = new JMXEnabledThreadPoolExecutor(1,
StageManager.KEEPALIVE,
TimeUnit.SECONDS,
new LinkedBlockingQueue<Runnable>(),
new NamedThreadFactory("MemtablePostFlush"),
"internal");
private static final ExecutorService reclaimExecutor = new JMXEnabledThreadPoolExecutor(1,
StageManager.KEEPALIVE,
TimeUnit.SECONDS,
new LinkedBlockingQueue<Runnable>(),
new NamedThreadFactory("MemtableReclaimMemory"),
"internal");
private static final String[] COUNTER_NAMES = new String[]{"raw", "count", "error", "string"};
private static final String[] COUNTER_DESCS = new String[]
{ "partition key in raw hex bytes",
"value of this partition for given sampler",
"value is within the error bounds plus or minus of this",
"the partition key turned into a human readable format" };
private static final CompositeType COUNTER_COMPOSITE_TYPE;
private static final TabularType COUNTER_TYPE;
private static final String[] SAMPLER_NAMES = new String[]{"cardinality", "partitions"};
private static final String[] SAMPLER_DESCS = new String[]
{ "cardinality of partitions",
"list of counter results" };
private static final String SAMPLING_RESULTS_NAME = "SAMPLING_RESULTS";
private static final CompositeType SAMPLING_RESULT;
public static final String SNAPSHOT_TRUNCATE_PREFIX = "truncated";
public static final String SNAPSHOT_DROP_PREFIX = "dropped";
static
{
try
{
OpenType<?>[] counterTypes = new OpenType[] { SimpleType.STRING, SimpleType.LONG, SimpleType.LONG, SimpleType.STRING };
COUNTER_COMPOSITE_TYPE = new CompositeType(SAMPLING_RESULTS_NAME, SAMPLING_RESULTS_NAME, COUNTER_NAMES, COUNTER_DESCS, counterTypes);
COUNTER_TYPE = new TabularType(SAMPLING_RESULTS_NAME, SAMPLING_RESULTS_NAME, COUNTER_COMPOSITE_TYPE, COUNTER_NAMES);
OpenType<?>[] samplerTypes = new OpenType[] { SimpleType.LONG, COUNTER_TYPE };
SAMPLING_RESULT = new CompositeType(SAMPLING_RESULTS_NAME, SAMPLING_RESULTS_NAME, SAMPLER_NAMES, SAMPLER_DESCS, samplerTypes);
} catch (OpenDataException e)
{
throw Throwables.propagate(e);
}
}
public final Keyspace keyspace;
public final String name;
public final CFMetaData metadata;
private final String mbeanName;
@Deprecated
private final String oldMBeanName;
private volatile boolean valid = true;
/**
* Memtables and SSTables on disk for this column family.
*
* We synchronize on the Tracker to ensure isolation when we want to make sure
* that the memtable we're acting on doesn't change out from under us. I.e., flush
* syncronizes on it to make sure it can submit on both executors atomically,
* so anyone else who wants to make sure flush doesn't interfere should as well.
*/
private final Tracker data;
/* The read order, used to track accesses to off-heap memtable storage */
public final OpOrder readOrdering = new OpOrder();
/* This is used to generate the next index for a SSTable */
private final AtomicInteger fileIndexGenerator = new AtomicInteger(0);
public final SecondaryIndexManager indexManager;
public final TableViews viewManager;
/* These are locally held copies to be changed from the config during runtime */
private volatile DefaultValue<Integer> minCompactionThreshold;
private volatile DefaultValue<Integer> maxCompactionThreshold;
private volatile DefaultValue<Double> crcCheckChance;
private final CompactionStrategyManager compactionStrategyManager;
private volatile Directories directories;
public final TableMetrics metric;
public volatile long sampleLatencyNanos;
private final ScheduledFuture<?> latencyCalculator;
private volatile boolean compactionSpaceCheck = true;
public static void shutdownPostFlushExecutor() throws InterruptedException
{
postFlushExecutor.shutdown();
postFlushExecutor.awaitTermination(60, TimeUnit.SECONDS);
}
public void reload()
{
// metadata object has been mutated directly. make all the members jibe with new settings.
// only update these runtime-modifiable settings if they have not been modified.
if (!minCompactionThreshold.isModified())
for (ColumnFamilyStore cfs : concatWithIndexes())
cfs.minCompactionThreshold = new DefaultValue(metadata.params.compaction.minCompactionThreshold());
if (!maxCompactionThreshold.isModified())
for (ColumnFamilyStore cfs : concatWithIndexes())
cfs.maxCompactionThreshold = new DefaultValue(metadata.params.compaction.maxCompactionThreshold());
if (!crcCheckChance.isModified())
for (ColumnFamilyStore cfs : concatWithIndexes())
cfs.crcCheckChance = new DefaultValue(metadata.params.crcCheckChance);
compactionStrategyManager.maybeReload(metadata);
scheduleFlush();
indexManager.reload();
// If the CF comparator has changed, we need to change the memtable,
// because the old one still aliases the previous comparator.
if (data.getView().getCurrentMemtable().initialComparator != metadata.comparator)
switchMemtable();
}
void scheduleFlush()
{
int period = metadata.params.memtableFlushPeriodInMs;
if (period > 0)
{
logger.trace("scheduling flush in {} ms", period);
WrappedRunnable runnable = new WrappedRunnable()
{
protected void runMayThrow()
{
synchronized (data)
{
Memtable current = data.getView().getCurrentMemtable();
// if we're not expired, we've been hit by a scheduled flush for an already flushed memtable, so ignore
if (current.isExpired())
{
if (current.isClean())
{
// if we're still clean, instead of swapping just reschedule a flush for later
scheduleFlush();
}
else
{
// we'll be rescheduled by the constructor of the Memtable.
forceFlush();
}
}
}
}
};
ScheduledExecutors.scheduledTasks.schedule(runnable, period, TimeUnit.MILLISECONDS);
}
}
public static Runnable getBackgroundCompactionTaskSubmitter()
{
return new Runnable()
{
public void run()
{
for (Keyspace keyspace : Keyspace.all())
for (ColumnFamilyStore cfs : keyspace.getColumnFamilyStores())
CompactionManager.instance.submitBackground(cfs);
}
};
}
public void setCompactionParametersJson(String options)
{
setCompactionParameters(FBUtilities.fromJsonMap(options));
}
public String getCompactionParametersJson()
{
return FBUtilities.json(getCompactionParameters());
}
public void setCompactionParameters(Map<String, String> options)
{
try
{
CompactionParams compactionParams = CompactionParams.fromMap(options);
compactionParams.validate();
compactionStrategyManager.setNewLocalCompactionStrategy(compactionParams);
}
catch (Throwable t)
{
logger.error("Could not set new local compaction strategy", t);
// dont propagate the ConfigurationException over jmx, user will only see a ClassNotFoundException
throw new IllegalArgumentException("Could not set new local compaction strategy: "+t.getMessage());
}
}
public Map<String, String> getCompactionParameters()
{
return compactionStrategyManager.getCompactionParams().asMap();
}
public Map<String,String> getCompressionParameters()
{
return metadata.params.compression.asMap();
}
public void setCompressionParameters(Map<String,String> opts)
{
try
{
metadata.compression(CompressionParams.fromMap(opts));
metadata.params.compression.validate();
}
catch (ConfigurationException e)
{
throw new IllegalArgumentException(e.getMessage());
}
}
@VisibleForTesting
public ColumnFamilyStore(Keyspace keyspace,
String columnFamilyName,
int generation,
CFMetaData metadata,
Directories directories,
boolean loadSSTables,
boolean registerBookeeping,
boolean offline)
{
assert directories != null;
assert metadata != null : "null metadata for " + keyspace + ":" + columnFamilyName;
this.keyspace = keyspace;
this.metadata = metadata;
name = columnFamilyName;
minCompactionThreshold = new DefaultValue<>(metadata.params.compaction.minCompactionThreshold());
maxCompactionThreshold = new DefaultValue<>(metadata.params.compaction.maxCompactionThreshold());
crcCheckChance = new DefaultValue<>(metadata.params.crcCheckChance);
indexManager = new SecondaryIndexManager(this);
viewManager = keyspace.viewManager.forTable(metadata);
metric = new TableMetrics(this);
fileIndexGenerator.set(generation);
sampleLatencyNanos = TimeUnit.MILLISECONDS.toNanos(DatabaseDescriptor.getReadRpcTimeout() / 2);
logger.info("Initializing {}.{}", keyspace.getName(), name);
// Create Memtable only on online
Memtable initialMemtable = null;
if (DatabaseDescriptor.isDaemonInitialized())
initialMemtable = new Memtable(new AtomicReference<>(CommitLog.instance.getCurrentPosition()), this);
data = new Tracker(initialMemtable, loadSSTables);
// scan for sstables corresponding to this cf and load them
if (data.loadsstables)
{
Directories.SSTableLister sstableFiles = directories.sstableLister(Directories.OnTxnErr.IGNORE).skipTemporary(true);
Collection<SSTableReader> sstables = SSTableReader.openAll(sstableFiles.list().entrySet(), metadata);
data.addInitialSSTables(sstables);
}
/**
* When creating a CFS offline we change the default logic needed by CASSANDRA-8671
* and link the passed directories to be picked up by the compaction strategy
*/
if (offline)
this.directories = directories;
else
this.directories = new Directories(metadata, Directories.dataDirectories);
// compaction strategy should be created after the CFS has been prepared
compactionStrategyManager = new CompactionStrategyManager(this);
if (maxCompactionThreshold.value() <= 0 || minCompactionThreshold.value() <=0)
{
logger.warn("Disabling compaction strategy by setting compaction thresholds to 0 is deprecated, set the compaction option 'enabled' to 'false' instead.");
this.compactionStrategyManager.disable();
}
// create the private ColumnFamilyStores for the secondary column indexes
for (IndexMetadata info : metadata.getIndexes())
indexManager.addIndex(info);
if (registerBookeeping)
{
// register the mbean
mbeanName = String.format("org.apache.cassandra.db:type=%s,keyspace=%s,table=%s",
isIndex() ? "IndexTables" : "Tables",
keyspace.getName(), name);
oldMBeanName = String.format("org.apache.cassandra.db:type=%s,keyspace=%s,columnfamily=%s",
isIndex() ? "IndexColumnFamilies" : "ColumnFamilies",
keyspace.getName(), name);
try
{
MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
ObjectName[] objectNames = {new ObjectName(mbeanName), new ObjectName(oldMBeanName)};
for (ObjectName objectName : objectNames)
{
mbs.registerMBean(this, objectName);
}
}
catch (Exception e)
{
throw new RuntimeException(e);
}
logger.trace("retryPolicy for {} is {}", name, this.metadata.params.speculativeRetry);
latencyCalculator = ScheduledExecutors.optionalTasks.scheduleWithFixedDelay(new Runnable()
{
public void run()
{
SpeculativeRetryParam retryPolicy = ColumnFamilyStore.this.metadata.params.speculativeRetry;
switch (retryPolicy.kind())
{
case PERCENTILE:
// get percentile in nanos
sampleLatencyNanos = (long) (metric.coordinatorReadLatency.getSnapshot().getValue(retryPolicy.threshold()));
break;
case CUSTOM:
sampleLatencyNanos = (long) retryPolicy.threshold();
break;
default:
sampleLatencyNanos = Long.MAX_VALUE;
break;
}
}
}, DatabaseDescriptor.getReadRpcTimeout(), DatabaseDescriptor.getReadRpcTimeout(), TimeUnit.MILLISECONDS);
}
else
{
latencyCalculator = ScheduledExecutors.optionalTasks.schedule(Runnables.doNothing(), 0, TimeUnit.NANOSECONDS);
mbeanName = null;
oldMBeanName= null;
}
}
public Directories getDirectories()
{
return directories;
}
public SSTableMultiWriter createSSTableMultiWriter(Descriptor descriptor, long keyCount, long repairedAt, int sstableLevel, SerializationHeader header, LifecycleTransaction txn)
{
MetadataCollector collector = new MetadataCollector(metadata.comparator).sstableLevel(sstableLevel);
return createSSTableMultiWriter(descriptor, keyCount, repairedAt, collector, header, txn);
}
public SSTableMultiWriter createSSTableMultiWriter(Descriptor descriptor, long keyCount, long repairedAt, MetadataCollector metadataCollector, SerializationHeader header, LifecycleTransaction txn)
{
return getCompactionStrategyManager().createSSTableMultiWriter(descriptor, keyCount, repairedAt, metadataCollector, header, indexManager.listIndexes(), txn);
}
public boolean supportsEarlyOpen()
{
return compactionStrategyManager.supportsEarlyOpen();
}
/** call when dropping or renaming a CF. Performs mbean housekeeping and invalidates CFS to other operations */
public void invalidate()
{
invalidate(true);
}
public void invalidate(boolean expectMBean)
{
// disable and cancel in-progress compactions before invalidating
valid = false;
try
{
unregisterMBean();
}
catch (Exception e)
{
if (expectMBean)
{
JVMStabilityInspector.inspectThrowable(e);
// this shouldn't block anything.
logger.warn("Failed unregistering mbean: {}", mbeanName, e);
}
}
latencyCalculator.cancel(false);
compactionStrategyManager.shutdown();
SystemKeyspace.removeTruncationRecord(metadata.cfId);
data.dropSSTables();
LifecycleTransaction.waitForDeletions();
indexManager.invalidateAllIndexesBlocking();
invalidateCaches();
}
/**
* Removes every SSTable in the directory from the Tracker's view.
* @param directory the unreadable directory, possibly with SSTables in it, but not necessarily.
*/
void maybeRemoveUnreadableSSTables(File directory)
{
data.removeUnreadableSSTables(directory);
}
void unregisterMBean() throws MalformedObjectNameException, InstanceNotFoundException, MBeanRegistrationException
{
MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
ObjectName[] objectNames = {new ObjectName(mbeanName), new ObjectName(oldMBeanName)};
for (ObjectName objectName : objectNames)
{
if (mbs.isRegistered(objectName))
mbs.unregisterMBean(objectName);
}
// unregister metrics
metric.release();
}
public static ColumnFamilyStore createColumnFamilyStore(Keyspace keyspace, CFMetaData metadata, boolean loadSSTables)
{
return createColumnFamilyStore(keyspace, metadata.cfName, metadata, loadSSTables);
}
public static synchronized ColumnFamilyStore createColumnFamilyStore(Keyspace keyspace,
String columnFamily,
CFMetaData metadata,
boolean loadSSTables)
{
Directories directories = new Directories(metadata, initialDirectories);
return createColumnFamilyStore(keyspace, columnFamily, metadata, directories, loadSSTables, true, false);
}
/** This is only directly used by offline tools */
public static synchronized ColumnFamilyStore createColumnFamilyStore(Keyspace keyspace,
String columnFamily,
CFMetaData metadata,
Directories directories,
boolean loadSSTables,
boolean registerBookkeeping,
boolean offline)
{
// get the max generation number, to prevent generation conflicts
Directories.SSTableLister lister = directories.sstableLister(Directories.OnTxnErr.IGNORE).includeBackups(true);
List<Integer> generations = new ArrayList<Integer>();
for (Map.Entry<Descriptor, Set<Component>> entry : lister.list().entrySet())
{
Descriptor desc = entry.getKey();
generations.add(desc.generation);
if (!desc.isCompatible())
throw new RuntimeException(String.format("Incompatible SSTable found. Current version %s is unable to read file: %s. Please run upgradesstables.",
desc.getFormat().getLatestVersion(), desc));
}
Collections.sort(generations);
int value = (generations.size() > 0) ? (generations.get(generations.size() - 1)) : 0;
return new ColumnFamilyStore(keyspace, columnFamily, value, metadata, directories, loadSSTables, registerBookkeeping, offline);
}
/**
* Removes unnecessary files from the cf directory at startup: these include temp files, orphans, zero-length files
* and compacted sstables. Files that cannot be recognized will be ignored.
*/
public static void scrubDataDirectories(CFMetaData metadata) throws StartupException
{
Directories directories = new Directories(metadata, initialDirectories);
Set<File> cleanedDirectories = new HashSet<>();
// clear ephemeral snapshots that were not properly cleared last session (CASSANDRA-7357)
clearEphemeralSnapshots(directories);
directories.removeTemporaryDirectories();
logger.trace("Removing temporary or obsoleted files from unfinished operations for table {}", metadata.cfName);
if (!LifecycleTransaction.removeUnfinishedLeftovers(metadata))
throw new StartupException(StartupException.ERR_WRONG_DISK_STATE,
String.format("Cannot remove temporary or obsoleted files for %s.%s due to a problem with transaction " +
"log files. Please check records with problems in the log messages above and fix them. " +
"Refer to the 3.0 upgrading instructions in NEWS.txt " +
"for a description of transaction log files.", metadata.ksName, metadata.cfName));
logger.trace("Further extra check for orphan sstable files for {}", metadata.cfName);
for (Map.Entry<Descriptor,Set<Component>> sstableFiles : directories.sstableLister(Directories.OnTxnErr.IGNORE).list().entrySet())
{
Descriptor desc = sstableFiles.getKey();
File directory = desc.directory;
Set<Component> components = sstableFiles.getValue();
if (!cleanedDirectories.contains(directory))
{
cleanedDirectories.add(directory);
for (File tmpFile : desc.getTemporaryFiles())
tmpFile.delete();
}
File dataFile = new File(desc.filenameFor(Component.DATA));
if (components.contains(Component.DATA) && dataFile.length() > 0)
// everything appears to be in order... moving on.
continue;
// missing the DATA file! all components are orphaned
logger.warn("Removing orphans for {}: {}", desc, components);
for (Component component : components)
{
File file = new File(desc.filenameFor(component));
if (file.exists())
FileUtils.deleteWithConfirm(desc.filenameFor(component));
}
}
// cleanup incomplete saved caches
Pattern tmpCacheFilePattern = Pattern.compile(metadata.ksName + "-" + metadata.cfName + "-(Key|Row)Cache.*\\.tmp$");
File dir = new File(DatabaseDescriptor.getSavedCachesLocation());
if (dir.exists())
{
assert dir.isDirectory();
for (File file : dir.listFiles())
if (tmpCacheFilePattern.matcher(file.getName()).matches())
if (!file.delete())
logger.warn("could not delete {}", file.getAbsolutePath());
}
// also clean out any index leftovers.
for (IndexMetadata index : metadata.getIndexes())
if (!index.isCustom())
{
CFMetaData indexMetadata = CassandraIndex.indexCfsMetadata(metadata, index);
scrubDataDirectories(indexMetadata);
}
}
/**
* See #{@code StorageService.loadNewSSTables(String, String)} for more info
*
* @param ksName The keyspace name
* @param cfName The columnFamily name
*/
public static synchronized void loadNewSSTables(String ksName, String cfName)
{
/** ks/cf existence checks will be done by open and getCFS methods for us */
Keyspace keyspace = Keyspace.open(ksName);
keyspace.getColumnFamilyStore(cfName).loadNewSSTables();
}
/**
* #{@inheritDoc}
*/
public synchronized void loadNewSSTables()
{
logger.info("Loading new SSTables for {}/{}...", keyspace.getName(), name);
Set<Descriptor> currentDescriptors = new HashSet<>();
for (SSTableReader sstable : getSSTables(SSTableSet.CANONICAL))
currentDescriptors.add(sstable.descriptor);
Set<SSTableReader> newSSTables = new HashSet<>();
Directories.SSTableLister lister = getDirectories().sstableLister(Directories.OnTxnErr.IGNORE).skipTemporary(true);
for (Map.Entry<Descriptor, Set<Component>> entry : lister.list().entrySet())
{
Descriptor descriptor = entry.getKey();
if (currentDescriptors.contains(descriptor))
continue; // old (initialized) SSTable found, skipping
if (!descriptor.isCompatible())
throw new RuntimeException(String.format("Can't open incompatible SSTable! Current version %s, found file: %s",
descriptor.getFormat().getLatestVersion(),
descriptor));
// force foreign sstables to level 0
try
{
if (new File(descriptor.filenameFor(Component.STATS)).exists())
descriptor.getMetadataSerializer().mutateLevel(descriptor, 0);
}
catch (IOException e)
{
FileUtils.handleCorruptSSTable(new CorruptSSTableException(e, entry.getKey().filenameFor(Component.STATS)));
logger.error("Cannot read sstable {}; other IO error, skipping table", entry, e);
continue;
}
// Increment the generation until we find a filename that doesn't exist. This is needed because the new
// SSTables that are being loaded might already use these generation numbers.
Descriptor newDescriptor;
do
{
newDescriptor = new Descriptor(descriptor.version,
descriptor.directory,
descriptor.ksname,
descriptor.cfname,
fileIndexGenerator.incrementAndGet(),
descriptor.formatType,
descriptor.digestComponent);
}
while (new File(newDescriptor.filenameFor(Component.DATA)).exists());
logger.info("Renaming new SSTable {} to {}", descriptor, newDescriptor);
SSTableWriter.rename(descriptor, newDescriptor, entry.getValue());
SSTableReader reader;
try
{
reader = SSTableReader.open(newDescriptor, entry.getValue(), metadata);
}
catch (CorruptSSTableException ex)
{
FileUtils.handleCorruptSSTable(ex);
logger.error("Corrupt sstable {}; skipping table", entry, ex);
continue;
}
catch (FSError ex)
{
FileUtils.handleFSError(ex);
logger.error("Cannot read sstable {}; file system error, skipping table", entry, ex);
continue;
}
catch (IOException ex)
{
FileUtils.handleCorruptSSTable(new CorruptSSTableException(ex, entry.getKey().filenameFor(Component.DATA)));
logger.error("Cannot read sstable {}; other IO error, skipping table", entry, ex);
continue;
}
newSSTables.add(reader);
}
if (newSSTables.isEmpty())
{
logger.info("No new SSTables were found for {}/{}", keyspace.getName(), name);
return;
}
logger.info("Loading new SSTables and building secondary indexes for {}/{}: {}", keyspace.getName(), name, newSSTables);
try (Refs<SSTableReader> refs = Refs.ref(newSSTables))
{
data.addSSTables(newSSTables);
indexManager.buildAllIndexesBlocking(newSSTables);
}
logger.info("Done loading load new SSTables for {}/{}", keyspace.getName(), name);
}
public void rebuildSecondaryIndex(String idxName)
{
rebuildSecondaryIndex(keyspace.getName(), metadata.cfName, idxName);
}
public static void rebuildSecondaryIndex(String ksName, String cfName, String... idxNames)
{
ColumnFamilyStore cfs = Keyspace.open(ksName).getColumnFamilyStore(cfName);
Set<String> indexes = new HashSet<String>(Arrays.asList(idxNames));
Iterable<SSTableReader> sstables = cfs.getSSTables(SSTableSet.CANONICAL);
try (Refs<SSTableReader> refs = Refs.ref(sstables))
{
logger.info("User Requested secondary index re-build for {}/{} indexes: {}", ksName, cfName, Joiner.on(',').join(idxNames));
cfs.indexManager.rebuildIndexesBlocking(refs, indexes);
}
}
@Deprecated
public String getColumnFamilyName()
{
return getTableName();
}
public String getTableName()
{
return name;
}
public String getSSTablePath(File directory)
{
return getSSTablePath(directory, SSTableFormat.Type.current().info.getLatestVersion(), SSTableFormat.Type.current());
}
public String getSSTablePath(File directory, SSTableFormat.Type format)
{
return getSSTablePath(directory, format.info.getLatestVersion(), format);
}
private String getSSTablePath(File directory, Version version, SSTableFormat.Type format)
{
Descriptor desc = new Descriptor(version,
directory,
keyspace.getName(),
name,
fileIndexGenerator.incrementAndGet(),
format,
Component.digestFor(BigFormat.latestVersion.uncompressedChecksumType()));
return desc.filenameFor(Component.DATA);
}
/**
* Switches the memtable iff the live memtable is the one provided
*
* @param memtable
*/
public ListenableFuture<CommitLogPosition> switchMemtableIfCurrent(Memtable memtable)
{
synchronized (data)
{
if (data.getView().getCurrentMemtable() == memtable)
return switchMemtable();
}
return waitForFlushes();
}
/*
* switchMemtable puts Memtable.getSortedContents on the writer executor. When the write is complete,
* we turn the writer into an SSTableReader and add it to ssTables where it is available for reads.
* This method does not block except for synchronizing on Tracker, but the Future it returns will
* not complete until the Memtable (and all prior Memtables) have been successfully flushed, and the CL
* marked clean up to the position owned by the Memtable.
*/
public ListenableFuture<CommitLogPosition> switchMemtable()
{
synchronized (data)
{
logFlush();
Flush flush = new Flush(false);
flushExecutor.execute(flush);
postFlushExecutor.execute(flush.postFlushTask);
return flush.postFlushTask;
}
}
// print out size of all memtables we're enqueuing
private void logFlush()
{
// reclaiming includes that which we are GC-ing;
float onHeapRatio = 0, offHeapRatio = 0;
long onHeapTotal = 0, offHeapTotal = 0;
Memtable memtable = getTracker().getView().getCurrentMemtable();
onHeapRatio += memtable.getAllocator().onHeap().ownershipRatio();
offHeapRatio += memtable.getAllocator().offHeap().ownershipRatio();
onHeapTotal += memtable.getAllocator().onHeap().owns();
offHeapTotal += memtable.getAllocator().offHeap().owns();
for (ColumnFamilyStore indexCfs : indexManager.getAllIndexColumnFamilyStores())
{
MemtableAllocator allocator = indexCfs.getTracker().getView().getCurrentMemtable().getAllocator();
onHeapRatio += allocator.onHeap().ownershipRatio();
offHeapRatio += allocator.offHeap().ownershipRatio();
onHeapTotal += allocator.onHeap().owns();
offHeapTotal += allocator.offHeap().owns();
}
logger.debug("Enqueuing flush of {}: {}",
name,
String.format("%s (%.0f%%) on-heap, %s (%.0f%%) off-heap",
FBUtilities.prettyPrintMemory(onHeapTotal),
onHeapRatio * 100,
FBUtilities.prettyPrintMemory(offHeapTotal),
offHeapRatio * 100));
}
/**
* Flush if there is unflushed data in the memtables
*
* @return a Future yielding the commit log position that can be guaranteed to have been successfully written
* to sstables for this table once the future completes
*/
public ListenableFuture<CommitLogPosition> forceFlush()
{
synchronized (data)
{
Memtable current = data.getView().getCurrentMemtable();
for (ColumnFamilyStore cfs : concatWithIndexes())
if (!cfs.data.getView().getCurrentMemtable().isClean())
return switchMemtableIfCurrent(current);
return waitForFlushes();
}
}
/**
* Flush if there is unflushed data that was written to the CommitLog before @param flushIfDirtyBefore
* (inclusive).
*
* @return a Future yielding the commit log position that can be guaranteed to have been successfully written
* to sstables for this table once the future completes
*/
public ListenableFuture<?> forceFlush(CommitLogPosition flushIfDirtyBefore)
{
// we don't loop through the remaining memtables since here we only care about commit log dirtiness
// and this does not vary between a table and its table-backed indexes
Memtable current = data.getView().getCurrentMemtable();
if (current.mayContainDataBefore(flushIfDirtyBefore))
return switchMemtableIfCurrent(current);
return waitForFlushes();
}
/**
* @return a Future yielding the commit log position that can be guaranteed to have been successfully written
* to sstables for this table once the future completes
*/
private ListenableFuture<CommitLogPosition> waitForFlushes()
{
// we grab the current memtable; once any preceding memtables have flushed, we know its
// commitLogLowerBound has been set (as this it is set with the upper bound of the preceding memtable)
final Memtable current = data.getView().getCurrentMemtable();
ListenableFutureTask<CommitLogPosition> task = ListenableFutureTask.create(() -> {
logger.debug("forceFlush requested but everything is clean in {}", name);
return current.getCommitLogLowerBound();
});
postFlushExecutor.execute(task);
return task;
}
public CommitLogPosition forceBlockingFlush()
{
return FBUtilities.waitOnFuture(forceFlush());
}
/**
* Both synchronises custom secondary indexes and provides ordering guarantees for futures on switchMemtable/flush
* etc, which expect to be able to wait until the flush (and all prior flushes) requested have completed.
*/
private final class PostFlush implements Callable<CommitLogPosition>
{
final CountDownLatch latch = new CountDownLatch(1);
final List<Memtable> memtables;
volatile Throwable flushFailure = null;
private PostFlush(List<Memtable> memtables)
{
this.memtables = memtables;
}
public CommitLogPosition call()
{
try
{
// we wait on the latch for the commitLogUpperBound to be set, and so that waiters
// on this task can rely on all prior flushes being complete
latch.await();
}
catch (InterruptedException e)