-
Notifications
You must be signed in to change notification settings - Fork 444
/
FileCompactor.java
592 lines (493 loc) · 21 KB
/
FileCompactor.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
/*
* 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
*
* https://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.accumulo.server.compaction;
import java.io.IOException;
import java.text.DateFormat;
import java.text.SimpleDateFormat;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Date;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.LongAdder;
import org.apache.accumulo.core.client.IteratorSetting;
import org.apache.accumulo.core.conf.AccumuloConfiguration;
import org.apache.accumulo.core.conf.Property;
import org.apache.accumulo.core.data.ByteSequence;
import org.apache.accumulo.core.data.Key;
import org.apache.accumulo.core.data.TableId;
import org.apache.accumulo.core.data.Value;
import org.apache.accumulo.core.dataImpl.KeyExtent;
import org.apache.accumulo.core.file.FileOperations;
import org.apache.accumulo.core.file.FileOperations.WriterBuilder;
import org.apache.accumulo.core.file.FileSKVIterator;
import org.apache.accumulo.core.file.FileSKVWriter;
import org.apache.accumulo.core.iterators.IteratorUtil;
import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
import org.apache.accumulo.core.iteratorsImpl.IteratorConfigUtil;
import org.apache.accumulo.core.iteratorsImpl.system.ColumnFamilySkippingIterator;
import org.apache.accumulo.core.iteratorsImpl.system.DeletingIterator;
import org.apache.accumulo.core.iteratorsImpl.system.IterationInterruptedException;
import org.apache.accumulo.core.iteratorsImpl.system.MultiIterator;
import org.apache.accumulo.core.iteratorsImpl.system.TimeSettingIterator;
import org.apache.accumulo.core.metadata.MetadataTable;
import org.apache.accumulo.core.metadata.RootTable;
import org.apache.accumulo.core.metadata.StoredTabletFile;
import org.apache.accumulo.core.metadata.TabletFile;
import org.apache.accumulo.core.metadata.schema.DataFileValue;
import org.apache.accumulo.core.spi.crypto.CryptoService;
import org.apache.accumulo.core.tabletserver.thrift.TCompactionReason;
import org.apache.accumulo.core.trace.TraceUtil;
import org.apache.accumulo.core.util.LocalityGroupUtil;
import org.apache.accumulo.core.util.LocalityGroupUtil.LocalityGroupConfigurationError;
import org.apache.accumulo.core.util.ratelimit.RateLimiter;
import org.apache.accumulo.server.ServerContext;
import org.apache.accumulo.server.fs.VolumeManager;
import org.apache.accumulo.server.iterators.SystemIteratorEnvironment;
import org.apache.accumulo.server.problems.ProblemReport;
import org.apache.accumulo.server.problems.ProblemReportingIterator;
import org.apache.accumulo.server.problems.ProblemReports;
import org.apache.accumulo.server.problems.ProblemType;
import org.apache.hadoop.fs.FileSystem;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.collect.Collections2;
import io.opentelemetry.api.trace.Span;
import io.opentelemetry.context.Scope;
public class FileCompactor implements Callable<CompactionStats> {
private static final Logger log = LoggerFactory.getLogger(FileCompactor.class);
private static final AtomicLong nextCompactorID = new AtomicLong(0);
public static class CompactionCanceledException extends Exception {
private static final long serialVersionUID = 1L;
}
public interface CompactionEnv {
boolean isCompactionEnabled();
IteratorScope getIteratorScope();
RateLimiter getReadLimiter();
RateLimiter getWriteLimiter();
SystemIteratorEnvironment createIteratorEnv(ServerContext context,
AccumuloConfiguration acuTableConf, TableId tableId);
SortedKeyValueIterator<Key,Value> getMinCIterator();
TCompactionReason getReason();
}
private final Map<StoredTabletFile,DataFileValue> filesToCompact;
private final TabletFile outputFile;
private final boolean propagateDeletes;
private final AccumuloConfiguration acuTableConf;
private final CompactionEnv env;
private final VolumeManager fs;
protected final KeyExtent extent;
private final List<IteratorSetting> iterators;
private final CryptoService cryptoService;
// things to report
private String currentLocalityGroup = "";
private volatile long startTime = -1;
private final AtomicLong currentEntriesRead = new AtomicLong(0);
private final AtomicLong currentEntriesWritten = new AtomicLong(0);
// These track the cumulative count of entries (read and written) that has been recorded in
// the global counts. Their purpose is to avoid double counting of metrics during the update of
// global statistics.
private final AtomicLong lastRecordedEntriesRead = new AtomicLong(0);
private final AtomicLong lastRecordedEntriesWritten = new AtomicLong(0);
private static final LongAdder totalEntriesRead = new LongAdder();
private static final LongAdder totalEntriesWritten = new LongAdder();
private static volatile long lastUpdateTime = 0;
private final DateFormat dateFormatter = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss.SSS");
// a unique id to identify a compactor
private final long compactorID = nextCompactorID.getAndIncrement();
protected volatile Thread thread;
private final ServerContext context;
private final AtomicBoolean interruptFlag = new AtomicBoolean(false);
public void interrupt() {
interruptFlag.set(true);
}
public long getCompactorID() {
return compactorID;
}
private synchronized void setLocalityGroup(String name) {
this.currentLocalityGroup = name;
}
public synchronized String getCurrentLocalityGroup() {
return currentLocalityGroup;
}
private void clearCurrentEntryCounts() {
currentEntriesRead.set(0);
currentEntriesWritten.set(0);
}
private void updateGlobalEntryCounts() {
updateTotalEntries(currentEntriesRead, lastRecordedEntriesRead, totalEntriesRead);
updateTotalEntries(currentEntriesWritten, lastRecordedEntriesWritten, totalEntriesWritten);
}
/**
* Updates the total count of entries by adding the difference between the current count and the
* last recorded count to the total.
*
* @param current The current count of entries
* @param recorded The last recorded count of entries
* @param total The total count to add the difference to
*/
private void updateTotalEntries(AtomicLong current, AtomicLong recorded, LongAdder total) {
long currentCount = current.get();
long lastRecorded =
recorded.getAndUpdate(recordedValue -> Math.max(recordedValue, currentCount));
if (lastRecorded < currentCount) {
total.add(currentCount - lastRecorded);
}
}
/**
* @return the total entries written by compactions over the lifetime of this process.
*/
public static long getTotalEntriesWritten() {
updateTotalEntries();
return totalEntriesWritten.sum();
}
/**
* @return the total entries read by compactions over the lifetime of this process.
*/
public static long getTotalEntriesRead() {
updateTotalEntries();
return totalEntriesRead.sum();
}
/**
* Updates total entries read and written for all currently running compactions. Compactions will
* update the global stats when they finish. This can be called to update them sooner. This method
* is rate limited, so it will not cause issues if called too frequently.
*/
private static void updateTotalEntries() {
long currentTime = System.nanoTime();
if (currentTime - lastUpdateTime < Duration.ofMillis(100).toNanos()) {
return;
}
runningCompactions.forEach(FileCompactor::updateGlobalEntryCounts);
lastUpdateTime = currentTime;
}
protected static final Set<FileCompactor> runningCompactions =
Collections.synchronizedSet(new HashSet<>());
public static List<CompactionInfo> getRunningCompactions() {
ArrayList<CompactionInfo> compactions = new ArrayList<>();
synchronized (runningCompactions) {
for (FileCompactor compactor : runningCompactions) {
compactions.add(new CompactionInfo(compactor));
}
}
return compactions;
}
public FileCompactor(ServerContext context, KeyExtent extent,
Map<StoredTabletFile,DataFileValue> files, TabletFile outputFile, boolean propagateDeletes,
CompactionEnv env, List<IteratorSetting> iterators, AccumuloConfiguration tableConfiguation,
CryptoService cs) {
this.context = context;
this.extent = extent;
this.fs = context.getVolumeManager();
this.acuTableConf = tableConfiguation;
this.filesToCompact = files;
this.outputFile = outputFile;
this.propagateDeletes = propagateDeletes;
this.env = env;
this.iterators = iterators;
this.cryptoService = cs;
}
public VolumeManager getVolumeManager() {
return fs;
}
public KeyExtent getExtent() {
return extent;
}
protected String getOutputFile() {
return outputFile.toString();
}
protected Map<String,Set<ByteSequence>> getLocalityGroups(AccumuloConfiguration acuTableConf)
throws IOException {
try {
return LocalityGroupUtil.getLocalityGroups(acuTableConf);
} catch (LocalityGroupConfigurationError e) {
throw new IOException(e);
}
}
@Override
public CompactionStats call() throws IOException, CompactionCanceledException {
FileSKVWriter mfw = null;
CompactionStats majCStats = new CompactionStats();
startTime = System.nanoTime();
boolean remove = runningCompactions.add(this);
String threadStartDate = dateFormatter.format(new Date());
clearCurrentEntryCounts();
String oldThreadName = Thread.currentThread().getName();
String newThreadName =
"MajC compacting " + extent + " started " + threadStartDate + " file: " + outputFile;
Thread.currentThread().setName(newThreadName);
thread = Thread.currentThread();
try {
FileOperations fileFactory = FileOperations.getInstance();
FileSystem ns = this.fs.getFileSystemByPath(outputFile.getPath());
final boolean isMinC = env.getIteratorScope() == IteratorUtil.IteratorScope.minc;
final boolean dropCacheBehindOutput = !RootTable.ID.equals(this.extent.tableId())
&& !MetadataTable.ID.equals(this.extent.tableId())
&& ((isMinC && acuTableConf.getBoolean(Property.TABLE_MINC_OUTPUT_DROP_CACHE))
|| (!isMinC && acuTableConf.getBoolean(Property.TABLE_MAJC_OUTPUT_DROP_CACHE)));
WriterBuilder outBuilder = fileFactory.newWriterBuilder()
.forFile(outputFile.getMetaInsert(), ns, ns.getConf(), cryptoService)
.withTableConfiguration(acuTableConf).withRateLimiter(env.getWriteLimiter());
if (dropCacheBehindOutput) {
outBuilder.dropCachesBehind();
}
mfw = outBuilder.build();
Map<String,Set<ByteSequence>> lGroups = getLocalityGroups(acuTableConf);
long t1 = System.currentTimeMillis();
HashSet<ByteSequence> allColumnFamilies = new HashSet<>();
if (mfw.supportsLocalityGroups()) {
for (Entry<String,Set<ByteSequence>> entry : lGroups.entrySet()) {
setLocalityGroup(entry.getKey());
compactLocalityGroup(entry.getKey(), entry.getValue(), true, mfw, majCStats);
allColumnFamilies.addAll(entry.getValue());
}
}
setLocalityGroup("");
compactLocalityGroup(null, allColumnFamilies, false, mfw, majCStats);
long t2 = System.currentTimeMillis();
FileSKVWriter mfwTmp = mfw;
mfw = null; // set this to null so we do not try to close it again in finally if the close
// fails
try {
mfwTmp.close(); // if the close fails it will cause the compaction to fail
} catch (IOException ex) {
if (!fs.deleteRecursively(outputFile.getPath())) {
if (fs.exists(outputFile.getPath())) {
log.error("Unable to delete {}", outputFile);
}
}
throw ex;
}
log.trace(String.format(
"Compaction %s %,d read | %,d written | %,6d entries/sec"
+ " | %,6.3f secs | %,12d bytes | %9.3f byte/sec",
extent, majCStats.getEntriesRead(), majCStats.getEntriesWritten(),
(int) (majCStats.getEntriesRead() / ((t2 - t1) / 1000.0)), (t2 - t1) / 1000.0,
mfwTmp.getLength(), mfwTmp.getLength() / ((t2 - t1) / 1000.0)));
majCStats.setFileSize(mfwTmp.getLength());
return majCStats;
} catch (CompactionCanceledException e) {
log.debug("Compaction canceled {}", extent);
throw e;
} catch (IterationInterruptedException iie) {
if (!env.isCompactionEnabled()) {
log.debug("Compaction canceled {}", extent);
throw new CompactionCanceledException();
}
log.debug("RFile interrupted {}", extent);
throw iie;
} catch (IOException | RuntimeException e) {
Collection<String> inputFileNames =
Collections2.transform(getFilesToCompact(), StoredTabletFile::getFileName);
String outputFileName = outputFile.getFileName();
log.error(
"Compaction error. Compaction info: "
+ "extent: {}, input files: {}, output file: {}, iterators: {}, start date: {}",
getExtent(), inputFileNames, outputFileName, getIterators(), threadStartDate, e);
throw e;
} finally {
Thread.currentThread().setName(oldThreadName);
if (remove) {
thread = null;
runningCompactions.remove(this);
}
updateGlobalEntryCounts();
try {
if (mfw != null) {
// compaction must not have finished successfully, so close its output file
try {
mfw.close();
} finally {
if (!fs.deleteRecursively(outputFile.getPath())) {
if (fs.exists(outputFile.getPath())) {
log.error("Unable to delete {}", outputFile);
}
}
}
}
} catch (IOException | RuntimeException e) {
/*
* If compaction is enabled then the compaction didn't finish due to a real error condition
* so log any errors on the output file close as a warning. However, if not enabled, then
* the compaction was canceled due to something like tablet split, user cancellation, or
* table deletion which is not an error so log any errors on output file close as a debug as
* this may happen due to an InterruptedException thrown due to the cancellation.
*/
if (env.isCompactionEnabled()) {
log.warn("{}", e.getMessage(), e);
} else {
log.debug("{}", e.getMessage(), e);
}
}
}
}
private List<SortedKeyValueIterator<Key,Value>>
openMapDataFiles(ArrayList<FileSKVIterator> readers) throws IOException {
List<SortedKeyValueIterator<Key,Value>> iters = new ArrayList<>(filesToCompact.size());
for (StoredTabletFile mapFile : filesToCompact.keySet()) {
try {
FileOperations fileFactory = FileOperations.getInstance();
FileSystem fs = this.fs.getFileSystemByPath(mapFile.getPath());
FileSKVIterator reader;
reader = fileFactory.newReaderBuilder()
.forFile(mapFile.getPathStr(), fs, fs.getConf(), cryptoService)
.withTableConfiguration(acuTableConf).withRateLimiter(env.getReadLimiter())
.dropCachesBehind().build();
readers.add(reader);
SortedKeyValueIterator<Key,Value> iter = new ProblemReportingIterator(context,
extent.tableId(), mapFile.getPathStr(), false, reader);
((ProblemReportingIterator) iter).setInterruptFlag(interruptFlag);
if (filesToCompact.get(mapFile).isTimeSet()) {
iter = new TimeSettingIterator(iter, filesToCompact.get(mapFile).getTime());
}
iters.add(iter);
} catch (Exception e) {
ProblemReports.getInstance(context).report(
new ProblemReport(extent.tableId(), ProblemType.FILE_READ, mapFile.getPathStr(), e));
log.warn("Some problem opening map file {} {}", mapFile, e.getMessage(), e);
// failed to open some map file... close the ones that were opened
for (FileSKVIterator reader : readers) {
try {
reader.close();
} catch (Exception e2) {
log.warn("Failed to close map file", e2);
}
}
readers.clear();
if (e instanceof IOException) {
throw (IOException) e;
}
throw new IOException("Failed to open map data files", e);
}
}
return iters;
}
private void compactLocalityGroup(String lgName, Set<ByteSequence> columnFamilies,
boolean inclusive, FileSKVWriter mfw, CompactionStats majCStats)
throws IOException, CompactionCanceledException {
ArrayList<FileSKVIterator> readers = new ArrayList<>(filesToCompact.size());
Span compactSpan = TraceUtil.startSpan(this.getClass(), "compact");
try (Scope span = compactSpan.makeCurrent()) {
long entriesCompacted = 0;
List<SortedKeyValueIterator<Key,Value>> iters = openMapDataFiles(readers);
if (env.getIteratorScope() == IteratorScope.minc) {
iters.add(env.getMinCIterator());
}
CountingIterator citr =
new CountingIterator(new MultiIterator(iters, extent.toDataRange()), currentEntriesRead);
SortedKeyValueIterator<Key,Value> delIter =
DeletingIterator.wrap(citr, propagateDeletes, DeletingIterator.getBehavior(acuTableConf));
ColumnFamilySkippingIterator cfsi = new ColumnFamilySkippingIterator(delIter);
SystemIteratorEnvironment iterEnv =
env.createIteratorEnv(context, acuTableConf, getExtent().tableId());
SortedKeyValueIterator<Key,Value> itr = iterEnv.getTopLevelIterator(IteratorConfigUtil
.convertItersAndLoad(env.getIteratorScope(), cfsi, acuTableConf, iterators, iterEnv));
itr.seek(extent.toDataRange(), columnFamilies, inclusive);
if (inclusive) {
mfw.startNewLocalityGroup(lgName, columnFamilies);
} else {
mfw.startDefaultLocalityGroup();
}
Span writeSpan = TraceUtil.startSpan(this.getClass(), "write");
try (Scope write = writeSpan.makeCurrent()) {
while (itr.hasTop() && env.isCompactionEnabled()) {
mfw.append(itr.getTopKey(), itr.getTopValue());
itr.next();
entriesCompacted++;
if (entriesCompacted % 1024 == 0) {
// Periodically update stats, do not want to do this too often since its volatile
currentEntriesWritten.addAndGet(1024);
}
}
if (itr.hasTop() && !env.isCompactionEnabled()) {
// cancel major compaction operation
try {
try {
mfw.close();
} catch (IOException e) {
log.warn("{}", e.getMessage());
log.debug("{}", e.getMessage(), e);
}
fs.deleteRecursively(outputFile.getPath());
} catch (Exception e) {
log.warn("Failed to delete Canceled compaction output file {}", outputFile, e);
}
throw new CompactionCanceledException();
}
} finally {
CompactionStats lgMajcStats = new CompactionStats(citr.getCount(), entriesCompacted);
majCStats.add(lgMajcStats);
writeSpan.end();
}
} catch (Exception e) {
TraceUtil.setException(compactSpan, e, true);
throw e;
} finally {
// close sequence files opened
for (FileSKVIterator reader : readers) {
try {
reader.close();
} catch (Exception e) {
log.warn("Failed to close map file", e);
}
}
compactSpan.end();
}
}
Collection<StoredTabletFile> getFilesToCompact() {
return filesToCompact.keySet();
}
boolean hasIMM() {
return env.getIteratorScope() == IteratorScope.minc;
}
boolean willPropagateDeletes() {
return propagateDeletes;
}
long getEntriesRead() {
return currentEntriesRead.get();
}
long getEntriesWritten() {
return currentEntriesWritten.get();
}
/**
* @return the duration since {@link #call()} was called
*/
Duration getAge() {
if (startTime == -1) {
// call() has not been called yet
return Duration.ZERO;
}
return Duration.ofNanos(System.nanoTime() - startTime);
}
Iterable<IteratorSetting> getIterators() {
return this.iterators;
}
public TCompactionReason getReason() {
return env.getReason();
}
}