You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by va...@apache.org on 2015/04/03 11:46:55 UTC

svn commit: r1671022 - in /lucene/dev/trunk/solr: ./ core/src/java/org/apache/solr/handler/ core/src/test/org/apache/solr/handler/

Author: varun
Date: Fri Apr  3 09:46:54 2015
New Revision: 1671022

URL: http://svn.apache.org/r1671022
Log:
SOLR-6637: Solr should have a way to restore a core

Added:
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/OldBackupDirectory.java   (with props)
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/RestoreCore.java   (with props)
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/TestRestoreCore.java   (with props)
Modified:
    lucene/dev/trunk/solr/CHANGES.txt
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/SnapShooter.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/TestReplicationHandlerBackup.java

Modified: lucene/dev/trunk/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=1671022&r1=1671021&r2=1671022&view=diff
==============================================================================
--- lucene/dev/trunk/solr/CHANGES.txt (original)
+++ lucene/dev/trunk/solr/CHANGES.txt Fri Apr  3 09:46:54 2015
@@ -78,6 +78,9 @@ Detailed Change List
 New Features
 ----------------------
 
+* SOLR-6637: Solr should have a way to restore a core from a backed up index.
+  (Varun Thacker, noble, shalin)
+
 Bug Fixes
 ----------------------
 

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java?rev=1671022&r1=1671021&r2=1671022&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java Fri Apr  3 09:46:54 2015
@@ -29,6 +29,7 @@ import java.nio.channels.FileChannel;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
 import java.nio.file.NoSuchFileException;
+import java.nio.file.Paths;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -246,31 +247,31 @@ public class IndexFetcher {
     }
   }
 
-  boolean fetchLatestIndex(final SolrCore core, boolean forceReplication) throws IOException, InterruptedException {
-    return fetchLatestIndex(core, forceReplication, false);
+  boolean fetchLatestIndex(boolean forceReplication) throws IOException, InterruptedException {
+    return fetchLatestIndex(forceReplication, false);
   }
   
   /**
    * This command downloads all the necessary files from master to install a index commit point. Only changed files are
    * downloaded. It also downloads the conf files (if they are modified).
    *
-   * @param core the SolrCore
    * @param forceReplication force a replication in all cases 
    * @param forceCoreReload force a core reload in all cases
    * @return true on success, false if slave is already in sync
    * @throws IOException if an exception occurs
    */
-   boolean fetchLatestIndex(final SolrCore core, boolean forceReplication, boolean forceCoreReload) throws IOException, InterruptedException {
+  boolean fetchLatestIndex(boolean forceReplication, boolean forceCoreReload) throws IOException, InterruptedException {
+    
     boolean cleanupDone = false;
     boolean successfulInstall = false;
     replicationStartTime = System.currentTimeMillis();
     Directory tmpIndexDir = null;
-    String tmpIndex = null;
+    String tmpIndex;
     Directory indexDir = null;
-    String indexDirPath = null;
+    String indexDirPath;
     boolean deleteTmpIdxDir = true;
     
-    if (!core.getSolrCoreState().getLastReplicateIndexSuccess()) {
+    if (!solrCore.getSolrCoreState().getLastReplicateIndexSuccess()) {
       // if the last replication was not a success, we force a full replication
       // when we are a bit more confident we may want to try a partial replication
       // if the error is connection related or something, but we have to be careful
@@ -279,7 +280,7 @@ public class IndexFetcher {
     
     try {
       //get the current 'replicateable' index version in the master
-      NamedList response = null;
+      NamedList response;
       try {
         response = getLatestVersion();
       } catch (Exception e) {
@@ -290,12 +291,12 @@ public class IndexFetcher {
       long latestGeneration = (Long) response.get(GENERATION);
 
       // TODO: make sure that getLatestCommit only returns commit points for the main index (i.e. no side-car indexes)
-      IndexCommit commit = core.getDeletionPolicy().getLatestCommit();
+      IndexCommit commit = solrCore.getDeletionPolicy().getLatestCommit();
       if (commit == null) {
         // Presumably the IndexWriter hasn't been opened yet, and hence the deletion policy hasn't been updated with commit points
         RefCounted<SolrIndexSearcher> searcherRefCounted = null;
         try {
-          searcherRefCounted = core.getNewestSearcher(false);
+          searcherRefCounted = solrCore.getNewestSearcher(false);
           if (searcherRefCounted == null) {
             LOG.warn("No open searcher found - fetch aborted");
             return false;
@@ -312,15 +313,14 @@ public class IndexFetcher {
         if (forceReplication && commit.getGeneration() != 0) {
           // since we won't get the files for an empty index,
           // we just clear ours and commit
-          RefCounted<IndexWriter> iw = core.getUpdateHandler().getSolrCoreState().getIndexWriter(core);
+          RefCounted<IndexWriter> iw = solrCore.getUpdateHandler().getSolrCoreState().getIndexWriter(solrCore);
           try {
             iw.get().deleteAll();
           } finally {
             iw.decref();
           }
-          SolrQueryRequest req = new LocalSolrQueryRequest(core,
-              new ModifiableSolrParams());
-          core.getUpdateHandler().commit(new CommitUpdateCommand(req, false));
+          SolrQueryRequest req = new LocalSolrQueryRequest(solrCore, new ModifiableSolrParams());
+          solrCore.getUpdateHandler().commit(new CommitUpdateCommand(req, false));
         }
         
         //there is nothing to be replicated
@@ -340,7 +340,9 @@ public class IndexFetcher {
       // get the list of files first
       fetchFileList(latestGeneration);
       // this can happen if the commit point is deleted before we fetch the file list.
-      if(filesToDownload.isEmpty()) return false;
+      if (filesToDownload.isEmpty()) {
+        return false;
+      }
       LOG.info("Number of files in latest index in master: " + filesToDownload.size());
 
       // Create the sync service
@@ -354,13 +356,13 @@ public class IndexFetcher {
           || commit.getGeneration() >= latestGeneration || forceReplication;
 
       String tmpIdxDirName = "index." + new SimpleDateFormat(SnapShooter.DATE_FMT, Locale.ROOT).format(new Date());
-      tmpIndex = createTempindexDir(core, tmpIdxDirName);
+      tmpIndex = Paths.get(solrCore.getDataDir(), tmpIdxDirName).toString();
 
-      tmpIndexDir = core.getDirectoryFactory().get(tmpIndex, DirContext.DEFAULT, core.getSolrConfig().indexConfig.lockType);
+      tmpIndexDir = solrCore.getDirectoryFactory().get(tmpIndex, DirContext.DEFAULT, solrCore.getSolrConfig().indexConfig.lockType);
       
       // cindex dir...
-      indexDirPath = core.getIndexDir();
-      indexDir = core.getDirectoryFactory().get(indexDirPath, DirContext.DEFAULT, core.getSolrConfig().indexConfig.lockType);
+      indexDirPath = solrCore.getIndexDir();
+      indexDir = solrCore.getDirectoryFactory().get(indexDirPath, DirContext.DEFAULT, solrCore.getSolrConfig().indexConfig.lockType);
 
       try {
 
@@ -404,7 +406,7 @@ public class IndexFetcher {
           } finally {
             writer.decref();
           }
-          solrCore.getUpdateHandler().getSolrCoreState().closeIndexWriter(core, true);
+          solrCore.getUpdateHandler().getSolrCoreState().closeIndexWriter(solrCore, true);
         }
         boolean reloadCore = false;
         
@@ -422,7 +424,7 @@ public class IndexFetcher {
             reloadCore = true;
             downloadConfFiles(confFilesToDownload, latestGeneration);
             if (isFullCopyNeeded) {
-              successfulInstall = modifyIndexProps(tmpIdxDirName);
+              successfulInstall = IndexFetcher.modifyIndexProps(solrCore, tmpIdxDirName);
               deleteTmpIdxDir = false;
             } else {
               successfulInstall = moveIndexFiles(tmpIndexDir, indexDir);
@@ -433,8 +435,8 @@ public class IndexFetcher {
                 // may be closed
                 if (indexDir != null) {
                   LOG.info("removing old index directory " + indexDir);
-                  core.getDirectoryFactory().doneWithDirectory(indexDir);
-                  core.getDirectoryFactory().remove(indexDir);
+                  solrCore.getDirectoryFactory().doneWithDirectory(indexDir);
+                  solrCore.getDirectoryFactory().remove(indexDir);
                 }
               }
               
@@ -446,7 +448,7 @@ public class IndexFetcher {
           } else {
             terminateAndWaitFsyncService();
             if (isFullCopyNeeded) {
-              successfulInstall = modifyIndexProps(tmpIdxDirName);
+              successfulInstall = IndexFetcher.modifyIndexProps(solrCore, tmpIdxDirName);
               deleteTmpIdxDir = false;
             } else {
               successfulInstall = moveIndexFiles(tmpIndexDir, indexDir);
@@ -458,13 +460,13 @@ public class IndexFetcher {
           }
         } finally {
           if (!isFullCopyNeeded) {
-            solrCore.getUpdateHandler().getSolrCoreState().openIndexWriter(core);
+            solrCore.getUpdateHandler().getSolrCoreState().openIndexWriter(solrCore);
           }
         }
         
         // we must reload the core after we open the IW back up
        if (successfulInstall && (reloadCore || forceCoreReload)) {
-          LOG.info("Reloading SolrCore {}", core.getName());
+          LOG.info("Reloading SolrCore {}", solrCore.getName());
           reloadCore();
         }
 
@@ -474,8 +476,8 @@ public class IndexFetcher {
             // may be closed
             if (indexDir != null) {
               LOG.info("removing old index directory " + indexDir);
-              core.getDirectoryFactory().doneWithDirectory(indexDir);
-              core.getDirectoryFactory().remove(indexDir);
+              solrCore.getDirectoryFactory().doneWithDirectory(indexDir);
+              solrCore.getDirectoryFactory().remove(indexDir);
             }
           }
           if (isFullCopyNeeded) {
@@ -486,13 +488,13 @@ public class IndexFetcher {
         }
         
         if (!isFullCopyNeeded && !forceReplication && !successfulInstall) {
-          cleanup(core, tmpIndexDir, indexDir, deleteTmpIdxDir, successfulInstall);
+          cleanup(solrCore, tmpIndexDir, indexDir, deleteTmpIdxDir, successfulInstall);
           cleanupDone = true;
           // we try with a full copy of the index
           LOG.warn(
               "Replication attempt was not successful - trying a full index replication reloadCore={}",
               reloadCore);
-          successfulInstall = fetchLatestIndex(core, true, reloadCore);
+          successfulInstall = fetchLatestIndex(true, reloadCore);
         }
         
         replicationStartTime = 0;
@@ -505,15 +507,15 @@ public class IndexFetcher {
       } catch (InterruptedException e) {
         throw new InterruptedException("Index fetch interrupted");
       } catch (Exception e) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Index fetch failed : ", e);
+        throw new SolrException(ErrorCode.SERVER_ERROR, "Index fetch failed : ", e);
       }
     } finally {
       if (!cleanupDone) {
-        cleanup(core, tmpIndexDir, indexDir, deleteTmpIdxDir, successfulInstall);
+        cleanup(solrCore, tmpIndexDir, indexDir, deleteTmpIdxDir, successfulInstall);
       }
     }
   }
-        
+
   private void cleanup(final SolrCore core, Directory tmpIndexDir,
       Directory indexDir, boolean deleteTmpIdxDir, boolean successfulInstall) throws IOException {
     try {
@@ -524,9 +526,9 @@ public class IndexFetcher {
           LOG.error("caught", e);
         }
       }
-      
+
       core.getUpdateHandler().getSolrCoreState().setLastReplicateIndexSuccess(successfulInstall);
-      
+
       filesToDownload = filesDownloaded = confFilesDownloaded = confFilesToDownload = null;
       replicationStartTime = 0;
       dirFileFetcher = null;
@@ -545,11 +547,11 @@ public class IndexFetcher {
           SolrException.log(LOG, "Error removing directory " + tmpIndexDir, e);
         }
       }
-      
+
       if (tmpIndexDir != null) {
         core.getDirectoryFactory().release(tmpIndexDir);
       }
-      
+
       if (indexDir != null) {
         core.getDirectoryFactory().release(indexDir);
       }
@@ -719,15 +721,6 @@ public class IndexFetcher {
     
   }
 
-  /**
-   * All the files are copied to a temp dir first
-   */
-  private String createTempindexDir(SolrCore core, String tmpIdxDirName) {
-    // TODO: there should probably be a DirectoryFactory#concatPath(parent, name)
-    // or something
-    return core.getDataDir() + tmpIdxDirName;
-  }
-
   private void reloadCore() {
     final CountDownLatch latch = new CountDownLatch(1);
     new Thread() {
@@ -815,12 +808,12 @@ public class IndexFetcher {
     || filename.startsWith("segments_") || size < _100K);
   }
 
-  static class CompareResult {
+  protected static class CompareResult {
     boolean equal = false;
     boolean checkSummed = false;
   }
-  
-  private CompareResult compareFile(Directory indexDir, String filename, Long backupIndexFileLen, Long backupIndexFileChecksum) {
+
+  protected static CompareResult compareFile(Directory indexDir, String filename, Long backupIndexFileLen, Long backupIndexFileChecksum) {
     CompareResult compareResult = new CompareResult();
     try {
       try (final IndexInput indexInput = indexDir.openInput(filename, IOContext.READONCE)) {
@@ -887,8 +880,8 @@ public class IndexFetcher {
   }  
 
   /**
-   * All the files which are common between master and slave must have same size else we assume they are
-   * not compatible (stale).
+   * All the files which are common between master and slave must have same size and same checksum else we assume
+   * they are not compatible (stale).
    *
    * @return true if the index stale and we need to download a fresh copy, false otherwise.
    * @throws IOException  if low level io error
@@ -1034,7 +1027,7 @@ public class IndexFetcher {
   /**
    * If the index is stale by any chance, load index from a different dir in the data dir.
    */
-  private boolean modifyIndexProps(String tmpIdxDirName) {
+  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;
@@ -1042,7 +1035,7 @@ public class IndexFetcher {
       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));
@@ -1083,7 +1076,7 @@ public class IndexFetcher {
         }
       }
     }
-    
+
   }
 
   private final Map<String, FileInfo> confFileInfoCache = new HashMap<>();

Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/OldBackupDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/OldBackupDirectory.java?rev=1671022&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/OldBackupDirectory.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/OldBackupDirectory.java Fri Apr  3 09:46:54 2015
@@ -0,0 +1,50 @@
+package org.apache.solr.handler;
+
+/*
+ * 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.
+ */
+
+import java.io.File;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Locale;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+class OldBackupDirectory implements Comparable<OldBackupDirectory> {
+  File dir;
+  Date timestamp;
+  private  final Pattern dirNamePattern = Pattern.compile("^snapshot[.](.*)$");
+
+  OldBackupDirectory(File dir) {
+    if(dir.isDirectory()) {
+      Matcher m = dirNamePattern.matcher(dir.getName());
+      if(m.find()) {
+        try {
+          this.dir = dir;
+          this.timestamp = new SimpleDateFormat(SnapShooter.DATE_FMT, Locale.ROOT).parse(m.group(1));
+        } catch(Exception e) {
+          this.dir = null;
+          this.timestamp = null;
+        }
+      }
+    }
+  }
+  @Override
+  public int compareTo(OldBackupDirectory that) {
+    return that.timestamp.compareTo(this.timestamp);
+  }
+}

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java?rev=1671022&r1=1671021&r2=1671022&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java Fri Apr  3 09:46:54 2015
@@ -36,7 +36,9 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
+import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -146,6 +148,13 @@ public class ReplicationHandler extends
 
   private ReentrantLock indexFetchLock = new ReentrantLock();
 
+  private ExecutorService restoreExecutor = Executors.newSingleThreadExecutor(
+      new DefaultSolrThreadFactory("restoreExecutor"));
+
+  private volatile Future<Boolean> restoreFuture;
+
+  private volatile String currentRestoreName;
+
   private String includeConfFiles;
 
   private NamedList<String> confFileNameAlias = new NamedList<>();
@@ -205,13 +214,13 @@ public class ReplicationHandler extends
     // It gives the current 'replicateable' index version
     if (command.equals(CMD_INDEX_VERSION)) {
       IndexCommit commitPoint = indexCommitPoint;  // make a copy so it won't change
- 
+
       if (commitPoint == null) {
         // if this handler is 'lazy', we may not have tracked the last commit
         // because our commit listener is registered on inform
         commitPoint = core.getDeletionPolicy().getLatestCommit();
       }
-      
+
       if (commitPoint != null && replicationEnabled.get()) {
         //
         // There is a race condition here.  The commit point may be changed / deleted by the time
@@ -235,6 +244,11 @@ public class ReplicationHandler extends
     } else if (command.equalsIgnoreCase(CMD_BACKUP)) {
       doSnapShoot(new ModifiableSolrParams(solrParams), rsp, req);
       rsp.add(STATUS, OK_STATUS);
+    } else if (command.equalsIgnoreCase(CMD_RESTORE)) {
+      restore(new ModifiableSolrParams(solrParams), rsp, req);
+      rsp.add(STATUS, OK_STATUS);
+    } else if (command.equalsIgnoreCase(CMD_RESTORE_STATUS)) {
+      rsp.add(CMD_RESTORE_STATUS, getRestoreStatus());
     } else if (command.equalsIgnoreCase(CMD_DELETE_BACKUP)) {
       deleteSnapshot(new ModifiableSolrParams(solrParams));
       rsp.add(STATUS, OK_STATUS);
@@ -302,7 +316,7 @@ public class ReplicationHandler extends
       throw new SolrException(ErrorCode.BAD_REQUEST, "Missing mandatory param: name");
     }
 
-    SnapShooter snapShooter = new SnapShooter(core, params.get("location"), params.get(NAME));
+    SnapShooter snapShooter = new SnapShooter(core, params.get(LOCATION), params.get(NAME));
     snapShooter.validateDeleteSnapshot();
     snapShooter.deleteSnapAsync(this);
   }
@@ -361,7 +375,7 @@ public class ReplicationHandler extends
       } else {
         currentIndexFetcher = pollingIndexFetcher;
       }
-      return currentIndexFetcher.fetchLatestIndex(core, forceReplication);
+      return currentIndexFetcher.fetchLatestIndex(forceReplication);
     } catch (Exception e) {
       SolrException.log(LOG, "Index fetch failed ", e);
     } finally {
@@ -377,6 +391,72 @@ public class ReplicationHandler extends
     return indexFetchLock.isLocked();
   }
 
+  private void restore(SolrParams params, SolrQueryResponse rsp, SolrQueryRequest req) {
+    if (restoreFuture != null && !restoreFuture.isDone()) {
+      throw new SolrException(ErrorCode.BAD_REQUEST, "Restore in progress. Cannot run multiple restore operations" +
+          "for the same core");
+    }
+    String name = params.get(NAME);
+    String location = params.get(LOCATION);
+
+    //If location is not provided then assume that the restore index is present inside the data directory.
+    if (location == null) {
+      location = core.getDataDir();
+    }
+
+    //If name is not provided then look for the last unnamed( the ones with the snapshot.timestamp format)
+    //snapshot folder since we allow snapshots to be taken without providing a name. Pick the latest timestamp.
+    if (name == null) {
+      File[] files = new File(location).listFiles();
+      List<OldBackupDirectory> dirs = new ArrayList<>();
+      for (File f : files) {
+        OldBackupDirectory obd = new OldBackupDirectory(f);
+        if (obd.dir != null) {
+          dirs.add(obd);
+        }
+      }
+      Collections.sort(dirs);
+      if (dirs.size() == 0) {
+        throw new SolrException(ErrorCode.BAD_REQUEST, "No backup name specified and none found in " + core.getDataDir());
+      }
+      name = dirs.get(0).dir.getName();
+    } else {
+      //"snapshot." is prefixed by snapshooter
+      name = "snapshot." + name;
+    }
+
+    RestoreCore restoreCore = new RestoreCore(core, location, name);
+    restoreFuture = restoreExecutor.submit(restoreCore);
+    currentRestoreName = name;
+  }
+
+  private NamedList<Object> getRestoreStatus() {
+    NamedList<Object> status = new SimpleOrderedMap<>();
+
+    if (restoreFuture == null) {
+      status.add(STATUS, "No restore actions in progress");
+      return status;
+    }
+
+    status.add("snapshotName", currentRestoreName);
+    if (restoreFuture.isDone()) {
+      try {
+        boolean success = restoreFuture.get();
+        if (success) {
+          status.add(STATUS, SUCCESS);
+        } else {
+          status.add(STATUS, FAILED);
+        }
+      } catch (Exception e) {
+        status.add(STATUS, FAILED);
+        status.add(EXCEPTION, e.getMessage());
+      }
+    } else {
+      status.add(STATUS, "In Progress");
+    }
+    return status;
+  }
+
   private void doSnapShoot(SolrParams params, SolrQueryResponse rsp,
       SolrQueryRequest req) {
     try {
@@ -391,19 +471,19 @@ public class ReplicationHandler extends
       if (numberToKeep < 1) {
         numberToKeep = Integer.MAX_VALUE;
       }
-      
+
       IndexDeletionPolicyWrapper delPolicy = core.getDeletionPolicy();
       IndexCommit indexCommit = delPolicy.getLatestCommit();
-      
+
       if (indexCommit == null) {
         indexCommit = req.getSearcher().getIndexReader().getIndexCommit();
       }
-      
+
       // small race here before the commit point is saved
       SnapShooter snapShooter = new SnapShooter(core, params.get("location"), params.get(NAME));
       snapShooter.validateCreateSnapshot();
       snapShooter.createSnapAsync(indexCommit, numberToKeep, this);
-      
+
     } catch (Exception e) {
       LOG.warn("Exception during creating a snapshot", e);
       rsp.add("exception", e);
@@ -420,7 +500,7 @@ public class ReplicationHandler extends
   private void getFileStream(SolrParams solrParams, SolrQueryResponse rsp) {
     ModifiableSolrParams rawParams = new ModifiableSolrParams(solrParams);
     rawParams.set(CommonParams.WT, FILE_STREAM);
-    
+
     String cfileName = solrParams.get(CONF_FILE_SHORT);
     if (cfileName != null) {
       rsp.add(FILE_STREAM, new LocalFsFileStream(solrParams));
@@ -438,7 +518,7 @@ public class ReplicationHandler extends
     }
     long gen = Long.parseLong(v);
     IndexCommit commit = core.getDeletionPolicy().getCommitPoint(gen);
- 
+
     //System.out.println("ask for files for gen:" + commit.getGeneration() + core.getCoreDescriptor().getCoreContainer().getZkController().getNodeName());
     if (commit == null) {
       rsp.add("status", "invalid index generation");
@@ -456,7 +536,7 @@ public class ReplicationHandler extends
           Map<String,Object> fileMeta = new HashMap<>();
           fileMeta.put(NAME, file);
           fileMeta.put(SIZE, dir.fileLength(file));
-          
+
           try (final IndexInput in = dir.openInput(file, IOContext.READONCE)) {
             try {
               long checksum = CodecUtil.retrieveChecksum(in);
@@ -465,13 +545,13 @@ public class ReplicationHandler extends
               LOG.warn("Could not read checksum from index file: " + file, e);
             }
           }
-          
+
           result.add(fileMeta);
         }
       }
 
       // add the segments_N file
-      
+
       Map<String,Object> fileMeta = new HashMap<>();
       fileMeta.put(NAME, infos.getSegmentsFileName());
       fileMeta.put(SIZE, dir.fileLength(infos.getSegmentsFileName()));
@@ -487,7 +567,7 @@ public class ReplicationHandler extends
       result.add(fileMeta);
     } catch (IOException e) {
       rsp.add("status", "unable to get file names for given index generation");
-      rsp.add("exception", e);
+      rsp.add(EXCEPTION, e);
       LOG.error("Unable to get file names for indexCommit generation: " + gen, e);
     } finally {
       if (dir != null) {
@@ -613,7 +693,7 @@ public class ReplicationHandler extends
     return "ReplicationHandler provides replication of index and configuration files from Master to Slaves";
   }
 
-  /** 
+  /**
    * returns the CommitVersionInfo for the current searcher, or null on error.
    */
   private CommitVersionInfo getIndexVersion() {
@@ -694,7 +774,7 @@ public class ReplicationHandler extends
     CommitVersionInfo vInfo = getIndexVersion();
     details.add("indexVersion", null == vInfo ? 0 : vInfo.version);
     details.add(GENERATION, null == vInfo ? 0 : vInfo.generation);
-    
+
     IndexCommit commit = indexCommitPoint;  // make a copy so it won't change
 
     if (isMaster) {
@@ -832,11 +912,11 @@ public class ReplicationHandler extends
       details.add("master", master);
     if (slave.size() > 0)
       details.add("slave", slave);
-    
+
     NamedList snapshotStats = snapShootDetails;
     if (snapshotStats != null)
       details.add(CMD_BACKUP, snapshotStats);
-    
+
     return details;
   }
 
@@ -971,12 +1051,12 @@ public class ReplicationHandler extends
             Boolean.toString(enableMaster) + " and slave setting is " + Boolean.toString(enableSlave));
       }
     }
-    
+
     if (!enableSlave && !enableMaster) {
       enableMaster = true;
       master = new NamedList<>();
     }
-    
+
     if (enableMaster) {
       includeConfFiles = (String) master.get(CONF_FILES);
       if (includeConfFiles != null && includeConfFiles.trim().length() > 0) {
@@ -999,7 +1079,7 @@ public class ReplicationHandler extends
       if (!replicateOnCommit && ! replicateOnOptimize) {
         replicateOnCommit = true;
       }
-      
+
       // if we only want to replicate on optimize, we need the deletion policy to
       // save the last optimized commit point.
       if (replicateOnOptimize) {
@@ -1068,7 +1148,7 @@ public class ReplicationHandler extends
       isMaster = true;
     }
   }
-  
+
   // check master or slave is enabled
   private boolean isEnabled( NamedList params ){
     if( params == null ) return false;
@@ -1106,6 +1186,19 @@ public class ReplicationHandler extends
       @Override
       public void postClose(SolrCore core) {}
     });
+
+    core.addCloseHook(new CloseHook() {
+      @Override
+      public void preClose(SolrCore core) {
+        ExecutorUtil.shutdownNowAndAwaitTermination(restoreExecutor);
+        if (restoreFuture != null) {
+          restoreFuture.cancel(true);
+        }
+      }
+
+      @Override
+      public void postClose(SolrCore core) {}
+    });
   }
 
   /**
@@ -1407,6 +1500,14 @@ public class ReplicationHandler extends
     return result;
   }
 
+  private static final String LOCATION = "location";
+
+  private static final String SUCCESS = "success";
+
+  private static final String FAILED = "failed";
+
+  private static final String EXCEPTION = "exception";
+
   public static final String MASTER_URL = "masterUrl";
 
   public static final String STATUS = "status";
@@ -1417,6 +1518,10 @@ public class ReplicationHandler extends
 
   public static final String CMD_BACKUP = "backup";
 
+  public static final String CMD_RESTORE = "restore";
+
+  public static final String CMD_RESTORE_STATUS = "restorestatus";
+
   public static final String CMD_FETCH_INDEX = "fetchindex";
 
   public static final String CMD_ABORT_FETCH = "abortfetch";

Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/RestoreCore.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/RestoreCore.java?rev=1671022&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/RestoreCore.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/RestoreCore.java Fri Apr  3 09:46:54 2015
@@ -0,0 +1,149 @@
+package org.apache.solr.handler;
+
+/*
+ * 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.
+ */
+
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Future;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FSDirectory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.core.DirectoryFactory;
+import org.apache.solr.core.SolrCore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class RestoreCore implements Callable<Boolean> {
+
+  private static final Logger log = LoggerFactory.getLogger(RestoreCore.class.getName());
+
+  private final String backupName;
+  private final String backupLocation;
+  private final SolrCore core;
+
+  public RestoreCore(SolrCore core, String location, String name) {
+    this.core = core;
+    this.backupLocation = location;
+    this.backupName = name;
+  }
+
+  @Override
+  public Boolean call() throws Exception {
+    return doRestore();
+  }
+
+  private boolean doRestore() throws Exception {
+
+    Path backupPath = Paths.get(backupLocation, backupName);
+    String restoreIndexName = "restore." + backupName;
+    Path restoreIndexPath = Paths.get(core.getDataDir(), restoreIndexName);
+
+    Directory restoreIndexDir = null;
+    Directory indexDir = null;
+    try (Directory backupDir = FSDirectory.open(backupPath)) {
+
+      restoreIndexDir = core.getDirectoryFactory().get(restoreIndexPath.toString(),
+          DirectoryFactory.DirContext.DEFAULT, core.getSolrConfig().indexConfig.lockType);
+
+      //Prefer local copy.
+      indexDir = core.getDirectoryFactory().get(core.getIndexDir(),
+          DirectoryFactory.DirContext.DEFAULT, core.getSolrConfig().indexConfig.lockType);
+
+      //Move all files from backupDir to restoreIndexDir
+      for (String filename : backupDir.listAll()) {
+        checkInterrupted();
+        log.info("Copying over file to restore directory " + filename);
+        try (IndexInput indexInput = backupDir.openInput(filename, IOContext.READONCE)) {
+          long checksum = CodecUtil.retrieveChecksum(indexInput);
+          long length = indexInput.length();
+          IndexFetcher.CompareResult compareResult = IndexFetcher.compareFile(indexDir, filename, length, checksum);
+          if (!compareResult.equal || (!compareResult.checkSummed && (filename.endsWith(".si")
+              || filename.endsWith(".liv") || filename.startsWith("segments_")))) {
+            restoreIndexDir.copyFrom(backupDir, filename, filename, IOContext.READONCE);
+          } else {
+            //prefer local copy
+            restoreIndexDir.copyFrom(indexDir, filename, filename, IOContext.READONCE);
+          }
+        } catch (Exception e) {
+          throw new SolrException(SolrException.ErrorCode.UNKNOWN, "Exception while restoring the backup index", e);
+        }
+      }
+      log.debug("Switching directories");
+      IndexFetcher.modifyIndexProps(core, restoreIndexName);
+
+      boolean success;
+      try {
+        core.getUpdateHandler().newIndexWriter(false);
+        openNewSearcher();
+        success = true;
+        log.info("Successfully restored to the backup index");
+      } catch (Exception e) {
+        //Rollback to the old index directory. Delete the restore index directory and mark the restore as failed.
+        log.info("Could not switch to restored index. Rolling back to the current index");
+        Directory dir = null;
+        try {
+          dir = core.getDirectoryFactory().get(core.getDataDir(), DirectoryFactory.DirContext.META_DATA,
+              core.getSolrConfig().indexConfig.lockType);
+          dir.deleteFile(IndexFetcher.INDEX_PROPERTIES);
+        } finally {
+          if (dir != null) {
+            core.getDirectoryFactory().release(dir);
+          }
+        }
+
+        core.getDirectoryFactory().doneWithDirectory(restoreIndexDir);
+        core.getDirectoryFactory().remove(restoreIndexDir);
+        core.getUpdateHandler().newIndexWriter(false);
+        openNewSearcher();
+        throw new SolrException(SolrException.ErrorCode.UNKNOWN, "Exception while restoring the backup index", e);
+      }
+      if (success) {
+        core.getDirectoryFactory().doneWithDirectory(indexDir);
+        core.getDirectoryFactory().remove(indexDir);
+      }
+
+      return true;
+    } finally {
+      if (restoreIndexDir != null) {
+        core.getDirectoryFactory().release(restoreIndexDir);
+      }
+      if (indexDir != null) {
+        core.getDirectoryFactory().release(indexDir);
+      }
+    }
+  }
+
+  private void checkInterrupted() throws InterruptedException {
+    if (Thread.currentThread().isInterrupted()) {
+      throw new InterruptedException("Stopping restore process. Thread was interrupted.");
+    }
+  }
+
+  private void openNewSearcher() throws Exception {
+    Future[] waitSearcher = new Future[1];
+    core.getSearcher(true, false, waitSearcher, true);
+    if (waitSearcher[0] != null) {
+      waitSearcher[0].get();
+    }
+  }
+}

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/SnapShooter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/SnapShooter.java?rev=1671022&r1=1671021&r2=1671022&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/SnapShooter.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/SnapShooter.java Fri Apr  3 09:46:54 2015
@@ -18,6 +18,7 @@ package org.apache.solr.handler;
 
 import java.io.File;
 import java.io.IOException;
+import java.nio.file.Paths;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -58,10 +59,11 @@ public class SnapShooter {
 
   public SnapShooter(SolrCore core, String location, String snapshotName) {
     solrCore = core;
-    if (location == null) snapDir = core.getDataDir();
+    if (location == null) {
+      snapDir = core.getDataDir();
+    }
     else  {
-      File base = new File(core.getCoreDescriptor().getInstanceDir());
-      snapDir = org.apache.solr.util.FileUtils.resolvePath(base, location).getAbsolutePath();
+      snapDir = Paths.get(core.getCoreDescriptor().getInstanceDir()).resolve(location).toAbsolutePath().toString();
     }
     this.snapshotName = snapshotName;
 
@@ -125,7 +127,7 @@ public class SnapShooter {
   }
 
   void createSnapshot(final IndexCommit indexCommit, ReplicationHandler replicationHandler) {
-    LOG.info("Creating backup snapshot...");
+    LOG.info("Creating backup snapshot " + (snapshotName == null ? "<not named>" : snapshotName));
     NamedList<Object> details = new NamedList<>();
     details.add("startTime", new Date().toString());
     try {
@@ -193,31 +195,6 @@ public class SnapShooter {
     replicationHandler.snapShootDetails = details;
   }
 
-  private class OldBackupDirectory implements Comparable<OldBackupDirectory>{
-    File dir;
-    Date timestamp;
-    final Pattern dirNamePattern = Pattern.compile("^snapshot[.](.*)$");
-
-    OldBackupDirectory(File dir) {
-      if(dir.isDirectory()) {
-        Matcher m = dirNamePattern.matcher(dir.getName());
-        if(m.find()) {
-          try {
-            this.dir = dir;
-            this.timestamp = new SimpleDateFormat(DATE_FMT, Locale.ROOT).parse(m.group(1));
-          } catch(Exception e) {
-            this.dir = null;
-            this.timestamp = null;
-          }
-        }
-      }
-    }
-    @Override
-    public int compareTo(OldBackupDirectory that) {
-      return that.timestamp.compareTo(this.timestamp);
-    }
-  }
-
   public static final String DATE_FMT = "yyyyMMddHHmmssSSS";
 
 

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/TestReplicationHandlerBackup.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/TestReplicationHandlerBackup.java?rev=1671022&r1=1671021&r2=1671022&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/TestReplicationHandlerBackup.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/TestReplicationHandlerBackup.java Fri Apr  3 09:46:54 2015
@@ -121,7 +121,7 @@ public class TestReplicationHandlerBacku
   @Test
   public void testBackupOnCommit() throws Exception {
     //Index
-    int nDocs = indexDocs();
+    int nDocs = indexDocs(masterClient);
 
     //Confirm if completed
     CheckBackupStatus checkBackupStatus = new CheckBackupStatus((HttpSolrClient) masterClient);
@@ -146,7 +146,7 @@ public class TestReplicationHandlerBacku
     }
   }
 
-  private int indexDocs() throws IOException, SolrServerException {
+  protected static int indexDocs(SolrClient masterClient) throws IOException, SolrServerException {
     int nDocs = TestUtil.nextInt(random(), 1, 100);
     masterClient.deleteByQuery("*:*");
     for (int i = 0; i < nDocs; i++) {
@@ -164,7 +164,7 @@ public class TestReplicationHandlerBacku
   @Test
   public void doTestBackup() throws Exception {
 
-    int nDocs = indexDocs();
+    int nDocs = indexDocs(masterClient);
 
     Path[] snapDir = new Path[5]; //One extra for the backup on commit
     //First snapshot location
@@ -180,18 +180,17 @@ public class TestReplicationHandlerBacku
       backupNames = new String[4];
     }
     for (int i = 0; i < 4; i++) {
-      BackupCommand backupCommand;
       final String backupName = TestUtil.randomSimpleString(random(), 1, 20);
       if (!namedBackup) {
-        backupCommand = new BackupCommand(addNumberToKeepInRequest, backupKeepParamName, ReplicationHandler.CMD_BACKUP);
+        if (addNumberToKeepInRequest) {
+          runBackupCommand(masterJetty, ReplicationHandler.CMD_BACKUP, "&" + backupKeepParamName + "=2");
+        } else {
+          runBackupCommand(masterJetty, ReplicationHandler.CMD_BACKUP, "");
+        }
       } else {
-        backupCommand = new BackupCommand(backupName, ReplicationHandler.CMD_BACKUP);
+          runBackupCommand(masterJetty, ReplicationHandler.CMD_BACKUP, "&name=" +  backupName);
         backupNames[i] = backupName;
       }
-      backupCommand.runCommand();
-      if (backupCommand.fail != null) {
-        fail(backupCommand.fail);
-      }
 
       CheckBackupStatus checkBackupStatus = new CheckBackupStatus((HttpSolrClient) masterClient, firstBackupTimestamp);
       while (!checkBackupStatus.success) {
@@ -253,8 +252,7 @@ public class TestReplicationHandlerBacku
   private void testDeleteNamedBackup(String backupNames[]) throws InterruptedException, IOException {
     String lastTimestamp = null;
     for (int i = 0; i < 2; i++) {
-      BackupCommand deleteBackupCommand = new BackupCommand(backupNames[i], ReplicationHandler.CMD_DELETE_BACKUP);
-      deleteBackupCommand.runCommand();
+      runBackupCommand(masterJetty, ReplicationHandler.CMD_DELETE_BACKUP, "&name=" +backupNames[i]);
       CheckDeleteBackupStatus checkDeleteBackupStatus = new CheckDeleteBackupStatus(backupNames[i], lastTimestamp);
       while (true) {
         boolean success = checkDeleteBackupStatus.fetchStatus();
@@ -267,52 +265,19 @@ public class TestReplicationHandlerBacku
         }
         Thread.sleep(200);
       }
-
-      if (deleteBackupCommand.fail != null) {
-        fail(deleteBackupCommand.fail);
-      }
     }
   }
 
-  private class BackupCommand {
-    String fail = null;
-    final boolean addNumberToKeepInRequest;
-    String backupKeepParamName;
-    String backupName;
-    String cmd;
-    
-    BackupCommand(boolean addNumberToKeepInRequest, String backupKeepParamName, String command) {
-      this.addNumberToKeepInRequest = addNumberToKeepInRequest;
-      this.backupKeepParamName = backupKeepParamName;
-      this.cmd = command;
-    }
-    BackupCommand(String backupName, String command) {
-      this.backupName = backupName;
-      addNumberToKeepInRequest = false;
-      this.cmd = command;
-    }
-    
-    public void runCommand() {
-      String masterUrl;
-      if(backupName != null) {
-        masterUrl = buildUrl(masterJetty.getLocalPort(), context) + "/" + DEFAULT_TEST_CORENAME + "/replication?command=" + cmd +
-            "&name=" +  backupName;
-      } else {
-        masterUrl = buildUrl(masterJetty.getLocalPort(), context) + "/" + DEFAULT_TEST_CORENAME + "/replication?command=" + cmd +
-            (addNumberToKeepInRequest ? "&" + backupKeepParamName + "=2" : "");
-      }
-
-      InputStream stream = null;
-      try {
-        URL url = new URL(masterUrl);
-        stream = url.openStream();
-        stream.close();
-      } catch (Exception e) {
-        fail = e.getMessage();
-      } finally {
-        IOUtils.closeQuietly(stream);
-      }
-
+  public static void runBackupCommand(JettySolrRunner masterJetty, String cmd, String params) throws IOException {
+    String masterUrl = buildUrl(masterJetty.getLocalPort(), context) + "/" + DEFAULT_TEST_CORENAME
+        + "/replication?command=" + cmd + params;
+    InputStream stream = null;
+    try {
+      URL url = new URL(masterUrl);
+      stream = url.openStream();
+      stream.close();
+    } finally {
+      IOUtils.closeQuietly(stream);
     }
   }
 
@@ -349,6 +314,6 @@ public class TestReplicationHandlerBacku
         IOUtils.closeQuietly(stream);
       }
       return false;
-    };
+    }
   }
 }

Added: lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/TestRestoreCore.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/TestRestoreCore.java?rev=1671022&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/TestRestoreCore.java (added)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/TestRestoreCore.java Fri Apr  3 09:46:54 2015
@@ -0,0 +1,243 @@
+package org.apache.solr.handler;
+
+/*
+ * 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.
+ */
+
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URL;
+import java.net.URLEncoder;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.regex.Pattern;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.util.TestUtil;
+import org.apache.solr.SolrJettyTestBase;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.util.FileUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+@SolrTestCaseJ4.SuppressSSL     // Currently unknown why SSL does not work with this test
+public class TestRestoreCore extends SolrJettyTestBase {
+
+  JettySolrRunner masterJetty;
+  TestReplicationHandler.SolrInstance master = null;
+  SolrClient masterClient;
+
+  private static final String CONF_DIR = "solr" + File.separator + "collection1" + File.separator + "conf"
+      + File.separator;
+
+  private static String context = "/solr";
+
+  private static JettySolrRunner createJetty(TestReplicationHandler.SolrInstance instance) throws Exception {
+    FileUtils.copyFile(new File(SolrTestCaseJ4.TEST_HOME(), "solr.xml"), new File(instance.getHomeDir(), "solr.xml"));
+    JettySolrRunner jetty = new JettySolrRunner(instance.getHomeDir(), "/solr", 0);
+    jetty.setDataDir(instance.getDataDir());
+    jetty.start();
+    return jetty;
+  }
+
+  private static SolrClient createNewSolrClient(int port) {
+    try {
+      // setup the client...
+      HttpSolrClient client = new HttpSolrClient(buildUrl(port, context) + "/" + DEFAULT_TEST_CORENAME);
+      client.setConnectionTimeout(15000);
+      client.setSoTimeout(60000);
+      client.setDefaultMaxConnectionsPerHost(100);
+      client.setMaxTotalConnections(100);
+      return client;
+    }
+    catch (Exception ex) {
+      throw new RuntimeException(ex);
+    }
+  }
+
+
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+    String configFile = "solrconfig-master.xml";
+
+    master = new TestReplicationHandler.SolrInstance(createTempDir("solr-instance").toFile(), "master", null);
+    master.setUp();
+    master.copyConfigFile(CONF_DIR + configFile, "solrconfig.xml");
+
+    masterJetty = createJetty(master);
+    masterClient = createNewSolrClient(masterJetty.getLocalPort());
+  }
+
+  @Override
+  @After
+  public void tearDown() throws Exception {
+    super.tearDown();
+    masterClient.close();
+    masterClient  = null;
+    masterJetty.stop();
+    masterJetty = null;
+    master = null;
+  }
+
+  @Test
+  public void testSimpleRestore() throws Exception {
+
+    int nDocs = TestReplicationHandlerBackup.indexDocs(masterClient);
+
+    String snapshotName;
+    String location;
+    String params = "";
+
+    //Use the default backup location or an externally provided location.
+    if (random().nextBoolean()) {
+      location = createTempDir().toFile().getAbsolutePath();
+      params += "&location=" + URLEncoder.encode(location, "UTF-8");
+    }
+
+    //named snapshot vs default snapshot name
+    if (random().nextBoolean()) {
+      snapshotName = TestUtil.randomSimpleString(random(), 1, 5);
+      params += "&name=" + snapshotName;
+    }
+
+    TestReplicationHandlerBackup.runBackupCommand(masterJetty, ReplicationHandler.CMD_BACKUP, params);
+
+    CheckBackupStatus checkBackupStatus = new CheckBackupStatus((HttpSolrClient) masterClient, null);
+    while (!checkBackupStatus.success) {
+      checkBackupStatus.fetchStatus();
+      Thread.sleep(1000);
+    }
+
+    //Modify existing index before we call restore.
+
+    //Delete a few docs
+    int numDeletes = TestUtil.nextInt(random(), 1, nDocs);
+    for(int i=0; i<numDeletes; i++) {
+      masterClient.deleteByQuery("id:" + i);
+    }
+    masterClient.commit();
+
+    //Add a few more
+    int moreAdds = TestUtil.nextInt(random(), 1, 100);
+    for (int i=0; i<moreAdds; i++) {
+      SolrInputDocument doc = new SolrInputDocument();
+      doc.addField("id", i + nDocs);
+      doc.addField("name", "name = " + (i + nDocs));
+      masterClient.add(doc);
+    }
+    //Purposely not calling commit once in a while. There can be some docs which are not committed
+    if (usually()) {
+      masterClient.commit();
+    }
+
+    TestReplicationHandlerBackup.runBackupCommand(masterJetty, ReplicationHandler.CMD_RESTORE, params);
+
+    while (!fetchRestoreStatus()) {
+      Thread.sleep(1000);
+    }
+
+    //See if restore was successful by checking if all the docs are present again
+    verifyDocs(nDocs);
+  }
+
+  @Test
+  public void testFailedRestore() throws Exception {
+    int nDocs = TestReplicationHandlerBackup.indexDocs(masterClient);
+
+    String location = createTempDir().toFile().getAbsolutePath();
+    String snapshotName = TestUtil.randomSimpleString(random(), 1, 5);
+    String params = "&name=" + snapshotName + "&location=" + URLEncoder.encode(location, "UTF-8");
+
+    TestReplicationHandlerBackup.runBackupCommand(masterJetty, ReplicationHandler.CMD_BACKUP, params);
+
+    CheckBackupStatus checkBackupStatus = new CheckBackupStatus((HttpSolrClient) masterClient, null);
+    while (!checkBackupStatus.success) {
+      checkBackupStatus.fetchStatus();
+      Thread.sleep(1000);
+    }
+
+    //Remove the segments_n file so that the backup index is corrupted.
+    //Restore should fail and it should automatically rollback to the original index.
+    Path restoreIndexPath = Paths.get(location, "snapshot." + snapshotName);
+    Path segmentFileName = Files.newDirectoryStream(restoreIndexPath, IndexFileNames.SEGMENTS + "*").iterator().next();
+    Files.delete(segmentFileName);
+
+    TestReplicationHandlerBackup.runBackupCommand(masterJetty, ReplicationHandler.CMD_RESTORE, params);
+
+    try {
+      while (!fetchRestoreStatus()) {
+        Thread.sleep(1000);
+      }
+      fail("Should have thrown an error because restore could not have been successful");
+    } catch (AssertionError e) {
+      //supposed to happen
+    }
+
+    verifyDocs(nDocs);
+
+    //make sure we can write to the index again
+    nDocs = TestReplicationHandlerBackup.indexDocs(masterClient);
+    verifyDocs(nDocs);
+
+  }
+
+  private void verifyDocs(int nDocs) throws SolrServerException, IOException {
+    ModifiableSolrParams queryParams = new ModifiableSolrParams();
+    queryParams.set("q", "*:*");
+    QueryResponse response = masterClient.query(queryParams);
+
+    assertEquals(0, response.getStatus());
+    assertEquals(nDocs, response.getResults().getNumFound());
+  }
+
+  private boolean fetchRestoreStatus() throws IOException {
+    String masterUrl = buildUrl(masterJetty.getLocalPort(), context) + "/" + DEFAULT_TEST_CORENAME +
+        "/replication?command=" + ReplicationHandler.CMD_RESTORE_STATUS;
+    final Pattern pException = Pattern.compile("<str name=\"exception\">(.*?)</str>");
+
+    InputStream stream = null;
+    try {
+      URL url = new URL(masterUrl);
+      stream = url.openStream();
+      String response = IOUtils.toString(stream, "UTF-8");
+      if(pException.matcher(response).find()) {
+        fail("Failed to complete restore action");
+      }
+      if(response.contains("<str name=\"status\">success</str>")) {
+        return true;
+      } else if (response.contains("<str name=\"status\">failed</str>")){
+        fail("Restore Failed");
+      }
+      stream.close();
+    } finally {
+      IOUtils.closeQuietly(stream);
+    }
+    return false;
+  }
+}