diff --git a/src/main/java/org/elasticsearch/index/gateway/IndexShardGateway.java b/src/main/java/org/elasticsearch/index/gateway/IndexShardGateway.java index 5cfb079d6e6..d18192531c6 100644 --- a/src/main/java/org/elasticsearch/index/gateway/IndexShardGateway.java +++ b/src/main/java/org/elasticsearch/index/gateway/IndexShardGateway.java @@ -103,6 +103,7 @@ public class IndexShardGateway extends AbstractIndexShardComponent implements Cl recoveryState.setStage(RecoveryState.Stage.INDEX); long version = -1; long translogId = -1; + final Set typesToUpdate = Sets.newHashSet(); indexShard.store().incRef(); try { try { @@ -232,7 +233,6 @@ public class IndexShardGateway extends AbstractIndexShardComponent implements Cl recoveryState.setStage(RecoveryState.Stage.TRANSLOG); StreamInput in = null; - final Set typesToUpdate = Sets.newHashSet(); try { logger.trace("recovering translog file: {} length: {}", recoveringTranslogFile, Files.size(recoveringTranslogFile)); TranslogStream stream = TranslogStreams.translogStreamFor(recoveringTranslogFile); @@ -291,35 +291,33 @@ public class IndexShardGateway extends AbstractIndexShardComponent implements Cl } catch (Exception ex) { logger.debug("Failed to delete recovering translog file {}", ex, recoveringTranslogFile); } - - for (final String type : typesToUpdate) { - final CountDownLatch latch = new CountDownLatch(1); - mappingUpdatedAction.updateMappingOnMaster(indexService.index().name(), indexService.mapperService().documentMapper(type), indexService.indexUUID(), new MappingUpdatedAction.MappingUpdateListener() { - @Override - public void onMappingUpdate() { - latch.countDown(); - } - - @Override - public void onFailure(Throwable t) { - latch.countDown(); - logger.debug("failed to send mapping update post recovery to master for [{}]", t, type); - } - }); - - try { - boolean waited = latch.await(waitForMappingUpdatePostRecovery.millis(), TimeUnit.MILLISECONDS); - if (!waited) { - logger.debug("waited for mapping update on master for [{}], yet timed out"); - } - } catch (InterruptedException e) { - logger.debug("interrupted while waiting for mapping update"); - } - } } finally { indexShard.store().decRef(); } + for (final String type : typesToUpdate) { + final CountDownLatch latch = new CountDownLatch(1); + mappingUpdatedAction.updateMappingOnMaster(indexService.index().name(), indexService.mapperService().documentMapper(type), indexService.indexUUID(), new MappingUpdatedAction.MappingUpdateListener() { + @Override + public void onMappingUpdate() { + latch.countDown(); + } + @Override + public void onFailure(Throwable t) { + latch.countDown(); + logger.debug("failed to send mapping update post recovery to master for [{}]", t, type); + } + }); + + try { + boolean waited = latch.await(waitForMappingUpdatePostRecovery.millis(), TimeUnit.MILLISECONDS); + if (!waited) { + logger.debug("waited for mapping update on master for [{}], yet timed out"); + } + } catch (InterruptedException e) { + logger.debug("interrupted while waiting for mapping update"); + } + } recoveryState.getTranslog().time(System.currentTimeMillis() - recoveryState.getTranslog().startTime()); }