You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by no...@apache.org on 2016/07/21 14:57:27 UTC

lucene-solr:branch_6x: SOLR-8995: Use lambdas in CoreAdminHandler

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_6x 0ec1c2509 -> c627ddc16


SOLR-8995: Use lambdas in CoreAdminHandler


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

Branch: refs/heads/branch_6x
Commit: c627ddc16c4bd1fedf767169df317623304b33be
Parents: 0ec1c25
Author: Noble Paul <no...@apache.org>
Authored: Thu Jul 21 20:26:14 2016 +0530
Committer: Noble Paul <no...@apache.org>
Committed: Thu Jul 21 20:27:11 2016 +0530

----------------------------------------------------------------------
 .../solr/handler/admin/CoreAdminHandler.java    |    9 +-
 .../solr/handler/admin/CoreAdminOperation.java  | 1406 +++++++++---------
 2 files changed, 686 insertions(+), 729 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c627ddc1/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
index 49da174..7b0ecfb 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
@@ -362,7 +362,7 @@ public class CoreAdminHandler extends RequestHandlerBase implements PermissionNa
     }
 
     void call() throws Exception {
-      op.call(this);
+      op.execute(this);
     }
 
   }
@@ -374,7 +374,10 @@ public class CoreAdminHandler extends RequestHandlerBase implements PermissionNa
   /**
    * used by the INVOKE action of core admin handler
    */
-  public static interface Invocable {
-    public Map<String, Object> invoke(SolrQueryRequest req);
+  public interface Invocable {
+    Map<String, Object> invoke(SolrQueryRequest req);
+  }
+  interface CoreAdminOp {
+    void execute(CallInfo it) throws Exception;
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c627ddc1/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 33fe19a..9b9aafa 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
@@ -64,6 +64,7 @@ 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.handler.admin.CoreAdminHandler.CoreAdminOp;
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.search.SolrIndexSearcher;
@@ -77,7 +78,6 @@ import org.apache.solr.util.NumberUtils;
 import org.apache.solr.util.PropertiesUtil;
 import org.apache.solr.util.RefCounted;
 import org.apache.solr.util.TestInjection;
-import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -96,839 +96,774 @@ import static org.apache.solr.handler.admin.CoreAdminHandler.RUNNING;
 import static org.apache.solr.handler.admin.CoreAdminHandler.buildCoreParams;
 import static org.apache.solr.handler.admin.CoreAdminHandler.normalizePath;
 
-enum CoreAdminOperation {
-  CREATE_OP(CREATE) {
-    @Override
-    public void call(CallInfo callInfo) {
-      assert TestInjection.injectRandomDelayInCoreCreation();
-      
-      SolrParams params = callInfo.req.getParams();
-      log.info("core create command {}", params);
-      String coreName = params.required().get(CoreAdminParams.NAME);
-      Map<String, String> coreParams = buildCoreParams(params);
-      CoreContainer coreContainer = callInfo.handler.coreContainer;
-      Path instancePath = coreContainer.getCoreRootDirectory().resolve(coreName);
-
-      // TODO: Should we nuke setting odd instance paths?  They break core discovery, generally
-      String instanceDir = callInfo.req.getParams().get(CoreAdminParams.INSTANCE_DIR);
-      if (instanceDir == null)
-        instanceDir = callInfo.req.getParams().get("property.instanceDir");
-      if (instanceDir != null) {
-        instanceDir = PropertiesUtil.substituteProperty(instanceDir, coreContainer.getContainerProperties());
-        instancePath = coreContainer.getCoreRootDirectory().resolve(instanceDir).normalize();
-      }
-
-      coreContainer.create(coreName, instancePath, coreParams);
-
-      callInfo.rsp.add("core", coreName);
+enum CoreAdminOperation implements CoreAdminOp {
+
+  CREATE_OP(CREATE, it -> {
+    assert TestInjection.injectRandomDelayInCoreCreation();
+
+    SolrParams params = it.req.getParams();
+    log().info("core create command {}", params);
+    String coreName = params.required().get(CoreAdminParams.NAME);
+    Map<String, String> coreParams = buildCoreParams(params);
+    CoreContainer coreContainer = it.handler.coreContainer;
+    Path instancePath = coreContainer.getCoreRootDirectory().resolve(coreName);
+
+    // TODO: Should we nuke setting odd instance paths?  They break core discovery, generally
+    String instanceDir = it.req.getParams().get(CoreAdminParams.INSTANCE_DIR);
+    if (instanceDir == null)
+      instanceDir = it.req.getParams().get("property.instanceDir");
+    if (instanceDir != null) {
+      instanceDir = PropertiesUtil.substituteProperty(instanceDir, coreContainer.getContainerProperties());
+      instancePath = coreContainer.getCoreRootDirectory().resolve(instanceDir).normalize();
     }
-  },
-  UNLOAD_OP(UNLOAD) {
-    @Override
-    public void call(CallInfo callInfo) throws IOException {
-      SolrParams params = callInfo.req.getParams();
-      String cname = params.get(CoreAdminParams.CORE);
-      boolean deleteIndexDir = params.getBool(CoreAdminParams.DELETE_INDEX, false);
-      boolean deleteDataDir = params.getBool(CoreAdminParams.DELETE_DATA_DIR, false);
-      boolean deleteInstanceDir = params.getBool(CoreAdminParams.DELETE_INSTANCE_DIR, false);
-      callInfo.handler.coreContainer.unload(cname, deleteIndexDir, deleteDataDir, deleteInstanceDir);
-
-      assert TestInjection.injectNonExistentCoreExceptionAfterUnload(cname);
-    }
-  },
-  RELOAD_OP(RELOAD) {
-    @Override
-    public void call(CallInfo callInfo) {
-      SolrParams params = callInfo.req.getParams();
-      String cname = params.get(CoreAdminParams.CORE);
 
-      if (cname == null || !callInfo.handler.coreContainer.getCoreNames().contains(cname)) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Core with core name [" + cname + "] does not exist.");
-      }
+    coreContainer.create(coreName, instancePath, coreParams);
 
-      try {
-        callInfo.handler.coreContainer.reload(cname);
-      } catch (Exception ex) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error handling 'reload' action", ex);
-      }
-    }
-  },
-  STATUS_OP(STATUS) {
-    @Override
-    public void call(CallInfo callInfo) {
-      SolrParams params = callInfo.req.getParams();
-
-      String cname = params.get(CoreAdminParams.CORE);
-      String indexInfo = params.get(CoreAdminParams.INDEX_INFO);
-      boolean isIndexInfoNeeded = Boolean.parseBoolean(null == indexInfo ? "true" : indexInfo);
-      NamedList<Object> status = new SimpleOrderedMap<>();
-      Map<String, Exception> failures = new HashMap<>();
-      for (Map.Entry<String, CoreContainer.CoreLoadFailure> failure : callInfo.handler.coreContainer.getCoreInitFailures().entrySet()) {
-        failures.put(failure.getKey(), failure.getValue().exception);
-      }
-      try {
-        if (cname == null) {
-          for (String name : callInfo.handler.coreContainer.getAllCoreNames()) {
-            status.add(name, getCoreStatus(callInfo.handler.coreContainer, name, isIndexInfoNeeded));
-          }
-          callInfo.rsp.add("initFailures", failures);
-        } else {
-          failures = failures.containsKey(cname)
-              ? Collections.singletonMap(cname, failures.get(cname))
-              : Collections.<String, Exception>emptyMap();
-          callInfo.rsp.add("initFailures", failures);
-          status.add(cname, getCoreStatus(callInfo.handler.coreContainer, cname, isIndexInfoNeeded));
-        }
-        callInfo.rsp.add("status", status);
-      } catch (Exception ex) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
-            "Error handling 'status' action ", ex);
-      }
-    }
+    it.rsp.add("core", coreName);
+  }),
+  UNLOAD_OP(UNLOAD, it -> {
+    SolrParams params = it.req.getParams();
+    String cname = params.get(CoreAdminParams.CORE);
+    boolean deleteIndexDir = params.getBool(CoreAdminParams.DELETE_INDEX, false);
+    boolean deleteDataDir = params.getBool(CoreAdminParams.DELETE_DATA_DIR, false);
+    boolean deleteInstanceDir = params.getBool(CoreAdminParams.DELETE_INSTANCE_DIR, false);
+    it.handler.coreContainer.unload(cname, deleteIndexDir, deleteDataDir, deleteInstanceDir);
 
+    assert TestInjection.injectNonExistentCoreExceptionAfterUnload(cname);
 
-  },
-
-  SWAP_OP(SWAP) {
-    @Override
-    public void call(CallInfo callInfo) {
-      final SolrParams params = callInfo.req.getParams();
-      final SolrParams required = params.required();
-
-      final String cname = params.get(CoreAdminParams.CORE);
-      String other = required.get(CoreAdminParams.OTHER);
-      callInfo.handler.coreContainer.swap(cname, other);
+  }),
+  RELOAD_OP(RELOAD, it -> {
+    SolrParams params = it.req.getParams();
+    String cname = params.get(CoreAdminParams.CORE);
 
+    if (cname == null || !it.handler.coreContainer.getCoreNames().contains(cname)) {
+      throw new SolrException(ErrorCode.BAD_REQUEST, "Core with core name [" + cname + "] does not exist.");
     }
-  },
-  RENAME_OP(RENAME) {
-    @Override
-    public void call(CallInfo callInfo) {
-      SolrParams params = callInfo.req.getParams();
 
-      String name = params.get(CoreAdminParams.OTHER);
-      String cname = params.get(CoreAdminParams.CORE);
-
-      if (cname.equals(name)) return;
-
-      callInfo.handler.coreContainer.rename(cname, name);
+    try {
+      it.handler.coreContainer.reload(cname);
+    } catch (Exception ex) {
+      throw new SolrException(ErrorCode.SERVER_ERROR, "Error handling 'reload' action", ex);
     }
-  },
-  MERGEINDEXES_OP(MERGEINDEXES) {
-    @Override
-    public void call(CallInfo callInfo) throws Exception {
-      SolrParams params = callInfo.req.getParams();
-      String cname = params.required().get(CoreAdminParams.CORE);
-      SolrCore core = callInfo.handler.coreContainer.getCore(cname);
-      SolrQueryRequest wrappedReq = null;
-
-      List<SolrCore> sourceCores = Lists.newArrayList();
-      List<RefCounted<SolrIndexSearcher>> searchers = Lists.newArrayList();
-      // stores readers created from indexDir param values
-      List<DirectoryReader> readersToBeClosed = Lists.newArrayList();
-      Map<Directory,Boolean> dirsToBeReleased = new HashMap<>();
-      if (core != null) {
-        try {
-          String[] dirNames = params.getParams(CoreAdminParams.INDEX_DIR);
-          if (dirNames == null || dirNames.length == 0) {
-            String[] sources = params.getParams("srcCore");
-            if (sources == null || sources.length == 0)
-              throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-                  "At least one indexDir or srcCore must be specified");
-
-            for (int i = 0; i < sources.length; i++) {
-              String source = sources[i];
-              SolrCore srcCore = callInfo.handler.coreContainer.getCore(source);
-              if (srcCore == null)
-                throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-                    "Core: " + source + " does not exist");
-              sourceCores.add(srcCore);
-            }
-          } else {
-            DirectoryFactory dirFactory = core.getDirectoryFactory();
-            for (int i = 0; i < dirNames.length; i++) {
-              boolean markAsDone = false;
-              if (dirFactory instanceof CachingDirectoryFactory) {
-                if (!((CachingDirectoryFactory)dirFactory).getLivePaths().contains(dirNames[i])) {
-                  markAsDone = true;
-                }
-              }
-              Directory dir = dirFactory.get(dirNames[i], DirectoryFactory.DirContext.DEFAULT, core.getSolrConfig().indexConfig.lockType);
-              dirsToBeReleased.put(dir, markAsDone);
-              // TODO: why doesn't this use the IR factory? what is going on here?
-              readersToBeClosed.add(DirectoryReader.open(dir));
-            }
+  }),
+
+  STATUS_OP(STATUS, it -> {
+    SolrParams params = it.req.getParams();
+
+    String cname = params.get(CoreAdminParams.CORE);
+    String indexInfo = params.get(CoreAdminParams.INDEX_INFO);
+    boolean isIndexInfoNeeded = Boolean.parseBoolean(null == indexInfo ? "true" : indexInfo);
+    NamedList<Object> status = new SimpleOrderedMap<>();
+    Map<String, Exception> failures = new HashMap<>();
+    for (Entry<String, CoreContainer.CoreLoadFailure> failure : it.handler.coreContainer.getCoreInitFailures().entrySet()) {
+      failures.put(failure.getKey(), failure.getValue().exception);
+    }
+    try {
+      if (cname == null) {
+        for (String name : it.handler.coreContainer.getAllCoreNames()) {
+          status.add(name, getCoreStatus(it.handler.coreContainer, name, isIndexInfoNeeded));
+        }
+        it.rsp.add("initFailures", failures);
+      } else {
+        failures = failures.containsKey(cname)
+            ? Collections.singletonMap(cname, failures.get(cname))
+            : Collections.<String, Exception>emptyMap();
+        it.rsp.add("initFailures", failures);
+        status.add(cname, getCoreStatus(it.handler.coreContainer, cname, isIndexInfoNeeded));
+      }
+      it.rsp.add("status", status);
+    } catch (Exception ex) {
+      throw new SolrException(ErrorCode.SERVER_ERROR,
+          "Error handling 'status' action ", ex);
+    }
+  }),
+
+  SWAP_OP(SWAP, it -> {
+    final SolrParams params = it.req.getParams();
+    final String cname = params.get(CoreAdminParams.CORE);
+    String other = params.required().get(CoreAdminParams.OTHER);
+    it.handler.coreContainer.swap(cname, other);
+  }),
+
+  RENAME_OP(RENAME, it -> {
+    SolrParams params = it.req.getParams();
+    String name = params.get(CoreAdminParams.OTHER);
+    String cname = params.get(CoreAdminParams.CORE);
+
+    if (cname.equals(name)) return;
+
+    it.handler.coreContainer.rename(cname, name);
+  }),
+
+  MERGEINDEXES_OP(MERGEINDEXES, it -> {
+    SolrParams params = it.req.getParams();
+    String cname = params.required().get(CoreAdminParams.CORE);
+    SolrCore core = it.handler.coreContainer.getCore(cname);
+    SolrQueryRequest wrappedReq = null;
+
+    List<SolrCore> sourceCores = Lists.newArrayList();
+    List<RefCounted<SolrIndexSearcher>> searchers = Lists.newArrayList();
+    // stores readers created from indexDir param values
+    List<DirectoryReader> readersToBeClosed = Lists.newArrayList();
+    Map<Directory, Boolean> dirsToBeReleased = new HashMap<>();
+    if (core != null) {
+      try {
+        String[] dirNames = params.getParams(CoreAdminParams.INDEX_DIR);
+        if (dirNames == null || dirNames.length == 0) {
+          String[] sources = params.getParams("srcCore");
+          if (sources == null || sources.length == 0)
+            throw new SolrException(ErrorCode.BAD_REQUEST,
+                "At least one indexDir or srcCore must be specified");
+
+          for (int i = 0; i < sources.length; i++) {
+            String source = sources[i];
+            SolrCore srcCore = it.handler.coreContainer.getCore(source);
+            if (srcCore == null)
+              throw new SolrException(ErrorCode.BAD_REQUEST,
+                  "Core: " + source + " does not exist");
+            sourceCores.add(srcCore);
           }
-
-          List<DirectoryReader> readers = null;
-          if (readersToBeClosed.size() > 0) {
-            readers = readersToBeClosed;
-          } else {
-            readers = Lists.newArrayList();
-            for (SolrCore solrCore : sourceCores) {
-              // record the searchers so that we can decref
-              RefCounted<SolrIndexSearcher> searcher = solrCore.getSearcher();
-              searchers.add(searcher);
-              readers.add(searcher.get().getIndexReader());
+        } else {
+          DirectoryFactory dirFactory = core.getDirectoryFactory();
+          for (int i = 0; i < dirNames.length; i++) {
+            boolean markAsDone = false;
+            if (dirFactory instanceof CachingDirectoryFactory) {
+              if (!((CachingDirectoryFactory) dirFactory).getLivePaths().contains(dirNames[i])) {
+                markAsDone = true;
+              }
             }
+            Directory dir = dirFactory.get(dirNames[i], DirectoryFactory.DirContext.DEFAULT, core.getSolrConfig().indexConfig.lockType);
+            dirsToBeReleased.put(dir, markAsDone);
+            // TODO: why doesn't this use the IR factory? what is going on here?
+            readersToBeClosed.add(DirectoryReader.open(dir));
           }
+        }
 
-          UpdateRequestProcessorChain processorChain =
-              core.getUpdateProcessingChain(params.get(UpdateParams.UPDATE_CHAIN));
-          wrappedReq = new LocalSolrQueryRequest(core, callInfo.req.getParams());
-          UpdateRequestProcessor processor =
-              processorChain.createProcessor(wrappedReq, callInfo.rsp);
-          processor.processMergeIndexes(new MergeIndexesCommand(readers, callInfo.req));
-        } catch (Exception e) {
-          // log and rethrow so that if the finally fails we don't lose the original problem
-          log.error("ERROR executing merge:", e);
-          throw e;
-        } finally {
-          for (RefCounted<SolrIndexSearcher> searcher : searchers) {
-            if (searcher != null) searcher.decref();
-          }
+        List<DirectoryReader> readers = null;
+        if (readersToBeClosed.size() > 0) {
+          readers = readersToBeClosed;
+        } else {
+          readers = Lists.newArrayList();
           for (SolrCore solrCore : sourceCores) {
-            if (solrCore != null) solrCore.close();
+            // record the searchers so that we can decref
+            RefCounted<SolrIndexSearcher> searcher = solrCore.getSearcher();
+            searchers.add(searcher);
+            readers.add(searcher.get().getIndexReader());
           }
-          IOUtils.closeWhileHandlingException(readersToBeClosed);
-          Set<Entry<Directory,Boolean>> entries = dirsToBeReleased.entrySet();
-          for (Entry<Directory,Boolean> entry : entries) {
-            DirectoryFactory dirFactory = core.getDirectoryFactory();
-            Directory dir = entry.getKey();
-            boolean markAsDone = entry.getValue();
-            if (markAsDone) {
-              dirFactory.doneWithDirectory(dir);
-            }
-            dirFactory.release(dir);
+        }
+
+        UpdateRequestProcessorChain processorChain =
+            core.getUpdateProcessingChain(params.get(UpdateParams.UPDATE_CHAIN));
+        wrappedReq = new LocalSolrQueryRequest(core, it.req.getParams());
+        UpdateRequestProcessor processor =
+            processorChain.createProcessor(wrappedReq, it.rsp);
+        processor.processMergeIndexes(new MergeIndexesCommand(readers, it.req));
+      } catch (Exception e) {
+        // log and rethrow so that if the finally fails we don't lose the original problem
+        log().error("ERROR executing merge:", e);
+        throw e;
+      } finally {
+        for (RefCounted<SolrIndexSearcher> searcher : searchers) {
+          if (searcher != null) searcher.decref();
+        }
+        for (SolrCore solrCore : sourceCores) {
+          if (solrCore != null) solrCore.close();
+        }
+        IOUtils.closeWhileHandlingException(readersToBeClosed);
+        Set<Entry<Directory, Boolean>> entries = dirsToBeReleased.entrySet();
+        for (Entry<Directory, Boolean> entry : entries) {
+          DirectoryFactory dirFactory = core.getDirectoryFactory();
+          Directory dir = entry.getKey();
+          boolean markAsDone = entry.getValue();
+          if (markAsDone) {
+            dirFactory.doneWithDirectory(dir);
           }
-          if (wrappedReq != null) wrappedReq.close();
-          core.close();
+          dirFactory.release(dir);
         }
+        if (wrappedReq != null) wrappedReq.close();
+        core.close();
       }
-
     }
-  },
-  SPLIT_OP(SPLIT) {
-    @Override
-    public void call(CallInfo callInfo) throws IOException {
-      SolrParams params = callInfo.req.getParams();
-      List<DocRouter.Range> ranges = null;
-
-      String[] pathsArr = params.getParams(PATH);
-      String rangesStr = params.get(CoreAdminParams.RANGES);    // ranges=a-b,c-d,e-f
-      if (rangesStr != null) {
-        String[] rangesArr = rangesStr.split(",");
-        if (rangesArr.length == 0) {
-          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "There must be at least one range specified to split an index");
-        } else {
-          ranges = new ArrayList<>(rangesArr.length);
-          for (String r : rangesArr) {
-            try {
-              ranges.add(DocRouter.DEFAULT.fromString(r));
-            } catch (Exception e) {
-              throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Exception parsing hexadecimal hash range: " + r, e);
-            }
+  }),
+
+  SPLIT_OP(SPLIT, it -> {
+    SolrParams params = it.req.getParams();
+    List<DocRouter.Range> ranges = null;
+
+    String[] pathsArr = params.getParams(PATH);
+    String rangesStr = params.get(CoreAdminParams.RANGES);    // ranges=a-b,c-d,e-f
+    if (rangesStr != null) {
+      String[] rangesArr = rangesStr.split(",");
+      if (rangesArr.length == 0) {
+        throw new SolrException(ErrorCode.BAD_REQUEST, "There must be at least one range specified to split an index");
+      } else {
+        ranges = new ArrayList<>(rangesArr.length);
+        for (String r : rangesArr) {
+          try {
+            ranges.add(DocRouter.DEFAULT.fromString(r));
+          } catch (Exception e) {
+            throw new SolrException(ErrorCode.BAD_REQUEST, "Exception parsing hexadecimal hash range: " + r, e);
           }
         }
       }
-      String splitKey = params.get("split.key");
-      String[] newCoreNames = params.getParams("targetCore");
-      String cname = params.get(CoreAdminParams.CORE, "");
+    }
+    String splitKey = params.get("split.key");
+    String[] newCoreNames = params.getParams("targetCore");
+    String cname = params.get(CoreAdminParams.CORE, "");
 
-      if ((pathsArr == null || pathsArr.length == 0) && (newCoreNames == null || newCoreNames.length == 0)) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Either path or targetCore param must be specified");
-      }
+    if ((pathsArr == null || pathsArr.length == 0) && (newCoreNames == null || newCoreNames.length == 0)) {
+      throw new SolrException(ErrorCode.BAD_REQUEST, "Either path or targetCore param must be specified");
+    }
 
-      log.info("Invoked split action for core: " + cname);
-      SolrCore core = callInfo.handler.coreContainer.getCore(cname);
-      SolrQueryRequest req = new LocalSolrQueryRequest(core, params);
-      List<SolrCore> newCores = null;
+    log().info("Invoked split action for core: " + cname);
+    SolrCore core = it.handler.coreContainer.getCore(cname);
+    SolrQueryRequest req = new LocalSolrQueryRequest(core, params);
+    List<SolrCore> newCores = null;
 
-      try {
-        // TODO: allow use of rangesStr in the future
-        List<String> paths = null;
-        int partitions = pathsArr != null ? pathsArr.length : newCoreNames.length;
-
-        DocRouter router = null;
-        String routeFieldName = null;
-        if (callInfo.handler.coreContainer.isZooKeeperAware()) {
-          ClusterState clusterState = callInfo.handler.coreContainer.getZkController().getClusterState();
-          String collectionName = req.getCore().getCoreDescriptor().getCloudDescriptor().getCollectionName();
-          DocCollection collection = clusterState.getCollection(collectionName);
-          String sliceName = req.getCore().getCoreDescriptor().getCloudDescriptor().getShardId();
-          Slice slice = collection.getSlice(sliceName);
-          router = collection.getRouter() != null ? collection.getRouter() : DocRouter.DEFAULT;
-          if (ranges == null) {
-            DocRouter.Range currentRange = slice.getRange();
-            ranges = currentRange != null ? router.partitionRange(partitions, currentRange) : null;
-          }
-          Object routerObj = collection.get(DOC_ROUTER); // for back-compat with Solr 4.4
-          if (routerObj != null && routerObj instanceof Map) {
-            Map routerProps = (Map) routerObj;
-            routeFieldName = (String) routerProps.get("field");
-          }
+    try {
+      // TODO: allow use of rangesStr in the future
+      List<String> paths = null;
+      int partitions = pathsArr != null ? pathsArr.length : newCoreNames.length;
+
+      DocRouter router = null;
+      String routeFieldName = null;
+      if (it.handler.coreContainer.isZooKeeperAware()) {
+        ClusterState clusterState = it.handler.coreContainer.getZkController().getClusterState();
+        String collectionName = req.getCore().getCoreDescriptor().getCloudDescriptor().getCollectionName();
+        DocCollection collection = clusterState.getCollection(collectionName);
+        String sliceName = req.getCore().getCoreDescriptor().getCloudDescriptor().getShardId();
+        Slice slice = collection.getSlice(sliceName);
+        router = collection.getRouter() != null ? collection.getRouter() : DocRouter.DEFAULT;
+        if (ranges == null) {
+          DocRouter.Range currentRange = slice.getRange();
+          ranges = currentRange != null ? router.partitionRange(partitions, currentRange) : null;
         }
+        Object routerObj = collection.get(DOC_ROUTER); // for back-compat with Solr 4.4
+        if (routerObj != null && routerObj instanceof Map) {
+          Map routerProps = (Map) routerObj;
+          routeFieldName = (String) routerProps.get("field");
+        }
+      }
 
-        if (pathsArr == null) {
-          newCores = new ArrayList<>(partitions);
-          for (String newCoreName : newCoreNames) {
-            SolrCore newcore = callInfo.handler.coreContainer.getCore(newCoreName);
-            if (newcore != null) {
-              newCores.add(newcore);
-            } else {
-              throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Core with core name " + newCoreName + " expected but doesn't exist.");
-            }
+      if (pathsArr == null) {
+        newCores = new ArrayList<>(partitions);
+        for (String newCoreName : newCoreNames) {
+          SolrCore newcore = it.handler.coreContainer.getCore(newCoreName);
+          if (newcore != null) {
+            newCores.add(newcore);
+          } else {
+            throw new SolrException(ErrorCode.BAD_REQUEST, "Core with core name " + newCoreName + " expected but doesn't exist.");
           }
-        } else {
-          paths = Arrays.asList(pathsArr);
         }
+      } else {
+        paths = Arrays.asList(pathsArr);
+      }
 
 
-        SplitIndexCommand cmd = new SplitIndexCommand(req, paths, newCores, ranges, router, routeFieldName, splitKey);
-        core.getUpdateHandler().split(cmd);
+      SplitIndexCommand cmd = new SplitIndexCommand(req, paths, newCores, ranges, router, routeFieldName, splitKey);
+      core.getUpdateHandler().split(cmd);
 
-        // After the split has completed, someone (here?) should start the process of replaying the buffered updates.
+      // After the split has completed, someone (here?) should start the process of replaying the buffered updates.
 
-      } catch (Exception e) {
-        log.error("ERROR executing split:", e);
-        throw new RuntimeException(e);
+    } catch (Exception e) {
+      log().error("ERROR executing split:", e);
+      throw new RuntimeException(e);
 
-      } finally {
-        if (req != null) req.close();
-        if (core != null) core.close();
-        if (newCores != null) {
-          for (SolrCore newCore : newCores) {
-            newCore.close();
-          }
+    } finally {
+      if (req != null) req.close();
+      if (core != null) core.close();
+      if (newCores != null) {
+        for (SolrCore newCore : newCores) {
+          newCore.close();
         }
       }
-
     }
-  },
-  PREPRECOVERY_OP(PREPRECOVERY) {
-    @Override
-    public void call(CallInfo callInfo) throws InterruptedException, IOException, KeeperException {
-      final SolrParams params = callInfo.req.getParams();
+  }),
 
-      String cname = params.get(CoreAdminParams.CORE);
-      if (cname == null) {
-        cname = "";
-      }
+  PREPRECOVERY_OP(PREPRECOVERY, it -> {
+    final SolrParams params = it.req.getParams();
 
-      String nodeName = params.get("nodeName");
-      String coreNodeName = params.get("coreNodeName");
-      Replica.State waitForState = Replica.State.getState(params.get(ZkStateReader.STATE_PROP));
-      Boolean checkLive = params.getBool("checkLive");
-      Boolean onlyIfLeader = params.getBool("onlyIfLeader");
-      Boolean onlyIfLeaderActive = params.getBool("onlyIfLeaderActive");
-
-      log.info("Going to wait for coreNodeName: " + coreNodeName + ", state: " + waitForState
-          + ", checkLive: " + checkLive + ", onlyIfLeader: " + onlyIfLeader
-          + ", onlyIfLeaderActive: " + onlyIfLeaderActive);
-
-      int maxTries = 0;
-      Replica.State state = null;
-      boolean live = false;
-      int retry = 0;
-      while (true) {
-        CoreContainer coreContainer = callInfo.handler.coreContainer;
-        try (SolrCore core = coreContainer.getCore(cname)) {
-          if (core == null && retry == 30) {
-            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "core not found:"
-                + cname);
-          }
-          if (core != null) {
-            if (onlyIfLeader != null && onlyIfLeader) {
-              if (!core.getCoreDescriptor().getCloudDescriptor().isLeader()) {
-                throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "We are not the leader");
-              }
-            }
+    String cname = params.get(CoreAdminParams.CORE);
+    if (cname == null) {
+      cname = "";
+    }
 
-            // wait until we are sure the recovering node is ready
-            // to accept updates
-            CloudDescriptor cloudDescriptor = core.getCoreDescriptor()
-                .getCloudDescriptor();
-            String collectionName = cloudDescriptor.getCollectionName();
-
-            if (retry % 15 == 0) {
-              if (retry > 0 && log.isInfoEnabled())
-                log.info("After " + retry + " seconds, core " + cname + " (" +
-                    cloudDescriptor.getShardId() + " of " +
-                    cloudDescriptor.getCollectionName() + ") still does not have state: " +
-                    waitForState + "; forcing ClusterState update from ZooKeeper");
-
-              // force a cluster state update
-              coreContainer.getZkController().getZkStateReader().forceUpdateCollection(collectionName);
+    String nodeName = params.get("nodeName");
+    String coreNodeName = params.get("coreNodeName");
+    Replica.State waitForState = Replica.State.getState(params.get(ZkStateReader.STATE_PROP));
+    Boolean checkLive = params.getBool("checkLive");
+    Boolean onlyIfLeader = params.getBool("onlyIfLeader");
+    Boolean onlyIfLeaderActive = params.getBool("onlyIfLeaderActive");
+
+    log().info("Going to wait for coreNodeName: " + coreNodeName + ", state: " + waitForState
+        + ", checkLive: " + checkLive + ", onlyIfLeader: " + onlyIfLeader
+        + ", onlyIfLeaderActive: " + onlyIfLeaderActive);
+
+    int maxTries = 0;
+    Replica.State state = null;
+    boolean live = false;
+    int retry = 0;
+    while (true) {
+      CoreContainer coreContainer = it.handler.coreContainer;
+      try (SolrCore core = coreContainer.getCore(cname)) {
+        if (core == null && retry == 30) {
+          throw new SolrException(ErrorCode.BAD_REQUEST, "core not found:"
+              + cname);
+        }
+        if (core != null) {
+          if (onlyIfLeader != null && onlyIfLeader) {
+            if (!core.getCoreDescriptor().getCloudDescriptor().isLeader()) {
+              throw new SolrException(ErrorCode.BAD_REQUEST, "We are not the leader");
             }
+          }
+
+          // wait until we are sure the recovering node is ready
+          // to accept updates
+          CloudDescriptor cloudDescriptor = core.getCoreDescriptor()
+              .getCloudDescriptor();
+          String collectionName = cloudDescriptor.getCollectionName();
 
-            if (maxTries == 0) {
-              // wait long enough for the leader conflict to work itself out plus a little extra
-              int conflictWaitMs = coreContainer.getZkController().getLeaderConflictResolveWait();
-              maxTries = (int) Math.round(conflictWaitMs / 1000) + 3;
-              log.info("Will wait a max of " + maxTries + " seconds to see " + cname + " (" +
+          if (retry % 15 == 0) {
+            if (retry > 0 && log().isInfoEnabled())
+              log().info("After " + retry + " seconds, core " + cname + " (" +
                   cloudDescriptor.getShardId() + " of " +
-                  cloudDescriptor.getCollectionName() + ") have state: " + waitForState);
-            }
+                  cloudDescriptor.getCollectionName() + ") still does not have state: " +
+                  waitForState + "; forcing ClusterState update from ZooKeeper");
 
-            ClusterState clusterState = coreContainer.getZkController().getClusterState();
-            DocCollection collection = clusterState.getCollection(collectionName);
-            Slice slice = collection.getSlice(cloudDescriptor.getShardId());
-            if (slice != null) {
-              final Replica replica = slice.getReplicasMap().get(coreNodeName);
-              if (replica != null) {
-                state = replica.getState();
-                live = clusterState.liveNodesContain(nodeName);
-
-                final Replica.State localState = cloudDescriptor.getLastPublished();
-
-                // TODO: This is funky but I've seen this in testing where the replica asks the
-                // leader to be in recovery? Need to track down how that happens ... in the meantime,
-                // this is a safeguard
-                boolean leaderDoesNotNeedRecovery = (onlyIfLeader != null &&
-                    onlyIfLeader &&
-                    core.getName().equals(replica.getStr("core")) &&
-                    waitForState == Replica.State.RECOVERING &&
-                    localState == Replica.State.ACTIVE &&
-                    state == Replica.State.ACTIVE);
-
-                if (leaderDoesNotNeedRecovery) {
-                  log.warn("Leader " + core.getName() + " ignoring request to be in the recovering state because it is live and active.");
-                }
+            // force a cluster state update
+            coreContainer.getZkController().getZkStateReader().forceUpdateCollection(collectionName);
+          }
 
-                boolean onlyIfActiveCheckResult = onlyIfLeaderActive != null && onlyIfLeaderActive && localState != Replica.State.ACTIVE;
-                log.info("In WaitForState(" + waitForState + "): collection=" + collectionName + ", shard=" + slice.getName() +
-                    ", thisCore=" + core.getName() + ", leaderDoesNotNeedRecovery=" + leaderDoesNotNeedRecovery +
-                    ", isLeader? " + core.getCoreDescriptor().getCloudDescriptor().isLeader() +
-                    ", live=" + live + ", checkLive=" + checkLive + ", currentState=" + state.toString() + ", localState=" + localState + ", nodeName=" + nodeName +
-                    ", coreNodeName=" + coreNodeName + ", onlyIfActiveCheckResult=" + onlyIfActiveCheckResult + ", nodeProps: " + replica);
-
-                if (!onlyIfActiveCheckResult && replica != null && (state == waitForState || leaderDoesNotNeedRecovery)) {
-                  if (checkLive == null) {
-                    break;
-                  } else if (checkLive && live) {
-                    break;
-                  } else if (!checkLive && !live) {
-                    break;
-                  }
+          if (maxTries == 0) {
+            // wait long enough for the leader conflict to work itself out plus a little extra
+            int conflictWaitMs = coreContainer.getZkController().getLeaderConflictResolveWait();
+            maxTries = (int) Math.round(conflictWaitMs / 1000) + 3;
+            log().info("Will wait a max of " + maxTries + " seconds to see " + cname + " (" +
+                cloudDescriptor.getShardId() + " of " +
+                cloudDescriptor.getCollectionName() + ") have state: " + waitForState);
+          }
+
+          ClusterState clusterState = coreContainer.getZkController().getClusterState();
+          DocCollection collection = clusterState.getCollection(collectionName);
+          Slice slice = collection.getSlice(cloudDescriptor.getShardId());
+          if (slice != null) {
+            final Replica replica = slice.getReplicasMap().get(coreNodeName);
+            if (replica != null) {
+              state = replica.getState();
+              live = clusterState.liveNodesContain(nodeName);
+
+              final Replica.State localState = cloudDescriptor.getLastPublished();
+
+              // TODO: This is funky but I've seen this in testing where the replica asks the
+              // leader to be in recovery? Need to track down how that happens ... in the meantime,
+              // this is a safeguard
+              boolean leaderDoesNotNeedRecovery = (onlyIfLeader != null &&
+                  onlyIfLeader &&
+                  core.getName().equals(replica.getStr("core")) &&
+                  waitForState == Replica.State.RECOVERING &&
+                  localState == Replica.State.ACTIVE &&
+                  state == Replica.State.ACTIVE);
+
+              if (leaderDoesNotNeedRecovery) {
+                log().warn("Leader " + core.getName() + " ignoring request to be in the recovering state because it is live and active.");
+              }
+
+              boolean onlyIfActiveCheckResult = onlyIfLeaderActive != null && onlyIfLeaderActive && localState != Replica.State.ACTIVE;
+              log().info("In WaitForState(" + waitForState + "): collection=" + collectionName + ", shard=" + slice.getName() +
+                  ", thisCore=" + core.getName() + ", leaderDoesNotNeedRecovery=" + leaderDoesNotNeedRecovery +
+                  ", isLeader? " + core.getCoreDescriptor().getCloudDescriptor().isLeader() +
+                  ", live=" + live + ", checkLive=" + checkLive + ", currentState=" + state.toString() + ", localState=" + localState + ", nodeName=" + nodeName +
+                  ", coreNodeName=" + coreNodeName + ", onlyIfActiveCheckResult=" + onlyIfActiveCheckResult + ", nodeProps: " + replica);
+
+              if (!onlyIfActiveCheckResult && replica != null && (state == waitForState || leaderDoesNotNeedRecovery)) {
+                if (checkLive == null) {
+                  break;
+                } else if (checkLive && live) {
+                  break;
+                } else if (!checkLive && !live) {
+                  break;
                 }
               }
             }
           }
+        }
 
-          if (retry++ == maxTries) {
-            String collection = null;
-            String leaderInfo = null;
-            String shardId = null;
-            try {
-              CloudDescriptor cloudDescriptor =
-                  core.getCoreDescriptor().getCloudDescriptor();
-              collection = cloudDescriptor.getCollectionName();
-              shardId = cloudDescriptor.getShardId();
-              leaderInfo = coreContainer.getZkController().
-                  getZkStateReader().getLeaderUrl(collection, shardId, 5000);
-            } catch (Exception exc) {
-              leaderInfo = "Not available due to: " + exc;
-            }
-
-            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-                "I was asked to wait on state " + waitForState + " for "
-                    + shardId + " in " + collection + " on " + nodeName
-                    + " but I still do not see the requested state. I see state: "
-                    + state.toString() + " live:" + live + " leader from ZK: " + leaderInfo
-            );
+        if (retry++ == maxTries) {
+          String collection = null;
+          String leaderInfo = null;
+          String shardId = null;
+          try {
+            CloudDescriptor cloudDescriptor =
+                core.getCoreDescriptor().getCloudDescriptor();
+            collection = cloudDescriptor.getCollectionName();
+            shardId = cloudDescriptor.getShardId();
+            leaderInfo = coreContainer.getZkController().
+                getZkStateReader().getLeaderUrl(collection, shardId, 5000);
+          } catch (Exception exc) {
+            leaderInfo = "Not available due to: " + exc;
           }
 
-          if (coreContainer.isShutDown()) {
-            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-                "Solr is shutting down");
-          }
+          throw new SolrException(ErrorCode.BAD_REQUEST,
+              "I was asked to wait on state " + waitForState + " for "
+                  + shardId + " in " + collection + " on " + nodeName
+                  + " but I still do not see the requested state. I see state: "
+                  + state.toString() + " live:" + live + " leader from ZK: " + leaderInfo
+          );
+        }
+
+        if (coreContainer.isShutDown()) {
+          throw new SolrException(ErrorCode.BAD_REQUEST,
+              "Solr is shutting down");
+        }
 
-          // solrcloud_debug
-          if (log.isDebugEnabled()) {
+        // solrcloud_debug
+        if (log().isDebugEnabled()) {
+          try {
+            LocalSolrQueryRequest r = new LocalSolrQueryRequest(core,
+                new ModifiableSolrParams());
+            CommitUpdateCommand commitCmd = new CommitUpdateCommand(r, false);
+            commitCmd.softCommit = true;
+            core.getUpdateHandler().commit(commitCmd);
+            RefCounted<SolrIndexSearcher> searchHolder = core
+                .getNewestSearcher(false);
+            SolrIndexSearcher searcher = searchHolder.get();
             try {
-              LocalSolrQueryRequest r = new LocalSolrQueryRequest(core,
-                  new ModifiableSolrParams());
-              CommitUpdateCommand commitCmd = new CommitUpdateCommand(r, false);
-              commitCmd.softCommit = true;
-              core.getUpdateHandler().commit(commitCmd);
-              RefCounted<SolrIndexSearcher> searchHolder = core
-                  .getNewestSearcher(false);
-              SolrIndexSearcher searcher = searchHolder.get();
-              try {
-                log.debug(core.getCoreDescriptor().getCoreContainer()
-                    .getZkController().getNodeName()
-                    + " to replicate "
-                    + searcher.search(new MatchAllDocsQuery(), 1).totalHits
-                    + " gen:"
-                    + core.getDeletionPolicy().getLatestCommit().getGeneration()
-                    + " data:" + core.getDataDir());
-              } finally {
-                searchHolder.decref();
-              }
-            } catch (Exception e) {
-              log.debug("Error in solrcloud_debug block", e);
+              log().debug(core.getCoreDescriptor().getCoreContainer()
+                  .getZkController().getNodeName()
+                  + " to replicate "
+                  + searcher.search(new MatchAllDocsQuery(), 1).totalHits
+                  + " gen:"
+                  + core.getDeletionPolicy().getLatestCommit().getGeneration()
+                  + " data:" + core.getDataDir());
+            } finally {
+              searchHolder.decref();
             }
+          } catch (Exception e) {
+            log().debug("Error in solrcloud_debug block", e);
           }
         }
-        Thread.sleep(1000);
       }
-
-      log.info("Waited coreNodeName: " + coreNodeName + ", state: " + waitForState
-          + ", checkLive: " + checkLive + ", onlyIfLeader: " + onlyIfLeader + " for: " + retry + " seconds.");
-    }
-  },
-  REQUESTRECOVERY_OP(REQUESTRECOVERY) {
-    @Override
-    public void call(final CallInfo callInfo) throws IOException {
-      final SolrParams params = callInfo.req.getParams();
-      log.info("It has been requested that we recover: core="+params.get(CoreAdminParams.CORE));
-      new Thread(() -> {
-        String cname = params.get(CoreAdminParams.CORE);
-        if (cname == null) {
-          cname = "";
-        }
-        try (SolrCore core = callInfo.handler.coreContainer.getCore(cname)) {
-          if (core != null) {
-            core.getUpdateHandler().getSolrCoreState().doRecovery(callInfo.handler.coreContainer, core.getCoreDescriptor());
-          } else {
-            SolrException.log(log, "Could not find core to call recovery:" + cname);
-          }
-        }
-      }).start();
-
+      Thread.sleep(1000);
     }
-  },
-  REQUESTSYNCSHARD_OP(REQUESTSYNCSHARD) {
-    @Override
-    public void call(CallInfo callInfo) throws IOException {
-      final SolrParams params = callInfo.req.getParams();
 
-      log.info("I have been requested to sync up my shard");
-      ZkController zkController = callInfo.handler.coreContainer.getZkController();
-      if (zkController == null) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Only valid for SolrCloud");
-      }
+    log().info("Waited coreNodeName: " + coreNodeName + ", state: " + waitForState
+        + ", checkLive: " + checkLive + ", onlyIfLeader: " + onlyIfLeader + " for: " + retry + " seconds.");
+  }),
 
+  REQUESTRECOVERY_OP(REQUESTRECOVERY, it -> {
+    final SolrParams params = it.req.getParams();
+    log().info("It has been requested that we recover: core=" + params.get(CoreAdminParams.CORE));
+    new Thread(() -> {
       String cname = params.get(CoreAdminParams.CORE);
       if (cname == null) {
-        throw new IllegalArgumentException(CoreAdminParams.CORE + " is required");
+        cname = "";
       }
-
-      SyncStrategy syncStrategy = null;
-      try (SolrCore core = callInfo.handler.coreContainer.getCore(cname)) {
-
+      try (SolrCore core = it.handler.coreContainer.getCore(cname)) {
         if (core != null) {
-          syncStrategy = new SyncStrategy(core.getCoreDescriptor().getCoreContainer());
-
-          Map<String, Object> props = new HashMap<>();
-          props.put(ZkStateReader.BASE_URL_PROP, zkController.getBaseUrl());
-          props.put(ZkStateReader.CORE_NAME_PROP, cname);
-          props.put(ZkStateReader.NODE_NAME_PROP, zkController.getNodeName());
-
-          boolean success = syncStrategy.sync(zkController, core, new ZkNodeProps(props), true);
-          // solrcloud_debug
-          if (log.isDebugEnabled()) {
-            try {
-              RefCounted<SolrIndexSearcher> searchHolder = core
-                  .getNewestSearcher(false);
-              SolrIndexSearcher searcher = searchHolder.get();
-              try {
-                log.debug(core.getCoreDescriptor().getCoreContainer()
-                    .getZkController().getNodeName()
-                    + " synched "
-                    + searcher.search(new MatchAllDocsQuery(), 1).totalHits);
-              } finally {
-                searchHolder.decref();
-              }
-            } catch (Exception e) {
-              log.debug("Error in solrcloud_debug block", e);
-            }
-          }
-          if (!success) {
-            throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Sync Failed");
-          }
+          core.getUpdateHandler().getSolrCoreState().doRecovery(it.handler.coreContainer, core.getCoreDescriptor());
         } else {
-          SolrException.log(log, "Could not find core to call sync:" + cname);
-        }
-      } finally {
-        // no recoveryStrat close for now
-        if (syncStrategy != null) {
-          syncStrategy.close();
+          SolrException.log(log(), "Could not find core to call recovery:" + cname);
         }
       }
+    }).start();
 
+  }),
+  REQUESTSYNCSHARD_OP(REQUESTSYNCSHARD, it -> {
+    final SolrParams params = it.req.getParams();
 
+    log().info("I have been requested to sync up my shard");
+    ZkController zkController = it.handler.coreContainer.getZkController();
+    if (zkController == null) {
+      throw new SolrException(ErrorCode.BAD_REQUEST, "Only valid for SolrCloud");
     }
-  },
-  REQUESTBUFFERUPDATES_OP(REQUESTBUFFERUPDATES) {
-    @Override
-    public void call(CallInfo callInfo) {
-      SolrParams params = callInfo.req.getParams();
-      String cname = params.get(CoreAdminParams.NAME, "");
-      log.info("Starting to buffer updates on core:" + cname);
 
-      try (SolrCore core = callInfo.handler.coreContainer.getCore(cname)) {
-        if (core == null)
-          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Core [" + cname + "] does not exist");
-        UpdateLog updateLog = core.getUpdateHandler().getUpdateLog();
-        if (updateLog.getState() != UpdateLog.State.ACTIVE)  {
-          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Core " + cname + " not in active state");
-        }
-        updateLog.bufferUpdates();
-        callInfo.rsp.add("core", cname);
-        callInfo.rsp.add("status", "BUFFERING");
-      } catch (Throwable e) {
-        if (e instanceof SolrException)
-          throw (SolrException)e;
-        else
-          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Could not start buffering updates", e);
-      } finally {
-        if (callInfo.req != null) callInfo.req.close();
-      }
+    String cname = params.get(CoreAdminParams.CORE);
+    if (cname == null) {
+      throw new IllegalArgumentException(CoreAdminParams.CORE + " is required");
     }
-  },
-  REQUESTAPPLYUPDATES_OP(REQUESTAPPLYUPDATES) {
-    @Override
-    public void call(CallInfo callInfo) {
-      {
-        SolrParams params = callInfo.req.getParams();
-        String cname = params.get(CoreAdminParams.NAME, "");
-        log.info("Applying buffered updates on core: " + cname);
-        CoreContainer coreContainer = callInfo.handler.coreContainer;
-        try (SolrCore core = coreContainer.getCore(cname)) {
-          if (core == null)
-            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Core [" + cname + "] not found");
-          UpdateLog updateLog = core.getUpdateHandler().getUpdateLog();
-          if (updateLog.getState() != UpdateLog.State.BUFFERING) {
-            throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Core " + cname + " not in buffering state");
-          }
-          Future<UpdateLog.RecoveryInfo> future = updateLog.applyBufferedUpdates();
-          if (future == null) {
-            log.info("No buffered updates available. core=" + cname);
-            callInfo.rsp.add("core", cname);
-            callInfo.rsp.add("status", "EMPTY_BUFFER");
-            return;
-          }
-          UpdateLog.RecoveryInfo report = future.get();
-          if (report.failed) {
-            SolrException.log(log, "Replay failed");
-            throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Replay failed");
+
+    SyncStrategy syncStrategy = null;
+    try (SolrCore core = it.handler.coreContainer.getCore(cname)) {
+
+      if (core != null) {
+        syncStrategy = new SyncStrategy(core.getCoreDescriptor().getCoreContainer());
+
+        Map<String, Object> props = new HashMap<>();
+        props.put(ZkStateReader.BASE_URL_PROP, zkController.getBaseUrl());
+        props.put(ZkStateReader.CORE_NAME_PROP, cname);
+        props.put(ZkStateReader.NODE_NAME_PROP, zkController.getNodeName());
+
+        boolean success = syncStrategy.sync(zkController, core, new ZkNodeProps(props), true);
+        // solrcloud_debug
+        if (log().isDebugEnabled()) {
+          try {
+            RefCounted<SolrIndexSearcher> searchHolder = core
+                .getNewestSearcher(false);
+            SolrIndexSearcher searcher = searchHolder.get();
+            try {
+              log().debug(core.getCoreDescriptor().getCoreContainer()
+                  .getZkController().getNodeName()
+                  + " synched "
+                  + searcher.search(new MatchAllDocsQuery(), 1).totalHits);
+            } finally {
+              searchHolder.decref();
+            }
+          } catch (Exception e) {
+            log().debug("Error in solrcloud_debug block", e);
           }
-          coreContainer.getZkController().publish(core.getCoreDescriptor(), Replica.State.ACTIVE);
-          callInfo.rsp.add("core", cname);
-          callInfo.rsp.add("status", "BUFFER_APPLIED");
-        } catch (InterruptedException e) {
-          Thread.currentThread().interrupt();
-          log.warn("Recovery was interrupted", e);
-        } catch (Exception e) {
-          if (e instanceof SolrException)
-            throw (SolrException) e;
-          else
-            throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Could not apply buffered updates", e);
-        } finally {
-          if (callInfo.req != null) callInfo.req.close();
         }
-
-      }
-    }
-  },
-  REQUESTSTATUS_OP(REQUESTSTATUS) {
-    @Override
-    public void call(CallInfo callInfo) {
-      SolrParams params = callInfo.req.getParams();
-      String requestId = params.get(CoreAdminParams.REQUESTID);
-      log.info("Checking request status for : " + requestId);
-
-      if (callInfo.handler.getRequestStatusMap(RUNNING).containsKey(requestId)) {
-        callInfo.rsp.add(RESPONSE_STATUS, RUNNING);
-      } else if (callInfo.handler.getRequestStatusMap(COMPLETED).containsKey(requestId)) {
-        callInfo.rsp.add(RESPONSE_STATUS, COMPLETED);
-        callInfo.rsp.add(RESPONSE, callInfo.handler.getRequestStatusMap(COMPLETED).get(requestId).getRspObject());
-      } else if (callInfo.handler.getRequestStatusMap(FAILED).containsKey(requestId)) {
-        callInfo.rsp.add(RESPONSE_STATUS, FAILED);
-        callInfo.rsp.add(RESPONSE, callInfo.handler.getRequestStatusMap(FAILED).get(requestId).getRspObject());
+        if (!success) {
+          throw new SolrException(ErrorCode.SERVER_ERROR, "Sync Failed");
+        }
       } else {
-        callInfo.rsp.add(RESPONSE_STATUS, "notfound");
-        callInfo.rsp.add(RESPONSE_MESSAGE, "No task found in running, completed or failed tasks");
+        SolrException.log(log(), "Could not find core to call sync:" + cname);
+      }
+    } finally {
+      // no recoveryStrat close for now
+      if (syncStrategy != null) {
+        syncStrategy.close();
       }
-
     }
-  },
-  OVERSEEROP_OP(OVERSEEROP) {
-    @Override
-    public void call(CallInfo callInfo) {
-      ZkController zkController = callInfo.handler.coreContainer.getZkController();
-      if (zkController != null) {
-        String op = callInfo.req.getParams().get("op");
-        String electionNode = callInfo.req.getParams().get("electionNode");
-        if (electionNode != null) {
-          zkController.rejoinOverseerElection(electionNode, "rejoinAtHead".equals(op));
-        } else {
-          log.info("electionNode is required param");
+  }),
+
+  REQUESTBUFFERUPDATES_OP(REQUESTBUFFERUPDATES, it -> {
+    SolrParams params = it.req.getParams();
+    String cname = params.get(CoreAdminParams.NAME, "");
+    log().info("Starting to buffer updates on core:" + cname);
+
+    try (SolrCore core = it.handler.coreContainer.getCore(cname)) {
+      if (core == null)
+        throw new SolrException(ErrorCode.BAD_REQUEST, "Core [" + cname + "] does not exist");
+      UpdateLog updateLog = core.getUpdateHandler().getUpdateLog();
+      if (updateLog.getState() != UpdateLog.State.ACTIVE) {
+        throw new SolrException(ErrorCode.SERVER_ERROR, "Core " + cname + " not in active state");
+      }
+      updateLog.bufferUpdates();
+      it.rsp.add("core", cname);
+      it.rsp.add("status", "BUFFERING");
+    } catch (Throwable e) {
+      if (e instanceof SolrException)
+        throw (SolrException) e;
+      else
+        throw new SolrException(ErrorCode.SERVER_ERROR, "Could not start buffering updates", e);
+    } finally {
+      if (it.req != null) it.req.close();
+    }
+  }),
+  REQUESTAPPLYUPDATES_OP(REQUESTAPPLYUPDATES, it -> {
+    SolrParams params = it.req.getParams();
+    String cname = params.get(CoreAdminParams.NAME, "");
+    log().info("Applying buffered updates on core: " + cname);
+    CoreContainer coreContainer = it.handler.coreContainer;
+    try (SolrCore core = coreContainer.getCore(cname)) {
+      if (core == null)
+        throw new SolrException(ErrorCode.BAD_REQUEST, "Core [" + cname + "] not found");
+      UpdateLog updateLog = core.getUpdateHandler().getUpdateLog();
+      if (updateLog.getState() != UpdateLog.State.BUFFERING) {
+        throw new SolrException(ErrorCode.SERVER_ERROR, "Core " + cname + " not in buffering state");
         }
-      }
+      Future<UpdateLog.RecoveryInfo> future = updateLog.applyBufferedUpdates();
+      if (future == null) {
+        log().info("No buffered updates available. core=" + cname);
+        it.rsp.add("core", cname);
+        it.rsp.add("status", "EMPTY_BUFFER");
+        return;
+      }
+      UpdateLog.RecoveryInfo report = future.get();
+      if (report.failed) {
+        SolrException.log(log(), "Replay failed");
+        throw new SolrException(ErrorCode.SERVER_ERROR, "Replay failed");
+      }
+      coreContainer.getZkController().publish(core.getCoreDescriptor(), Replica.State.ACTIVE);
+      it.rsp.add("core", cname);
+      it.rsp.add("status", "BUFFER_APPLIED");
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      log().warn("Recovery was interrupted", e);
+    } catch (Exception e) {
+      if (e instanceof SolrException)
+        throw (SolrException) e;
+      else
+        throw new SolrException(ErrorCode.SERVER_ERROR, "Could not apply buffered updates", e);
+    } finally {
+      if (it.req != null) it.req.close();
+      }
+  }),
+
+  REQUESTSTATUS_OP(REQUESTSTATUS, it -> {
+    SolrParams params = it.req.getParams();
+    String requestId = params.get(CoreAdminParams.REQUESTID);
+    log().info("Checking request status for : " + requestId);
+
+    if (it.handler.getRequestStatusMap(RUNNING).containsKey(requestId)) {
+      it.rsp.add(RESPONSE_STATUS, RUNNING);
+    } else if (it.handler.getRequestStatusMap(COMPLETED).containsKey(requestId)) {
+      it.rsp.add(RESPONSE_STATUS, COMPLETED);
+      it.rsp.add(RESPONSE, it.handler.getRequestStatusMap(COMPLETED).get(requestId).getRspObject());
+    } else if (it.handler.getRequestStatusMap(FAILED).containsKey(requestId)) {
+      it.rsp.add(RESPONSE_STATUS, FAILED);
+      it.rsp.add(RESPONSE, it.handler.getRequestStatusMap(FAILED).get(requestId).getRspObject());
+    } else {
+      it.rsp.add(RESPONSE_STATUS, "notfound");
+      it.rsp.add(RESPONSE_MESSAGE, "No task found in running, completed or failed tasks");
     }
-  },
-  REJOINLEADERELECTION_OP(REJOINLEADERELECTION) {
-    @Override
-    public void call(CallInfo callInfo) {
-      ZkController zkController = callInfo.handler.coreContainer.getZkController();
-
-      if (zkController != null) {
-        zkController.rejoinShardLeaderElection(callInfo.req.getParams());
+  }),
+
+  OVERSEEROP_OP(OVERSEEROP, it -> {
+    ZkController zkController = it.handler.coreContainer.getZkController();
+    if (zkController != null) {
+      String op = it.req.getParams().get("op");
+      String electionNode = it.req.getParams().get("electionNode");
+      if (electionNode != null) {
+        zkController.rejoinOverseerElection(electionNode, "rejoinAtHead".equals(op));
       } else {
-        log.warn("zkController is null in CoreAdminHandler.handleRequestInternal:REJOINLEADERELECTION. No action taken.");
+        log().info("electionNode is required param");
       }
     }
-  },
-  INVOKE_OP(INVOKE) {
-    @Override
-    public void call(CallInfo callInfo) throws Exception {
-      String[] klas = callInfo.req.getParams().getParams("class");
-      if (klas == null || klas.length == 0) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "class is a required param");
-      }
-      for (String c : klas) {
-        Map<String, Object> result = invokeAClass(callInfo.req, c);
-        callInfo.rsp.add(c, result);
-      }
+  }),
+
+  REJOINLEADERELECTION_OP(REJOINLEADERELECTION, it -> {
+    ZkController zkController = it.handler.coreContainer.getZkController();
 
+    if (zkController != null) {
+      zkController.rejoinShardLeaderElection(it.req.getParams());
+    } else {
+      log().warn("zkController is null in CoreAdminHandler.handleRequestInternal:REJOINLEADERELECTION. No action taken.");
     }
+  }),
 
-    Map<String, Object> invokeAClass(SolrQueryRequest req, String c) {
-      SolrResourceLoader loader = null;
-      if (req.getCore() != null) loader = req.getCore().getResourceLoader();
-      else if (req.getContext().get(CoreContainer.class.getName()) != null) {
-        CoreContainer cc = (CoreContainer) req.getContext().get(CoreContainer.class.getName());
-        loader = cc.getResourceLoader();
-      }
+  INVOKE_OP(INVOKE, it -> {
+    String[] klas = it.req.getParams().getParams("class");
+    if (klas == null || klas.length == 0) {
+      throw new SolrException(ErrorCode.BAD_REQUEST, "class is a required param");
+    }
+    for (String c : klas) {
+      Map<String, Object> result = invokeAClass(it.req, c);
+      it.rsp.add(c, result);
+    }
+  }),
 
-      Invocable invokable = loader.newInstance(c, Invocable.class);
-      Map<String, Object> result = invokable.invoke(req);
-      log.info("Invocable_invoked {}", result);
-      return result;
-    }
-  },
-  FORCEPREPAREFORLEADERSHIP_OP(FORCEPREPAREFORLEADERSHIP) {
-    @Override
-    public void call(CallInfo callInfo) throws IOException {
-      final SolrParams params = callInfo.req.getParams();
-
-      log.info("I have been forcefully prepare myself for leadership.");
-      ZkController zkController = callInfo.handler.coreContainer.getZkController();
-      if (zkController == null) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Only valid for SolrCloud");
-      }
+  FORCEPREPAREFORLEADERSHIP_OP(FORCEPREPAREFORLEADERSHIP, it -> {
+    final SolrParams params = it.req.getParams();
 
-      String cname = params.get(CoreAdminParams.CORE);
-      if (cname == null) {
-        throw new IllegalArgumentException(CoreAdminParams.CORE + " is required");
-      }
-      try (SolrCore core = callInfo.handler.coreContainer.getCore(cname)) {
+    log().info("I have been forcefully prepare myself for leadership.");
+    ZkController zkController = it.handler.coreContainer.getZkController();
+    if (zkController == null) {
+      throw new SolrException(ErrorCode.BAD_REQUEST, "Only valid for SolrCloud");
+    }
 
-        // Setting the last published state for this core to be ACTIVE
-        if (core != null) {
-          core.getCoreDescriptor().getCloudDescriptor().setLastPublished(Replica.State.ACTIVE);
-          log.info("Setting the last published state for this core, {}, to {}", core.getName(), Replica.State.ACTIVE);
-        } else {
-          SolrException.log(log, "Could not find core: " + cname);
-        }
+    String cname = params.get(CoreAdminParams.CORE);
+    if (cname == null) {
+      throw new IllegalArgumentException(CoreAdminParams.CORE + " is required");
+    }
+    try (SolrCore core = it.handler.coreContainer.getCore(cname)) {
+
+      // Setting the last published state for this core to be ACTIVE
+      if (core != null) {
+        core.getCoreDescriptor().getCloudDescriptor().setLastPublished(Replica.State.ACTIVE);
+        log().info("Setting the last published state for this core, {}, to {}", core.getName(), Replica.State.ACTIVE);
+      } else {
+        SolrException.log(log(), "Could not find core: " + cname);
       }
+    }
+  }),
 
+  BACKUPCORE_OP(BACKUPCORE, it -> {
+    ZkController zkController = it.handler.coreContainer.getZkController();
+    if (zkController == null) {
+      throw new SolrException(ErrorCode.BAD_REQUEST, "Internal SolrCloud API");
     }
-  },
-  BACKUPCORE_OP(BACKUPCORE) {
-    @Override
-    public void call(CallInfo callInfo) throws IOException {
-      ZkController zkController = callInfo.handler.coreContainer.getZkController();
-      if (zkController == null) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Internal SolrCloud API");
-      }
 
-      final SolrParams params = callInfo.req.getParams();
-      String cname = params.get(CoreAdminParams.CORE);
-      if (cname == null) {
-        throw new IllegalArgumentException(CoreAdminParams.CORE + " is required");
-      }
+    final SolrParams params = it.req.getParams();
+    String cname = params.get(CoreAdminParams.CORE);
+    if (cname == null) {
+      throw new IllegalArgumentException(CoreAdminParams.CORE + " is required");
+    }
 
-      String name = params.get(NAME);
-      if (name == null) {
-        throw new IllegalArgumentException(CoreAdminParams.NAME + " is required");
-      }
+    String name = params.get(NAME);
+    if (name == null) {
+      throw new IllegalArgumentException(CoreAdminParams.NAME + " is required");
+    }
 
-      String repoName = params.get(CoreAdminParams.BACKUP_REPOSITORY);
-      BackupRepository repository = callInfo.handler.coreContainer.newBackupRepository(Optional.ofNullable(repoName));
+    String repoName = params.get(CoreAdminParams.BACKUP_REPOSITORY);
+    BackupRepository repository = it.handler.coreContainer.newBackupRepository(Optional.ofNullable(repoName));
 
-      String location = repository.getBackupLocation(params.get(CoreAdminParams.BACKUP_LOCATION));
-      if (location == null) {
-        throw new SolrException(ErrorCode.BAD_REQUEST, "'location' is not specified as a query"
-            + " parameter or as a default repository property");
-      }
+    String location = repository.getBackupLocation(params.get(CoreAdminParams.BACKUP_LOCATION));
+    if (location == null) {
+      throw new SolrException(ErrorCode.BAD_REQUEST, "'location' is not specified as a query"
+          + " parameter or as a default repository property");
+    }
 
-      try (SolrCore core = callInfo.handler.coreContainer.getCore(cname)) {
-        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 (!snapShooter.getBackupRepository().exists(snapShooter.getLocation())) {
-          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-              "Directory to contain snapshots doesn't exist: " + snapShooter.getLocation());
-        }
-        snapShooter.validateCreateSnapshot();
-        snapShooter.createSnapshot();
-      } catch (Exception e) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
-            "Failed to backup core=" + cname + " because " + e, e);
-      }
+    try (SolrCore core = it.handler.coreContainer.getCore(cname)) {
+      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 (!snapShooter.getBackupRepository().exists(snapShooter.getLocation())) {
+        throw new SolrException(ErrorCode.BAD_REQUEST,
+            "Directory to contain snapshots doesn't exist: " + snapShooter.getLocation());
+      }
+      snapShooter.validateCreateSnapshot();
+      snapShooter.createSnapshot();
+    } catch (Exception e) {
+      throw new SolrException(ErrorCode.SERVER_ERROR,
+          "Failed to backup core=" + cname + " because " + e, e);
     }
-  },
-  RESTORECORE_OP(RESTORECORE) {
-    @Override
-    public void call(CallInfo callInfo) throws Exception {
-      ZkController zkController = callInfo.handler.coreContainer.getZkController();
-      if (zkController == null) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Only valid for SolrCloud");
-      }
+  }),
 
-      final SolrParams params = callInfo.req.getParams();
-      String cname = params.get(CoreAdminParams.CORE);
-      if (cname == null) {
-        throw new IllegalArgumentException(CoreAdminParams.CORE + " is required");
-      }
+  RESTORECORE_OP(RESTORECORE, it -> {
+    ZkController zkController = it.handler.coreContainer.getZkController();
+    if (zkController == null) {
+      throw new SolrException(ErrorCode.BAD_REQUEST, "Only valid for SolrCloud");
+    }
 
-      String name = params.get(NAME);
-      if (name == null) {
-        throw new IllegalArgumentException(CoreAdminParams.NAME + " is required");
-      }
+    final SolrParams params = it.req.getParams();
+    String cname = params.get(CoreAdminParams.CORE);
+    if (cname == null) {
+      throw new IllegalArgumentException(CoreAdminParams.CORE + " is required");
+    }
 
-      String repoName = params.get(CoreAdminParams.BACKUP_REPOSITORY);
-      BackupRepository repository = callInfo.handler.coreContainer.newBackupRepository(Optional.ofNullable(repoName));
+    String name = params.get(NAME);
+    if (name == null) {
+      throw new IllegalArgumentException(CoreAdminParams.NAME + " is required");
+    }
 
-      String location = repository.getBackupLocation(params.get(CoreAdminParams.BACKUP_LOCATION));
-      if (location == null) {
-        throw new SolrException(ErrorCode.BAD_REQUEST, "'location' is not specified as a query"
-            + " parameter or as a default repository property");
-      }
+    String repoName = params.get(CoreAdminParams.BACKUP_REPOSITORY);
+    BackupRepository repository = it.handler.coreContainer.newBackupRepository(Optional.ofNullable(repoName));
 
-      try (SolrCore core = callInfo.handler.coreContainer.getCore(cname)) {
-        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());
-        }
+    String location = repository.getBackupLocation(params.get(CoreAdminParams.BACKUP_LOCATION));
+    if (location == null) {
+      throw new SolrException(ErrorCode.BAD_REQUEST, "'location' is not specified as a query"
+          + " parameter or as a default repository property");
+    }
+
+    try (SolrCore core = it.handler.coreContainer.getCore(cname)) {
+      RestoreCore restoreCore = new RestoreCore(repository, core, location, name);
+      boolean success = restoreCore.doRestore();
+      if (!success) {
+        throw new SolrException(ErrorCode.SERVER_ERROR, "Failed to restore core=" + core.getName());
       }
     }
-  };
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  });
 
   final CoreAdminParams.CoreAdminAction action;
+  final CoreAdminOp fun;
 
-  public abstract void call(CallInfo callInfo) throws Exception;
-
-  CoreAdminOperation(CoreAdminParams.CoreAdminAction action) {
+  CoreAdminOperation(CoreAdminParams.CoreAdminAction action, CoreAdminOp fun) {
     this.action = action;
+    this.fun = fun;
+  }
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  static Logger log() {
+    return log;
   }
+
+
+
+
   /**
    * Returns the core status for a particular core.
    * @param cores - the enclosing core container
@@ -937,7 +872,7 @@ enum CoreAdminOperation {
    * @return - a named list of key/value pairs from the core.
    * @throws IOException - LukeRequestHandler can throw an I/O exception
    */
-  NamedList<Object> getCoreStatus(CoreContainer cores, String cname, boolean isIndexInfoNeeded)  throws IOException {
+  static NamedList<Object> getCoreStatus(CoreContainer cores, String cname, boolean isIndexInfoNeeded) throws IOException {
     NamedList<Object> info = new SimpleOrderedMap<>();
 
     if (!cores.isLoaded(cname)) { // Lazily-loaded core, fill in what we can.
@@ -986,7 +921,7 @@ enum CoreAdminOperation {
     return info;
   }
 
-  long getIndexSize(SolrCore core) {
+  static long getIndexSize(SolrCore core) {
     Directory dir;
     long size = 0;
     try {
@@ -1004,4 +939,23 @@ enum CoreAdminOperation {
     return size;
   }
 
+  static Map<String, Object> invokeAClass(SolrQueryRequest req, String c) {
+    SolrResourceLoader loader = null;
+    if (req.getCore() != null) loader = req.getCore().getResourceLoader();
+    else if (req.getContext().get(CoreContainer.class.getName()) != null) {
+      CoreContainer cc = (CoreContainer) req.getContext().get(CoreContainer.class.getName());
+      loader = cc.getResourceLoader();
+    }
+
+    Invocable invokable = loader.newInstance(c, Invocable.class);
+    Map<String, Object> result = invokable.invoke(req);
+    log().info("Invocable_invoked {}", result);
+    return result;
+  }
+
+  @Override
+  public void execute(CallInfo it) throws Exception {
+    fun.execute(it);
+  }
+
 }