Skip to content

Commit f1930ae

Browse files
authored
Expose the list of completed ranges in SparseFileTracker (#64145)
This commit adds a new method getCompletedRanges() to the SparseFileTracker class. This will be useful for the persistent cache that will need to retrieve the list of completed ranges before fsyncing a cache file.
1 parent ebed2d2 commit f1930ae

File tree

2 files changed

+77
-1
lines changed

2 files changed

+77
-1
lines changed

x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/cache/SparseFileTracker.java

+17
Original file line numberDiff line numberDiff line change
@@ -49,6 +49,23 @@ public long getLength() {
4949
return length;
5050
}
5151

52+
public List<Tuple<Long, Long>> getCompletedRanges() {
53+
List<Tuple<Long, Long>> completedRanges = null;
54+
synchronized (mutex) {
55+
assert invariant();
56+
for (Range range : ranges) {
57+
if (range.isPending()) {
58+
continue;
59+
}
60+
if (completedRanges == null) {
61+
completedRanges = new ArrayList<>();
62+
}
63+
completedRanges.add(Tuple.tuple(range.start, range.end));
64+
}
65+
}
66+
return completedRanges == null ? Collections.emptyList() : completedRanges;
67+
}
68+
5269
/**
5370
* @return the sum of the length of the ranges
5471
*/

x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/cache/SparseFileTrackerTests.java

+60-1
Original file line numberDiff line numberDiff line change
@@ -12,9 +12,13 @@
1212
import org.elasticsearch.common.settings.Settings;
1313
import org.elasticsearch.test.ESTestCase;
1414

15+
import java.util.Collections;
16+
import java.util.Comparator;
1517
import java.util.HashSet;
18+
import java.util.LinkedList;
1619
import java.util.List;
1720
import java.util.Set;
21+
import java.util.TreeSet;
1822
import java.util.concurrent.CountDownLatch;
1923
import java.util.concurrent.Semaphore;
2024
import java.util.concurrent.atomic.AtomicBoolean;
@@ -26,6 +30,7 @@
2630
import static org.hamcrest.Matchers.empty;
2731
import static org.hamcrest.Matchers.equalTo;
2832
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
33+
import static org.hamcrest.Matchers.hasSize;
2934
import static org.hamcrest.Matchers.is;
3035
import static org.hamcrest.Matchers.lessThanOrEqualTo;
3136

@@ -408,6 +413,58 @@ public void testThreadSafety() throws InterruptedException {
408413
checkThread.join();
409414
}
410415

416+
public void testCompletedRanges() {
417+
final byte[] fileContents = new byte[between(0, 1000)];
418+
final SparseFileTracker sparseFileTracker = new SparseFileTracker("test", fileContents.length);
419+
420+
final Set<AtomicBoolean> listenersCalled = new HashSet<>();
421+
final Set<SparseFileTracker.Gap> gapsProcessed = Collections.synchronizedSet(
422+
new TreeSet<>(Comparator.comparingLong(SparseFileTracker.Gap::start))
423+
);
424+
for (int i = between(0, 10); i > 0; i--) {
425+
waitForRandomRange(fileContents, sparseFileTracker, listenersCalled::add, gap -> {
426+
if (processGap(fileContents, gap)) {
427+
gapsProcessed.add(gap);
428+
}
429+
});
430+
assertTrue(listenersCalled.stream().allMatch(AtomicBoolean::get));
431+
}
432+
433+
// merge adjacent processed ranges as the SparseFileTracker does internally when a gap is completed
434+
// in order to check that SparseFileTracker.getCompletedRanges() returns the expected values
435+
final List<Tuple<Long, Long>> expectedCompletedRanges = gapsProcessed.stream()
436+
.map(gap -> Tuple.tuple(gap.start(), gap.end()))
437+
.collect(LinkedList::new, (gaps, gap) -> {
438+
if (gaps.isEmpty()) {
439+
gaps.add(gap);
440+
} else {
441+
final Tuple<Long, Long> previous = gaps.removeLast();
442+
if (previous.v2().equals(gap.v1())) {
443+
gaps.add(Tuple.tuple(previous.v1(), gap.v2()));
444+
} else {
445+
gaps.add(previous);
446+
gaps.add(gap);
447+
}
448+
}
449+
}, (gaps1, gaps2) -> {
450+
if (gaps1.isEmpty() == false && gaps2.isEmpty() == false) {
451+
final Tuple<Long, Long> last = gaps1.removeLast();
452+
final Tuple<Long, Long> first = gaps2.removeFirst();
453+
if (last.v2().equals(first.v1())) {
454+
gaps1.add(Tuple.tuple(last.v1(), first.v2()));
455+
} else {
456+
gaps1.add(last);
457+
gaps2.add(first);
458+
}
459+
}
460+
gaps1.addAll(gaps2);
461+
});
462+
463+
final List<Tuple<Long, Long>> completedRanges = sparseFileTracker.getCompletedRanges();
464+
assertThat(completedRanges, hasSize(expectedCompletedRanges.size()));
465+
assertThat(completedRanges, equalTo(expectedCompletedRanges));
466+
}
467+
411468
private static void checkRandomAbsentRange(byte[] fileContents, SparseFileTracker sparseFileTracker, boolean expectExact) {
412469
final long checkStart = randomLongBetween(0, fileContents.length - 1);
413470
final long checkEnd = randomLongBetween(0, fileContents.length);
@@ -487,19 +544,21 @@ public void onFailure(Exception e) {
487544
}
488545
}
489546

490-
private static void processGap(byte[] fileContents, SparseFileTracker.Gap gap) {
547+
private static boolean processGap(byte[] fileContents, SparseFileTracker.Gap gap) {
491548
for (long i = gap.start(); i < gap.end(); i++) {
492549
assertThat(fileContents[toIntBytes(i)], equalTo(UNAVAILABLE));
493550
}
494551

495552
if (randomBoolean()) {
496553
gap.onFailure(new ElasticsearchException("simulated"));
554+
return false;
497555
} else {
498556
for (long i = gap.start(); i < gap.end(); i++) {
499557
fileContents[toIntBytes(i)] = AVAILABLE;
500558
gap.onProgress(i + 1L);
501559
}
502560
gap.onCompletion();
561+
return true;
503562
}
504563
}
505564
}

0 commit comments

Comments
 (0)