You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ma...@apache.org on 2017/01/19 02:18:53 UTC

lucene-solr:branch_6x: SOLR-9836: Add ability to recover from leader when index corruption is detected on SolrCore creation.

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_6x 37687c9b8 -> 59d7bc5ed


SOLR-9836: Add ability to recover from leader when index corruption is detected on SolrCore creation.

# Conflicts:
#	solr/CHANGES.txt
#	solr/core/src/java/org/apache/solr/core/CoreContainer.java


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/59d7bc5e
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/59d7bc5e
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/59d7bc5e

Branch: refs/heads/branch_6x
Commit: 59d7bc5ede7cf4d50b5efb79b31bc0343d6f10dc
Parents: 37687c9
Author: markrmiller <ma...@apache.org>
Authored: Wed Jan 18 21:18:27 2017 -0500
Committer: markrmiller <ma...@apache.org>
Committed: Wed Jan 18 21:18:27 2017 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../org/apache/solr/core/CoreContainer.java     | 110 ++++++++++++++++-
 .../org/apache/solr/core/DirectoryFactory.java  |  27 ++++
 .../src/java/org/apache/solr/core/SolrCore.java | 120 ++++++++++++++----
 .../org/apache/solr/handler/IndexFetcher.java   |  60 +--------
 .../org/apache/solr/handler/RestoreCore.java    |   2 +-
 .../solr/cloud/MissingSegmentRecoveryTest.java  | 123 +++++++++++++++++++
 7 files changed, 362 insertions(+), 83 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/59d7bc5e/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index cb2f376..dde2d05 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -43,6 +43,9 @@ New Features
 
 * SOLR-9979: Macro expansion should not be done in shard requests (Tom�s Fern�ndez L�bbe)
 
+* SOLR-9836: Add ability to recover from leader when index corruption is detected on SolrCore creation.
+  (Mike Drob via Mark Miller)
+
 Optimizations
 ----------------------
 * SOLR-9941: Clear the deletes lists at UpdateLog before replaying from log. This prevents redundantly pre-applying

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/59d7bc5e/solr/core/src/java/org/apache/solr/core/CoreContainer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index 0facddb..b63375a 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -20,9 +20,12 @@ import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.nio.file.Path;
 import java.nio.file.Paths;
+import java.text.SimpleDateFormat;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Date;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
@@ -36,20 +39,28 @@ import java.util.concurrent.Future;
 import com.codahale.metrics.Gauge;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
+
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.store.Directory;
 import org.apache.solr.client.solrj.impl.HttpClientConfigurer;
 import org.apache.solr.client.solrj.impl.HttpClientUtil;
 import org.apache.solr.client.solrj.util.SolrIdentifierValidator;
+import org.apache.solr.cloud.CloudDescriptor;
 import org.apache.solr.cloud.Overseer;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Replica.State;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.IOUtils;
 import org.apache.solr.common.util.Utils;
+import org.apache.solr.core.DirectoryFactory.DirContext;
 import org.apache.solr.core.backup.repository.BackupRepository;
 import org.apache.solr.core.backup.repository.BackupRepositoryFactory;
 import org.apache.solr.handler.RequestHandlerBase;
+import org.apache.solr.handler.SnapShooter;
 import org.apache.solr.handler.admin.CollectionsHandler;
 import org.apache.solr.handler.admin.ConfigSetsHandler;
 import org.apache.solr.handler.admin.CoreAdminHandler;
@@ -163,6 +174,8 @@ public class CoreContainer {
 
   protected MetricsHandler metricsHandler;
 
+  private enum CoreInitFailedAction { fromleader, none }
+
   /**
    * This method instantiates a new instance of {@linkplain BackupRepository}.
    *
@@ -886,7 +899,11 @@ public class CoreContainer {
 
       ConfigSet coreConfig = coreConfigService.getConfig(dcore);
       log.info("Creating SolrCore '{}' using configuration from {}", dcore.getName(), coreConfig.getName());
-      core = new SolrCore(dcore, coreConfig);
+      try {
+        core = new SolrCore(dcore, coreConfig);
+      } catch (SolrException e) {
+        core = processCoreCreateException(e, dcore, coreConfig);
+      }
 
       // always kick off recovery if we are in non-Cloud mode
       if (!isZooKeeperAware() && core.getUpdateHandler().getUpdateLog() != null) {
@@ -898,14 +915,12 @@ public class CoreContainer {
       return core;
     } catch (Exception e) {
       coreInitFailures.put(dcore.getName(), new CoreLoadFailure(dcore, e));
-      log.error("Error creating core [{}]: {}", dcore.getName(), e.getMessage(), e);
       final SolrException solrException = new SolrException(ErrorCode.SERVER_ERROR, "Unable to create core [" + dcore.getName() + "]", e);
       if(core != null && !core.isClosed())
         IOUtils.closeQuietly(core);
       throw solrException;
     } catch (Throwable t) {
       SolrException e = new SolrException(ErrorCode.SERVER_ERROR, "JVM Error creating core [" + dcore.getName() + "]: " + t.getMessage(), t);
-      log.error("Error creating core [{}]: {}", dcore.getName(), t.getMessage(), t);
       coreInitFailures.put(dcore.getName(), new CoreLoadFailure(dcore, e));
       if(core != null && !core.isClosed())
         IOUtils.closeQuietly(core);
@@ -913,7 +928,96 @@ public class CoreContainer {
     } finally {
       MDCLoggingContext.clear();
     }
+  }
+  
+  /**
+   * Take action when we failed to create a SolrCore. If error is due to corrupt index, try to recover. Various recovery
+   * strategies can be specified via system properties "-DCoreInitFailedAction={fromleader, none}"
+   *
+   * @see CoreInitFailedAction
+   *
+   * @param original
+   *          the problem seen when loading the core the first time.
+   * @param dcore
+   *          core descriptor for the core to create
+   * @param coreConfig
+   *          core config for the core to create
+   * @return if possible
+   * @throws SolrException
+   *           rethrows the original exception if we will not attempt to recover, throws a new SolrException with the
+   *           original exception as a suppressed exception if there is a second problem creating the solr core.
+   */
+  private SolrCore processCoreCreateException(SolrException original, CoreDescriptor dcore, ConfigSet coreConfig) {
+    // Traverse full chain since CIE may not be root exception
+    Throwable cause = original;
+    while ((cause = cause.getCause()) != null) {
+      if (cause instanceof CorruptIndexException) {
+        break;
+      }
+    }
+    
+    // If no CorruptIndexExeption, nothing we can try here
+    if (cause == null) throw original;
+    
+    CoreInitFailedAction action = CoreInitFailedAction.valueOf(System.getProperty(CoreInitFailedAction.class.getSimpleName(), "none"));
+    log.debug("CorruptIndexException while creating core, will attempt to repair via {}", action);
+    
+    switch (action) {
+      case fromleader: // Recovery from leader on a CorruptedIndexException
+        if (isZooKeeperAware()) {
+          CloudDescriptor desc = dcore.getCloudDescriptor();
+          try {
+            Replica leader = getZkController().getClusterState()
+                .getCollection(desc.getCollectionName())
+                .getSlice(desc.getShardId())
+                .getLeader();
+            if (leader != null && leader.getState() == State.ACTIVE) {
+              log.info("Found active leader, will attempt to create fresh core and recover.");
+              resetIndexDirectory(dcore, coreConfig);
+              return new SolrCore(dcore, coreConfig);
+            }
+          } catch (SolrException se) {
+            se.addSuppressed(original);
+            throw se;
+          }
+        }
+        throw original;
+      case none:
+        throw original;
+      default:
+        log.warn("Failed to create core, and did not recognize specified 'CoreInitFailedAction': [{}]. Valid options are {}.",
+            action, Arrays.asList(CoreInitFailedAction.values()));
+        throw original;
+    }
+  }
+
+  /**
+   * Write a new index directory for the a SolrCore, but do so without loading it.
+   */
+  private void resetIndexDirectory(CoreDescriptor dcore, ConfigSet coreConfig) {
+    SolrConfig config = coreConfig.getSolrConfig();
+
+    String registryName = SolrMetricManager.getRegistryName(SolrInfoMBean.Group.core, dcore.getName());
+    DirectoryFactory df = DirectoryFactory.loadDirectoryFactory(config, this, registryName);
+    String dataDir = SolrCore.findDataDir(df, null, config, dcore);
 
+    String tmpIdxDirName = "index." + new SimpleDateFormat(SnapShooter.DATE_FMT, Locale.ROOT).format(new Date());
+    SolrCore.modifyIndexProps(df, dataDir, config, tmpIdxDirName);
+
+    // Free the directory object that we had to create for this
+    Directory dir = null;
+    try {
+      dir = df.get(dataDir, DirContext.META_DATA, config.indexConfig.lockType);
+    } catch (IOException e) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+    } finally {
+      try {
+        df.release(dir);
+        df.doneWithDirectory(dir);
+      } catch (IOException e) {
+        SolrException.log(log, e);
+      }
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/59d7bc5e/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java b/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
index ac18d7e..9dd0d8a 100644
--- a/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
@@ -383,4 +383,31 @@ public abstract class DirectoryFactory implements NamedListInitializedPlugin,
     
     return baseDir;
   }
+
+  /**
+   * Create a new DirectoryFactory instance from the given SolrConfig and tied to the specified core container.
+   */
+  static DirectoryFactory loadDirectoryFactory(SolrConfig config, CoreContainer cc, String registryName) {
+    final PluginInfo info = config.getPluginInfo(DirectoryFactory.class.getName());
+    final DirectoryFactory dirFactory;
+    if (info != null) {
+      log.debug(info.className);
+      dirFactory = config.getResourceLoader().newInstance(info.className, DirectoryFactory.class);
+      // allow DirectoryFactory instances to access the CoreContainer
+      dirFactory.initCoreContainer(cc);
+      dirFactory.init(info.initArgs);
+    } else {
+      log.debug("solr.NRTCachingDirectoryFactory");
+      dirFactory = new NRTCachingDirectoryFactory();
+      dirFactory.initCoreContainer(cc);
+    }
+    if (config.indexConfig.metricsInfo != null && config.indexConfig.metricsInfo.isEnabled()) {
+      final DirectoryFactory factory = new MetricsDirectoryFactory(cc.getMetricManager(),
+          registryName, dirFactory);
+        factory.init(config.indexConfig.metricsInfo.initArgs);
+      return factory;
+    } else {
+      return dirFactory;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/59d7bc5e/solr/core/src/java/org/apache/solr/core/SolrCore.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrCore.java b/solr/core/src/java/org/apache/solr/core/SolrCore.java
index ce72b58..0d83b53 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -23,6 +23,8 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
 import java.lang.invoke.MethodHandles;
 import java.lang.reflect.Constructor;
 import java.net.URL;
@@ -67,6 +69,7 @@ import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.LockObtainFailedException;
 import org.apache.solr.client.solrj.impl.BinaryResponseParser;
 import org.apache.solr.cloud.CloudDescriptor;
@@ -148,6 +151,7 @@ import org.apache.solr.update.processor.UpdateRequestProcessorFactory;
 import org.apache.solr.util.DefaultSolrThreadFactory;
 import org.apache.solr.util.NumberUtils;
 import org.apache.solr.util.PropertiesInputStream;
+import org.apache.solr.util.PropertiesOutputStream;
 import org.apache.solr.util.RefCounted;
 import org.apache.solr.util.plugin.NamedListInitializedPlugin;
 import org.apache.solr.util.plugin.PluginInfoInitialized;
@@ -646,27 +650,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
   }
 
   private DirectoryFactory initDirectoryFactory() {
-    final PluginInfo info = solrConfig.getPluginInfo(DirectoryFactory.class.getName());
-    final DirectoryFactory dirFactory;
-    if (info != null) {
-      log.debug(info.className);
-      dirFactory = getResourceLoader().newInstance(info.className, DirectoryFactory.class);
-      // allow DirectoryFactory instances to access the CoreContainer
-      dirFactory.initCoreContainer(getCoreDescriptor().getCoreContainer());
-      dirFactory.init(info.initArgs);
-    } else {
-      log.debug("solr.NRTCachingDirectoryFactory");
-      dirFactory = new NRTCachingDirectoryFactory();
-      dirFactory.initCoreContainer(getCoreDescriptor().getCoreContainer());
-    }
-    if (solrConfig.indexConfig.metricsInfo != null && solrConfig.indexConfig.metricsInfo.isEnabled()) {
-      final DirectoryFactory factory = new MetricsDirectoryFactory(coreDescriptor.getCoreContainer().getMetricManager(),
-          coreMetricManager.getRegistryName(), dirFactory);
-        factory.init(solrConfig.indexConfig.metricsInfo.initArgs);
-      return factory;
-    } else {
-      return dirFactory;
-    }
+    return DirectoryFactory.loadDirectoryFactory(solrConfig, getCoreDescriptor().getCoreContainer(), coreMetricManager.getRegistryName());
   }
 
   private void initIndexReaderFactory() {
@@ -1127,6 +1111,26 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
   }
 
   private String initDataDir(String dataDir, SolrConfig config, CoreDescriptor coreDescriptor) {
+    return findDataDir(getDirectoryFactory(), dataDir, config, coreDescriptor);
+  }
+
+  /**
+   * Locate the data directory for a given config and core descriptor.
+   *
+   * @param directoryFactory
+   *          The directory factory to use if necessary to calculate an absolute path. Should be the same as what will
+   *          be used to open the data directory later.
+   * @param dataDir
+   *          An optional hint to the data directory location. Will be normalized and used if not null.
+   * @param config
+   *          A solr config to retrieve the default data directory location, if used.
+   * @param coreDescriptor
+   *          descriptor to load the actual data dir from, if not using the defualt.
+   * @return a normalized data directory name
+   * @throws SolrException
+   *           if the data directory cannot be loaded from the core descriptor
+   */
+  static String findDataDir(DirectoryFactory directoryFactory, String dataDir, SolrConfig config, CoreDescriptor coreDescriptor) {
     if (dataDir == null) {
       if (coreDescriptor.usingDefaultDataDir()) {
         dataDir = config.getDataDir();
@@ -1145,6 +1149,80 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
     return SolrResourceLoader.normalizeDir(dataDir);
   }
 
+
+  public boolean modifyIndexProps(String tmpIdxDirName) {
+    return SolrCore.modifyIndexProps(getDirectoryFactory(), getDataDir(), getSolrConfig(), tmpIdxDirName);
+  }
+  
+  /**
+   * Update the index.properties file with the new index sub directory name
+   */
+  // package private
+  static boolean modifyIndexProps(DirectoryFactory directoryFactory, String dataDir, SolrConfig solrConfig, String tmpIdxDirName) {
+    log.info("Updating index properties... index="+tmpIdxDirName);
+    Directory dir = null;
+    try {
+      dir = directoryFactory.get(dataDir, DirContext.META_DATA, solrConfig.indexConfig.lockType);
+      String tmpIdxPropName = IndexFetcher.INDEX_PROPERTIES + "." + System.nanoTime();
+      writeNewIndexProps(dir, tmpIdxPropName, tmpIdxDirName);
+      directoryFactory.renameWithOverwrite(dir, tmpIdxPropName, IndexFetcher.INDEX_PROPERTIES);
+      return true;
+    } catch (IOException e1) {
+      throw new RuntimeException(e1);
+    } finally {
+      if (dir != null) {
+        try {
+          directoryFactory.release(dir);
+        } catch (IOException e) {
+          SolrException.log(log, "", e);
+        }
+      }
+    }
+  }
+  
+  /**
+   * Write the index.properties file with the new index sub directory name
+   * @param dir a data directory (containing an index.properties file)
+   * @param tmpFileName the file name to write the new index.properties to
+   * @param tmpIdxDirName new index directory name
+   */
+  private static void writeNewIndexProps(Directory dir, String tmpFileName, String tmpIdxDirName) {
+    if (tmpFileName == null) {
+      tmpFileName = IndexFetcher.INDEX_PROPERTIES;
+    }
+    final Properties p = new Properties();
+    
+    // Read existing properties
+    try {
+      final IndexInput input = dir.openInput(IndexFetcher.INDEX_PROPERTIES, DirectoryFactory.IOCONTEXT_NO_CACHE);
+      final InputStream is = new PropertiesInputStream(input);
+      try {
+        p.load(new InputStreamReader(is, StandardCharsets.UTF_8));
+      } catch (Exception e) {
+        log.error("Unable to load " + IndexFetcher.INDEX_PROPERTIES, e);
+      } finally {
+        IOUtils.closeQuietly(is);
+      }
+    } catch (IOException e) {
+      // ignore; file does not exist
+    }
+    
+    p.put("index", tmpIdxDirName);
+
+    // Write new properties
+    Writer os = null;
+    try {
+      IndexOutput out = dir.createOutput(tmpFileName, DirectoryFactory.IOCONTEXT_NO_CACHE);
+      os = new OutputStreamWriter(new PropertiesOutputStream(out), StandardCharsets.UTF_8);
+      p.store(os, IndexFetcher.INDEX_PROPERTIES);
+      dir.sync(Collections.singleton(tmpFileName));
+    } catch (Exception e) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unable to write " + IndexFetcher.INDEX_PROPERTIES, e);
+    } finally {
+      IOUtils.closeQuietly(os);
+    }
+  }
+
   private String initUpdateLogDir(CoreDescriptor coreDescriptor) {
     String updateLogDir = coreDescriptor.getUlogDir();
     if (updateLogDir == null) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/59d7bc5e/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
index 432bb8b..a5d4b1a 100644
--- a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
+++ b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
@@ -21,7 +21,6 @@ import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
-import java.io.InputStreamReader;
 import java.io.OutputStreamWriter;
 import java.io.Writer;
 import java.lang.invoke.MethodHandles;
@@ -91,7 +90,6 @@ import org.apache.solr.update.UpdateLog;
 import org.apache.solr.update.VersionInfo;
 import org.apache.solr.util.DefaultSolrThreadFactory;
 import org.apache.solr.util.FileUtils;
-import org.apache.solr.util.PropertiesInputStream;
 import org.apache.solr.util.PropertiesOutputStream;
 import org.apache.solr.util.RTimer;
 import org.apache.solr.util.RefCounted;
@@ -446,7 +444,7 @@ public class IndexFetcher {
             reloadCore = true;
             downloadConfFiles(confFilesToDownload, latestGeneration);
             if (isFullCopyNeeded) {
-              successfulInstall = IndexFetcher.modifyIndexProps(solrCore, tmpIdxDirName);
+              successfulInstall = solrCore.modifyIndexProps(tmpIdxDirName);
               deleteTmpIdxDir = false;
             } else {
               successfulInstall = moveIndexFiles(tmpIndexDir, indexDir);
@@ -474,7 +472,7 @@ public class IndexFetcher {
           } else {
             terminateAndWaitFsyncService();
             if (isFullCopyNeeded) {
-              successfulInstall = IndexFetcher.modifyIndexProps(solrCore, tmpIdxDirName);
+              successfulInstall = solrCore.modifyIndexProps(tmpIdxDirName);
               deleteTmpIdxDir = false;
             } else {
               successfulInstall = moveIndexFiles(tmpIndexDir, indexDir);
@@ -1175,60 +1173,6 @@ public class IndexFetcher {
     return new SimpleDateFormat(SnapShooter.DATE_FMT, Locale.ROOT).format(d);
   }
 
-  /**
-   * If the index is stale by any chance, load index from a different dir in the data dir.
-   */
-  protected static boolean modifyIndexProps(SolrCore solrCore, String tmpIdxDirName) {
-    LOG.info("New index installed. Updating index properties... index="+tmpIdxDirName);
-    Properties p = new Properties();
-    Directory dir = null;
-    try {
-      dir = solrCore.getDirectoryFactory().get(solrCore.getDataDir(), DirContext.META_DATA, solrCore.getSolrConfig().indexConfig.lockType);
-      if (slowFileExists(dir, IndexFetcher.INDEX_PROPERTIES)){
-        final IndexInput input = dir.openInput(IndexFetcher.INDEX_PROPERTIES, DirectoryFactory.IOCONTEXT_NO_CACHE);
-
-        final InputStream is = new PropertiesInputStream(input);
-        try {
-          p.load(new InputStreamReader(is, StandardCharsets.UTF_8));
-        } catch (Exception e) {
-          LOG.error("Unable to load " + IndexFetcher.INDEX_PROPERTIES, e);
-        } finally {
-          IOUtils.closeQuietly(is);
-        }
-      }
-
-      String tmpFileName = IndexFetcher.INDEX_PROPERTIES + "." + System.nanoTime();
-      final IndexOutput out = dir.createOutput(tmpFileName, DirectoryFactory.IOCONTEXT_NO_CACHE);
-      p.put("index", tmpIdxDirName);
-      Writer os = null;
-      try {
-        os = new OutputStreamWriter(new PropertiesOutputStream(out), StandardCharsets.UTF_8);
-        p.store(os, tmpFileName);
-        dir.sync(Collections.singleton(tmpFileName));
-      } catch (Exception e) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
-            "Unable to write " + IndexFetcher.INDEX_PROPERTIES, e);
-      } finally {
-        IOUtils.closeQuietly(os);
-      }
-      
-      solrCore.getDirectoryFactory().renameWithOverwrite(dir, tmpFileName, IndexFetcher.INDEX_PROPERTIES);
-      return true;
-
-    } catch (IOException e1) {
-      throw new RuntimeException(e1);
-    } finally {
-      if (dir != null) {
-        try {
-          solrCore.getDirectoryFactory().release(dir);
-        } catch (IOException e) {
-          SolrException.log(LOG, "", e);
-        }
-      }
-    }
-
-  }
-
   private final Map<String, FileInfo> confFileInfoCache = new HashMap<>();
 
   /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/59d7bc5e/solr/core/src/java/org/apache/solr/handler/RestoreCore.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/RestoreCore.java b/solr/core/src/java/org/apache/solr/handler/RestoreCore.java
index c00d7bd..e750631 100644
--- a/solr/core/src/java/org/apache/solr/handler/RestoreCore.java
+++ b/solr/core/src/java/org/apache/solr/handler/RestoreCore.java
@@ -101,7 +101,7 @@ public class RestoreCore implements Callable<Boolean> {
         }
       }
       log.debug("Switching directories");
-      IndexFetcher.modifyIndexProps(core, restoreIndexName);
+      core.modifyIndexProps(restoreIndexName);
 
       boolean success;
       try {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/59d7bc5e/solr/core/src/test/org/apache/solr/cloud/MissingSegmentRecoveryTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/MissingSegmentRecoveryTest.java b/solr/core/src/test/org/apache/solr/cloud/MissingSegmentRecoveryTest.java
new file mode 100644
index 0000000..a7bfa20
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/MissingSegmentRecoveryTest.java
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.cloud;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.util.LuceneTestCase.Slow;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.core.SolrCore;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+@Slow
+public class MissingSegmentRecoveryTest extends SolrCloudTestCase {
+  final String collection = getClass().getSimpleName();
+  
+  Replica leader;
+  Replica replica;
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(2)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
+    useFactory("solr.StandardDirectoryFactory");
+  }
+
+  @Before
+  public void setup() throws SolrServerException, IOException {
+    CollectionAdminRequest.createCollection(collection, "conf", 1, 2)
+        .setMaxShardsPerNode(1)
+        .process(cluster.getSolrClient());
+    waitForState("Expected a collection with one shard and two replicas", collection, clusterShape(1, 2));
+    cluster.getSolrClient().setDefaultCollection(collection);
+
+    List<SolrInputDocument> docs = new ArrayList<>();
+    for (int i = 0; i < 10; i++) {
+      SolrInputDocument doc = new SolrInputDocument();
+      doc.addField("id", i);
+      docs.add(doc);
+    }
+
+    cluster.getSolrClient().add(docs);
+    cluster.getSolrClient().commit();
+    
+    DocCollection state = getCollectionState(collection);
+    leader = state.getLeader("shard1");
+    replica = getRandomReplica(state.getSlice("shard1"), (r) -> leader != r);
+  }
+  
+  @After
+  public void teardown() throws Exception {
+    System.clearProperty("CoreInitFailedAction");
+    CollectionAdminRequest.deleteCollection(collection).process(cluster.getSolrClient());
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws Exception {
+    resetFactory();
+  }
+
+  @Test
+  public void testLeaderRecovery() throws Exception {
+    System.setProperty("CoreInitFailedAction", "fromleader");
+
+    // Simulate failure by truncating the segment_* files
+    for (File segment : getSegmentFiles(replica)) {
+      truncate(segment);
+    }
+
+    // Might not need a sledge-hammer to reload the core
+    JettySolrRunner jetty = cluster.getReplicaJetty(replica);
+    jetty.stop();
+    jetty.start();
+
+    waitForState("Expected a collection with one shard and two replicas", collection, clusterShape(1, 2));
+    
+    QueryResponse resp = cluster.getSolrClient().query(collection, new SolrQuery("*:*"));
+    assertEquals(10, resp.getResults().getNumFound());
+  }
+
+  private File[] getSegmentFiles(Replica replica) {
+    try (SolrCore core = cluster.getReplicaJetty(replica).getCoreContainer().getCore(replica.getCoreName())) {
+      File indexDir = new File(core.getDataDir(), "index");
+      return indexDir.listFiles((File dir, String name) -> {
+        return name.startsWith("segments_");
+      });
+    }
+  }
+  
+  private void truncate(File file) throws IOException {
+    Files.write(file.toPath(), new byte[0], StandardOpenOption.TRUNCATE_EXISTING);
+  }
+}