Skip to content

Commit 6742d9c

Browse files
Cleanup Redundant Futures in Recovery Code (#48805)
Follow up to #48110 cleaning up the redundant future uses that were left over from that change.
1 parent 0298b6e commit 6742d9c

File tree

9 files changed

+74
-81
lines changed

9 files changed

+74
-81
lines changed

server/src/main/java/org/elasticsearch/index/shard/IndexShard.java

+30-41
Original file line numberDiff line numberDiff line change
@@ -54,6 +54,7 @@
5454
import org.elasticsearch.cluster.routing.RecoverySource.SnapshotRecoverySource;
5555
import org.elasticsearch.cluster.routing.ShardRouting;
5656
import org.elasticsearch.common.Booleans;
57+
import org.elasticsearch.common.CheckedConsumer;
5758
import org.elasticsearch.common.CheckedFunction;
5859
import org.elasticsearch.common.CheckedRunnable;
5960
import org.elasticsearch.common.Nullable;
@@ -1787,34 +1788,38 @@ public ShardPath shardPath() {
17871788
return path;
17881789
}
17891790

1790-
public boolean recoverFromLocalShards(BiConsumer<String, MappingMetaData> mappingUpdateConsumer,
1791-
List<IndexShard> localShards) throws IOException {
1791+
public void recoverFromLocalShards(BiConsumer<String, MappingMetaData> mappingUpdateConsumer, List<IndexShard> localShards,
1792+
ActionListener<Boolean> listener) throws IOException {
17921793
assert shardRouting.primary() : "recover from local shards only makes sense if the shard is a primary shard";
17931794
assert recoveryState.getRecoverySource().getType() == RecoverySource.Type.LOCAL_SHARDS : "invalid recovery type: " +
17941795
recoveryState.getRecoverySource();
17951796
final List<LocalShardSnapshot> snapshots = new ArrayList<>();
1797+
final ActionListener<Boolean> recoveryListener = ActionListener.runBefore(listener, () -> IOUtils.close(snapshots));
1798+
boolean success = false;
17961799
try {
17971800
for (IndexShard shard : localShards) {
17981801
snapshots.add(new LocalShardSnapshot(shard));
17991802
}
1800-
18011803
// we are the first primary, recover from the gateway
18021804
// if its post api allocation, the index should exists
18031805
assert shardRouting.primary() : "recover from local shards only makes sense if the shard is a primary shard";
18041806
StoreRecovery storeRecovery = new StoreRecovery(shardId, logger);
1805-
return storeRecovery.recoverFromLocalShards(mappingUpdateConsumer, this, snapshots);
1807+
storeRecovery.recoverFromLocalShards(mappingUpdateConsumer, this, snapshots, recoveryListener);
1808+
success = true;
18061809
} finally {
1807-
IOUtils.close(snapshots);
1810+
if (success == false) {
1811+
IOUtils.close(snapshots);
1812+
}
18081813
}
18091814
}
18101815

1811-
public boolean recoverFromStore() {
1816+
public void recoverFromStore(ActionListener<Boolean> listener) {
18121817
// we are the first primary, recover from the gateway
18131818
// if its post api allocation, the index should exists
18141819
assert shardRouting.primary() : "recover from store only makes sense if the shard is a primary shard";
18151820
assert shardRouting.initializing() : "can only start recovery on initializing shard";
18161821
StoreRecovery storeRecovery = new StoreRecovery(shardId, logger);
1817-
return storeRecovery.recoverFromStore(this);
1822+
storeRecovery.recoverFromStore(this, listener);
18181823
}
18191824

18201825
public void restoreFromRepository(Repository repository, ActionListener<Boolean> listener) {
@@ -2484,17 +2489,7 @@ public void startRecovery(RecoveryState recoveryState, PeerRecoveryTargetService
24842489
switch (recoveryState.getRecoverySource().getType()) {
24852490
case EMPTY_STORE:
24862491
case EXISTING_STORE:
2487-
markAsRecovering("from store", recoveryState); // mark the shard as recovering on the cluster state thread
2488-
threadPool.generic().execute(() -> {
2489-
try {
2490-
if (recoverFromStore()) {
2491-
recoveryListener.onRecoveryDone(recoveryState);
2492-
}
2493-
} catch (Exception e) {
2494-
recoveryListener.onRecoveryFailure(recoveryState,
2495-
new RecoveryFailedException(recoveryState, null, e), true);
2496-
}
2497-
});
2492+
executeRecovery("from store", recoveryState, recoveryListener, this::recoverFromStore);
24982493
break;
24992494
case PEER:
25002495
try {
@@ -2507,17 +2502,9 @@ public void startRecovery(RecoveryState recoveryState, PeerRecoveryTargetService
25072502
}
25082503
break;
25092504
case SNAPSHOT:
2510-
markAsRecovering("from snapshot", recoveryState); // mark the shard as recovering on the cluster state thread
2511-
SnapshotRecoverySource recoverySource = (SnapshotRecoverySource) recoveryState.getRecoverySource();
2512-
threadPool.generic().execute(
2513-
ActionRunnable.<Boolean>wrap(ActionListener.wrap(r -> {
2514-
if (r) {
2515-
recoveryListener.onRecoveryDone(recoveryState);
2516-
}
2517-
},
2518-
e -> recoveryListener.onRecoveryFailure(recoveryState, new RecoveryFailedException(recoveryState, null, e), true)),
2519-
restoreListener -> restoreFromRepository(
2520-
repositoriesService.repository(recoverySource.snapshot().getRepository()), restoreListener)));
2505+
final String repo = ((SnapshotRecoverySource) recoveryState.getRecoverySource()).snapshot().getRepository();
2506+
executeRecovery("from snapshot",
2507+
recoveryState, recoveryListener, l -> restoreFromRepository(repositoriesService.repository(repo), l));
25212508
break;
25222509
case LOCAL_SHARDS:
25232510
final IndexMetaData indexMetaData = indexSettings().getIndexMetaData();
@@ -2542,18 +2529,9 @@ public void startRecovery(RecoveryState recoveryState, PeerRecoveryTargetService
25422529

25432530
if (numShards == startedShards.size()) {
25442531
assert requiredShards.isEmpty() == false;
2545-
markAsRecovering("from local shards", recoveryState); // mark the shard as recovering on the cluster state thread
2546-
threadPool.generic().execute(() -> {
2547-
try {
2548-
if (recoverFromLocalShards(mappingUpdateConsumer, startedShards.stream()
2549-
.filter((s) -> requiredShards.contains(s.shardId())).collect(Collectors.toList()))) {
2550-
recoveryListener.onRecoveryDone(recoveryState);
2551-
}
2552-
} catch (Exception e) {
2553-
recoveryListener.onRecoveryFailure(recoveryState,
2554-
new RecoveryFailedException(recoveryState, null, e), true);
2555-
}
2556-
});
2532+
executeRecovery("from local shards", recoveryState, recoveryListener,
2533+
l -> recoverFromLocalShards(mappingUpdateConsumer,
2534+
startedShards.stream().filter((s) -> requiredShards.contains(s.shardId())).collect(Collectors.toList()), l));
25572535
} else {
25582536
final RuntimeException e;
25592537
if (numShards == -1) {
@@ -2571,6 +2549,17 @@ public void startRecovery(RecoveryState recoveryState, PeerRecoveryTargetService
25712549
}
25722550
}
25732551

2552+
private void executeRecovery(String reason, RecoveryState recoveryState, PeerRecoveryTargetService.RecoveryListener recoveryListener,
2553+
CheckedConsumer<ActionListener<Boolean>, Exception> action) {
2554+
markAsRecovering(reason, recoveryState); // mark the shard as recovering on the cluster state thread
2555+
threadPool.generic().execute(ActionRunnable.wrap(ActionListener.wrap(r -> {
2556+
if (r) {
2557+
recoveryListener.onRecoveryDone(recoveryState);
2558+
}
2559+
},
2560+
e -> recoveryListener.onRecoveryFailure(recoveryState, new RecoveryFailedException(recoveryState, null, e), true)), action));
2561+
}
2562+
25742563
/**
25752564
* Returns whether the shard is a relocated primary, i.e. not in charge anymore of replicating changes (see {@link ReplicationTracker}).
25762565
*/

server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java

+13-20
Original file line numberDiff line numberDiff line change
@@ -31,7 +31,6 @@
3131
import org.apache.lucene.store.IndexInput;
3232
import org.elasticsearch.ExceptionsHelper;
3333
import org.elasticsearch.action.ActionListener;
34-
import org.elasticsearch.action.support.PlainActionFuture;
3534
import org.elasticsearch.cluster.metadata.IndexMetaData;
3635
import org.elasticsearch.cluster.metadata.MappingMetaData;
3736
import org.elasticsearch.cluster.routing.RecoverySource;
@@ -81,31 +80,27 @@ final class StoreRecovery {
8180
* exist on disk ie. has been previously allocated or if the shard is a brand new allocation without pre-existing index
8281
* files / transaction logs. This
8382
* @param indexShard the index shard instance to recovery the shard into
84-
* @return <code>true</code> if the shard has been recovered successfully, <code>false</code> if the recovery
85-
* has been ignored due to a concurrent modification of if the clusters state has changed due to async updates.
83+
* @param listener resolves to <code>true</code> if the shard has been recovered successfully, <code>false</code> if the recovery
84+
* has been ignored due to a concurrent modification of if the clusters state has changed due to async updates.
8685
* @see Store
8786
*/
88-
boolean recoverFromStore(final IndexShard indexShard) {
87+
void recoverFromStore(final IndexShard indexShard, ActionListener<Boolean> listener) {
8988
if (canRecover(indexShard)) {
9089
RecoverySource.Type recoveryType = indexShard.recoveryState().getRecoverySource().getType();
9190
assert recoveryType == RecoverySource.Type.EMPTY_STORE || recoveryType == RecoverySource.Type.EXISTING_STORE :
9291
"expected store recovery type but was: " + recoveryType;
93-
final PlainActionFuture<Boolean> future = PlainActionFuture.newFuture();
94-
final ActionListener<Boolean> recoveryListener = recoveryListener(indexShard, future);
95-
try {
92+
ActionListener.completeWith(recoveryListener(indexShard, listener), () -> {
9693
logger.debug("starting recovery from store ...");
9794
internalRecoverFromStore(indexShard);
98-
recoveryListener.onResponse(true);
99-
} catch (Exception e) {
100-
recoveryListener.onFailure(e);
101-
}
102-
return future.actionGet();
95+
return true;
96+
});
97+
} else {
98+
listener.onResponse(false);
10399
}
104-
return false;
105100
}
106101

107-
boolean recoverFromLocalShards(BiConsumer<String, MappingMetaData> mappingUpdateConsumer,
108-
final IndexShard indexShard, final List<LocalShardSnapshot> shards) {
102+
void recoverFromLocalShards(BiConsumer<String, MappingMetaData> mappingUpdateConsumer, IndexShard indexShard,
103+
List<LocalShardSnapshot> shards, ActionListener<Boolean> listener) {
109104
if (canRecover(indexShard)) {
110105
RecoverySource.Type recoveryType = indexShard.recoveryState().getRecoverySource().getType();
111106
assert recoveryType == RecoverySource.Type.LOCAL_SHARDS: "expected local shards recovery type: " + recoveryType;
@@ -125,8 +120,7 @@ boolean recoverFromLocalShards(BiConsumer<String, MappingMetaData> mappingUpdate
125120
Sort indexSort = indexShard.getIndexSort();
126121
final boolean hasNested = indexShard.mapperService().hasNested();
127122
final boolean isSplit = sourceMetaData.getNumberOfShards() < indexShard.indexSettings().getNumberOfShards();
128-
final PlainActionFuture<Boolean> future = PlainActionFuture.newFuture();
129-
ActionListener.completeWith(recoveryListener(indexShard, future), () -> {
123+
ActionListener.completeWith(recoveryListener(indexShard, listener), () -> {
130124
logger.debug("starting recovery from local shards {}", shards);
131125
try {
132126
final Directory directory = indexShard.store().directory(); // don't close this directory!!
@@ -146,10 +140,9 @@ boolean recoverFromLocalShards(BiConsumer<String, MappingMetaData> mappingUpdate
146140
throw new IndexShardRecoveryException(indexShard.shardId(), "failed to recover from local shards", ex);
147141
}
148142
});
149-
assert future.isDone();
150-
return future.actionGet();
143+
} else {
144+
listener.onResponse(false);
151145
}
152-
return false;
153146
}
154147

155148
void addIndices(final RecoveryState.Index indexRecoveryStats, final Directory target, final Sort indexSort, final Directory[] sources,

server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java

+2-1
Original file line numberDiff line numberDiff line change
@@ -118,6 +118,7 @@
118118
import static org.elasticsearch.index.seqno.SequenceNumbers.NO_OPS_PERFORMED;
119119
import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO;
120120
import static org.elasticsearch.index.shard.IndexShardTestCase.getTranslog;
121+
import static org.elasticsearch.index.shard.IndexShardTestCase.recoverFromStore;
121122
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
122123
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
123124
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures;
@@ -641,7 +642,7 @@ public void testCircuitBreakerIncrementedByIndexShard() throws Exception {
641642
public static final IndexShard recoverShard(IndexShard newShard) throws IOException {
642643
DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT);
643644
newShard.markAsRecovering("store", new RecoveryState(newShard.routingEntry(), localNode, null));
644-
assertTrue(newShard.recoverFromStore());
645+
recoverFromStore(newShard);
645646
IndexShardTestCase.updateRoutingEntry(newShard, newShard.routingEntry().moveToStarted());
646647
return newShard;
647648
}

0 commit comments

Comments
 (0)