Skip to content

Commit

Permalink
Avoid bubbling up failures from a shard that is recovering (#42287)
Browse files Browse the repository at this point in the history
A shard that is undergoing peer recovery is subject to logging warnings of the form

org.elasticsearch.action.FailedNodeException: Failed node [XYZ]
...
Caused by: org.apache.lucene.index.IndexNotFoundException: no segments* file found in ...

These failures are actually harmless, and expected to happen while a peer recovery is ongoing (i.e.
there is an IndexShard instance, but no proper IndexCommit just yet).
As these failures are currently bubbled up to the master, they cause unnecessary reroutes and
confusion amongst users due to being logged as warnings.

Closes  #40107
  • Loading branch information
ywelsch committed May 22, 2019
1 parent b775755 commit badfcdf
Show file tree
Hide file tree
Showing 2 changed files with 58 additions and 2 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

package org.elasticsearch.indices.store;

import org.apache.logging.log4j.message.ParameterizedMessage;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.FailedNodeException;
Expand Down Expand Up @@ -125,8 +126,17 @@ private StoreFilesMetaData listStoreMetaData(ShardId shardId) throws IOException
if (indexService != null) {
IndexShard indexShard = indexService.getShardOrNull(shardId.id());
if (indexShard != null) {
exists = true;
return new StoreFilesMetaData(shardId, indexShard.snapshotStoreMetadata());
try {
final StoreFilesMetaData storeFilesMetaData = new StoreFilesMetaData(shardId, indexShard.snapshotStoreMetadata());
exists = true;
return storeFilesMetaData;
} catch (org.apache.lucene.index.IndexNotFoundException e) {
logger.trace(new ParameterizedMessage("[{}] node is missing index, responding with empty", shardId), e);
return new StoreFilesMetaData(shardId, Store.MetadataSnapshot.EMPTY);
} catch (IOException e) {
logger.warn(new ParameterizedMessage("[{}] can't read metadata from store, responding with empty", shardId), e);
return new StoreFilesMetaData(shardId, Store.MetadataSnapshot.EMPTY);
}
}
}
// try and see if we an list unallocated
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -76,6 +76,7 @@
import java.util.Map;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Semaphore;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Consumer;
Expand All @@ -90,6 +91,7 @@
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.lessThanOrEqualTo;
import static org.hamcrest.Matchers.not;
import static org.junit.Assert.assertFalse;

@ClusterScope(scope = Scope.TEST, numDataNodes = 0)
public class IndexRecoveryIT extends ESIntegTestCase {
Expand Down Expand Up @@ -789,4 +791,48 @@ public void sendRequest(Transport.Connection connection, long requestId, String
assertHitCount(client().prepareSearch(indexName).get(), numDocs);
}
}

/** Makes sure the new master does not repeatedly fetch index metadata from recovering replicas */
public void testOngoingRecoveryAndMasterFailOver() throws Exception {
String indexName = "test";
internalCluster().startNodes(2);
String nodeWithPrimary = internalCluster().startDataOnlyNode();
assertAcked(client().admin().indices().prepareCreate(indexName)
.setSettings(Settings.builder()
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0)
.put("index.routing.allocation.include._name", nodeWithPrimary)));
MockTransportService transport = (MockTransportService) internalCluster().getInstance(TransportService.class, nodeWithPrimary);
CountDownLatch phase1ReadyBlocked = new CountDownLatch(1);
CountDownLatch allowToCompletePhase1Latch = new CountDownLatch(1);
Semaphore blockRecovery = new Semaphore(1);
transport.addSendBehavior((connection, requestId, action, request, options) -> {
if (PeerRecoveryTargetService.Actions.CLEAN_FILES.equals(action) && blockRecovery.tryAcquire()) {
phase1ReadyBlocked.countDown();
try {
allowToCompletePhase1Latch.await();
} catch (InterruptedException e) {
throw new AssertionError(e);
}
}
connection.sendRequest(requestId, action, request, options);
});
try {
String nodeWithReplica = internalCluster().startDataOnlyNode();
assertAcked(client().admin().indices().prepareUpdateSettings(indexName).setSettings(Settings.builder()
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1)
.put("index.routing.allocation.include._name", nodeWithPrimary + "," + nodeWithReplica)));
phase1ReadyBlocked.await();
internalCluster().restartNode(clusterService().state().nodes().getMasterNode().getName(),
new InternalTestCluster.RestartCallback());
internalCluster().ensureAtLeastNumDataNodes(3);
assertAcked(client().admin().indices().prepareUpdateSettings(indexName).setSettings(Settings.builder()
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 2)
.putNull("index.routing.allocation.include._name")));
assertFalse(client().admin().cluster().prepareHealth(indexName).setWaitForActiveShards(2).get().isTimedOut());
} finally {
allowToCompletePhase1Latch.countDown();
}
ensureGreen(indexName);
}
}

0 comments on commit badfcdf

Please sign in to comment.