/
LeveledCompactionStrategy.java
536 lines (468 loc) · 20.3 KB
/
LeveledCompactionStrategy.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
/*
* 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.compaction;
import java.util.*;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Joiner;
import com.google.common.collect.*;
import com.google.common.primitives.Doubles;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.db.rows.UnfilteredRowIterator;
import org.apache.cassandra.dht.Range;
import org.apache.cassandra.dht.Token;
import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.io.sstable.ISSTableScanner;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.codehaus.jackson.JsonNode;
import org.codehaus.jackson.node.JsonNodeFactory;
import org.codehaus.jackson.node.ObjectNode;
public class LeveledCompactionStrategy extends AbstractCompactionStrategy
{
private static final Logger logger = LoggerFactory.getLogger(LeveledCompactionStrategy.class);
private static final String SSTABLE_SIZE_OPTION = "sstable_size_in_mb";
@VisibleForTesting
final LeveledManifest manifest;
private final int maxSSTableSizeInMB;
public LeveledCompactionStrategy(ColumnFamilyStore cfs, Map<String, String> options)
{
super(cfs, options);
int configuredMaxSSTableSize = 160;
SizeTieredCompactionStrategyOptions localOptions = new SizeTieredCompactionStrategyOptions(options);
if (options != null)
{
if (options.containsKey(SSTABLE_SIZE_OPTION))
{
configuredMaxSSTableSize = Integer.parseInt(options.get(SSTABLE_SIZE_OPTION));
if (!Boolean.getBoolean("cassandra.tolerate_sstable_size"))
{
if (configuredMaxSSTableSize >= 1000)
logger.warn("Max sstable size of {}MB is configured for {}.{}; having a unit of compaction this large is probably a bad idea",
configuredMaxSSTableSize, cfs.name, cfs.getColumnFamilyName());
if (configuredMaxSSTableSize < 50)
logger.warn("Max sstable size of {}MB is configured for {}.{}. Testing done for CASSANDRA-5727 indicates that performance improves up to 160MB",
configuredMaxSSTableSize, cfs.name, cfs.getColumnFamilyName());
}
}
}
maxSSTableSizeInMB = configuredMaxSSTableSize;
manifest = new LeveledManifest(cfs, this.maxSSTableSizeInMB, localOptions);
logger.trace("Created {}", manifest);
}
public int getLevelSize(int i)
{
return manifest.getLevelSize(i);
}
public int[] getAllLevelSize()
{
return manifest.getAllLevelSize();
}
@Override
public void startup()
{
manifest.calculateLastCompactedKeys();
super.startup();
}
/**
* the only difference between background and maximal in LCS is that maximal is still allowed
* (by explicit user request) even when compaction is disabled.
*/
@SuppressWarnings("resource") // transaction is closed by AbstractCompactionTask::execute
public AbstractCompactionTask getNextBackgroundTask(int gcBefore)
{
while (true)
{
OperationType op;
LeveledManifest.CompactionCandidate candidate = manifest.getCompactionCandidates();
if (candidate == null)
{
// if there is no sstable to compact in standard way, try compacting based on droppable tombstone ratio
SSTableReader sstable = findDroppableSSTable(gcBefore);
if (sstable == null)
{
logger.trace("No compaction necessary for {}", this);
return null;
}
candidate = new LeveledManifest.CompactionCandidate(Collections.singleton(sstable),
sstable.getSSTableLevel(),
getMaxSSTableBytes());
op = OperationType.TOMBSTONE_COMPACTION;
}
else
{
op = OperationType.COMPACTION;
}
LifecycleTransaction txn = cfs.getTracker().tryModify(candidate.sstables, OperationType.COMPACTION);
if (txn != null)
{
LeveledCompactionTask newTask = new LeveledCompactionTask(cfs, txn, candidate.level, gcBefore, candidate.maxSSTableBytes, false);
newTask.setCompactionType(op);
return newTask;
}
}
}
@SuppressWarnings("resource") // transaction is closed by AbstractCompactionTask::execute
public synchronized Collection<AbstractCompactionTask> getMaximalTask(int gcBefore, boolean splitOutput)
{
Iterable<SSTableReader> sstables = manifest.getAllSSTables();
Iterable<SSTableReader> filteredSSTables = filterSuspectSSTables(sstables);
if (Iterables.isEmpty(sstables))
return null;
LifecycleTransaction txn = cfs.getTracker().tryModify(filteredSSTables, OperationType.COMPACTION);
if (txn == null)
return null;
return Arrays.<AbstractCompactionTask>asList(new LeveledCompactionTask(cfs, txn, 0, gcBefore, getMaxSSTableBytes(), true));
}
@Override
@SuppressWarnings("resource") // transaction is closed by AbstractCompactionTask::execute
public AbstractCompactionTask getUserDefinedTask(Collection<SSTableReader> sstables, int gcBefore)
{
if (sstables.isEmpty())
return null;
LifecycleTransaction transaction = cfs.getTracker().tryModify(sstables, OperationType.COMPACTION);
if (transaction == null)
{
logger.trace("Unable to mark {} for compaction; probably a background compaction got to it first. You can disable background compactions temporarily if this is a problem", sstables);
return null;
}
int level = sstables.size() > 1 ? 0 : sstables.iterator().next().getSSTableLevel();
return new LeveledCompactionTask(cfs, transaction, level, gcBefore, level == 0 ? Long.MAX_VALUE : getMaxSSTableBytes(), false);
}
@Override
public AbstractCompactionTask getCompactionTask(LifecycleTransaction txn, int gcBefore, long maxSSTableBytes)
{
assert txn.originals().size() > 0;
int level = -1;
// if all sstables are in the same level, we can set that level:
for (SSTableReader sstable : txn.originals())
{
if (level == -1)
level = sstable.getSSTableLevel();
if (level != sstable.getSSTableLevel())
level = 0;
}
return new LeveledCompactionTask(cfs, txn, level, gcBefore, maxSSTableBytes, false);
}
/**
* Leveled compaction strategy has guarantees on the data contained within each level so we
* have to make sure we only create groups of SSTables with members from the same level.
* This way we won't end up creating invalid sstables during anti-compaction.
* @param ssTablesToGroup
* @return Groups of sstables from the same level
*/
@Override
public Collection<Collection<SSTableReader>> groupSSTablesForAntiCompaction(Collection<SSTableReader> ssTablesToGroup)
{
int groupSize = 2;
Map<Integer, Collection<SSTableReader>> sstablesByLevel = new HashMap<>();
for (SSTableReader sstable : ssTablesToGroup)
{
Integer level = sstable.getSSTableLevel();
Collection<SSTableReader> sstablesForLevel = sstablesByLevel.get(level);
if (sstablesForLevel == null)
{
sstablesForLevel = new ArrayList<SSTableReader>();
sstablesByLevel.put(level, sstablesForLevel);
}
sstablesForLevel.add(sstable);
}
Collection<Collection<SSTableReader>> groupedSSTables = new ArrayList<>();
for (Collection<SSTableReader> levelOfSSTables : sstablesByLevel.values())
{
Collection<SSTableReader> currGroup = new ArrayList<>();
for (SSTableReader sstable : levelOfSSTables)
{
currGroup.add(sstable);
if (currGroup.size() == groupSize)
{
groupedSSTables.add(currGroup);
currGroup = new ArrayList<>();
}
}
if (currGroup.size() != 0)
groupedSSTables.add(currGroup);
}
return groupedSSTables;
}
public int getEstimatedRemainingTasks()
{
int n = manifest.getEstimatedTasks();
cfs.getCompactionStrategyManager().compactionLogger.pending(this, n);
return n;
}
public long getMaxSSTableBytes()
{
return maxSSTableSizeInMB * 1024L * 1024L;
}
public ScannerList getScanners(Collection<SSTableReader> sstables, Collection<Range<Token>> ranges)
{
Set<SSTableReader>[] sstablesPerLevel = manifest.getSStablesPerLevelSnapshot();
Multimap<Integer, SSTableReader> byLevel = ArrayListMultimap.create();
for (SSTableReader sstable : sstables)
{
int level = sstable.getSSTableLevel();
// if an sstable is not on the manifest, it was recently added or removed
// so we add it to level -1 and create exclusive scanners for it - see below (#9935)
if (level >= sstablesPerLevel.length || !sstablesPerLevel[level].contains(sstable))
{
logger.warn("Live sstable {} from level {} is not on corresponding level in the leveled manifest." +
" This is not a problem per se, but may indicate an orphaned sstable due to a failed" +
" compaction not cleaned up properly.",
sstable.getFilename(), level);
level = -1;
}
byLevel.get(level).add(sstable);
}
List<ISSTableScanner> scanners = new ArrayList<ISSTableScanner>(sstables.size());
try
{
for (Integer level : byLevel.keySet())
{
// level can be -1 when sstables are added to Tracker but not to LeveledManifest
// since we don't know which level those sstable belong yet, we simply do the same as L0 sstables.
if (level <= 0)
{
// L0 makes no guarantees about overlapping-ness. Just create a direct scanner for each
for (SSTableReader sstable : byLevel.get(level))
scanners.add(sstable.getScanner(ranges, null));
}
else
{
// Create a LeveledScanner that only opens one sstable at a time, in sorted order
Collection<SSTableReader> intersecting = LeveledScanner.intersecting(byLevel.get(level), ranges);
if (!intersecting.isEmpty())
{
@SuppressWarnings("resource") // The ScannerList will be in charge of closing (and we close properly on errors)
ISSTableScanner scanner = new LeveledScanner(intersecting, ranges);
scanners.add(scanner);
}
}
}
}
catch (Throwable t)
{
try
{
new ScannerList(scanners).close();
}
catch (Throwable t2)
{
t.addSuppressed(t2);
}
throw t;
}
return new ScannerList(scanners);
}
@Override
public void replaceSSTables(Collection<SSTableReader> removed, Collection<SSTableReader> added)
{
manifest.replace(removed, added);
}
@Override
public void addSSTable(SSTableReader added)
{
manifest.add(added);
}
@Override
public void removeSSTable(SSTableReader sstable)
{
manifest.remove(sstable);
}
// Lazily creates SSTableBoundedScanner for sstable that are assumed to be from the
// same level (e.g. non overlapping) - see #4142
private static class LeveledScanner extends AbstractIterator<UnfilteredRowIterator> implements ISSTableScanner
{
private final Collection<Range<Token>> ranges;
private final List<SSTableReader> sstables;
private final Iterator<SSTableReader> sstableIterator;
private final long totalLength;
private final long compressedLength;
private ISSTableScanner currentScanner;
private long positionOffset;
private long totalBytesScanned = 0;
public LeveledScanner(Collection<SSTableReader> sstables, Collection<Range<Token>> ranges)
{
this.ranges = ranges;
// add only sstables that intersect our range, and estimate how much data that involves
this.sstables = new ArrayList<>(sstables.size());
long length = 0;
long cLength = 0;
for (SSTableReader sstable : sstables)
{
this.sstables.add(sstable);
long estimatedKeys = sstable.estimatedKeys();
double estKeysInRangeRatio = 1.0;
if (estimatedKeys > 0 && ranges != null)
estKeysInRangeRatio = ((double) sstable.estimatedKeysForRanges(ranges)) / estimatedKeys;
length += sstable.uncompressedLength() * estKeysInRangeRatio;
cLength += sstable.onDiskLength() * estKeysInRangeRatio;
}
totalLength = length;
compressedLength = cLength;
Collections.sort(this.sstables, SSTableReader.sstableComparator);
sstableIterator = this.sstables.iterator();
assert sstableIterator.hasNext(); // caller should check intersecting first
SSTableReader currentSSTable = sstableIterator.next();
currentScanner = currentSSTable.getScanner(ranges, null);
}
public static Collection<SSTableReader> intersecting(Collection<SSTableReader> sstables, Collection<Range<Token>> ranges)
{
if (ranges == null)
return Lists.newArrayList(sstables);
Set<SSTableReader> filtered = new HashSet<>();
for (Range<Token> range : ranges)
{
for (SSTableReader sstable : sstables)
{
Range<Token> sstableRange = new Range<>(sstable.first.getToken(), sstable.last.getToken());
if (range == null || sstableRange.intersects(range))
filtered.add(sstable);
}
}
return filtered;
}
public boolean isForThrift()
{
return false;
}
public CFMetaData metadata()
{
return sstables.get(0).metadata; // The ctor checks we have at least one sstable
}
protected UnfilteredRowIterator computeNext()
{
if (currentScanner == null)
return endOfData();
while (true)
{
if (currentScanner.hasNext())
return currentScanner.next();
positionOffset += currentScanner.getLengthInBytes();
totalBytesScanned += currentScanner.getBytesScanned();
currentScanner.close();
if (!sstableIterator.hasNext())
{
// reset to null so getCurrentPosition does not return wrong value
currentScanner = null;
return endOfData();
}
SSTableReader currentSSTable = sstableIterator.next();
currentScanner = currentSSTable.getScanner(ranges, null);
}
}
public void close()
{
if (currentScanner != null)
currentScanner.close();
}
public long getLengthInBytes()
{
return totalLength;
}
public long getCurrentPosition()
{
return positionOffset + (currentScanner == null ? 0L : currentScanner.getCurrentPosition());
}
public long getCompressedLengthInBytes()
{
return compressedLength;
}
public long getBytesScanned()
{
return currentScanner == null ? totalBytesScanned : totalBytesScanned + currentScanner.getBytesScanned();
}
public String getBackingFiles()
{
return Joiner.on(", ").join(sstables);
}
}
@Override
public String toString()
{
return String.format("LCS@%d(%s)", hashCode(), cfs.name);
}
private SSTableReader findDroppableSSTable(final int gcBefore)
{
level:
for (int i = manifest.getLevelCount(); i >= 0; i--)
{
// sort sstables by droppable ratio in descending order
SortedSet<SSTableReader> sstables = manifest.getLevelSorted(i, new Comparator<SSTableReader>()
{
public int compare(SSTableReader o1, SSTableReader o2)
{
double r1 = o1.getEstimatedDroppableTombstoneRatio(gcBefore);
double r2 = o2.getEstimatedDroppableTombstoneRatio(gcBefore);
return -1 * Doubles.compare(r1, r2);
}
});
if (sstables.isEmpty())
continue;
Set<SSTableReader> compacting = cfs.getTracker().getCompacting();
for (SSTableReader sstable : sstables)
{
if (sstable.getEstimatedDroppableTombstoneRatio(gcBefore) <= tombstoneThreshold)
continue level;
else if (!compacting.contains(sstable) && !sstable.isMarkedSuspect() && worthDroppingTombstones(sstable, gcBefore))
return sstable;
}
}
return null;
}
public CompactionLogger.Strategy strategyLogger()
{
return new CompactionLogger.Strategy()
{
public JsonNode sstable(SSTableReader sstable)
{
ObjectNode node = JsonNodeFactory.instance.objectNode();
node.put("level", sstable.getSSTableLevel());
node.put("min_token", sstable.first.getToken().toString());
node.put("max_token", sstable.last.getToken().toString());
return node;
}
public JsonNode options()
{
return null;
}
};
}
public static Map<String, String> validateOptions(Map<String, String> options) throws ConfigurationException
{
Map<String, String> uncheckedOptions = AbstractCompactionStrategy.validateOptions(options);
String size = options.containsKey(SSTABLE_SIZE_OPTION) ? options.get(SSTABLE_SIZE_OPTION) : "1";
try
{
int ssSize = Integer.parseInt(size);
if (ssSize < 1)
{
throw new ConfigurationException(String.format("%s must be larger than 0, but was %s", SSTABLE_SIZE_OPTION, ssSize));
}
}
catch (NumberFormatException ex)
{
throw new ConfigurationException(String.format("%s is not a parsable int (base10) for %s", size, SSTABLE_SIZE_OPTION), ex);
}
uncheckedOptions.remove(SSTABLE_SIZE_OPTION);
uncheckedOptions = SizeTieredCompactionStrategyOptions.validateOptions(options, uncheckedOptions);
return uncheckedOptions;
}
}