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 2016/06/23 18:19:47 UTC

[2/2] lucene-solr:master: SOLR-7374: Core level backup/restore now supports specifying a directory implementation

SOLR-7374: Core level backup/restore now supports specifying a directory implementation


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

Branch: refs/heads/master
Commit: 07be2c42ba24fea7c4e84836aa4c3f8d059f71d6
Parents: 1e4d51f
Author: Varun Thacker <va...@gmail.com>
Authored: Thu Jun 23 23:49:14 2016 +0530
Committer: Varun Thacker <va...@gmail.com>
Committed: Thu Jun 23 23:49:28 2016 +0530

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../cloud/OverseerCollectionMessageHandler.java |   4 +-
 .../org/apache/solr/core/CoreContainer.java     |   9 +
 .../apache/solr/core/HdfsDirectoryFactory.java  |   2 +-
 .../java/org/apache/solr/core/NodeConfig.java   |  20 +-
 .../org/apache/solr/core/SolrXmlConfig.java     |  12 +
 .../backup/repository/BackupRepository.java     | 166 ++++++++++++
 .../repository/BackupRepositoryFactory.java     |  89 +++++++
 .../backup/repository/HdfsBackupRepository.java | 159 ++++++++++++
 .../repository/LocalFileSystemRepository.java   | 136 ++++++++++
 .../core/backup/repository/package-info.java    |  23 ++
 .../apache/solr/handler/OldBackupDirectory.java |  55 ++--
 .../apache/solr/handler/ReplicationHandler.java |  61 ++++-
 .../org/apache/solr/handler/RestoreCore.java    |  22 +-
 .../org/apache/solr/handler/SnapShooter.java    | 153 ++++++-----
 .../solr/handler/admin/CollectionsHandler.java  |   4 +-
 .../solr/handler/admin/CoreAdminOperation.java  |  42 +++-
 .../apache/solr/store/hdfs/HdfsDirectory.java   |   5 +-
 solr/core/src/test-files/solr/solr-50-all.xml   |   4 +
 .../solr/cloud/BasicDistributedZk2Test.java     |  27 +-
 .../solr/core/TestBackupRepositoryFactory.java  | 152 +++++++++++
 .../test/org/apache/solr/core/TestSolrXml.java  |  13 +
 .../apache/solr/handler/BackupRestoreUtils.java |  69 +++++
 .../apache/solr/handler/CheckBackupStatus.java  |  10 +-
 .../solr/handler/TestHdfsBackupRestoreCore.java | 251 +++++++++++++++++++
 .../handler/TestReplicationHandlerBackup.java   |  39 ++-
 .../apache/solr/handler/TestRestoreCore.java    |  52 ++--
 27 files changed, 1399 insertions(+), 183 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/07be2c42/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 683b98b..c8b6914 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -66,6 +66,9 @@ New Features
 
 * SOLR-8048: bin/solr script should support basic auth credentials provided in solr.in.sh (noble)
 
+* SOLR-7374: Core level Backup/Restore now supports specifying the directory implementation to use
+  via the "repository" parameter. (Hrishikesh Gadre, Varun Thacker, Mark Miller)
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/07be2c42/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java b/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
index 2cd09d1..d7c7ad2 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionMessageHandler.java
@@ -2195,7 +2195,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
   private void processBackupAction(ZkNodeProps message, NamedList results) throws IOException, KeeperException, InterruptedException {
     String collectionName =  message.getStr(COLLECTION_PROP);
     String backupName =  message.getStr(NAME);
-    String location = message.getStr("location");
+    String location = message.getStr(ZkStateReader.BACKUP_LOCATION);
     ShardHandler shardHandler = shardHandlerFactory.getShardHandler();
     String asyncId = message.getStr(ASYNC);
     Map<String, String> requestMap = new HashMap<>();
@@ -2267,7 +2267,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler
     // TODO maybe we can inherit createCollection's options/code
     String restoreCollectionName =  message.getStr(COLLECTION_PROP);
     String backupName =  message.getStr(NAME); // of backup
-    String location = message.getStr("location");
+    String location = message.getStr(ZkStateReader.BACKUP_LOCATION);
     ShardHandler shardHandler = shardHandlerFactory.getShardHandler();
     String asyncId = message.getStr(ASYNC);
     Map<String, String> requestMap = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/07be2c42/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 e5e16cb..bfab1b8 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -59,6 +59,7 @@ 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.backup.repository.BackupRepositoryFactory;
 import org.apache.solr.handler.RequestHandlerBase;
 import org.apache.solr.handler.admin.CollectionsHandler;
 import org.apache.solr.handler.admin.ConfigSetsHandler;
@@ -152,6 +153,12 @@ public class CoreContainer {
 
   private SecurityPluginHolder<AuthenticationPlugin> authenticationPlugin;
 
+  private BackupRepositoryFactory backupRepoFactory;
+
+  public BackupRepositoryFactory getBackupRepoFactory() {
+    return backupRepoFactory;
+  }
+
   public ExecutorService getCoreZkRegisterExecutorService() {
     return zkSys.getCoreZkRegisterExecutorService();
   }
@@ -441,6 +448,8 @@ public class CoreContainer {
     initializeAuthorizationPlugin((Map<String, Object>) securityConfig.data.get("authorization"));
     initializeAuthenticationPlugin((Map<String, Object>) securityConfig.data.get("authentication"));
 
+    this.backupRepoFactory = new BackupRepositoryFactory(cfg.getBackupRepositoryPlugins());
+
     containerHandlers.put(ZK_PATH, new ZookeeperInfoHandler(this));
     securityConfHandler = new SecurityConfHandler(this);
     collectionsHandler = createHandler(cfg.getCollectionsHandlerClass(), CollectionsHandler.class);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/07be2c42/solr/core/src/java/org/apache/solr/core/HdfsDirectoryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/HdfsDirectoryFactory.java b/solr/core/src/java/org/apache/solr/core/HdfsDirectoryFactory.java
index c911ac5..b003287 100644
--- a/solr/core/src/java/org/apache/solr/core/HdfsDirectoryFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/HdfsDirectoryFactory.java
@@ -331,7 +331,7 @@ public class HdfsDirectoryFactory extends CachingDirectoryFactory implements Sol
     }
   }
   
-  private Configuration getConf() {
+  public Configuration getConf() {
     Configuration conf = new Configuration();
     confDir = getConfig(CONFIG_DIRECTORY, null);
     HdfsUtil.addHdfsResources(conf, confDir);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/07be2c42/solr/core/src/java/org/apache/solr/core/NodeConfig.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/NodeConfig.java b/solr/core/src/java/org/apache/solr/core/NodeConfig.java
index 0783355..e72fbc9 100644
--- a/solr/core/src/java/org/apache/solr/core/NodeConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/NodeConfig.java
@@ -58,13 +58,15 @@ public class NodeConfig {
 
   private final String managementPath;
 
+  private final PluginInfo[] backupRepositoryPlugins;
+
   private NodeConfig(String nodeName, Path coreRootDirectory, Path configSetBaseDirectory, String sharedLibDirectory,
                      PluginInfo shardHandlerFactoryConfig, UpdateShardHandlerConfig updateShardHandlerConfig,
                      String coreAdminHandlerClass, String collectionsAdminHandlerClass,
                      String infoHandlerClass, String configSetsHandlerClass,
                      LogWatcherConfig logWatcherConfig, CloudConfig cloudConfig, int coreLoadThreads,
-                     int transientCacheSize, boolean useSchemaCache, String managementPath,
-                     SolrResourceLoader loader, Properties solrProperties) {
+                     int transientCacheSize, boolean useSchemaCache, String managementPath, SolrResourceLoader loader,
+                     Properties solrProperties, PluginInfo[] backupRepositoryPlugins) {
     this.nodeName = nodeName;
     this.coreRootDirectory = coreRootDirectory;
     this.configSetBaseDirectory = configSetBaseDirectory;
@@ -83,6 +85,7 @@ public class NodeConfig {
     this.managementPath = managementPath;
     this.loader = loader;
     this.solrProperties = solrProperties;
+    this.backupRepositoryPlugins = backupRepositoryPlugins;
 
     if (this.cloudConfig != null && this.coreLoadThreads < 2) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
@@ -165,6 +168,10 @@ public class NodeConfig {
     return loader;
   }
 
+  public PluginInfo[] getBackupRepositoryPlugins() {
+    return backupRepositoryPlugins;
+  }
+
   public static class NodeConfigBuilder {
 
     private Path coreRootDirectory;
@@ -183,6 +190,7 @@ public class NodeConfig {
     private boolean useSchemaCache = false;
     private String managementPath;
     private Properties solrProperties = new Properties();
+    private PluginInfo[] backupRepositoryPlugins;
 
     private final SolrResourceLoader loader;
     private final String nodeName;
@@ -283,10 +291,16 @@ public class NodeConfig {
       return this;
     }
 
+    public NodeConfigBuilder setBackupRepositoryPlugins(PluginInfo[] backupRepositoryPlugins) {
+      this.backupRepositoryPlugins = backupRepositoryPlugins;
+      return this;
+    }
+
     public NodeConfig build() {
       return new NodeConfig(nodeName, coreRootDirectory, configSetBaseDirectory, sharedLibDirectory, shardHandlerFactoryConfig,
                             updateShardHandlerConfig, coreAdminHandlerClass, collectionsAdminHandlerClass, infoHandlerClass, configSetsHandlerClass,
-                            logWatcherConfig, cloudConfig, coreLoadThreads, transientCacheSize, useSchemaCache, managementPath, loader, solrProperties);
+                            logWatcherConfig, cloudConfig, coreLoadThreads, transientCacheSize, useSchemaCache, managementPath, loader, solrProperties,
+                            backupRepositoryPlugins);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/07be2c42/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java b/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
index fedaf56..65b248d 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
@@ -95,6 +95,7 @@ public class SolrXmlConfig {
     configBuilder.setSolrProperties(loadProperties(config));
     if (cloudConfig != null)
       configBuilder.setCloudConfig(cloudConfig);
+    configBuilder.setBackupRepositoryPlugins((getBackupRepositoryPluginInfos(config)));
     return fillSolrSection(configBuilder, entries);
   }
 
@@ -154,6 +155,7 @@ public class SolrXmlConfig {
     assertSingleInstance("solrcloud", config);
     assertSingleInstance("logging", config);
     assertSingleInstance("logging/watcher", config);
+    assertSingleInstance("backup", config);
   }
 
   private static void assertSingleInstance(String section, Config config) {
@@ -424,5 +426,15 @@ public class SolrXmlConfig {
     return (node == null) ? null : new PluginInfo(node, "shardHandlerFactory", false, true);
   }
 
+  private static PluginInfo[] getBackupRepositoryPluginInfos(Config config) {
+    NodeList nodes = (NodeList) config.evaluate("solr/backup/repository", XPathConstants.NODESET);
+    if (nodes == null || nodes.getLength() == 0)
+      return new PluginInfo[0];
+    PluginInfo[] configs = new PluginInfo[nodes.getLength()];
+    for (int i = 0; i < nodes.getLength(); i++) {
+      configs[i] = new PluginInfo(nodes.item(i), "BackupRepositoryFactory", true, true);
+    }
+    return configs;
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/07be2c42/solr/core/src/java/org/apache/solr/core/backup/repository/BackupRepository.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/backup/repository/BackupRepository.java b/solr/core/src/java/org/apache/solr/core/backup/repository/BackupRepository.java
new file mode 100644
index 0000000..f209b87
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/core/backup/repository/BackupRepository.java
@@ -0,0 +1,166 @@
+package org.apache.solr.core.backup.repository;
+
+/*
+ * 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.Closeable;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.URI;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.solr.util.plugin.NamedListInitializedPlugin;
+
+/**
+ * This interface defines the functionality required to backup/restore Solr indexes to an arbitrary storage system.
+ */
+public interface BackupRepository extends NamedListInitializedPlugin, Closeable {
+  /**
+   * A parameter to specify the name of the backup repository to be used.
+   */
+  String REPOSITORY_PROPERTY_NAME = "repository";
+
+
+  /**
+   * This enumeration defines the type of a given path.
+   */
+  enum PathType {
+    DIRECTORY, FILE
+  }
+
+  /**
+   * This method returns the value of the specified configuration property.
+   */
+  <T> T getConfigProperty(String name);
+
+  /**
+   * This method creates a URI using the specified path components (as method arguments).
+   *
+   * @param pathComponents
+   *          The directory (or file-name) to be included in the URI.
+   * @return A URI containing absolute path
+   */
+  URI createURI(String... pathComponents);
+
+  /**
+   * This method checks if the specified path exists in this repository.
+   *
+   * @param path
+   *          The path whose existence needs to be checked.
+   * @return if the specified path exists in this repository.
+   * @throws IOException
+   *           in case of errors
+   */
+  boolean exists(URI path) throws IOException;
+
+  /**
+   * This method returns the type of a specified path
+   *
+   * @param path
+   *          The path whose type needs to be checked.
+   * @return the {@linkplain PathType} for the specified path
+   * @throws IOException
+   *           in case of errors
+   */
+  PathType getPathType(URI path) throws IOException;
+
+  /**
+   * This method returns all the entries (files and directories) in the specified directory.
+   *
+   * @param path
+   *          The directory path
+   * @return an array of strings, one for each entry in the directory
+   * @throws IOException
+   *           in case of errors
+   */
+  String[] listAll(URI path) throws IOException;
+
+  /**
+   * This method returns a Lucene input stream reading an existing file.
+   *
+   * @param dirPath
+   *          The parent directory of the file to be read
+   * @param fileName
+   *          The name of the file to be read
+   * @param ctx
+   *          the Lucene IO context
+   * @return Lucene {@linkplain IndexInput} reference
+   * @throws IOException
+   *           in case of errors
+   */
+  IndexInput openInput(URI dirPath, String fileName, IOContext ctx) throws IOException;
+
+  /**
+   * This method returns a {@linkplain OutputStream} instance for the specified <code>path</code>
+   *
+   * @param path
+   *          The path for which {@linkplain OutputStream} needs to be created
+   * @return {@linkplain OutputStream} instance for the specified <code>path</code>
+   * @throws IOException
+   *           in case of errors
+   */
+  OutputStream createOutput(URI path) throws IOException;
+
+  /**
+   * This method creates a directory at the specified path.
+   *
+   * @param path
+   *          The path where the directory needs to be created.
+   * @throws IOException
+   *           in case of errors
+   */
+  void createDirectory(URI path) throws IOException;
+
+  /**
+   * This method deletes a directory at the specified path.
+   *
+   * @param path
+   *          The path referring to the directory to be deleted.
+   * @throws IOException
+   *           in case of errors
+   */
+  void deleteDirectory(URI path) throws IOException;
+
+  /**
+   * Copy a file from specified <code>sourceDir</code> to the destination repository (i.e. backup).
+   *
+   * @param sourceDir
+   *          The source directory hosting the file to be copied.
+   * @param fileName
+   *          The name of the file to by copied
+   * @param dest
+   *          The destination backup location.
+   * @throws IOException
+   *           in case of errors
+   */
+  void copyFileFrom(Directory sourceDir, String fileName, URI dest) throws IOException;
+
+  /**
+   * Copy a file from specified <code>sourceRepo</code> to the destination directory (i.e. restore).
+   *
+   * @param sourceRepo
+   *          The source URI hosting the file to be copied.
+   * @param fileName
+   *          The name of the file to by copied
+   * @param dest
+   *          The destination where the file should be copied.
+   * @throws IOException
+   *           in case of errors.
+   */
+  void copyFileTo(URI sourceRepo, String fileName, Directory dest) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/07be2c42/solr/core/src/java/org/apache/solr/core/backup/repository/BackupRepositoryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/backup/repository/BackupRepositoryFactory.java b/solr/core/src/java/org/apache/solr/core/backup/repository/BackupRepositoryFactory.java
new file mode 100644
index 0000000..d035874
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/core/backup/repository/BackupRepositoryFactory.java
@@ -0,0 +1,89 @@
+package org.apache.solr.core.backup.repository;
+
+/*
+ * 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.lang.invoke.MethodHandles;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.core.SolrResourceLoader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+
+public class BackupRepositoryFactory {
+  private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private final Map<String,PluginInfo> backupRepoPluginByName = new HashMap<>();
+  private PluginInfo defaultBackupRepoPlugin = null;
+
+  public BackupRepositoryFactory(PluginInfo[] backupRepoPlugins) {
+    if (backupRepoPlugins != null) {
+      for (int i = 0; i < backupRepoPlugins.length; i++) {
+        String name = backupRepoPlugins[i].name;
+        boolean isDefault = backupRepoPlugins[i].isDefault();
+
+        if (backupRepoPluginByName.containsKey(name)) {
+          throw new SolrException(ErrorCode.SERVER_ERROR, "Duplicate backup repository with name " + name);
+        }
+        if (isDefault) {
+          if (this.defaultBackupRepoPlugin != null) {
+            throw new SolrException(ErrorCode.SERVER_ERROR, "More than one backup repository is configured as default");
+          }
+          this.defaultBackupRepoPlugin = backupRepoPlugins[i];
+        }
+        backupRepoPluginByName.put(name, backupRepoPlugins[i]);
+        LOG.info("Added backup repository with configuration params {}", backupRepoPlugins[i]);
+      }
+      if (backupRepoPlugins.length == 1) {
+        this.defaultBackupRepoPlugin = backupRepoPlugins[0];
+      }
+
+      if (this.defaultBackupRepoPlugin != null) {
+        LOG.info("Default configuration for backup repository is with configuration params {}",
+            defaultBackupRepoPlugin);
+      }
+    }
+  }
+
+  public BackupRepository newInstance(SolrResourceLoader loader, String name) {
+    Preconditions.checkNotNull(loader);
+    Preconditions.checkNotNull(name);
+    PluginInfo repo = Preconditions.checkNotNull(backupRepoPluginByName.get(name),
+        "Could not find a backup repository with name " + name);
+
+    BackupRepository result = loader.newInstance(repo.className, BackupRepository.class);
+    result.init(repo.initArgs);
+    return result;
+  }
+
+  public BackupRepository newInstance(SolrResourceLoader loader) {
+    if (defaultBackupRepoPlugin != null) {
+      return newInstance(loader, defaultBackupRepoPlugin.name);
+    }
+
+    LocalFileSystemRepository repo = new LocalFileSystemRepository();
+    repo.init(new NamedList<>());
+    return repo;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/07be2c42/solr/core/src/java/org/apache/solr/core/backup/repository/HdfsBackupRepository.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/backup/repository/HdfsBackupRepository.java b/solr/core/src/java/org/apache/solr/core/backup/repository/HdfsBackupRepository.java
new file mode 100644
index 0000000..596c271
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/core/backup/repository/HdfsBackupRepository.java
@@ -0,0 +1,159 @@
+package org.apache.solr.core.backup.repository;
+
+/*
+ * 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.IOException;
+import java.io.OutputStream;
+import java.net.URI;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.NoLockFactory;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.DirectoryFactory;
+import org.apache.solr.core.HdfsDirectoryFactory;
+import org.apache.solr.store.hdfs.HdfsDirectory;
+import org.apache.solr.store.hdfs.HdfsDirectory.HdfsIndexInput;
+
+import com.google.common.base.Preconditions;
+
+public class HdfsBackupRepository implements BackupRepository {
+  private HdfsDirectoryFactory factory;
+  private Configuration hdfsConfig = null;
+  private FileSystem fileSystem = null;
+  private Path baseHdfsPath = null;
+  private NamedList config = null;
+
+  @SuppressWarnings("rawtypes")
+  @Override
+  public void init(NamedList args) {
+    this.config = args;
+
+    // We don't really need this factory instance. But we want to initialize it here to
+    // make sure that all HDFS related initialization is at one place (and not duplicated here).
+    factory = new HdfsDirectoryFactory();
+    factory.init(args);
+    this.hdfsConfig = factory.getConf();
+
+    String hdfsSolrHome = (String) Preconditions.checkNotNull(args.get(HdfsDirectoryFactory.HDFS_HOME),
+        "Please specify " + HdfsDirectoryFactory.HDFS_HOME + " property.");
+    Path path = new Path(hdfsSolrHome);
+    while (path != null) { // Compute the path of root file-system (without requiring an additional system property).
+      baseHdfsPath = path;
+      path = path.getParent();
+    }
+
+    try {
+      this.fileSystem = FileSystem.get(this.baseHdfsPath.toUri(), this.hdfsConfig);
+    } catch (IOException e) {
+      throw new SolrException(ErrorCode.SERVER_ERROR, e);
+    }
+  }
+
+  public void close() throws IOException {
+    if (this.fileSystem != null) {
+      this.fileSystem.close();
+    }
+    if (this.factory != null) {
+      this.factory.close();
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public <T> T getConfigProperty(String name) {
+    return (T) this.config.get(name);
+  }
+
+  @Override
+  public URI createURI(String... pathComponents) {
+    Path result = baseHdfsPath;
+    for (String p : pathComponents) {
+      result = new Path(result, p);
+    }
+    return result.toUri();
+  }
+
+  @Override
+  public boolean exists(URI path) throws IOException {
+    return this.fileSystem.exists(new Path(path));
+  }
+
+  @Override
+  public PathType getPathType(URI path) throws IOException {
+    return this.fileSystem.isDirectory(new Path(path)) ? PathType.DIRECTORY : PathType.FILE;
+  }
+
+  @Override
+  public String[] listAll(URI path) throws IOException {
+    FileStatus[] status = this.fileSystem.listStatus(new Path(path));
+    String[] result = new String[status.length];
+    for (int i = 0; i < status.length; i++) {
+      result[i] = status[i].getPath().getName();
+    }
+    return result;
+  }
+
+  @Override
+  public IndexInput openInput(URI dirPath, String fileName, IOContext ctx) throws IOException {
+    Path p = new Path(new Path(dirPath), fileName);
+    return new HdfsIndexInput(fileName, this.fileSystem, p, HdfsDirectory.DEFAULT_BUFFER_SIZE);
+  }
+
+  @Override
+  public OutputStream createOutput(URI path) throws IOException {
+    return this.fileSystem.create(new Path(path));
+  }
+
+  @Override
+  public void createDirectory(URI path) throws IOException {
+    if (!this.fileSystem.mkdirs(new Path(path))) {
+      throw new IOException("Unable to create a directory at following location " + path);
+    }
+  }
+
+  @Override
+  public void deleteDirectory(URI path) throws IOException {
+    if (!this.fileSystem.delete(new Path(path), true)) {
+      throw new IOException("Unable to delete a directory at following location " + path);
+    }
+  }
+
+  @Override
+  public void copyFileFrom(Directory sourceDir, String fileName, URI dest) throws IOException {
+    try (HdfsDirectory dir = new HdfsDirectory(new Path(dest), NoLockFactory.INSTANCE,
+        hdfsConfig, HdfsDirectory.DEFAULT_BUFFER_SIZE)) {
+      dir.copyFrom(sourceDir, fileName, fileName, DirectoryFactory.IOCONTEXT_NO_CACHE);
+    }
+  }
+
+  @Override
+  public void copyFileTo(URI sourceRepo, String fileName, Directory dest) throws IOException {
+    try (HdfsDirectory dir = new HdfsDirectory(new Path(sourceRepo), NoLockFactory.INSTANCE,
+        hdfsConfig, HdfsDirectory.DEFAULT_BUFFER_SIZE)) {
+      dest.copyFrom(dir, fileName, fileName, DirectoryFactory.IOCONTEXT_NO_CACHE);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/07be2c42/solr/core/src/java/org/apache/solr/core/backup/repository/LocalFileSystemRepository.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/backup/repository/LocalFileSystemRepository.java b/solr/core/src/java/org/apache/solr/core/backup/repository/LocalFileSystemRepository.java
new file mode 100644
index 0000000..bb75a9e
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/core/backup/repository/LocalFileSystemRepository.java
@@ -0,0 +1,136 @@
+package org.apache.solr.core.backup.repository;
+
+/*
+ * 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.IOException;
+import java.io.OutputStream;
+import java.net.URI;
+import java.nio.file.FileVisitResult;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.SimpleFileVisitor;
+import java.nio.file.attribute.BasicFileAttributes;
+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.lucene.store.NoLockFactory;
+import org.apache.lucene.store.SimpleFSDirectory;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.DirectoryFactory;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * A concrete implementation of {@linkplain BackupRepository} interface supporting backup/restore of Solr indexes to a
+ * local file-system. (Note - This can even be used for a shared file-system if it is exposed via a local file-system
+ * interface e.g. NFS).
+ */
+public class LocalFileSystemRepository implements BackupRepository {
+  private NamedList config = null;
+
+  @Override
+  public void init(NamedList args) {
+    this.config = args;
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public <T> T getConfigProperty(String name) {
+    return (T) this.config.get(name);
+  }
+
+  @Override
+  public URI createURI(String... pathComponents) {
+    Preconditions.checkArgument(pathComponents.length > 0);
+    Path result = Paths.get(pathComponents[0]);
+    for (int i = 1; i < pathComponents.length; i++) {
+      result = result.resolve(pathComponents[i]);
+    }
+    return result.toUri();
+  }
+
+  @Override
+  public void createDirectory(URI path) throws IOException {
+    Files.createDirectory(Paths.get(path));
+  }
+
+  @Override
+  public void deleteDirectory(URI path) throws IOException {
+    Files.walkFileTree(Paths.get(path), new SimpleFileVisitor<Path>() {
+      @Override
+      public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException {
+        Files.delete(file);
+        return FileVisitResult.CONTINUE;
+      }
+
+      @Override
+      public FileVisitResult postVisitDirectory(Path dir, IOException exc) throws IOException {
+        Files.delete(dir);
+        return FileVisitResult.CONTINUE;
+      }
+    });
+  }
+
+  @Override
+  public boolean exists(URI path) throws IOException {
+    return Files.exists(Paths.get(path));
+  }
+
+  @Override
+  public IndexInput openInput(URI dirPath, String fileName, IOContext ctx) throws IOException {
+    try (FSDirectory dir = new SimpleFSDirectory(Paths.get(dirPath), NoLockFactory.INSTANCE)) {
+      return dir.openInput(fileName, ctx);
+    }
+  }
+
+  @Override
+  public OutputStream createOutput(URI path) throws IOException {
+    return Files.newOutputStream(Paths.get(path));
+  }
+
+  @Override
+  public String[] listAll(URI dirPath) throws IOException {
+    try (FSDirectory dir = new SimpleFSDirectory(Paths.get(dirPath), NoLockFactory.INSTANCE)) {
+      return dir.listAll();
+    }
+  }
+
+  @Override
+  public PathType getPathType(URI path) throws IOException {
+    return Files.isDirectory(Paths.get(path)) ? PathType.DIRECTORY : PathType.FILE;
+  }
+
+  @Override
+  public void copyFileFrom(Directory sourceDir, String fileName, URI dest) throws IOException {
+    try (FSDirectory dir = new SimpleFSDirectory(Paths.get(dest), NoLockFactory.INSTANCE)) {
+      dir.copyFrom(sourceDir, fileName, fileName, DirectoryFactory.IOCONTEXT_NO_CACHE);
+    }
+  }
+
+  @Override
+  public void copyFileTo(URI sourceDir, String fileName, Directory dest) throws IOException {
+    try (FSDirectory dir = new SimpleFSDirectory(Paths.get(sourceDir), NoLockFactory.INSTANCE)) {
+      dest.copyFrom(dir, fileName, fileName, DirectoryFactory.IOCONTEXT_NO_CACHE);
+    }
+  }
+
+  @Override
+  public void close() throws IOException {}
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/07be2c42/solr/core/src/java/org/apache/solr/core/backup/repository/package-info.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/backup/repository/package-info.java b/solr/core/src/java/org/apache/solr/core/backup/repository/package-info.java
new file mode 100644
index 0000000..fb3cfd5
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/core/backup/repository/package-info.java
@@ -0,0 +1,23 @@
+/*
+* 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.
+*/
+
+
+/**
+* {@link org.apache.solr.core.backup.repository.BackupRepository} Providing backup/restore
+* repository interfaces to plug different storage systems
+*/
+package org.apache.solr.core.backup.repository;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/07be2c42/solr/core/src/java/org/apache/solr/handler/OldBackupDirectory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/OldBackupDirectory.java b/solr/core/src/java/org/apache/solr/handler/OldBackupDirectory.java
index 7fd0fec..2b19116 100644
--- a/solr/core/src/java/org/apache/solr/handler/OldBackupDirectory.java
+++ b/solr/core/src/java/org/apache/solr/handler/OldBackupDirectory.java
@@ -16,34 +16,55 @@
  */
 package org.apache.solr.handler;
 
-import java.io.File;
+import java.net.URI;
+import java.text.ParseException;
 import java.text.SimpleDateFormat;
 import java.util.Date;
 import java.util.Locale;
+import java.util.Optional;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import com.google.common.base.Preconditions;
+
 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;
-        }
+  private static final Pattern dirNamePattern = Pattern.compile("^snapshot[.](.*)$");
+
+  private URI basePath;
+  private String dirName;
+  private Optional<Date> timestamp;
+
+  public OldBackupDirectory(URI basePath, String dirName) {
+    this.dirName = Preconditions.checkNotNull(dirName);
+    this.basePath = Preconditions.checkNotNull(basePath);
+    Matcher m = dirNamePattern.matcher(dirName);
+    if (m.find()) {
+      try {
+        this.timestamp = Optional.of(new SimpleDateFormat(SnapShooter.DATE_FMT, Locale.ROOT).parse(m.group(1)));
+      } catch (ParseException e) {
+        this.timestamp = Optional.empty();
       }
     }
   }
+
+  public URI getPath() {
+    return this.basePath.resolve(dirName);
+  }
+
+  public String getDirName() {
+    return dirName;
+  }
+
+  public Optional<Date> getTimestamp() {
+    return timestamp;
+  }
+
   @Override
   public int compareTo(OldBackupDirectory that) {
-    return that.timestamp.compareTo(this.timestamp);
+    if(this.timestamp.isPresent() && that.timestamp.isPresent()) {
+      return that.timestamp.get().compareTo(this.timestamp.get());
+    }
+    // Use absolute value of path in case the time-stamp is missing on either side.
+    return that.getPath().compareTo(this.getPath());
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/07be2c42/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
index 9de4a78..0e6960c 100644
--- a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
@@ -24,6 +24,7 @@ import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.OutputStream;
 import java.lang.invoke.MethodHandles;
+import java.net.URI;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
 import java.nio.charset.StandardCharsets;
@@ -66,6 +67,7 @@ import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.RateLimiter;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
@@ -76,11 +78,15 @@ import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.SuppressForbidden;
 import org.apache.solr.core.CloseHook;
+import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.DirectoryFactory.DirContext;
 import org.apache.solr.core.IndexDeletionPolicyWrapper;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrDeletionPolicy;
 import org.apache.solr.core.SolrEventListener;
+import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.core.backup.repository.BackupRepository;
+import org.apache.solr.core.backup.repository.LocalFileSystemRepository;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.search.SolrIndexSearcher;
@@ -407,7 +413,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
     return indexFetchLock.isLocked();
   }
 
-  private void restore(SolrParams params, SolrQueryResponse rsp, SolrQueryRequest req) {
+  private void restore(SolrParams params, SolrQueryResponse rsp, SolrQueryRequest req) throws IOException {
     if (restoreFuture != null && !restoreFuture.isDone()) {
       throw new SolrException(ErrorCode.BAD_REQUEST, "Restore in progress. Cannot run multiple restore operations" +
           "for the same core");
@@ -415,6 +421,22 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
     String name = params.get(NAME);
     String location = params.get(LOCATION);
 
+    String repoName = params.get(BackupRepository.REPOSITORY_PROPERTY_NAME);
+    CoreContainer cc = core.getCoreDescriptor().getCoreContainer();
+    SolrResourceLoader rl = cc.getResourceLoader();
+    BackupRepository repo = null;
+    if(repoName != null) {
+      repo = cc.getBackupRepoFactory().newInstance(rl, repoName);
+      if (location == null) {
+        location = repo.getConfigProperty(ZkStateReader.BACKUP_LOCATION);
+        if(location == null) {
+          throw new IllegalArgumentException("location is required");
+        }
+      }
+    } else {
+      repo = new LocalFileSystemRepository();
+    }
+
     //If location is not provided then assume that the restore index is present inside the data directory.
     if (location == null) {
       location = core.getDataDir();
@@ -423,11 +445,12 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
     //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();
+      URI basePath = repo.createURI(location);
+      String[] filePaths = repo.listAll(basePath);
       List<OldBackupDirectory> dirs = new ArrayList<>();
-      for (File f : files) {
-        OldBackupDirectory obd = new OldBackupDirectory(f);
-        if (obd.dir != null) {
+      for (String f : filePaths) {
+        OldBackupDirectory obd = new OldBackupDirectory(basePath, f);
+        if (obd.getTimestamp().isPresent()) {
           dirs.add(obd);
         }
       }
@@ -435,13 +458,13 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
       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();
+      name = dirs.get(0).getDirName();
     } else {
       //"snapshot." is prefixed by snapshooter
       name = "snapshot." + name;
     }
 
-    RestoreCore restoreCore = new RestoreCore(core, location, name);
+    RestoreCore restoreCore = new RestoreCore(repo, core, location, name);
     try {
       MDC.put("RestoreCore.core", core.getName());
       MDC.put("RestoreCore.backupLocation", location);
@@ -504,8 +527,30 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw
         indexCommit = req.getSearcher().getIndexReader().getIndexCommit();
       }
 
+      String location = params.get(ZkStateReader.BACKUP_LOCATION);
+      String repoName = params.get(BackupRepository.REPOSITORY_PROPERTY_NAME);
+      CoreContainer cc = core.getCoreDescriptor().getCoreContainer();
+      SolrResourceLoader rl = cc.getResourceLoader();
+      BackupRepository repo = null;
+      if(repoName != null) {
+        repo = cc.getBackupRepoFactory().newInstance(rl, repoName);
+        if (location == null) {
+          location = repo.getConfigProperty(ZkStateReader.BACKUP_LOCATION);
+          if(location == null) {
+            throw new IllegalArgumentException("location is required");
+          }
+        }
+      } else {
+        repo = new LocalFileSystemRepository();
+        if (location == null) {
+          location = core.getDataDir();
+        } else {
+          location = core.getCoreDescriptor().getInstanceDir().resolve(location).normalize().toString();
+        }
+      }
+
       // small race here before the commit point is saved
-      SnapShooter snapShooter = new SnapShooter(core, params.get("location"), params.get(NAME));
+      SnapShooter snapShooter = new SnapShooter(repo, core, location, params.get(NAME));
       snapShooter.validateCreateSnapshot();
       snapShooter.createSnapAsync(indexCommit, numberToKeep, (nl) -> snapShootDetails = nl);
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/07be2c42/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 34109c6..d3c98fa 100644
--- a/solr/core/src/java/org/apache/solr/handler/RestoreCore.java
+++ b/solr/core/src/java/org/apache/solr/handler/RestoreCore.java
@@ -17,8 +17,7 @@
 package org.apache.solr.handler;
 
 import java.lang.invoke.MethodHandles;
-import java.nio.file.Path;
-import java.nio.file.Paths;
+import java.net.URI;
 import java.text.SimpleDateFormat;
 import java.util.Date;
 import java.util.Locale;
@@ -27,12 +26,12 @@ 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.apache.solr.core.backup.repository.BackupRepository;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -43,8 +42,10 @@ public class RestoreCore implements Callable<Boolean> {
   private final String backupName;
   private final String backupLocation;
   private final SolrCore core;
+  private final BackupRepository backupRepo;
 
-  public RestoreCore(SolrCore core, String location, String name) {
+  public RestoreCore(BackupRepository backupRepo, SolrCore core, String location, String name) {
+    this.backupRepo = backupRepo;
     this.core = core;
     this.backupLocation = location;
     this.backupName = name;
@@ -57,14 +58,14 @@ public class RestoreCore implements Callable<Boolean> {
 
   public boolean doRestore() throws Exception {
 
-    Path backupPath = Paths.get(backupLocation).resolve(backupName);
+    URI backupPath = backupRepo.createURI(backupLocation, backupName);
     SimpleDateFormat dateFormat = new SimpleDateFormat(SnapShooter.DATE_FMT, Locale.ROOT);
     String restoreIndexName = "restore." + dateFormat.format(new Date());
     String restoreIndexPath = core.getDataDir() + restoreIndexName;
 
     Directory restoreIndexDir = null;
     Directory indexDir = null;
-    try (Directory backupDir = FSDirectory.open(backupPath)) {
+    try {
 
       restoreIndexDir = core.getDirectoryFactory().get(restoreIndexPath,
           DirectoryFactory.DirContext.DEFAULT, core.getSolrConfig().indexConfig.lockType);
@@ -74,10 +75,10 @@ public class RestoreCore implements Callable<Boolean> {
           DirectoryFactory.DirContext.DEFAULT, core.getSolrConfig().indexConfig.lockType);
 
       //Move all files from backupDir to restoreIndexDir
-      for (String filename : backupDir.listAll()) {
+      for (String filename : backupRepo.listAll(backupPath)) {
         checkInterrupted();
         log.info("Copying file {} to restore directory ", filename);
-        try (IndexInput indexInput = backupDir.openInput(filename, IOContext.READONCE)) {
+        try (IndexInput indexInput = backupRepo.openInput(backupPath, filename, IOContext.READONCE)) {
           Long checksum = null;
           try {
             checksum = CodecUtil.retrieveChecksum(indexInput);
@@ -88,12 +89,13 @@ public class RestoreCore implements Callable<Boolean> {
           IndexFetcher.CompareResult compareResult = IndexFetcher.compareFile(indexDir, filename, length, checksum);
           if (!compareResult.equal ||
               (IndexFetcher.filesToAlwaysDownloadIfNoChecksums(filename, length, compareResult))) {
-            restoreIndexDir.copyFrom(backupDir, filename, filename, IOContext.READONCE);
+            backupRepo.copyFileTo(backupPath, filename, restoreIndexDir);
           } else {
             //prefer local copy
             restoreIndexDir.copyFrom(indexDir, filename, filename, IOContext.READONCE);
           }
         } catch (Exception e) {
+          log.warn("Exception while restoring the backup index ", e);
           throw new SolrException(SolrException.ErrorCode.UNKNOWN, "Exception while restoring the backup index", e);
         }
       }
@@ -108,7 +110,7 @@ public class RestoreCore implements Callable<Boolean> {
         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.warn("Could not switch to restored index. Rolling back to the current index");
+        log.warn("Could not switch to restored index. Rolling back to the current index", e);
         Directory dir = null;
         try {
           dir = core.getDirectoryFactory().get(core.getDataDir(), DirectoryFactory.DirContext.META_DATA,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/07be2c42/solr/core/src/java/org/apache/solr/handler/SnapShooter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/SnapShooter.java b/solr/core/src/java/org/apache/solr/handler/SnapShooter.java
index 2365fca..cc3f69e 100644
--- a/solr/core/src/java/org/apache/solr/handler/SnapShooter.java
+++ b/solr/core/src/java/org/apache/solr/handler/SnapShooter.java
@@ -16,11 +16,9 @@
  */
 package org.apache.solr.handler;
 
-import java.io.File;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
-import java.nio.file.Path;
-import java.nio.file.Paths;
+import java.net.URI;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -32,20 +30,21 @@ import java.util.function.Consumer;
 
 import org.apache.lucene.index.IndexCommit;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.FSDirectory;
-import org.apache.lucene.store.NoLockFactory;
-import org.apache.lucene.store.SimpleFSDirectory;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.NamedList;
-import org.apache.solr.core.DirectoryFactory;
 import org.apache.solr.core.DirectoryFactory.DirContext;
 import org.apache.solr.core.IndexDeletionPolicyWrapper;
 import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.backup.repository.BackupRepository;
+import org.apache.solr.core.backup.repository.BackupRepository.PathType;
+import org.apache.solr.core.backup.repository.LocalFileSystemRepository;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.util.RefCounted;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Preconditions;
+
 /**
  * <p> Provides functionality equivalent to the snapshooter script </p>
  * This is no longer used in standard replication.
@@ -55,48 +54,76 @@ import org.slf4j.LoggerFactory;
  */
 public class SnapShooter {
   private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  private String snapDir = null;
   private SolrCore solrCore;
   private String snapshotName = null;
   private String directoryName = null;
-  private File snapShotDir = null;
-  //TODO update to NIO Path API
+  private URI baseSnapDirPath = null;
+  private URI snapshotDirPath = null;
+  private BackupRepository backupRepo = null;
 
+  @Deprecated
   public SnapShooter(SolrCore core, String location, String snapshotName) {
-    solrCore = core;
+    String snapDirStr = null;
+    // Note - This logic is only applicable to the usecase where a shared file-system is exposed via
+    // local file-system interface (primarily for backwards compatibility). For other use-cases, users
+    // will be required to specify "location" where the backup should be stored.
     if (location == null) {
-      snapDir = core.getDataDir();
-    }
-    else  {
-      snapDir = core.getCoreDescriptor().getInstanceDir().resolve(location).normalize().toString();
+      snapDirStr = core.getDataDir();
+    } else {
+      snapDirStr = core.getCoreDescriptor().getInstanceDir().resolve(location).normalize().toString();
     }
-    this.snapshotName = snapshotName;
+    initialize(new LocalFileSystemRepository(), core, snapDirStr, snapshotName);
+  }
+
+  public SnapShooter(BackupRepository backupRepo, SolrCore core, String location, String snapshotName) {
+    initialize(backupRepo, core, location, snapshotName);
+  }
 
-    if(snapshotName != null) {
+  private void initialize(BackupRepository backupRepo, SolrCore core, String location, String snapshotName) {
+    this.solrCore = Preconditions.checkNotNull(core);
+    this.backupRepo = Preconditions.checkNotNull(backupRepo);
+    this.baseSnapDirPath = backupRepo.createURI(Preconditions.checkNotNull(location)).normalize();
+    this.snapshotName = snapshotName;
+    if (snapshotName != null) {
       directoryName = "snapshot." + snapshotName;
     } else {
       SimpleDateFormat fmt = new SimpleDateFormat(DATE_FMT, Locale.ROOT);
       directoryName = "snapshot." + fmt.format(new Date());
     }
+    this.snapshotDirPath = backupRepo.createURI(location, directoryName);
   }
 
-  /** Gets the parent directory of the snapshots.  This is the {@code location} given in the constructor after
-   * being resolved against the core instance dir. */
-  public Path getLocation() {
-    return Paths.get(snapDir);
+  public BackupRepository getBackupRepository() {
+    return backupRepo;
+  }
+
+  /**
+   * Gets the parent directory of the snapshots. This is the {@code location}
+   * given in the constructor.
+   */
+  public URI getLocation() {
+    return this.baseSnapDirPath;
   }
 
   public void validateDeleteSnapshot() {
+    Preconditions.checkNotNull(this.snapshotName);
+
     boolean dirFound = false;
-    File[] files = new File(snapDir).listFiles();
-    for(File f : files) {
-      if (f.getName().equals("snapshot." + snapshotName)) {
-        dirFound = true;
-        break;
+    String[] paths;
+    try {
+      paths = backupRepo.listAll(baseSnapDirPath);
+      for (String path : paths) {
+        if (path.equals(this.directoryName)
+            && backupRepo.getPathType(baseSnapDirPath.resolve(path)) == PathType.DIRECTORY) {
+          dirFound = true;
+          break;
+        }
       }
-    }
-    if(dirFound == false) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Snapshot cannot be found in directory: " + snapDir);
+      if(dirFound == false) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Snapshot " + snapshotName + " cannot be found in directory: " + baseSnapDirPath);
+      }
+    } catch (IOException e) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unable to find snapshot " + snapshotName + " in directory: " + baseSnapDirPath, e);
     }
   }
 
@@ -110,14 +137,16 @@ public class SnapShooter {
   }
 
   public void validateCreateSnapshot() throws IOException {
-    snapShotDir = new File(snapDir, directoryName);
-    if (snapShotDir.exists()) {
+    // Note - Removed the current behavior of creating the directory hierarchy.
+    // Do we really need to provide this support?
+    if (!backupRepo.exists(baseSnapDirPath)) {
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-          "Snapshot directory already exists: " + snapShotDir.getAbsolutePath());
+          " Directory does not exist: " + snapshotDirPath);
     }
-    if (!snapShotDir.mkdirs()) { // note: TODO reconsider mkdirs vs mkdir
+
+    if (backupRepo.exists(snapshotDirPath)) {
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-          "Unable to create snapshot directory: " + snapShotDir.getAbsolutePath());
+          "Snapshot directory already exists: " + snapshotDirPath);
     }
   }
 
@@ -155,7 +184,11 @@ public class SnapShooter {
           solrCore.getDeletionPolicy().releaseCommitPoint(indexCommit.getGeneration());
         }
         if (snapshotName == null) {
-          deleteOldBackups(numberToKeep);
+          try {
+            deleteOldBackups(numberToKeep);
+          } catch (IOException e) {
+            LOG.warn("Unable to delete old snapshots ", e);
+          }
         }
       }
     }.start();
@@ -163,7 +196,7 @@ public class SnapShooter {
 
   // note: remember to reserve the indexCommit first so it won't get deleted concurrently
   protected NamedList createSnapshot(final IndexCommit indexCommit) throws Exception {
-    LOG.info("Creating backup snapshot " + (snapshotName == null ? "<not named>" : snapshotName) + " at " + snapDir);
+    LOG.info("Creating backup snapshot " + (snapshotName == null ? "<not named>" : snapshotName) + " at " + baseSnapDirPath);
     boolean success = false;
     try {
       NamedList<Object> details = new NamedList<>();
@@ -172,7 +205,9 @@ public class SnapShooter {
       Collection<String> files = indexCommit.getFileNames();
       Directory dir = solrCore.getDirectoryFactory().get(solrCore.getIndexDir(), DirContext.DEFAULT, solrCore.getSolrConfig().indexConfig.lockType);
       try {
-        copyFiles(dir, files, snapShotDir);
+        for(String fileName : files) {
+          backupRepo.copyFileFrom(dir, fileName, snapshotDirPath);
+        }
       } finally {
         solrCore.getDirectoryFactory().release(dir);
       }
@@ -182,34 +217,35 @@ public class SnapShooter {
       details.add("snapshotCompletedAt", new Date().toString());//bad; should be Instant.now().toString()
       details.add("snapshotName", snapshotName);
       LOG.info("Done creating backup snapshot: " + (snapshotName == null ? "<not named>" : snapshotName) +
-          " at " + snapDir);
+          " at " + baseSnapDirPath);
       success = true;
       return details;
     } finally {
       if (!success) {
-        IndexFetcher.delTree(snapShotDir);
+        backupRepo.deleteDirectory(snapshotDirPath);
       }
     }
   }
 
-  private void deleteOldBackups(int numberToKeep) {
-    File[] files = new File(snapDir).listFiles();
+  private void deleteOldBackups(int numberToKeep) throws IOException {
+    String[] paths = backupRepo.listAll(baseSnapDirPath);
     List<OldBackupDirectory> dirs = new ArrayList<>();
-    for (File f : files) {
-      OldBackupDirectory obd = new OldBackupDirectory(f);
-      if(obd.dir != null) {
-        dirs.add(obd);
+    for (String f : paths) {
+      if (backupRepo.getPathType(baseSnapDirPath.resolve(f)) == PathType.DIRECTORY) {
+        OldBackupDirectory obd = new OldBackupDirectory(baseSnapDirPath, f);
+        if (obd.getTimestamp().isPresent()) {
+          dirs.add(obd);
+        }
       }
     }
     if (numberToKeep > dirs.size() -1) {
       return;
     }
-
     Collections.sort(dirs);
     int i=1;
     for (OldBackupDirectory dir : dirs) {
       if (i++ > numberToKeep) {
-        IndexFetcher.delTree(dir.dir);
+        backupRepo.deleteDirectory(dir.getPath());
       }
     }
   }
@@ -218,29 +254,22 @@ public class SnapShooter {
     LOG.info("Deleting snapshot: " + snapshotName);
 
     NamedList<Object> details = new NamedList<>();
-    boolean isSuccess;
-    File f = new File(snapDir, "snapshot." + snapshotName);
-    isSuccess = IndexFetcher.delTree(f);
 
-    if(isSuccess) {
+    try {
+      URI path = baseSnapDirPath.resolve("snapshot." + snapshotName);
+      backupRepo.deleteDirectory(path);
+
       details.add("status", "success");
       details.add("snapshotDeletedAt", new Date().toString());
-    } else {
+
+    } catch (IOException e) {
       details.add("status", "Unable to delete snapshot: " + snapshotName);
-      LOG.warn("Unable to delete snapshot: " + snapshotName);
+      LOG.warn("Unable to delete snapshot: " + snapshotName, e);
     }
+
     replicationHandler.snapShootDetails = details;
   }
 
   public static final String DATE_FMT = "yyyyMMddHHmmssSSS";
 
-
-  private static void copyFiles(Directory sourceDir, Collection<String> files, File destDir) throws IOException {
-    try (FSDirectory dir = new SimpleFSDirectory(destDir.toPath(), NoLockFactory.INSTANCE)) {
-      for (String indexFile : files) {
-        dir.copyFrom(sourceDir, indexFile, indexFile, DirectoryFactory.IOCONTEXT_NO_CACHE);
-      }
-    }
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/07be2c42/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
index 6d501a1..6acd86a 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
@@ -798,7 +798,7 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
           throw new SolrException(ErrorCode.BAD_REQUEST, "Collection '" + collectionName + "' does not exist, no action taken.");
         }
 
-        String location = req.getParams().get("location");
+        String location = req.getParams().get(ZkStateReader.BACKUP_LOCATION);
         if (location == null) {
           location = h.coreContainer.getZkController().getZkStateReader().getClusterProperty("location", (String) null);
         }
@@ -822,7 +822,7 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
           throw new SolrException(ErrorCode.BAD_REQUEST, "Collection '" + collectionName + "' exists, no action taken.");
         }
 
-        String location = req.getParams().get("location");
+        String location = req.getParams().get(ZkStateReader.BACKUP_LOCATION);
         if (location == null) {
           location = h.coreContainer.getZkController().getZkStateReader().getClusterProperty("location", (String) null);
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/07be2c42/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java
index 3fdf3ef..3c52bea 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java
@@ -18,7 +18,6 @@ package org.apache.solr.handler.admin;
 
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
-import java.nio.file.Files;
 import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -60,6 +59,7 @@ import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.core.DirectoryFactory;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.core.backup.repository.BackupRepository;
 import org.apache.solr.handler.RestoreCore;
 import org.apache.solr.handler.SnapShooter;
 import org.apache.solr.request.LocalSolrQueryRequest;
@@ -858,20 +858,32 @@ enum CoreAdminOperation {
         throw new IllegalArgumentException(CoreAdminParams.NAME + " is required");
       }
 
-      String location = params.get("location");
+      SolrResourceLoader loader = callInfo.handler.coreContainer.getResourceLoader();
+      BackupRepository repository;
+      String repoName = params.get(BackupRepository.REPOSITORY_PROPERTY_NAME);
+      if(repoName != null) {
+        repository = callInfo.handler.coreContainer.getBackupRepoFactory().newInstance(loader, repoName);
+      } else { // Fetch the default.
+        repository = callInfo.handler.coreContainer.getBackupRepoFactory().newInstance(loader);
+      }
+
+      String location = params.get(ZkStateReader.BACKUP_LOCATION);
       if (location == null) {
-        throw new IllegalArgumentException("location is required");
+        location = repository.getConfigProperty(ZkStateReader.BACKUP_LOCATION);
+        if (location == null) {
+          throw new IllegalArgumentException("location is required");
+        }
       }
 
       try (SolrCore core = callInfo.handler.coreContainer.getCore(cname)) {
-        SnapShooter snapShooter = new SnapShooter(core, location, name);
+        SnapShooter snapShooter = new SnapShooter(repository, core, location, name);
         // validateCreateSnapshot will create parent dirs instead of throw; that choice is dubious.
         //  But we want to throw. One reason is that
         //  this dir really should, in fact must, already exist here if triggered via a collection backup on a shared
         //  file system. Otherwise, perhaps the FS location isn't shared -- we want an error.
-        if (!Files.exists(snapShooter.getLocation())) {
+        if (!snapShooter.getBackupRepository().exists(snapShooter.getLocation())) {
           throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-              "Directory to contain snapshots doesn't exist: " + snapShooter.getLocation().toAbsolutePath());
+              "Directory to contain snapshots doesn't exist: " + snapShooter.getLocation());
         }
         snapShooter.validateCreateSnapshot();
         snapShooter.createSnapshot();
@@ -900,13 +912,25 @@ enum CoreAdminOperation {
         throw new IllegalArgumentException(CoreAdminParams.NAME + " is required");
       }
 
-      String location = params.get("location");
+      SolrResourceLoader loader = callInfo.handler.coreContainer.getResourceLoader();
+      BackupRepository repository;
+      String repoName = params.get(BackupRepository.REPOSITORY_PROPERTY_NAME);
+      if(repoName != null) {
+        repository = callInfo.handler.coreContainer.getBackupRepoFactory().newInstance(loader, repoName);
+      } else { // Fetch the default.
+        repository = callInfo.handler.coreContainer.getBackupRepoFactory().newInstance(loader);
+      }
+
+      String location = params.get(ZkStateReader.BACKUP_LOCATION);
       if (location == null) {
-        throw new IllegalArgumentException("location is required");
+        location = repository.getConfigProperty(ZkStateReader.BACKUP_LOCATION);
+        if (location == null) {
+          throw new IllegalArgumentException("location is required");
+        }
       }
 
       try (SolrCore core = callInfo.handler.coreContainer.getCore(cname)) {
-        RestoreCore restoreCore = new RestoreCore(core, location, name);
+        RestoreCore restoreCore = new RestoreCore(repository, core, location, name);
         boolean success = restoreCore.doRestore();
         if (!success) {
           throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Failed to restore core=" + core.getName());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/07be2c42/solr/core/src/java/org/apache/solr/store/hdfs/HdfsDirectory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/store/hdfs/HdfsDirectory.java b/solr/core/src/java/org/apache/solr/store/hdfs/HdfsDirectory.java
index 9a8b36c..0a25692 100644
--- a/solr/core/src/java/org/apache/solr/store/hdfs/HdfsDirectory.java
+++ b/solr/core/src/java/org/apache/solr/store/hdfs/HdfsDirectory.java
@@ -42,6 +42,7 @@ import org.slf4j.LoggerFactory;
 
 public class HdfsDirectory extends BaseDirectory {
   private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  public static final int DEFAULT_BUFFER_SIZE = 4096;
   
   private static final String LF_EXT = ".lf";
   protected final Path hdfsDirPath;
@@ -53,7 +54,7 @@ public class HdfsDirectory extends BaseDirectory {
   private final int bufferSize;
   
   public HdfsDirectory(Path hdfsDirPath, Configuration configuration) throws IOException {
-    this(hdfsDirPath, HdfsLockFactory.INSTANCE, configuration, 4096);
+    this(hdfsDirPath, HdfsLockFactory.INSTANCE, configuration, DEFAULT_BUFFER_SIZE);
   }
   
   public HdfsDirectory(Path hdfsDirPath, LockFactory lockFactory, Configuration configuration, int bufferSize)
@@ -190,7 +191,7 @@ public class HdfsDirectory extends BaseDirectory {
     return configuration;
   }
   
-  static class HdfsIndexInput extends CustomBufferedIndexInput {
+  public static class HdfsIndexInput extends CustomBufferedIndexInput {
     private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
     
     private final Path path;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/07be2c42/solr/core/src/test-files/solr/solr-50-all.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/solr-50-all.xml b/solr/core/src/test-files/solr/solr-50-all.xml
index a0f3165..e2ce924 100644
--- a/solr/core/src/test-files/solr/solr-50-all.xml
+++ b/solr/core/src/test-files/solr/solr-50-all.xml
@@ -56,4 +56,8 @@
     <int name="connTimeout">${connTimeout:110}</int>
   </shardHandlerFactory>
 
+  <backup>
+    <repository name="local" class="a.b.C" default="true"/>
+  </backup>
+
 </solr>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/07be2c42/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZk2Test.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZk2Test.java b/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZk2Test.java
index 47d8212..582c8b4 100644
--- a/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZk2Test.java
+++ b/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZk2Test.java
@@ -399,22 +399,25 @@ public class BasicDistributedZk2Test extends AbstractFullDistribZkTestBase {
     checkShardConsistency(true, false);
     
     // try a backup command
-    final HttpSolrClient client = (HttpSolrClient) shardToJetty.get(SHARD2).get(0).client.solrClient;
-    ModifiableSolrParams params = new ModifiableSolrParams();
-    params.set("qt", ReplicationHandler.PATH);
-    params.set("command", "backup");
-    Path location = createTempDir();
-    location = FilterPath.unwrap(location).toRealPath();
-    params.set("location", location.toString());
+    try(final HttpSolrClient client = getHttpSolrClient((String) shardToJetty.get(SHARD2).get(0).info.get("base_url"))) {
+      ModifiableSolrParams params = new ModifiableSolrParams();
+      params.set("qt", ReplicationHandler.PATH);
+      params.set("command", "backup");
+      Path location = createTempDir();
+      location = FilterPath.unwrap(location).toRealPath();
+      params.set("location", location.toString());
+
+      QueryRequest request = new QueryRequest(params);
+      client.request(request, DEFAULT_TEST_COLLECTION_NAME);
+
+      checkForBackupSuccess(client, location);
+      client.close();
+    }
 
-    QueryRequest request = new QueryRequest(params);
-    client.request(request);
-    
-    checkForBackupSuccess(client, location);
   }
 
   private void checkForBackupSuccess(HttpSolrClient client, Path location) throws InterruptedException, IOException {
-    CheckBackupStatus checkBackupStatus = new CheckBackupStatus(client);
+    CheckBackupStatus checkBackupStatus = new CheckBackupStatus(client, DEFAULT_TEST_COLLECTION_NAME);
     while (!checkBackupStatus.success) {
       checkBackupStatus.fetchStatus();
       Thread.sleep(1000);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/07be2c42/solr/core/src/test/org/apache/solr/core/TestBackupRepositoryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/TestBackupRepositoryFactory.java b/solr/core/src/test/org/apache/solr/core/TestBackupRepositoryFactory.java
new file mode 100644
index 0000000..81d3c40
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/core/TestBackupRepositoryFactory.java
@@ -0,0 +1,152 @@
+package org.apache.solr.core;
+
+/*
+ * 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.util.HashMap;
+import java.util.Map;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CoreAdminParams;
+import org.apache.solr.core.backup.repository.BackupRepository;
+import org.apache.solr.core.backup.repository.BackupRepositoryFactory;
+import org.apache.solr.core.backup.repository.LocalFileSystemRepository;
+import org.apache.solr.schema.FieldType;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.rules.RuleChain;
+import org.junit.rules.TestRule;
+
+import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
+
+public class TestBackupRepositoryFactory extends SolrTestCaseJ4 {
+  @Rule
+  public TestRule solrTestRules = RuleChain.outerRule(new SystemPropertiesRestoreRule());
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
+
+  // tmp dir, cleaned up automatically.
+  private static File solrHome = null;
+  private static SolrResourceLoader loader = null;
+
+  @BeforeClass
+  public static void setupLoader() throws Exception {
+    solrHome = createTempDir().toFile();
+    loader = new SolrResourceLoader(solrHome.toPath());
+  }
+
+  @AfterClass
+  public static void cleanupLoader() throws Exception {
+    solrHome = null;
+    loader = null;
+  }
+
+  @Test
+  public void testMultipleDefaultRepositories() {
+    PluginInfo[] plugins = new PluginInfo[2];
+
+    {
+      Map<String, Object> attrs = new HashMap<>();
+      attrs.put(CoreAdminParams.NAME, "repo1");
+      attrs.put(FieldType.CLASS_NAME, "a.b.C");
+      attrs.put("default" , "true");
+      plugins[0] = new PluginInfo("repository", attrs);
+    }
+
+    {
+      Map<String, Object> attrs = new HashMap<>();
+      attrs.put(CoreAdminParams.NAME, "repo2");
+      attrs.put(FieldType.CLASS_NAME, "p.q.R");
+      attrs.put("default" , "true");
+      plugins[1] = new PluginInfo("repository", attrs);
+    }
+
+    expectedException.expect(SolrException.class);
+    expectedException.expectMessage("More than one backup repository is configured as default");
+    new BackupRepositoryFactory(plugins);
+  }
+
+  @Test
+  public void testMultipleRepositoriesWithSameName() {
+    PluginInfo[] plugins = new PluginInfo[2];
+
+    {
+      Map<String, Object> attrs = new HashMap<>();
+      attrs.put(CoreAdminParams.NAME, "repo1");
+      attrs.put(FieldType.CLASS_NAME, "a.b.C");
+      attrs.put("default" , "true");
+      plugins[0] = new PluginInfo("repository", attrs);
+    }
+
+    {
+      Map<String, Object> attrs = new HashMap<>();
+      attrs.put(CoreAdminParams.NAME, "repo1");
+      attrs.put(FieldType.CLASS_NAME, "p.q.R");
+      plugins[1] = new PluginInfo("repository", attrs);
+    }
+
+    expectedException.expect(SolrException.class);
+    expectedException.expectMessage("Duplicate backup repository with name repo1");
+    new BackupRepositoryFactory(plugins);
+  }
+
+  @Test
+  public void testNonExistantBackupRepository() {
+    PluginInfo[] plugins = new PluginInfo[0];
+    BackupRepositoryFactory f = new BackupRepositoryFactory(plugins);
+
+    expectedException.expect(NullPointerException.class);
+    expectedException.expectMessage("Could not find a backup repository with name repo1");
+    f.newInstance(loader, "repo1");
+  }
+
+  @Test
+  public void testRepositoryConfig() {
+    PluginInfo[] plugins = new PluginInfo[1];
+
+    {
+      Map<String, Object> attrs = new HashMap<>();
+      attrs.put(CoreAdminParams.NAME, "repo1");
+      attrs.put(FieldType.CLASS_NAME, LocalFileSystemRepository.class.getName());
+      attrs.put("default" , "true");
+      attrs.put(ZkStateReader.BACKUP_LOCATION, "/tmp");
+      plugins[0] = new PluginInfo("repository", attrs);
+    }
+
+    BackupRepositoryFactory f = new BackupRepositoryFactory(plugins);
+
+    {
+      BackupRepository repo = f.newInstance(loader);
+
+      assertTrue(repo instanceof LocalFileSystemRepository);
+      assertEquals("/tmp", repo.getConfigProperty(ZkStateReader.BACKUP_LOCATION));
+    }
+
+    {
+      BackupRepository repo = f.newInstance(loader, "repo1");
+
+      assertTrue(repo instanceof LocalFileSystemRepository);
+      assertEquals("/tmp", repo.getConfigProperty(ZkStateReader.BACKUP_LOCATION));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/07be2c42/solr/core/src/test/org/apache/solr/core/TestSolrXml.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/TestSolrXml.java b/solr/core/src/test/org/apache/solr/core/TestSolrXml.java
index 1cfeb3f..4343efe 100644
--- a/solr/core/src/test/org/apache/solr/core/TestSolrXml.java
+++ b/solr/core/src/test/org/apache/solr/core/TestSolrXml.java
@@ -67,6 +67,7 @@ public class TestSolrXml extends SolrTestCaseJ4 {
     NodeConfig cfg = SolrXmlConfig.fromSolrHome(loader, solrHome.toPath());
     CloudConfig ccfg = cfg.getCloudConfig();
     UpdateShardHandlerConfig ucfg = cfg.getUpdateShardHandlerConfig();
+    PluginInfo[] backupRepoConfigs = cfg.getBackupRepositoryPlugins();
     
     assertEquals("core admin handler class", "testAdminHandler", cfg.getCoreAdminHandlerClass());
     assertEquals("collection handler class", "testCollectionsHandler", cfg.getCollectionsHandlerClass());
@@ -98,6 +99,11 @@ public class TestSolrXml extends SolrTestCaseJ4 {
     assertEquals("zk host", "testZkHost", ccfg.getZkHost());
     assertEquals("zk ACL provider", "DefaultZkACLProvider", ccfg.getZkACLProviderClass());
     assertEquals("zk credentials provider", "DefaultZkCredentialsProvider", ccfg.getZkCredentialsProviderClass());
+    assertEquals(1, backupRepoConfigs.length);
+    assertEquals("local", backupRepoConfigs[0].name);
+    assertEquals("a.b.C", backupRepoConfigs[0].className);
+    assertEquals("true", backupRepoConfigs[0].attributes.get("default"));
+    assertEquals(0, backupRepoConfigs[0].initArgs.size());
   }
 
   // Test  a few property substitutions that happen to be in solr-50-all.xml.
@@ -321,4 +327,11 @@ public class TestSolrXml extends SolrTestCaseJ4 {
 
     SolrXmlConfig.fromString(loader, "<solr><solrcloud><str name=\"host\">host</str><int name=\"hostPort\">8983</int></solrcloud></solr>");
   }
+
+  public void testMultiBackupSectionError() throws IOException {
+    String solrXml = "<solr><backup></backup><backup></backup></solr>";
+    expectedException.expect(SolrException.class);
+    expectedException.expectMessage("Multiple instances of backup section found in solr.xml");
+    SolrXmlConfig.fromString(loader, solrXml); // return not used, only for validation
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/07be2c42/solr/core/src/test/org/apache/solr/handler/BackupRestoreUtils.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/BackupRestoreUtils.java b/solr/core/src/test/org/apache/solr/handler/BackupRestoreUtils.java
new file mode 100644
index 0000000..bbc80be
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/BackupRestoreUtils.java
@@ -0,0 +1,69 @@
+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.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class BackupRestoreUtils extends LuceneTestCase {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public static int indexDocs(SolrClient masterClient, String collectionName, long docsSeed) throws IOException, SolrServerException {
+    masterClient.deleteByQuery(collectionName, "*:*");
+
+    Random random = new Random(docsSeed);// use a constant seed for the whole test run so that we can easily re-index.
+    int nDocs = random.nextInt(100);
+    log.info("Indexing {} test docs", nDocs);
+    if (nDocs == 0) {
+      return 0;
+    }
+
+    List<SolrInputDocument> docs = new ArrayList<>(nDocs);
+    for (int i = 0; i < nDocs; i++) {
+      SolrInputDocument doc = new SolrInputDocument();
+      doc.addField("id", i);
+      doc.addField("name", "name = " + i);
+      docs.add(doc);
+    }
+    masterClient.add(collectionName, docs);
+    masterClient.commit(collectionName);
+    return nDocs;
+  }
+
+  public static void verifyDocs(int nDocs, SolrClient masterClient, String collectionName) throws SolrServerException, IOException {
+    ModifiableSolrParams queryParams = new ModifiableSolrParams();
+    queryParams.set("q", "*:*");
+    QueryResponse response = masterClient.query(collectionName, queryParams);
+
+    assertEquals(0, response.getStatus());
+    assertEquals(nDocs, response.getResults().getNumFound());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/07be2c42/solr/core/src/test/org/apache/solr/handler/CheckBackupStatus.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/CheckBackupStatus.java b/solr/core/src/test/org/apache/solr/handler/CheckBackupStatus.java
index 706a2fe..f84d89f 100644
--- a/solr/core/src/test/org/apache/solr/handler/CheckBackupStatus.java
+++ b/solr/core/src/test/org/apache/solr/handler/CheckBackupStatus.java
@@ -33,18 +33,20 @@ public class CheckBackupStatus extends SolrTestCaseJ4 {
   final Pattern p = Pattern.compile("<str name=\"snapshotCompletedAt\">(.*?)</str>");
   final Pattern pException = Pattern.compile("<str name=\"snapShootException\">(.*?)</str>");
   final HttpSolrClient client;
+  final String coreName;
 
-  public CheckBackupStatus(final HttpSolrClient client, String lastBackupTimestamp) {
+  public CheckBackupStatus(final HttpSolrClient client, String coreName, String lastBackupTimestamp) {
     this.client = client;
     this.lastBackupTimestamp = lastBackupTimestamp;
+    this.coreName = coreName;
   }
 
-  public CheckBackupStatus(final HttpSolrClient client) {
-    this(client, null);
+  public CheckBackupStatus(final HttpSolrClient client, String coreName) {
+    this(client, coreName, null);
   }
 
   public void fetchStatus() throws IOException {
-    String masterUrl = client.getBaseURL() + ReplicationHandler.PATH + "?command=" + ReplicationHandler.CMD_DETAILS;
+    String masterUrl = client.getBaseURL() + "/"  + coreName + ReplicationHandler.PATH + "?command=" + ReplicationHandler.CMD_DETAILS;
     response = client.getHttpClient().execute(new HttpGet(masterUrl), new BasicResponseHandler());
     if(pException.matcher(response).find()) {
       fail("Failed to create backup");