-
Notifications
You must be signed in to change notification settings - Fork 24.9k
/
Translog.java
1904 lines (1709 loc) · 79.3 KB
/
Translog.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 Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch 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.elasticsearch.index.translog;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.lucene.index.Term;
import org.apache.lucene.store.AlreadyClosedException;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.bytes.ReleasablePagedBytesReference;
import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.lucene.uid.Versions;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.concurrent.ReleasableLock;
import org.elasticsearch.core.internal.io.IOUtils;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.VersionType;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.seqno.SequenceNumbers;
import org.elasticsearch.index.shard.AbstractIndexShardComponent;
import org.elasticsearch.index.shard.IndexShardComponent;
import org.elasticsearch.index.shard.ShardId;
import java.io.Closeable;
import java.io.EOFException;
import java.io.IOException;
import java.nio.channels.FileChannel;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.StandardCopyOption;
import java.nio.file.StandardOpenOption;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Objects;
import java.util.Optional;
import java.util.OptionalLong;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.function.LongConsumer;
import java.util.function.LongSupplier;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
* A Translog is a per index shard component that records all non-committed index operations in a durable manner.
* In Elasticsearch there is one Translog instance per {@link org.elasticsearch.index.engine.InternalEngine}. The engine
* records the current translog generation {@link Translog#getGeneration()} in it's commit metadata using {@link #TRANSLOG_GENERATION_KEY}
* to reference the generation that contains all operations that have not yet successfully been committed to the engines lucene index.
* Additionally, since Elasticsearch 2.0 the engine also records a {@link #TRANSLOG_UUID_KEY} with each commit to ensure a strong
* association between the lucene index an the transaction log file. This UUID is used to prevent accidental recovery from a transaction
* log that belongs to a
* different engine.
* <p>
* Each Translog has only one translog file open for writes at any time referenced by a translog generation ID. This ID is written to a
* {@code translog.ckp} file that is designed to fit in a single disk block such that a write of the file is atomic. The checkpoint file
* is written on each fsync operation of the translog and records the number of operations written, the current translog's file generation,
* its fsynced offset in bytes, and other important statistics.
* </p>
* <p>
* When the current translog file reaches a certain size ({@link IndexSettings#INDEX_TRANSLOG_GENERATION_THRESHOLD_SIZE_SETTING}, or when
* a clear separation between old and new operations (upon change in primary term), the current file is reopened for read only and a new
* write only file is created. Any non-current, read only translog file always has a {@code translog-${gen}.ckp} associated with it
* which is an fsynced copy of its last {@code translog.ckp} such that in disaster recovery last fsynced offsets, number of
* operation etc. are still preserved.
* </p>
*/
public class Translog extends AbstractIndexShardComponent implements IndexShardComponent, Closeable {
/*
* TODO
* - we might need something like a deletion policy to hold on to more than one translog eventually (I think sequence IDs needs this)
* but we can refactor as we go
* - use a simple BufferedOutputStream to write stuff and fold BufferedTranslogWriter into it's super class... the tricky bit is we
* need to be able to do random access reads even from the buffer
* - we need random exception on the FileSystem API tests for all this.
* - we need to page align the last write before we sync, we can take advantage of ensureSynced for this since we might have already
* fsynced far enough
*/
public static final String TRANSLOG_GENERATION_KEY = "translog_generation";
public static final String TRANSLOG_UUID_KEY = "translog_uuid";
public static final String TRANSLOG_FILE_PREFIX = "translog-";
public static final String TRANSLOG_FILE_SUFFIX = ".tlog";
public static final String CHECKPOINT_SUFFIX = ".ckp";
public static final String CHECKPOINT_FILE_NAME = "translog" + CHECKPOINT_SUFFIX;
static final Pattern PARSE_STRICT_ID_PATTERN = Pattern.compile("^" + TRANSLOG_FILE_PREFIX + "(\\d+)(\\.tlog)$");
public static final int DEFAULT_HEADER_SIZE_IN_BYTES = TranslogHeader.headerSizeInBytes(UUIDs.randomBase64UUID());
// the list of translog readers is guaranteed to be in order of translog generation
private final List<TranslogReader> readers = new ArrayList<>();
private BigArrays bigArrays;
protected final ReleasableLock readLock;
protected final ReleasableLock writeLock;
private final Path location;
private TranslogWriter current;
protected final TragicExceptionHolder tragedy = new TragicExceptionHolder();
private final AtomicBoolean closed = new AtomicBoolean();
private final TranslogConfig config;
private final LongSupplier globalCheckpointSupplier;
private final LongSupplier primaryTermSupplier;
private final String translogUUID;
private final TranslogDeletionPolicy deletionPolicy;
private final LongConsumer persistedSequenceNumberConsumer;
/**
* Creates a new Translog instance. This method will create a new transaction log unless the given {@link TranslogGeneration} is
* {@code null}. If the generation is {@code null} this method is destructive and will delete all files in the translog path given. If
* the generation is not {@code null}, this method tries to open the given translog generation. The generation is treated as the last
* generation referenced from already committed data. This means all operations that have not yet been committed should be in the
* translog file referenced by this generation. The translog creation will fail if this generation can't be opened.
*
* @param config the configuration of this translog
* @param translogUUID the translog uuid to open, null for a new translog
* @param deletionPolicy an instance of {@link TranslogDeletionPolicy} that controls when a translog file can be safely
* deleted
* @param globalCheckpointSupplier a supplier for the global checkpoint
* @param primaryTermSupplier a supplier for the latest value of primary term of the owning index shard. The latest term value is
* examined and stored in the header whenever a new generation is rolled. It's guaranteed from outside
* that a new generation is rolled when the term is increased. This guarantee allows to us to validate
* and reject operation whose term is higher than the primary term stored in the translog header.
* @param persistedSequenceNumberConsumer a callback that's called whenever an operation with a given sequence number is successfully
* persisted.
*/
public Translog(
final TranslogConfig config, final String translogUUID, TranslogDeletionPolicy deletionPolicy,
final LongSupplier globalCheckpointSupplier, final LongSupplier primaryTermSupplier,
final LongConsumer persistedSequenceNumberConsumer) throws IOException {
super(config.getShardId(), config.getIndexSettings());
this.config = config;
this.globalCheckpointSupplier = globalCheckpointSupplier;
this.primaryTermSupplier = primaryTermSupplier;
this.persistedSequenceNumberConsumer = persistedSequenceNumberConsumer;
this.deletionPolicy = deletionPolicy;
this.translogUUID = translogUUID;
bigArrays = config.getBigArrays();
ReadWriteLock rwl = new ReentrantReadWriteLock();
readLock = new ReleasableLock(rwl.readLock());
writeLock = new ReleasableLock(rwl.writeLock());
this.location = config.getTranslogPath();
Files.createDirectories(this.location);
try {
final Checkpoint checkpoint = readCheckpoint(location);
final Path nextTranslogFile = location.resolve(getFilename(checkpoint.generation + 1));
final Path currentCheckpointFile = location.resolve(getCommitCheckpointFileName(checkpoint.generation));
// this is special handling for error condition when we create a new writer but we fail to bake
// the newly written file (generation+1) into the checkpoint. This is still a valid state
// we just need to cleanup before we continue
// we hit this before and then blindly deleted the new generation even though we managed to bake it in and then hit this:
// https://discuss.elastic.co/t/cannot-recover-index-because-of-missing-tanslog-files/38336 as an example
//
// For this to happen we must have already copied the translog.ckp file into translog-gen.ckp so we first check if that
// file exists. If not we don't even try to clean it up and wait until we fail creating it
assert Files.exists(nextTranslogFile) == false ||
Files.size(nextTranslogFile) <= TranslogHeader.headerSizeInBytes(translogUUID) :
"unexpected translog file: [" + nextTranslogFile + "]";
if (Files.exists(currentCheckpointFile) // current checkpoint is already copied
&& Files.deleteIfExists(nextTranslogFile)) { // delete it and log a warning
logger.warn("deleted previously created, but not yet committed, next generation [{}]. This can happen due to a" +
" tragic exception when creating a new generation", nextTranslogFile.getFileName());
}
this.readers.addAll(recoverFromFiles(checkpoint));
if (readers.isEmpty()) {
throw new IllegalStateException("at least one reader must be recovered");
}
boolean success = false;
current = null;
try {
current = createWriter(checkpoint.generation + 1, getMinFileGeneration(), checkpoint.globalCheckpoint,
persistedSequenceNumberConsumer);
success = true;
} finally {
// we have to close all the recovered ones otherwise we leak file handles here
// for instance if we have a lot of tlog and we can't create the writer we keep on holding
// on to all the uncommitted tlog files if we don't close
if (success == false) {
IOUtils.closeWhileHandlingException(readers);
}
}
} catch (Exception e) {
// close the opened translog files if we fail to create a new translog...
IOUtils.closeWhileHandlingException(current);
IOUtils.closeWhileHandlingException(readers);
throw e;
}
}
/** recover all translog files found on disk */
private ArrayList<TranslogReader> recoverFromFiles(Checkpoint checkpoint) throws IOException {
boolean success = false;
ArrayList<TranslogReader> foundTranslogs = new ArrayList<>();
try (ReleasableLock ignored = writeLock.acquire()) {
logger.debug("open uncommitted translog checkpoint {}", checkpoint);
final long minGenerationToRecoverFrom = checkpoint.minTranslogGeneration;
assert minGenerationToRecoverFrom >= 0 : "minTranslogGeneration should be non-negative";
// we open files in reverse order in order to validate the translog uuid before we start traversing the translog based on
// the generation id we found in the lucene commit. This gives for better error messages if the wrong
// translog was found.
for (long i = checkpoint.generation; i >= minGenerationToRecoverFrom; i--) {
Path committedTranslogFile = location.resolve(getFilename(i));
if (Files.exists(committedTranslogFile) == false) {
throw new TranslogCorruptedException(committedTranslogFile.toString(),
"translog file doesn't exist with generation: " + i + " recovering from: " + minGenerationToRecoverFrom
+ " checkpoint: " + checkpoint.generation + " - translog ids must be consecutive");
}
final Checkpoint readerCheckpoint = i == checkpoint.generation ? checkpoint
: Checkpoint.read(location.resolve(getCommitCheckpointFileName(i)));
final TranslogReader reader = openReader(committedTranslogFile, readerCheckpoint);
assert reader.getPrimaryTerm() <= primaryTermSupplier.getAsLong() :
"Primary terms go backwards; current term [" + primaryTermSupplier.getAsLong() + "] translog path [ "
+ committedTranslogFile + ", existing term [" + reader.getPrimaryTerm() + "]";
foundTranslogs.add(reader);
logger.debug("recovered local translog from checkpoint {}", checkpoint);
}
Collections.reverse(foundTranslogs);
// when we clean up files, we first update the checkpoint with a new minReferencedTranslog and then delete them;
// if we crash just at the wrong moment, it may be that we leave one unreferenced file behind so we delete it if there
IOUtils.deleteFilesIgnoringExceptions(location.resolve(getFilename(minGenerationToRecoverFrom - 1)),
location.resolve(getCommitCheckpointFileName(minGenerationToRecoverFrom - 1)));
Path commitCheckpoint = location.resolve(getCommitCheckpointFileName(checkpoint.generation));
if (Files.exists(commitCheckpoint)) {
Checkpoint checkpointFromDisk = Checkpoint.read(commitCheckpoint);
if (checkpoint.equals(checkpointFromDisk) == false) {
throw new TranslogCorruptedException(commitCheckpoint.toString(),
"checkpoint file " + commitCheckpoint.getFileName() + " already exists but has corrupted content: expected "
+ checkpoint + " but got " + checkpointFromDisk);
}
} else {
copyCheckpointTo(commitCheckpoint);
}
success = true;
} finally {
if (success == false) {
IOUtils.closeWhileHandlingException(foundTranslogs);
}
}
return foundTranslogs;
}
private void copyCheckpointTo(Path targetPath) throws IOException {
// a temp file to copy checkpoint to - note it must be in on the same FS otherwise atomic move won't work
final Path tempFile = Files.createTempFile(location, TRANSLOG_FILE_PREFIX, CHECKPOINT_SUFFIX);
boolean tempFileRenamed = false;
try {
// we first copy this into the temp-file and then fsync it followed by an atomic move into the target file
// that way if we hit a disk-full here we are still in an consistent state.
Files.copy(location.resolve(CHECKPOINT_FILE_NAME), tempFile, StandardCopyOption.REPLACE_EXISTING);
IOUtils.fsync(tempFile, false);
Files.move(tempFile, targetPath, StandardCopyOption.ATOMIC_MOVE);
tempFileRenamed = true;
// we only fsync the directory the tempFile was already fsynced
IOUtils.fsync(targetPath.getParent(), true);
} finally {
if (tempFileRenamed == false) {
try {
Files.delete(tempFile);
} catch (IOException ex) {
logger.warn(() -> new ParameterizedMessage("failed to delete temp file {}", tempFile), ex);
}
}
}
}
TranslogReader openReader(Path path, Checkpoint checkpoint) throws IOException {
FileChannel channel = FileChannel.open(path, StandardOpenOption.READ);
try {
assert Translog.parseIdFromFileName(path) == checkpoint.generation : "expected generation: " +
Translog.parseIdFromFileName(path) + " but got: " + checkpoint.generation;
TranslogReader reader = TranslogReader.open(channel, path, checkpoint, translogUUID);
channel = null;
return reader;
} finally {
IOUtils.close(channel);
}
}
/**
* Extracts the translog generation from a file name.
*
* @throws IllegalArgumentException if the path doesn't match the expected pattern.
*/
public static long parseIdFromFileName(Path translogFile) {
final String fileName = translogFile.getFileName().toString();
final Matcher matcher = PARSE_STRICT_ID_PATTERN.matcher(fileName);
if (matcher.matches()) {
try {
return Long.parseLong(matcher.group(1));
} catch (NumberFormatException e) {
throw new IllegalStateException("number formatting issue in a file that passed PARSE_STRICT_ID_PATTERN: " +
fileName + "]", e);
}
}
throw new IllegalArgumentException("can't parse id from file: " + fileName);
}
/** Returns {@code true} if this {@code Translog} is still open. */
public boolean isOpen() {
return closed.get() == false;
}
private static boolean calledFromOutsideOrViaTragedyClose() {
List<StackTraceElement> frames = Stream.of(Thread.currentThread().getStackTrace()).
skip(3). //skip getStackTrace, current method and close method frames
limit(10). //limit depth of analysis to 10 frames, it should be enough to catch closing with, e.g. IOUtils
filter(f ->
{
try {
return Translog.class.isAssignableFrom(Class.forName(f.getClassName()));
} catch (Exception ignored) {
return false;
}
}
). //find all inner callers including Translog subclasses
collect(Collectors.toList());
//the list of inner callers should be either empty or should contain closeOnTragicEvent method
return frames.isEmpty() || frames.stream().anyMatch(f -> f.getMethodName().equals("closeOnTragicEvent"));
}
@Override
public void close() throws IOException {
assert calledFromOutsideOrViaTragedyClose() :
"Translog.close method is called from inside Translog, but not via closeOnTragicEvent method";
if (closed.compareAndSet(false, true)) {
try (ReleasableLock lock = writeLock.acquire()) {
try {
current.sync();
} finally {
closeFilesIfNoPendingRetentionLocks();
}
} finally {
logger.debug("translog closed");
}
}
}
/**
* Returns all translog locations as absolute paths.
* These paths don't contain actual translog files they are
* directories holding the transaction logs.
*/
public Path location() {
return location;
}
/**
* Returns the generation of the current transaction log.
*/
public long currentFileGeneration() {
try (ReleasableLock ignored = readLock.acquire()) {
return current.getGeneration();
}
}
/**
* Returns the minimum file generation referenced by the translog
*/
public long getMinFileGeneration() {
try (ReleasableLock ignored = readLock.acquire()) {
if (readers.isEmpty()) {
return current.getGeneration();
} else {
assert readers.stream().map(TranslogReader::getGeneration).min(Long::compareTo).get()
.equals(readers.get(0).getGeneration()) : "the first translog isn't the one with the minimum generation:" + readers;
return readers.get(0).getGeneration();
}
}
}
/**
* Returns the number of operations in the translog files
*/
public int totalOperations() {
return totalOperationsByMinGen(-1);
}
/**
* Returns the size in bytes of the v files
*/
public long sizeInBytes() {
return sizeInBytesByMinGen(-1);
}
long earliestLastModifiedAge() {
try (ReleasableLock ignored = readLock.acquire()) {
ensureOpen();
return findEarliestLastModifiedAge(System.currentTimeMillis(), readers, current);
} catch (IOException e) {
throw new TranslogException(shardId, "Unable to get the earliest last modified time for the transaction log");
}
}
/**
* Returns the age of the oldest entry in the translog files in seconds
*/
static long findEarliestLastModifiedAge(long currentTime, Iterable<TranslogReader> readers, TranslogWriter writer) throws IOException {
long earliestTime = currentTime;
for (BaseTranslogReader r : readers) {
earliestTime = Math.min(r.getLastModifiedTime(), earliestTime);
}
return Math.max(0, currentTime - Math.min(earliestTime, writer.getLastModifiedTime()));
}
/**
* Returns the number of operations in the translog files at least the given generation
*/
public int totalOperationsByMinGen(long minGeneration) {
try (ReleasableLock ignored = readLock.acquire()) {
ensureOpen();
return Stream.concat(readers.stream(), Stream.of(current))
.filter(r -> r.getGeneration() >= minGeneration)
.mapToInt(BaseTranslogReader::totalOperations)
.sum();
}
}
/**
* Returns the number of operations in the transaction files that contain operations with seq# above the given number.
*/
public int estimateTotalOperationsFromMinSeq(long minSeqNo) {
try (ReleasableLock ignored = readLock.acquire()) {
ensureOpen();
return readersAboveMinSeqNo(minSeqNo).mapToInt(BaseTranslogReader::totalOperations).sum();
}
}
/**
* Returns the size in bytes of the translog files at least the given generation
*/
public long sizeInBytesByMinGen(long minGeneration) {
try (ReleasableLock ignored = readLock.acquire()) {
ensureOpen();
return Stream.concat(readers.stream(), Stream.of(current))
.filter(r -> r.getGeneration() >= minGeneration)
.mapToLong(BaseTranslogReader::sizeInBytes)
.sum();
}
}
/**
* Creates a new translog for the specified generation.
*
* @param fileGeneration the translog generation
* @return a writer for the new translog
* @throws IOException if creating the translog failed
*/
TranslogWriter createWriter(long fileGeneration) throws IOException {
final TranslogWriter writer = createWriter(fileGeneration, getMinFileGeneration(), globalCheckpointSupplier.getAsLong(),
persistedSequenceNumberConsumer);
assert writer.sizeInBytes() == DEFAULT_HEADER_SIZE_IN_BYTES : "Mismatch translog header size; " +
"empty translog size [" + writer.sizeInBytes() + ", header size [" + DEFAULT_HEADER_SIZE_IN_BYTES + "]";
return writer;
}
/**
* creates a new writer
*
* @param fileGeneration the generation of the write to be written
* @param initialMinTranslogGen the minimum translog generation to be written in the first checkpoint. This is
* needed to solve and initialization problem while constructing an empty translog.
* With no readers and no current, a call to {@link #getMinFileGeneration()} would not work.
* @param initialGlobalCheckpoint the global checkpoint to be written in the first checkpoint.
*/
TranslogWriter createWriter(long fileGeneration, long initialMinTranslogGen, long initialGlobalCheckpoint,
LongConsumer persistedSequenceNumberConsumer) throws IOException {
final TranslogWriter newFile;
try {
newFile = TranslogWriter.create(
shardId,
translogUUID,
fileGeneration,
location.resolve(getFilename(fileGeneration)),
getChannelFactory(),
config.getBufferSize(),
initialMinTranslogGen, initialGlobalCheckpoint,
globalCheckpointSupplier, this::getMinFileGeneration, primaryTermSupplier.getAsLong(), tragedy,
persistedSequenceNumberConsumer);
} catch (final IOException e) {
throw new TranslogException(shardId, "failed to create new translog file", e);
}
return newFile;
}
/**
* Adds an operation to the transaction log.
*
* @param operation the operation to add
* @return the location of the operation in the translog
* @throws IOException if adding the operation to the translog resulted in an I/O exception
*/
public Location add(final Operation operation) throws IOException {
final ReleasableBytesStreamOutput out = new ReleasableBytesStreamOutput(bigArrays);
try {
final long start = out.position();
out.skip(Integer.BYTES);
writeOperationNoSize(new BufferedChecksumStreamOutput(out), operation);
final long end = out.position();
final int operationSize = (int) (end - Integer.BYTES - start);
out.seek(start);
out.writeInt(operationSize);
out.seek(end);
final ReleasablePagedBytesReference bytes = out.bytes();
try (ReleasableLock ignored = readLock.acquire()) {
ensureOpen();
if (operation.primaryTerm() > current.getPrimaryTerm()) {
assert false :
"Operation term is newer than the current term; "
+ "current term[" + current.getPrimaryTerm() + "], operation term[" + operation + "]";
throw new IllegalArgumentException("Operation term is newer than the current term; "
+ "current term[" + current.getPrimaryTerm() + "], operation term[" + operation + "]");
}
return current.add(bytes, operation.seqNo());
}
} catch (final AlreadyClosedException | IOException ex) {
closeOnTragicEvent(ex);
throw ex;
} catch (final Exception ex) {
closeOnTragicEvent(ex);
throw new TranslogException(shardId, "Failed to write operation [" + operation + "]", ex);
} finally {
Releasables.close(out);
}
}
/**
* Tests whether or not the translog generation should be rolled to a new generation. This test
* is based on the size of the current generation compared to the configured generation
* threshold size.
*
* @return {@code true} if the current generation should be rolled to a new generation
*/
public boolean shouldRollGeneration() {
final long threshold = this.indexSettings.getGenerationThresholdSize().getBytes();
try (ReleasableLock ignored = readLock.acquire()) {
return this.current.sizeInBytes() > threshold;
}
}
/**
* The a {@linkplain Location} that will sort after the {@linkplain Location} returned by the last write but before any locations which
* can be returned by the next write.
*/
public Location getLastWriteLocation() {
try (ReleasableLock lock = readLock.acquire()) {
/*
* We use position = current - 1 and size = Integer.MAX_VALUE here instead of position current and size = 0 for two reasons:
* 1. Translog.Location's compareTo doesn't actually pay attention to size even though it's equals method does.
* 2. It feels more right to return a *position* that is before the next write's position rather than rely on the size.
*/
return new Location(current.generation, current.sizeInBytes() - 1, Integer.MAX_VALUE);
}
}
/**
* The last synced checkpoint for this translog.
*
* @return the last synced checkpoint
*/
public long getLastSyncedGlobalCheckpoint() {
return getLastSyncedCheckpoint().globalCheckpoint;
}
final Checkpoint getLastSyncedCheckpoint() {
try (ReleasableLock ignored = readLock.acquire()) {
return current.getLastSyncedCheckpoint();
}
}
/**
* Snapshots the current transaction log allowing to safely iterate over the snapshot.
* Snapshots are fixed in time and will not be updated with future operations.
*/
public Snapshot newSnapshot() throws IOException {
try (ReleasableLock ignored = readLock.acquire()) {
return newSnapshotFromGen(new TranslogGeneration(translogUUID, getMinFileGeneration()), Long.MAX_VALUE);
}
}
public Snapshot newSnapshotFromGen(TranslogGeneration fromGeneration, long upToSeqNo) throws IOException {
try (ReleasableLock ignored = readLock.acquire()) {
ensureOpen();
final long fromFileGen = fromGeneration.translogFileGeneration;
if (fromFileGen < getMinFileGeneration()) {
throw new IllegalArgumentException("requested snapshot generation [" + fromFileGen + "] is not available. " +
"Min referenced generation is [" + getMinFileGeneration() + "]");
}
TranslogSnapshot[] snapshots = Stream.concat(readers.stream(), Stream.of(current))
.filter(reader -> reader.getGeneration() >= fromFileGen && reader.getCheckpoint().minSeqNo <= upToSeqNo)
.map(BaseTranslogReader::newSnapshot).toArray(TranslogSnapshot[]::new);
final Snapshot snapshot = newMultiSnapshot(snapshots);
if (upToSeqNo == Long.MAX_VALUE) {
return snapshot;
} else {
return new SeqNoFilterSnapshot(snapshot, Long.MIN_VALUE, upToSeqNo);
}
}
}
/**
* Reads and returns the operation from the given location if the generation it references is still available. Otherwise
* this method will return <code>null</code>.
*/
public Operation readOperation(Location location) throws IOException {
try (ReleasableLock ignored = readLock.acquire()) {
ensureOpen();
if (location.generation < getMinFileGeneration()) {
return null;
}
if (current.generation == location.generation) {
// no need to fsync here the read operation will ensure that buffers are written to disk
// if they are still in RAM and we are reading onto that position
return current.read(location);
} else {
// read backwards - it's likely we need to read on that is recent
for (int i = readers.size() - 1; i >= 0; i--) {
TranslogReader translogReader = readers.get(i);
if (translogReader.generation == location.generation) {
return translogReader.read(location);
}
}
}
} catch (final Exception ex) {
closeOnTragicEvent(ex);
throw ex;
}
return null;
}
public Snapshot newSnapshotFromMinSeqNo(long minSeqNo) throws IOException {
try (ReleasableLock ignored = readLock.acquire()) {
ensureOpen();
TranslogSnapshot[] snapshots = readersAboveMinSeqNo(minSeqNo).map(BaseTranslogReader::newSnapshot)
.toArray(TranslogSnapshot[]::new);
return newMultiSnapshot(snapshots);
}
}
private Snapshot newMultiSnapshot(TranslogSnapshot[] snapshots) throws IOException {
final Closeable onClose;
if (snapshots.length == 0) {
onClose = () -> {};
} else {
assert Arrays.stream(snapshots).map(BaseTranslogReader::getGeneration).min(Long::compareTo).get()
== snapshots[0].generation : "first reader generation of " + snapshots + " is not the smallest";
onClose = acquireTranslogGenFromDeletionPolicy(snapshots[0].generation);
}
boolean success = false;
try {
Snapshot result = new MultiSnapshot(snapshots, onClose);
success = true;
return result;
} finally {
if (success == false) {
onClose.close();
}
}
}
private Stream<? extends BaseTranslogReader> readersAboveMinSeqNo(long minSeqNo) {
assert readLock.isHeldByCurrentThread() || writeLock.isHeldByCurrentThread() :
"callers of readersAboveMinSeqNo must hold a lock: readLock ["
+ readLock.isHeldByCurrentThread() + "], writeLock [" + readLock.isHeldByCurrentThread() + "]";
return Stream.concat(readers.stream(), Stream.of(current))
.filter(reader -> {
final long maxSeqNo = reader.getCheckpoint().maxSeqNo;
return maxSeqNo == SequenceNumbers.UNASSIGNED_SEQ_NO || maxSeqNo >= minSeqNo;
});
}
/**
* Acquires a lock on the translog files, preventing them from being trimmed
*/
public Closeable acquireRetentionLock() {
try (ReleasableLock lock = readLock.acquire()) {
ensureOpen();
final long viewGen = getMinFileGeneration();
return acquireTranslogGenFromDeletionPolicy(viewGen);
}
}
private Closeable acquireTranslogGenFromDeletionPolicy(long viewGen) {
Releasable toClose = deletionPolicy.acquireTranslogGen(viewGen);
return () -> {
try {
toClose.close();
} finally {
trimUnreferencedReaders();
closeFilesIfNoPendingRetentionLocks();
}
};
}
/**
* Sync's the translog.
*/
public void sync() throws IOException {
try (ReleasableLock lock = readLock.acquire()) {
if (closed.get() == false) {
current.sync();
}
} catch (final Exception ex) {
closeOnTragicEvent(ex);
throw ex;
}
}
/**
* Returns <code>true</code> if an fsync is required to ensure durability of the translogs operations or it's metadata.
*/
public boolean syncNeeded() {
try (ReleasableLock lock = readLock.acquire()) {
return current.syncNeeded();
}
}
/** package private for testing */
public static String getFilename(long generation) {
return TRANSLOG_FILE_PREFIX + generation + TRANSLOG_FILE_SUFFIX;
}
static String getCommitCheckpointFileName(long generation) {
return TRANSLOG_FILE_PREFIX + generation + CHECKPOINT_SUFFIX;
}
/**
* Trims translog for terms of files below <code>belowTerm</code> and seq# above <code>aboveSeqNo</code>.
* Effectively it moves max visible seq# {@link Checkpoint#trimmedAboveSeqNo} therefore {@link TranslogSnapshot} skips those operations.
*/
public void trimOperations(long belowTerm, long aboveSeqNo) throws IOException {
assert aboveSeqNo >= SequenceNumbers.NO_OPS_PERFORMED : "aboveSeqNo has to a valid sequence number";
try (ReleasableLock lock = writeLock.acquire()) {
ensureOpen();
if (current.getPrimaryTerm() < belowTerm) {
throw new IllegalArgumentException("Trimming the translog can only be done for terms lower than the current one. " +
"Trim requested for term [ " + belowTerm + " ] , current is [ " + current.getPrimaryTerm() + " ]");
}
// we assume that the current translog generation doesn't have trimmable ops. Verify that.
assert current.assertNoSeqAbove(belowTerm, aboveSeqNo);
// update all existed ones (if it is necessary) as checkpoint and reader are immutable
final List<TranslogReader> newReaders = new ArrayList<>(readers.size());
try {
for (TranslogReader reader : readers) {
final TranslogReader newReader =
reader.getPrimaryTerm() < belowTerm
? reader.closeIntoTrimmedReader(aboveSeqNo, getChannelFactory())
: reader;
newReaders.add(newReader);
}
} catch (IOException e) {
IOUtils.closeWhileHandlingException(newReaders);
tragedy.setTragicException(e);
closeOnTragicEvent(e);
throw e;
}
this.readers.clear();
this.readers.addAll(newReaders);
}
}
/**
* Ensures that the given location has be synced / written to the underlying storage.
*
* @return Returns <code>true</code> iff this call caused an actual sync operation otherwise <code>false</code>
*/
public boolean ensureSynced(Location location) throws IOException {
try (ReleasableLock lock = readLock.acquire()) {
if (location.generation == current.getGeneration()) { // if we have a new one it's already synced
ensureOpen();
return current.syncUpTo(location.translogLocation + location.size);
}
} catch (final Exception ex) {
closeOnTragicEvent(ex);
throw ex;
}
return false;
}
/**
* Ensures that all locations in the given stream have been synced / written to the underlying storage.
* This method allows for internal optimization to minimize the amount of fsync operations if multiple
* locations must be synced.
*
* @return Returns <code>true</code> iff this call caused an actual sync operation otherwise <code>false</code>
*/
public boolean ensureSynced(Stream<Location> locations) throws IOException {
final Optional<Location> max = locations.max(Location::compareTo);
// we only need to sync the max location since it will sync all other
// locations implicitly
if (max.isPresent()) {
return ensureSynced(max.get());
} else {
return false;
}
}
/**
* Closes the translog if the current translog writer experienced a tragic exception.
*
* Note that in case this thread closes the translog it must not already be holding a read lock on the translog as it will acquire a
* write lock in the course of closing the translog
*
* @param ex if an exception occurs closing the translog, it will be suppressed into the provided exception
*/
protected void closeOnTragicEvent(final Exception ex) {
// we can not hold a read lock here because closing will attempt to obtain a write lock and that would result in self-deadlock
assert readLock.isHeldByCurrentThread() == false : Thread.currentThread().getName();
if (tragedy.get() != null) {
try {
close();
} catch (final AlreadyClosedException inner) {
/*
* Don't do anything in this case. The AlreadyClosedException comes from TranslogWriter and we should not add it as
* suppressed because it will contain the provided exception as its cause. See also
* https://github.com/elastic/elasticsearch/issues/15941.
*/
} catch (final Exception inner) {
assert ex != inner.getCause();
ex.addSuppressed(inner);
}
}
}
/**
* return stats
*/
public TranslogStats stats() {
// acquire lock to make the two numbers roughly consistent (no file change half way)
try (ReleasableLock lock = readLock.acquire()) {
final long uncommittedGen = deletionPolicy.getTranslogGenerationOfLastCommit();
return new TranslogStats(totalOperations(), sizeInBytes(), totalOperationsByMinGen(uncommittedGen),
sizeInBytesByMinGen(uncommittedGen), earliestLastModifiedAge());
}
}
public TranslogConfig getConfig() {
return config;
}
// public for testing
public TranslogDeletionPolicy getDeletionPolicy() {
return deletionPolicy;
}
public static class Location implements Comparable<Location> {
public final long generation;
public final long translogLocation;
public final int size;
public Location(long generation, long translogLocation, int size) {
this.generation = generation;
this.translogLocation = translogLocation;
this.size = size;
}
@Override
public String toString() {
return "[generation: " + generation + ", location: " + translogLocation + ", size: " + size + "]";
}
@Override
public int compareTo(Location o) {
if (generation == o.generation) {
return Long.compare(translogLocation, o.translogLocation);
}
return Long.compare(generation, o.generation);
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
Location location = (Location) o;
if (generation != location.generation) {
return false;
}
if (translogLocation != location.translogLocation) {
return false;
}
return size == location.size;
}
@Override
public int hashCode() {
int result = Long.hashCode(generation);
result = 31 * result + Long.hashCode(translogLocation);
result = 31 * result + size;
return result;
}
}
/**
* A snapshot of the transaction log, allows to iterate over all the transaction log operations.
*/
public interface Snapshot extends Closeable {
/**
* The total estimated number of operations in the snapshot.
*/
int totalOperations();
/**
* The number of operations have been skipped (overridden or trimmed) in the snapshot so far.
* Unlike {@link #totalOperations()}, this value is updated each time after {@link #next()}) is called.
*/
default int skippedOperations() {
return 0;
}
/**
* Returns the next operation in the snapshot or <code>null</code> if we reached the end.
*/
Translog.Operation next() throws IOException;
}
/**
* A filtered snapshot consisting of only operations whose sequence numbers are in the given range
* between {@code fromSeqNo} (inclusive) and {@code toSeqNo} (inclusive). This filtered snapshot
* shares the same underlying resources with the {@code delegate} snapshot, therefore we should not
* use the {@code delegate} after passing it to this filtered snapshot.
*/
static final class SeqNoFilterSnapshot implements Snapshot {
private final Snapshot delegate;
private int filteredOpsCount;
private final long fromSeqNo; // inclusive
private final long toSeqNo; // inclusive
SeqNoFilterSnapshot(Snapshot delegate, long fromSeqNo, long toSeqNo) {
assert fromSeqNo <= toSeqNo : "from_seq_no[" + fromSeqNo + "] > to_seq_no[" + toSeqNo + "]";
this.delegate = delegate;
this.fromSeqNo = fromSeqNo;
this.toSeqNo = toSeqNo;
}
@Override
public int totalOperations() {
return delegate.totalOperations();
}
@Override
public int skippedOperations() {
return filteredOpsCount + delegate.skippedOperations();
}
@Override
public Operation next() throws IOException {
Translog.Operation op;
while ((op = delegate.next()) != null) {
if (fromSeqNo <= op.seqNo() && op.seqNo() <= toSeqNo) {
return op;
} else {
filteredOpsCount++;
}
}
return null;
}