Skip to content

Commit 6dc03ec

Browse files
authored
Remove unused Translog#read method (#20598)
Translog#read is a left-over from realtime-get that allows to read from an arbitrary location in the transaction log. This method is unused and can be replaced with snapshots in tests.
1 parent b3e5e6a commit 6dc03ec

File tree

4 files changed

+51
-138
lines changed

4 files changed

+51
-138
lines changed

core/src/main/java/org/elasticsearch/index/translog/BaseTranslogReader.java

Lines changed: 11 additions & 24 deletions
Original file line numberDiff line numberDiff line change
@@ -58,34 +58,21 @@ public final long getFirstOperationOffset() {
5858
return firstOperationOffset;
5959
}
6060

61-
public Translog.Operation read(Translog.Location location) throws IOException {
62-
assert location.generation == generation : "read location's translog generation [" + location.generation + "] is not [" + generation + "]";
63-
ByteBuffer buffer = ByteBuffer.allocate(location.size);
64-
try (BufferedChecksumStreamInput checksumStreamInput = checksummedStream(buffer, location.translogLocation, location.size, null)) {
65-
return read(checksumStreamInput);
66-
}
67-
}
68-
6961
/** read the size of the op (i.e., number of bytes, including the op size) written at the given position */
70-
protected final int readSize(ByteBuffer reusableBuffer, long position) {
62+
protected final int readSize(ByteBuffer reusableBuffer, long position) throws IOException {
7163
// read op size from disk
7264
assert reusableBuffer.capacity() >= 4 : "reusable buffer must have capacity >=4 when reading opSize. got [" + reusableBuffer.capacity() + "]";
73-
try {
74-
reusableBuffer.clear();
75-
reusableBuffer.limit(4);
76-
readBytes(reusableBuffer, position);
77-
reusableBuffer.flip();
78-
// Add an extra 4 to account for the operation size integer itself
79-
final int size = reusableBuffer.getInt() + 4;
80-
final long maxSize = sizeInBytes() - position;
81-
if (size < 0 || size > maxSize) {
82-
throw new TranslogCorruptedException("operation size is corrupted must be [0.." + maxSize + "] but was: " + size);
83-
}
84-
85-
return size;
86-
} catch (IOException e) {
87-
throw new ElasticsearchException("unexpected exception reading from translog snapshot of " + this.path, e);
65+
reusableBuffer.clear();
66+
reusableBuffer.limit(4);
67+
readBytes(reusableBuffer, position);
68+
reusableBuffer.flip();
69+
// Add an extra 4 to account for the operation size integer itself
70+
final int size = reusableBuffer.getInt() + 4;
71+
final long maxSize = sizeInBytes() - position;
72+
if (size < 0 || size > maxSize) {
73+
throw new TranslogCorruptedException("operation size is corrupted must be [0.." + maxSize + "] but was: " + size);
8874
}
75+
return size;
8976
}
9077

9178
public Translog.Snapshot newSnapshot() {

core/src/main/java/org/elasticsearch/index/translog/Translog.java

Lines changed: 0 additions & 32 deletions
Original file line numberDiff line numberDiff line change
@@ -384,31 +384,6 @@ TranslogWriter createWriter(long fileGeneration) throws IOException {
384384
return newFile;
385385
}
386386

387-
388-
/**
389-
* Read the Operation object from the given location. This method will try to read the given location from
390-
* the current or from the currently committing translog file. If the location is in a file that has already
391-
* been closed or even removed the method will return <code>null</code> instead.
392-
*/
393-
Translog.Operation read(Location location) { // TODO this is only here for testing - we can remove it?
394-
try (ReleasableLock lock = readLock.acquire()) {
395-
final BaseTranslogReader reader;
396-
final long currentGeneration = current.getGeneration();
397-
if (currentGeneration == location.generation) {
398-
reader = current;
399-
} else if (readers.isEmpty() == false && readers.get(readers.size() - 1).getGeneration() == location.generation) {
400-
reader = readers.get(readers.size() - 1);
401-
} else if (currentGeneration < location.generation) {
402-
throw new IllegalStateException("location generation [" + location.generation + "] is greater than the current generation [" + currentGeneration + "]");
403-
} else {
404-
return null;
405-
}
406-
return reader.read(location);
407-
} catch (IOException e) {
408-
throw new ElasticsearchException("failed to read source from translog location " + location, e);
409-
}
410-
}
411-
412387
/**
413388
* Adds a delete / index operations to the transaction log.
414389
*
@@ -432,7 +407,6 @@ public Location add(Operation operation) throws IOException {
432407
try (ReleasableLock lock = readLock.acquire()) {
433408
ensureOpen();
434409
Location location = current.add(bytes);
435-
assert assertBytesAtLocation(location, bytes);
436410
return location;
437411
}
438412
} catch (AlreadyClosedException | IOException ex) {
@@ -469,12 +443,6 @@ public Location getLastWriteLocation() {
469443
}
470444
}
471445

472-
boolean assertBytesAtLocation(Translog.Location location, BytesReference expectedBytes) throws IOException {
473-
// tests can override this
474-
ByteBuffer buffer = ByteBuffer.allocate(location.size);
475-
current.readBytes(buffer, location.translogLocation);
476-
return new BytesArray(buffer.array()).equals(expectedBytes);
477-
}
478446

479447
/**
480448
* Snapshots the current transaction log allowing to safely iterate over the snapshot.

core/src/main/java/org/elasticsearch/index/translog/TranslogSnapshot.java

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -26,7 +26,7 @@
2626
import java.nio.channels.FileChannel;
2727
import java.nio.file.Path;
2828

29-
public class TranslogSnapshot extends BaseTranslogReader implements Translog.Snapshot {
29+
final class TranslogSnapshot extends BaseTranslogReader implements Translog.Snapshot {
3030

3131
private final int totalOperations;
3232
protected final long length;
@@ -51,7 +51,7 @@ public TranslogSnapshot(long generation, FileChannel channel, Path path, long fi
5151
}
5252

5353
@Override
54-
public final int totalOperations() {
54+
public int totalOperations() {
5555
return totalOperations;
5656
}
5757

@@ -64,7 +64,7 @@ public Translog.Operation next() throws IOException {
6464
}
6565
}
6666

67-
protected final Translog.Operation readOperation() throws IOException {
67+
protected Translog.Operation readOperation() throws IOException {
6868
final int opSize = readSize(reusableBuffer, position);
6969
reuse = checksummedStream(reusableBuffer, position, opSize, reuse);
7070
Translog.Operation op = read(reuse);

core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java

Lines changed: 37 additions & 79 deletions
Original file line numberDiff line numberDiff line change
@@ -85,6 +85,7 @@
8585
import java.util.concurrent.atomic.AtomicInteger;
8686
import java.util.concurrent.atomic.AtomicLong;
8787
import java.util.concurrent.atomic.AtomicReference;
88+
import java.util.stream.Collectors;
8889

8990
import static org.hamcrest.Matchers.equalTo;
9091
import static org.hamcrest.Matchers.greaterThan;
@@ -206,53 +207,6 @@ private String randomNonTranslogPatternString(int min, int max) {
206207
return string;
207208
}
208209

209-
public void testRead() throws IOException {
210-
Location loc0 = translog.getLastWriteLocation();
211-
assertNotNull(loc0);
212-
213-
Translog.Location loc1 = translog.add(new Translog.Index("test", "1", new byte[]{1}));
214-
assertThat(loc1, greaterThan(loc0));
215-
assertThat(translog.getLastWriteLocation(), greaterThan(loc1));
216-
Translog.Location loc2 = translog.add(new Translog.Index("test", "2", new byte[]{2}));
217-
assertThat(loc2, greaterThan(loc1));
218-
assertThat(translog.getLastWriteLocation(), greaterThan(loc2));
219-
assertThat(translog.read(loc1).getSource().source, equalTo(new BytesArray(new byte[]{1})));
220-
assertThat(translog.read(loc2).getSource().source, equalTo(new BytesArray(new byte[]{2})));
221-
222-
Translog.Location lastLocBeforeSync = translog.getLastWriteLocation();
223-
translog.sync();
224-
assertEquals(lastLocBeforeSync, translog.getLastWriteLocation());
225-
assertThat(translog.read(loc1).getSource().source, equalTo(new BytesArray(new byte[]{1})));
226-
assertThat(translog.read(loc2).getSource().source, equalTo(new BytesArray(new byte[]{2})));
227-
228-
Translog.Location loc3 = translog.add(new Translog.Index("test", "2", new byte[]{3}));
229-
assertThat(loc3, greaterThan(loc2));
230-
assertThat(translog.getLastWriteLocation(), greaterThan(loc3));
231-
assertThat(translog.read(loc3).getSource().source, equalTo(new BytesArray(new byte[]{3})));
232-
233-
lastLocBeforeSync = translog.getLastWriteLocation();
234-
translog.sync();
235-
assertEquals(lastLocBeforeSync, translog.getLastWriteLocation());
236-
assertThat(translog.read(loc3).getSource().source, equalTo(new BytesArray(new byte[]{3})));
237-
translog.prepareCommit();
238-
/*
239-
* The commit adds to the lastWriteLocation even though is isn't really a write. This is just an implementation artifact but it can
240-
* safely be ignored because the lastWriteLocation continues to be greater than the Location returned from the last write operation
241-
* and less than the location of the next write operation.
242-
*/
243-
assertThat(translog.getLastWriteLocation(), greaterThan(lastLocBeforeSync));
244-
assertThat(translog.read(loc3).getSource().source, equalTo(new BytesArray(new byte[]{3})));
245-
translog.commit();
246-
assertNull(translog.read(loc1));
247-
assertNull(translog.read(loc2));
248-
assertNull(translog.read(loc3));
249-
try {
250-
translog.read(new Translog.Location(translog.currentFileGeneration() + 1, 17, 35));
251-
fail("generation is greater than the current");
252-
} catch (IllegalStateException ex) {
253-
// expected
254-
}
255-
}
256210

257211
public void testSimpleOperations() throws IOException {
258212
ArrayList<Translog.Operation> ops = new ArrayList<>();
@@ -441,7 +395,7 @@ public void assertFileDeleted(Translog translog, long id) {
441395
assertFalse("translog [" + id + "] still exists", Files.exists(translog.location().resolve(Translog.getFilename(id))));
442396
}
443397

444-
static class LocationOperation {
398+
static class LocationOperation implements Comparable<LocationOperation> {
445399
final Translog.Operation operation;
446400
final Translog.Location location;
447401

@@ -450,6 +404,10 @@ public LocationOperation(Translog.Operation operation, Translog.Location locatio
450404
this.location = location;
451405
}
452406

407+
@Override
408+
public int compareTo(LocationOperation o) {
409+
return location.compareTo(o.location);
410+
}
453411
}
454412

455413
public void testConcurrentWritesWithVaryingSize() throws Throwable {
@@ -478,8 +436,12 @@ public void testConcurrentWritesWithVaryingSize() throws Throwable {
478436
threads[i].join(60 * 1000);
479437
}
480438

481-
for (LocationOperation locationOperation : writtenOperations) {
482-
Translog.Operation op = translog.read(locationOperation.location);
439+
List<LocationOperation> collect = writtenOperations.stream().collect(Collectors.toList());
440+
Collections.sort(collect);
441+
Translog.Snapshot snapshot = translog.newSnapshot();
442+
for (LocationOperation locationOperation : collect) {
443+
Translog.Operation op = snapshot.next();
444+
assertNotNull(op);
483445
Translog.Operation expectedOp = locationOperation.operation;
484446
assertEquals(expectedOp.opType(), op.opType());
485447
switch (op.opType()) {
@@ -505,6 +467,7 @@ public void testConcurrentWritesWithVaryingSize() throws Throwable {
505467
}
506468

507469
}
470+
assertNull(snapshot.next());
508471

509472
}
510473

@@ -521,13 +484,16 @@ public void testTranslogChecksums() throws Exception {
521484
corruptTranslogs(translogDir);
522485

523486
AtomicInteger corruptionsCaught = new AtomicInteger(0);
487+
Translog.Snapshot snapshot = translog.newSnapshot();
524488
for (Translog.Location location : locations) {
525489
try {
526-
translog.read(location);
490+
Translog.Operation next = snapshot.next();
491+
assertNotNull(next);
527492
} catch (TranslogCorruptedException e) {
528493
corruptionsCaught.incrementAndGet();
529494
}
530495
}
496+
expectThrows(TranslogCorruptedException.class, () -> snapshot.next());
531497
assertThat("at least one corruption was caused and caught", corruptionsCaught.get(), greaterThanOrEqualTo(1));
532498
}
533499

@@ -544,15 +510,12 @@ public void testTruncatedTranslogs() throws Exception {
544510
truncateTranslogs(translogDir);
545511

546512
AtomicInteger truncations = new AtomicInteger(0);
513+
Translog.Snapshot snap = translog.newSnapshot();
547514
for (Translog.Location location : locations) {
548515
try {
549-
translog.read(location);
550-
} catch (ElasticsearchException e) {
551-
if (e.getCause() instanceof EOFException) {
552-
truncations.incrementAndGet();
553-
} else {
554-
throw e;
555-
}
516+
assertNotNull(snap.next());
517+
} catch (EOFException e) {
518+
truncations.incrementAndGet();
556519
}
557520
}
558521
assertThat("at least one truncation was caused and caught", truncations.get(), greaterThanOrEqualTo(1));
@@ -860,8 +823,14 @@ public void testLocationComparison() throws IOException {
860823
}
861824

862825
assertEquals(max.generation, translog.currentFileGeneration());
863-
final Translog.Operation read = translog.read(max);
864-
assertEquals(read.getSource().source.utf8ToString(), Integer.toString(count));
826+
Translog.Snapshot snap = translog.newSnapshot();
827+
Translog.Operation next;
828+
Translog.Operation maxOp = null;
829+
while ((next = snap.next()) != null) {
830+
maxOp = next;
831+
}
832+
assertNotNull(maxOp);
833+
assertEquals(maxOp.getSource().source.utf8ToString(), Integer.toString(count));
865834
}
866835

867836
public static Translog.Location max(Translog.Location a, Translog.Location b) {
@@ -884,30 +853,24 @@ public void testBasicCheckpoint() throws IOException {
884853
}
885854
}
886855
assertEquals(translogOperations, translog.totalOperations());
887-
final Translog.Location lastLocation = translog.add(new Translog.Index("test", "" + translogOperations, Integer.toString(translogOperations).getBytes(Charset.forName("UTF-8"))));
856+
translog.add(new Translog.Index("test", "" + translogOperations, Integer.toString(translogOperations).getBytes(Charset.forName("UTF-8"))));
888857

889858
final Checkpoint checkpoint = Checkpoint.read(translog.location().resolve(Translog.CHECKPOINT_FILE_NAME));
890859
try (final TranslogReader reader = translog.openReader(translog.location().resolve(Translog.getFilename(translog.currentFileGeneration())), checkpoint)) {
891860
assertEquals(lastSynced + 1, reader.totalOperations());
861+
Translog.Snapshot snapshot = reader.newSnapshot();
862+
892863
for (int op = 0; op < translogOperations; op++) {
893-
Translog.Location location = locations.get(op);
894864
if (op <= lastSynced) {
895-
final Translog.Operation read = reader.read(location);
865+
final Translog.Operation read = snapshot.next();
896866
assertEquals(Integer.toString(op), read.getSource().source.utf8ToString());
897867
} else {
898-
try {
899-
reader.read(location);
900-
fail("read past checkpoint");
901-
} catch (EOFException ex) {
902-
903-
}
868+
Translog.Operation next = snapshot.next();
869+
assertNull(next);
904870
}
905871
}
906-
try {
907-
reader.read(lastLocation);
908-
fail("read past checkpoint");
909-
} catch (EOFException ex) {
910-
}
872+
Translog.Operation next = snapshot.next();
873+
assertNull(next);
911874
}
912875
assertEquals(translogOperations + 1, translog.totalOperations());
913876
translog.close();
@@ -1618,11 +1581,6 @@ ChannelFactory getChannelFactory() {
16181581
}
16191582
};
16201583
}
1621-
1622-
@Override
1623-
protected boolean assertBytesAtLocation(Location location, BytesReference expectedBytes) throws IOException {
1624-
return true; // we don't wanna fail in the assert
1625-
}
16261584
};
16271585
}
16281586

0 commit comments

Comments
 (0)