forked from apache/cassandra
-
Notifications
You must be signed in to change notification settings - Fork 0
/
CompactionTask.java
424 lines (356 loc) · 18.5 KB
/
CompactionTask.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
/*
* 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.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
import com.google.common.base.Predicate;
import com.google.common.collect.Iterables;
import com.google.common.collect.Sets;
import com.google.common.util.concurrent.RateLimiter;
import org.apache.cassandra.db.Directories;
import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter;
import org.apache.cassandra.db.compaction.writers.DefaultCompactionWriter;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.commons.lang3.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.SystemKeyspace;
import org.apache.cassandra.db.compaction.CompactionManager.CompactionExecutorStatsCollector;
import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
import org.apache.cassandra.service.ActiveRepairService;
import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.concurrent.Refs;
public class CompactionTask extends AbstractCompactionTask
{
protected static final Logger logger = LoggerFactory.getLogger(CompactionTask.class);
protected final int gcBefore;
protected final boolean keepOriginals;
protected static long totalBytesCompacted = 0;
private CompactionExecutorStatsCollector collector;
public CompactionTask(ColumnFamilyStore cfs, LifecycleTransaction txn, int gcBefore)
{
this(cfs, txn, gcBefore, false);
}
@Deprecated
public CompactionTask(ColumnFamilyStore cfs, LifecycleTransaction txn, int gcBefore, boolean offline, boolean keepOriginals)
{
this(cfs, txn, gcBefore, keepOriginals);
}
public CompactionTask(ColumnFamilyStore cfs, LifecycleTransaction txn, int gcBefore, boolean keepOriginals)
{
super(cfs, txn);
this.gcBefore = gcBefore;
this.keepOriginals = keepOriginals;
}
public static synchronized long addToTotalBytesCompacted(long bytesCompacted)
{
return totalBytesCompacted += bytesCompacted;
}
protected int executeInternal(CompactionExecutorStatsCollector collector)
{
this.collector = collector;
run();
return transaction.originals().size();
}
public boolean reduceScopeForLimitedSpace(Set<SSTableReader> nonExpiredSSTables, long expectedSize)
{
if (partialCompactionsAcceptable() && transaction.originals().size() > 1)
{
// Try again w/o the largest one.
logger.warn("insufficient space to compact all requested files. {}MB required, {}",
(float) expectedSize / 1024 / 1024,
StringUtils.join(transaction.originals(), ", "));
// Note that we have removed files that are still marked as compacting.
// This suboptimal but ok since the caller will unmark all the sstables at the end.
SSTableReader removedSSTable = cfs.getMaxSizeFile(nonExpiredSSTables);
transaction.cancel(removedSSTable);
return true;
}
return false;
}
/**
* For internal use and testing only. The rest of the system should go through the submit* methods,
* which are properly serialized.
* Caller is in charge of marking/unmarking the sstables as compacting.
*/
protected void runMayThrow() throws Exception
{
// The collection of sstables passed may be empty (but not null); even if
// it is not empty, it may compact down to nothing if all rows are deleted.
assert transaction != null;
if (transaction.originals().isEmpty())
return;
// Note that the current compaction strategy, is not necessarily the one this task was created under.
// This should be harmless; see comments to CFS.maybeReloadCompactionStrategy.
CompactionStrategyManager strategy = cfs.getCompactionStrategyManager();
if (DatabaseDescriptor.isSnapshotBeforeCompaction())
cfs.snapshotWithoutFlush(System.currentTimeMillis() + "-compact-" + cfs.name);
try (CompactionController controller = getCompactionController(transaction.originals()))
{
final Set<SSTableReader> fullyExpiredSSTables = controller.getFullyExpiredSSTables();
// select SSTables to compact based on available disk space.
buildCompactionCandidatesForAvailableDiskSpace(fullyExpiredSSTables);
// sanity check: all sstables must belong to the same cfs
assert !Iterables.any(transaction.originals(), new Predicate<SSTableReader>()
{
@Override
public boolean apply(SSTableReader sstable)
{
return !sstable.descriptor.cfname.equals(cfs.name);
}
});
UUID taskId = transaction.opId();
// new sstables from flush can be added during a compaction, but only the compaction can remove them,
// so in our single-threaded compaction world this is a valid way of determining if we're compacting
// all the sstables (that existed when we started)
StringBuilder ssTableLoggerMsg = new StringBuilder("[");
for (SSTableReader sstr : transaction.originals())
{
ssTableLoggerMsg.append(String.format("%s:level=%d, ", sstr.getFilename(), sstr.getSSTableLevel()));
}
ssTableLoggerMsg.append("]");
logger.debug("Compacting ({}) {}", taskId, ssTableLoggerMsg);
RateLimiter limiter = CompactionManager.instance.getRateLimiter();
long start = System.nanoTime();
long startTime = System.currentTimeMillis();
long totalKeysWritten = 0;
long estimatedKeys = 0;
long inputSizeBytes;
Set<SSTableReader> actuallyCompact = Sets.difference(transaction.originals(), fullyExpiredSSTables);
Collection<SSTableReader> newSStables;
long[] mergedRowCounts;
long totalSourceCQLRows;
// SSTableScanners need to be closed before markCompactedSSTablesReplaced call as scanners contain references
// to both ifile and dfile and SSTR will throw deletion errors on Windows if it tries to delete before scanner is closed.
// See CASSANDRA-8019 and CASSANDRA-8399
int nowInSec = FBUtilities.nowInSeconds();
try (Refs<SSTableReader> refs = Refs.ref(actuallyCompact);
AbstractCompactionStrategy.ScannerList scanners = strategy.getScanners(actuallyCompact);
CompactionIterator ci = new CompactionIterator(compactionType, scanners.scanners, controller, nowInSec, taskId))
{
long lastCheckObsoletion = start;
inputSizeBytes = scanners.getTotalCompressedSize();
double compressionRatio = scanners.getCompressionRatio();
if (compressionRatio == MetadataCollector.NO_COMPRESSION_RATIO)
compressionRatio = 1.0;
long lastBytesScanned = 0;
if (!controller.cfs.getCompactionStrategyManager().isActive())
throw new CompactionInterruptedException(ci.getCompactionInfo());
if (collector != null)
collector.beginCompaction(ci);
try (CompactionAwareWriter writer = getCompactionAwareWriter(cfs, getDirectories(), transaction, actuallyCompact))
{
estimatedKeys = writer.estimatedKeys();
while (ci.hasNext())
{
if (ci.isStopRequested())
throw new CompactionInterruptedException(ci.getCompactionInfo());
if (writer.append(ci.next()))
totalKeysWritten++;
long bytesScanned = scanners.getTotalBytesScanned();
//Rate limit the scanners, and account for compression
CompactionManager.compactionRateLimiterAcquire(limiter, bytesScanned, lastBytesScanned, compressionRatio);
lastBytesScanned = bytesScanned;
if (System.nanoTime() - lastCheckObsoletion > TimeUnit.MINUTES.toNanos(1L))
{
controller.maybeRefreshOverlaps();
lastCheckObsoletion = System.nanoTime();
}
}
// point of no return
newSStables = writer.finish();
}
finally
{
if (collector != null)
collector.finishCompaction(ci);
mergedRowCounts = ci.getMergedRowCounts();
totalSourceCQLRows = ci.getTotalSourceCQLRows();
}
}
if (transaction.isOffline())
{
Refs.release(Refs.selfRefs(newSStables));
}
else
{
// log a bunch of statistics about the result and save to system table compaction_history
long durationInNano = System.nanoTime() - start;
long dTime = TimeUnit.NANOSECONDS.toMillis(durationInNano);
long startsize = inputSizeBytes;
long endsize = SSTableReader.getTotalBytes(newSStables);
double ratio = (double) endsize / (double) startsize;
StringBuilder newSSTableNames = new StringBuilder();
for (SSTableReader reader : newSStables)
newSSTableNames.append(reader.descriptor.baseFilename()).append(",");
long totalSourceRows = 0;
for (int i = 0; i < mergedRowCounts.length; i++)
totalSourceRows += mergedRowCounts[i] * (i + 1);
String mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getTableName(), mergedRowCounts, startsize, endsize);
logger.debug(String.format("Compacted (%s) %d sstables to [%s] to level=%d. %s to %s (~%d%% of original) in %,dms. Read Throughput = %s, Write Throughput = %s, Row Throughput = ~%,d/s. %,d total partitions merged to %,d. Partition merge counts were {%s}",
taskId,
transaction.originals().size(),
newSSTableNames.toString(),
getLevel(),
FBUtilities.prettyPrintMemory(startsize),
FBUtilities.prettyPrintMemory(endsize),
(int) (ratio * 100),
dTime,
FBUtilities.prettyPrintMemoryPerSecond(startsize, durationInNano),
FBUtilities.prettyPrintMemoryPerSecond(endsize, durationInNano),
(int) totalSourceCQLRows / (TimeUnit.NANOSECONDS.toSeconds(durationInNano) + 1),
totalSourceRows,
totalKeysWritten,
mergeSummary));
logger.trace("CF Total Bytes Compacted: {}", FBUtilities.prettyPrintMemory(CompactionTask.addToTotalBytesCompacted(endsize)));
logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double)(totalKeysWritten - estimatedKeys)/totalKeysWritten));
cfs.getCompactionStrategyManager().compactionLogger.compaction(startTime, transaction.originals(), System.currentTimeMillis(), newSStables);
// update the metrics
cfs.metric.compactionBytesWritten.inc(endsize);
}
}
}
@Override
public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs,
Directories directories,
LifecycleTransaction transaction,
Set<SSTableReader> nonExpiredSSTables)
{
return new DefaultCompactionWriter(cfs, directories, transaction, nonExpiredSSTables, keepOriginals, getLevel());
}
public static String updateCompactionHistory(String keyspaceName, String columnFamilyName, long[] mergedRowCounts, long startSize, long endSize)
{
StringBuilder mergeSummary = new StringBuilder(mergedRowCounts.length * 10);
Map<Integer, Long> mergedRows = new HashMap<>();
for (int i = 0; i < mergedRowCounts.length; i++)
{
long count = mergedRowCounts[i];
if (count == 0)
continue;
int rows = i + 1;
mergeSummary.append(String.format("%d:%d, ", rows, count));
mergedRows.put(rows, count);
}
SystemKeyspace.updateCompactionHistory(keyspaceName, columnFamilyName, System.currentTimeMillis(), startSize, endSize, mergedRows);
return mergeSummary.toString();
}
protected Directories getDirectories()
{
return cfs.getDirectories();
}
public static long getMinRepairedAt(Set<SSTableReader> actuallyCompact)
{
long minRepairedAt= Long.MAX_VALUE;
for (SSTableReader sstable : actuallyCompact)
minRepairedAt = Math.min(minRepairedAt, sstable.getSSTableMetadata().repairedAt);
if (minRepairedAt == Long.MAX_VALUE)
return ActiveRepairService.UNREPAIRED_SSTABLE;
return minRepairedAt;
}
public static UUID getPendingRepair(Set<SSTableReader> sstables)
{
if (sstables.isEmpty())
{
return ActiveRepairService.NO_PENDING_REPAIR;
}
Set<UUID> ids = new HashSet<>();
for (SSTableReader sstable: sstables)
ids.add(sstable.getSSTableMetadata().pendingRepair);
if (ids.size() != 1)
throw new RuntimeException(String.format("Attempting to compact pending repair sstables with sstables from other repair, or sstables not pending repair: %s", ids));
return ids.iterator().next();
}
/*
* Checks if we have enough disk space to execute the compaction. Drops the largest sstable out of the Task until
* there's enough space (in theory) to handle the compaction. Does not take into account space that will be taken by
* other compactions.
*/
protected void buildCompactionCandidatesForAvailableDiskSpace(final Set<SSTableReader> fullyExpiredSSTables)
{
if(!cfs.isCompactionDiskSpaceCheckEnabled() && compactionType == OperationType.COMPACTION)
{
logger.info("Compaction space check is disabled");
return; // try to compact all SSTables
}
final Set<SSTableReader> nonExpiredSSTables = Sets.difference(transaction.originals(), fullyExpiredSSTables);
CompactionStrategyManager strategy = cfs.getCompactionStrategyManager();
int sstablesRemoved = 0;
while(!nonExpiredSSTables.isEmpty())
{
// Only consider write size of non expired SSTables
long expectedWriteSize = cfs.getExpectedCompactedFileSize(nonExpiredSSTables, compactionType);
long estimatedSSTables = Math.max(1, expectedWriteSize / strategy.getMaxSSTableBytes());
if(cfs.getDirectories().hasAvailableDiskSpace(estimatedSSTables, expectedWriteSize))
break;
if (!reduceScopeForLimitedSpace(nonExpiredSSTables, expectedWriteSize))
{
// we end up here if we can't take any more sstables out of the compaction.
// usually means we've run out of disk space
// but we can still compact expired SSTables
if(partialCompactionsAcceptable() && fullyExpiredSSTables.size() > 0 )
{
// sanity check to make sure we compact only fully expired SSTables.
assert transaction.originals().equals(fullyExpiredSSTables);
break;
}
String msg = String.format("Not enough space for compaction, estimated sstables = %d, expected write size = %d", estimatedSSTables, expectedWriteSize);
logger.warn(msg);
CompactionManager.instance.incrementAborted();
throw new RuntimeException(msg);
}
sstablesRemoved++;
logger.warn("Not enough space for compaction, {}MB estimated. Reducing scope.",
(float) expectedWriteSize / 1024 / 1024);
}
if(sstablesRemoved > 0)
{
CompactionManager.instance.incrementCompactionsReduced();
CompactionManager.instance.incrementSstablesDropppedFromCompactions(sstablesRemoved);
}
}
protected int getLevel()
{
return 0;
}
protected CompactionController getCompactionController(Set<SSTableReader> toCompact)
{
return new CompactionController(cfs, toCompact, gcBefore);
}
protected boolean partialCompactionsAcceptable()
{
return !isUserDefined;
}
public static long getMaxDataAge(Collection<SSTableReader> sstables)
{
long max = 0;
for (SSTableReader sstable : sstables)
{
if (sstable.maxDataAge > max)
max = sstable.maxDataAge;
}
return max;
}
}