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/09/29 14:16:10 UTC

lucene-solr:master: SOLR-8146: refactored the replica rules classes so that they can be accessed from SolrJ

Repository: lucene-solr
Updated Branches:
  refs/heads/master ce24de5cd -> 063d624cd


SOLR-8146: refactored the replica rules classes so that they can be accessed from SolrJ


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

Branch: refs/heads/master
Commit: 063d624cdcf73e0eeb3c11487a76d4c3de7f40dc
Parents: ce24de5
Author: Noble Paul <no...@apache.org>
Authored: Thu Sep 29 19:37:35 2016 +0530
Committer: Noble Paul <no...@apache.org>
Committed: Thu Sep 29 19:39:33 2016 +0530

----------------------------------------------------------------------
 .../apache/solr/cloud/rule/ImplicitSnitch.java  | 145 +---------------
 .../apache/solr/cloud/rule/RemoteCallback.java  |  23 ---
 .../apache/solr/cloud/rule/ReplicaAssigner.java |  17 +-
 .../java/org/apache/solr/cloud/rule/Rule.java   |   2 +-
 .../solr/cloud/rule/ServerSnitchContext.java    | 108 ++++++++++++
 .../java/org/apache/solr/cloud/rule/Snitch.java |   2 +-
 .../apache/solr/cloud/rule/SnitchContext.java   | 168 -------------------
 .../solr/cloud/rule/ImplicitSnitchTest.java     |  16 +-
 .../apache/solr/cloud/rule/RuleEngineTest.java  |   9 +-
 .../solr/common/cloud/rule/ImplicitSnitch.java  | 163 ++++++++++++++++++
 .../solr/common/cloud/rule/RemoteCallback.java  |  23 +++
 .../apache/solr/common/cloud/rule/Snitch.java   |  34 ++++
 .../solr/common/cloud/rule/SnitchContext.java   | 103 ++++++++++++
 .../solr/common/cloud/rule/package-info.java    |  23 +++
 14 files changed, 488 insertions(+), 348 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/063d624c/solr/core/src/java/org/apache/solr/cloud/rule/ImplicitSnitch.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/rule/ImplicitSnitch.java b/solr/core/src/java/org/apache/solr/cloud/rule/ImplicitSnitch.java
index 09f8e2c..00e27c3 100644
--- a/solr/core/src/java/org/apache/solr/cloud/rule/ImplicitSnitch.java
+++ b/solr/core/src/java/org/apache/solr/cloud/rule/ImplicitSnitch.java
@@ -14,91 +14,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.solr.cloud.rule;
 
 import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.net.InetAddress;
 import java.nio.file.Files;
 import java.nio.file.Path;
-import java.util.ArrayList;
-import java.util.Collections;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
-import java.util.Set;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
 
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.handler.admin.CoreAdminHandler;
 import org.apache.solr.request.SolrQueryRequest;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class ImplicitSnitch extends Snitch implements CoreAdminHandler.Invocable {
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-  public static final Pattern hostAndPortPattern = Pattern.compile("(?:https?://)?([^:]+):(\\d+)");
-
-  //well known tags
-  public static final String NODE = "node";
-  public static final String PORT = "port";
-  public static final String HOST = "host";
-  public static final String CORES = "cores";
-  public static final String DISK = "freedisk";
-  public static final String ROLE = "role";
-  public static final String SYSPROP = "sysprop.";
-  public static final List<String> IP_SNITCHES = ImmutableList.of("ip_1", "ip_2", "ip_3", "ip_4");
-
-  public static final Set<String> tags = ImmutableSet.<String>builder().add(NODE, PORT, HOST, CORES, DISK, ROLE).addAll(IP_SNITCHES).build();
-
 
+import static org.apache.solr.common.cloud.rule.ImplicitSnitch.CORES;
+import static org.apache.solr.common.cloud.rule.ImplicitSnitch.DISK;
+import static org.apache.solr.common.cloud.rule.ImplicitSnitch.SYSPROP;
 
-  @Override
-  public void getTags(String solrNode, Set<String> requestedTags, SnitchContext ctx) {
-    if (requestedTags.contains(NODE)) ctx.getTags().put(NODE, solrNode);
-    if (requestedTags.contains(HOST)) {
-      Matcher hostAndPortMatcher = hostAndPortPattern.matcher(solrNode);
-      if (hostAndPortMatcher.find()) ctx.getTags().put(HOST, hostAndPortMatcher.group(1));
-    }
-    if (requestedTags.contains(PORT)) {
-      Matcher hostAndPortMatcher = hostAndPortPattern.matcher(solrNode);
-      if (hostAndPortMatcher.find()) ctx.getTags().put(PORT, hostAndPortMatcher.group(2));
-    }
-    if (requestedTags.contains(ROLE)) fillRole(solrNode, ctx);
-    addIpTags(solrNode, requestedTags, ctx);
-
-    ModifiableSolrParams params = new ModifiableSolrParams();
-    if (requestedTags.contains(CORES)) params.add(CORES, "1");
-    if (requestedTags.contains(DISK)) params.add(DISK, "1");
-    for (String tag : requestedTags) {
-      if (tag.startsWith(SYSPROP)) params.add(SYSPROP, tag.substring(SYSPROP.length()));
-    }
-    if (params.size() > 0) ctx.invokeRemote(solrNode, params, ImplicitSnitch.class.getName(), null);
-  }
-
-  private void fillRole(String solrNode, SnitchContext ctx) {
-    Map roles = (Map) ctx.retrieve(ZkStateReader.ROLES); // we don't want to hit the ZK for each node
-    // so cache and reuse
-    if(roles == null) roles = ctx.getZkJson(ZkStateReader.ROLES);
-    ctx.store(ZkStateReader.ROLES, roles == null ? Collections.emptyMap() : roles);
-    if (roles != null) {
-      for (Object o : roles.entrySet()) {
-        Map.Entry e = (Map.Entry) o;
-        if (e.getValue() instanceof List) {
-          if(((List) e.getValue()).contains(solrNode)) {
-            ctx.getTags().put(ROLE, e.getKey());
-            break;
-          }
-        }
-      }
-    }
-  }
+//this is the server-side component which provides the tag values
+public class ImplicitSnitch implements CoreAdminHandler.Invocable {
 
   static long getUsableSpaceInGB(Path path) throws IOException {
     long space = Files.getFileStore(path).getUsableSpace();
@@ -106,6 +40,7 @@ public class ImplicitSnitch extends Snitch implements CoreAdminHandler.Invocable
     return spaceInGB;
   }
 
+  @Override
   public Map<String, Object> invoke(SolrQueryRequest req) {
     Map<String, Object> result = new HashMap<>();
     CoreContainer cc = (CoreContainer) req.getContext().get(CoreContainer.class.getName());
@@ -127,70 +62,4 @@ public class ImplicitSnitch extends Snitch implements CoreAdminHandler.Invocable
     return result;
   }
 
-  private static final String HOST_FRAG_SEPARATOR_REGEX = "\\.";
-
-  @Override
-  public boolean isKnownTag(String tag) {
-    return tags.contains(tag) ||
-        tag.startsWith(SYSPROP);
-  }
-
-  private void addIpTags(String solrNode, Set<String> requestedTags, SnitchContext context) {
-
-    List<String> requestedHostTags = new ArrayList<>();
-    for (String tag : requestedTags) {
-      if (IP_SNITCHES.contains(tag)) {
-        requestedHostTags.add(tag);
-      }
-    }
-
-    if (requestedHostTags.isEmpty()) {
-      return;
-    }
-
-    String[] ipFragments = getIpFragments(solrNode);
-
-    if (ipFragments == null) {
-      return;
-    }
-
-    int ipSnitchCount = IP_SNITCHES.size();
-    for (int i = 0; i < ipSnitchCount; i++) {
-      String currentTagValue = ipFragments[i];
-      String currentTagKey = IP_SNITCHES.get(ipSnitchCount - i - 1);
-
-      if (requestedHostTags.contains(currentTagKey)) {
-        context.getTags().put(currentTagKey, currentTagValue);
-      }
-
-    }
-
-  }
-
-  private String[] getIpFragments(String solrNode) {
-    Matcher hostAndPortMatcher = hostAndPortPattern.matcher(solrNode);
-    if (hostAndPortMatcher.find()) {
-      String host = hostAndPortMatcher.group(1);
-      if (host != null) {
-        String ip = getHostIp(host);
-        if (ip != null) {
-          return ip.split(HOST_FRAG_SEPARATOR_REGEX); //IPv6 support will be provided by SOLR-8523
-        }
-      }
-    }
-
-    log.warn("Failed to match host IP address from node URL [{}] using regex [{}]", solrNode, hostAndPortPattern.pattern());
-    return null;
-  }
-
-  protected String getHostIp(String host) {
-    try {
-      InetAddress address = InetAddress.getByName(host);
-      return address.getHostAddress();
-    } catch (Exception e) {
-      log.warn("Failed to get IP address from host [{}], with exception [{}] ", host, e);
-      return null;
-    }
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/063d624c/solr/core/src/java/org/apache/solr/cloud/rule/RemoteCallback.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/rule/RemoteCallback.java b/solr/core/src/java/org/apache/solr/cloud/rule/RemoteCallback.java
deleted file mode 100644
index ca3662e..0000000
--- a/solr/core/src/java/org/apache/solr/cloud/rule/RemoteCallback.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.cloud.rule;
-
-import java.util.Map;
-
-public interface RemoteCallback {
-  void remoteCallback(SnitchContext ctx, Map<String, Object> returnedVal);
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/063d624c/solr/core/src/java/org/apache/solr/cloud/rule/ReplicaAssigner.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/rule/ReplicaAssigner.java b/solr/core/src/java/org/apache/solr/cloud/rule/ReplicaAssigner.java
index b1a1f2c..4ecda47 100644
--- a/solr/core/src/java/org/apache/solr/cloud/rule/ReplicaAssigner.java
+++ b/solr/core/src/java/org/apache/solr/cloud/rule/ReplicaAssigner.java
@@ -36,6 +36,9 @@ import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.rule.ImplicitSnitch;
+import org.apache.solr.common.cloud.rule.Snitch;
+import org.apache.solr.common.cloud.rule.SnitchContext;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.CoreContainer;
@@ -360,10 +363,7 @@ public class ReplicaAssigner {
       return myTags;
     }
 
-    @Override
-    public CoreContainer getCoreContainer() {
-      return cc;
-    }
+
   }
 
   /**
@@ -399,7 +399,7 @@ public class ReplicaAssigner {
       //now use the Snitch to get the tags
       for (SnitchInfoImpl info : snitches.values()) {
         if (!info.myTags.isEmpty()) {
-          SnitchContext context = getSnitchCtx(node, info);
+          SnitchContext context = getSnitchCtx(node, info, cc);
           info.nodeVsContext.put(node, context);
           try {
             info.snitch.getTags(node, info.myTags, context);
@@ -418,7 +418,7 @@ public class ReplicaAssigner {
         if (context.exception != null) {
           failedNodes.put(node, context);
           participatingLiveNodes.remove(node);
-          log.warn("Not all tags were obtained from node " + node);
+          log.warn("Not all tags were obtained from node " + node, context.exception);
           context.exception = new SolrException(SolrException.ErrorCode.SERVER_ERROR,
               "Not all tags were obtained from node " + node);
         } else {
@@ -443,8 +443,9 @@ public class ReplicaAssigner {
   }
 
   private Map<String, Object> snitchSession = new HashMap<>();
-  protected SnitchContext getSnitchCtx( String node, SnitchInfoImpl info) {
-    return new SnitchContext(info, node, snitchSession);
+
+  protected SnitchContext getSnitchCtx(String node, SnitchInfoImpl info, CoreContainer cc) {
+    return new ServerSnitchContext(info, node, snitchSession, cc);
   }
 
   public static void verifySnitchConf(CoreContainer cc, List snitchConf) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/063d624c/solr/core/src/java/org/apache/solr/cloud/rule/Rule.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/rule/Rule.java b/solr/core/src/java/org/apache/solr/cloud/rule/Rule.java
index 8db6ac3..97947cf 100644
--- a/solr/core/src/java/org/apache/solr/cloud/rule/Rule.java
+++ b/solr/core/src/java/org/apache/solr/cloud/rule/Rule.java
@@ -25,7 +25,7 @@ import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.Utils;
 
-import static org.apache.solr.cloud.rule.ImplicitSnitch.CORES;
+import static org.apache.solr.common.cloud.rule.ImplicitSnitch.CORES;
 import static org.apache.solr.cloud.rule.Rule.MatchStatus.CANNOT_ASSIGN_FAIL;
 import static org.apache.solr.cloud.rule.Rule.MatchStatus.NODE_CAN_BE_ASSIGNED;
 import static org.apache.solr.cloud.rule.Rule.MatchStatus.NOT_APPLICABLE;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/063d624c/solr/core/src/java/org/apache/solr/cloud/rule/ServerSnitchContext.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/rule/ServerSnitchContext.java b/solr/core/src/java/org/apache/solr/cloud/rule/ServerSnitchContext.java
new file mode 100644
index 0000000..2d526fd
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cloud/rule/ServerSnitchContext.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.cloud.rule;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.Map;
+
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.impl.BinaryResponseParser;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.GenericSolrRequest;
+import org.apache.solr.client.solrj.response.SimpleSolrResponse;
+import org.apache.solr.common.cloud.rule.RemoteCallback;
+import org.apache.solr.common.cloud.rule.SnitchContext;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.update.UpdateShardHandler;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.common.params.CoreAdminParams.ACTION;
+import static org.apache.solr.common.params.CoreAdminParams.CoreAdminAction.INVOKE;
+
+public class ServerSnitchContext extends SnitchContext {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  final CoreContainer coreContainer;
+  public ServerSnitchContext(SnitchInfo perSnitch,
+                             String node, Map<String, Object> session,
+                             CoreContainer coreContainer) {
+    super(perSnitch, node, session);
+    this.coreContainer = coreContainer;
+  }
+
+
+  public  Map getZkJson(String path) {
+    if (coreContainer.isZooKeeperAware()) {
+      try {
+        byte[] data = coreContainer.getZkController().getZkClient().getData(path, null, new Stat(), true);
+        if (data == null) return null;
+        return (Map) Utils.fromJSON(data);
+      } catch (Exception e) {
+        log.warn("Unable to read from ZK path : " + path, e);
+        return null;
+
+      }
+    } else {
+      return null;
+    }
+
+  }
+
+  public void invokeRemote(String node, ModifiableSolrParams params, String klas, RemoteCallback callback) {
+    if (callback == null) callback = this;
+    String url = coreContainer.getZkController().getZkStateReader().getBaseUrlForNodeName(node);
+    params.add("class", klas);
+    params.add(ACTION, INVOKE.toString());
+    //todo batch all requests to the same server
+
+    try {
+      SimpleSolrResponse rsp = invoke(coreContainer.getUpdateShardHandler(), url, CommonParams.CORES_HANDLER_PATH, params);
+      Map<String, Object> returnedVal = (Map<String, Object>) rsp.getResponse().get(klas);
+      if(exception == null){
+//        log this
+      } else {
+        callback.remoteCallback(ServerSnitchContext.this,returnedVal);
+      }
+      callback.remoteCallback(this, returnedVal);
+    } catch (Exception e) {
+      log.error("Unable to invoke snitch counterpart", e);
+      exception = e;
+    }
+  }
+
+  public SimpleSolrResponse invoke(UpdateShardHandler shardHandler,  final String url, String path, SolrParams params)
+      throws IOException, SolrServerException {
+    GenericSolrRequest request = new GenericSolrRequest(SolrRequest.METHOD.GET, path, params);
+    try (HttpSolrClient client = new HttpSolrClient.Builder(url).withHttpClient(shardHandler.getHttpClient())
+        .withResponseParser(new BinaryResponseParser()).build()) {
+      NamedList<Object> rsp = client.request(request);
+      request.response.nl = rsp;
+      return request.response;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/063d624c/solr/core/src/java/org/apache/solr/cloud/rule/Snitch.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/rule/Snitch.java b/solr/core/src/java/org/apache/solr/cloud/rule/Snitch.java
index a469e19..e47184f 100644
--- a/solr/core/src/java/org/apache/solr/cloud/rule/Snitch.java
+++ b/solr/core/src/java/org/apache/solr/cloud/rule/Snitch.java
@@ -27,7 +27,7 @@ public abstract class Snitch {
   static Set<Class> WELL_KNOWN_SNITCHES = ImmutableSet.of(ImplicitSnitch.class);
 
 
-  public abstract void getTags(String solrNode, Set<String> requestedTags, SnitchContext ctx);
+  public abstract void getTags(String solrNode, Set<String> requestedTags, org.apache.solr.common.cloud.rule.SnitchContext ctx);
 
   public abstract boolean isKnownTag(String tag);
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/063d624c/solr/core/src/java/org/apache/solr/cloud/rule/SnitchContext.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/rule/SnitchContext.java b/solr/core/src/java/org/apache/solr/cloud/rule/SnitchContext.java
deleted file mode 100644
index bfb80c0..0000000
--- a/solr/core/src/java/org/apache/solr/cloud/rule/SnitchContext.java
+++ /dev/null
@@ -1,168 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.solr.cloud.rule;
-
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.solr.client.solrj.SolrRequest;
-import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.client.solrj.impl.BinaryResponseParser;
-import org.apache.solr.client.solrj.impl.HttpSolrClient;
-import org.apache.solr.client.solrj.request.GenericSolrRequest;
-import org.apache.solr.client.solrj.response.SimpleSolrResponse;
-import org.apache.solr.common.params.CommonParams;
-import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.solr.common.params.SolrParams;
-import org.apache.solr.common.util.NamedList;
-import org.apache.solr.common.util.Utils;
-import org.apache.solr.core.CoreContainer;
-import org.apache.solr.update.UpdateShardHandler;
-import org.apache.zookeeper.data.Stat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.apache.solr.common.params.CoreAdminParams.ACTION;
-import static org.apache.solr.common.params.CoreAdminParams.CoreAdminAction.INVOKE;
-
-/**
- * This is the context provided to the snitches to interact with the system. This is a per-node-per-snitch
- * instance.
- */
-public class SnitchContext implements RemoteCallback {
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  private final Map<String, Object> tags = new HashMap<>();
-  private String node;
-  private Map<String, Object> session;
-  final SnitchInfo snitchInfo;
-  Exception exception;
-
-
-  SnitchContext(SnitchInfo perSnitch, String node, Map<String, Object> session) {
-    this.snitchInfo = perSnitch;
-    this.node = node;
-    this.session = session;
-  }
-
-  public SnitchInfo getSnitchInfo() {
-    return snitchInfo;
-  }
-
-  public Map<String, Object> getTags() {
-    return tags;
-  }
-
-  public void store(String s, Object val) {
-    if (session != null) session.put(s, val);
-
-  }
-
-  public Object retrieve(String s) {
-    return session != null ? session.get(s) : null;
-
-  }
-
-  public Map getZkJson(String path) {
-    if (snitchInfo.getCoreContainer().isZooKeeperAware()) {
-      try {
-        byte[] data = snitchInfo.getCoreContainer().getZkController().getZkClient().getData(path, null, new Stat(), true);
-        if (data == null) return null;
-        return (Map) Utils.fromJSON(data);
-      } catch (Exception e) {
-        log.warn("Unable to read from ZK path : " + path, e);
-        return null;
-
-      }
-    } else {
-      return null;
-    }
-
-  }
-
-  public String getNode() {
-    return node;
-  }
-
-  /**
-   * make a call to solrnode/admin/cores with the given params and give a callback. This is designed to be
-   * asynchronous because the system would want to batch the calls made to any given node
-   *
-   * @param node     The node for which this call is made
-   * @param params   The params to be passed to the Snitch counterpart
-   * @param klas     The  name of the class to be invoked in the remote node
-   * @param callback The callback to be called when the response is obtained from remote node.
-   *                 If this is passed as null the entire response map will be added as tags
-   */
-  public void invokeRemote(String node, ModifiableSolrParams params, String klas, RemoteCallback callback) {
-    if (callback == null) callback = this;
-    String url = snitchInfo.getCoreContainer().getZkController().getZkStateReader().getBaseUrlForNodeName(node);
-    params.add("class", klas);
-    params.add(ACTION, INVOKE.toString());
-    //todo batch all requests to the same server
-
-    try {
-      SimpleSolrResponse rsp = invoke(snitchInfo.getCoreContainer().getUpdateShardHandler(), url, CommonParams.CORES_HANDLER_PATH, params);
-      Map<String, Object> returnedVal = (Map<String, Object>) rsp.getResponse().get(klas);
-      if(exception == null){
-//        log this
-      } else {
-        callback.remoteCallback(SnitchContext.this,returnedVal);
-      }
-      callback.remoteCallback(this, returnedVal);
-    } catch (Exception e) {
-      log.error("Unable to invoke snitch counterpart", e);
-      exception = e;
-    }
-  }
-
-
-  public SimpleSolrResponse invoke(UpdateShardHandler shardHandler,  final String url, String path, SolrParams params)
-      throws IOException, SolrServerException {
-    GenericSolrRequest request = new GenericSolrRequest(SolrRequest.METHOD.GET, path, params);
-    try (HttpSolrClient client = new HttpSolrClient.Builder(url).withHttpClient(shardHandler.getHttpClient())
-        .withResponseParser(new BinaryResponseParser()).build()) {
-      NamedList<Object> rsp = client.request(request);
-      request.response.nl = rsp;
-      return request.response;
-    }
-  }
-
-
-  @Override
-  public void remoteCallback(SnitchContext ctx, Map<String, Object> returnedVal) {
-    tags.putAll(returnedVal);
-  }
-
-  public String getErrMsg() {
-    return exception == null ? null : exception.getMessage();
-  }
-
-  public static abstract class SnitchInfo {
-    private final Map<String, Object> conf;
-
-    SnitchInfo(Map<String, Object> conf) {
-      this.conf = conf;
-    }
-
-    public abstract Set<String> getTagNames();
-
-    public abstract CoreContainer getCoreContainer();
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/063d624c/solr/core/src/test/org/apache/solr/cloud/rule/ImplicitSnitchTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/rule/ImplicitSnitchTest.java b/solr/core/src/test/org/apache/solr/cloud/rule/ImplicitSnitchTest.java
index 424b6a5..94ca771 100644
--- a/solr/core/src/test/org/apache/solr/cloud/rule/ImplicitSnitchTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/rule/ImplicitSnitchTest.java
@@ -21,6 +21,8 @@ import java.util.HashMap;
 import java.util.Map;
 
 import com.google.common.collect.Sets;
+import org.apache.solr.common.cloud.rule.ImplicitSnitch;
+import org.apache.solr.common.cloud.rule.SnitchContext;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
@@ -45,7 +47,7 @@ public class ImplicitSnitchTest {
   @Before
   public void beforeImplicitSnitchTest() {
     snitch = new ImplicitSnitch();
-    context = new SnitchContext(null, null, new HashMap<>());
+    context = new ServerSnitchContext(null, null, new HashMap<>(),null);
   }
 
 
@@ -82,7 +84,7 @@ public class ImplicitSnitchTest {
   public void testGetTags_withIPv4RequestedTags_ip2_and_ip4_returns_two_tags() throws Exception {
     String node = "192.168.1.2:8983_solr";
 
-    SnitchContext context = new SnitchContext(null, node, new HashMap<>());
+    SnitchContext context = new ServerSnitchContext(null, node, new HashMap<>(),null);
     snitch.getTags(node, Sets.newHashSet(IP_2, IP_4), context);
 
     Map<String, Object> tags = context.getTags();
@@ -95,7 +97,7 @@ public class ImplicitSnitchTest {
   public void testGetTags_with_wrong_ipv4_format_ip_returns_nothing() throws Exception {
     String node = "192.168.1.2.1:8983_solr";
 
-    SnitchContext context = new SnitchContext(null, node, new HashMap<>());
+    SnitchContext context = new ServerSnitchContext(null, node, new HashMap<>(),null);
     snitch.getTags(node, Sets.newHashSet(IP_1), context);
 
     Map<String, Object> tags = context.getTags();
@@ -107,7 +109,7 @@ public class ImplicitSnitchTest {
   public void testGetTags_with_correct_ipv6_format_ip_returns_nothing() throws Exception {
     String node = "[0:0:0:0:0:0:0:1]:8983_solr";
 
-    SnitchContext context = new SnitchContext(null, node, new HashMap<>());
+    SnitchContext context = new ServerSnitchContext(null, node, new HashMap<>(),null);
     snitch.getTags(node, Sets.newHashSet(IP_1), context);
 
     Map<String, Object> tags = context.getTags();
@@ -130,7 +132,7 @@ public class ImplicitSnitchTest {
   public void testGetTags_withAllHostNameRequestedTags_returns_all_Tags() throws Exception {
     String node = "serv01.dc01.london.uk.apache.org:8983_solr";
 
-    SnitchContext context = new SnitchContext(null, node, new HashMap<>());
+    SnitchContext context = new ServerSnitchContext(null, node, new HashMap<>(),null);
     //We need mocking here otherwise, we would need proper DNS entry for this test to pass
     ImplicitSnitch mockedSnitch = Mockito.spy(snitch);
     when(mockedSnitch.getHostIp(anyString())).thenReturn("10.11.12.13");
@@ -149,7 +151,7 @@ public class ImplicitSnitchTest {
   public void testGetTags_withHostNameRequestedTag_ip3_returns_1_tag() throws Exception {
     String node = "serv01.dc01.london.uk.apache.org:8983_solr";
 
-    SnitchContext context = new SnitchContext(null, node, new HashMap<>());
+    SnitchContext context = new ServerSnitchContext(null, node, new HashMap<>(),null);
     //We need mocking here otherwise, we would need proper DNS entry for this test to pass
     ImplicitSnitch mockedSnitch = Mockito.spy(snitch);
     when(mockedSnitch.getHostIp(anyString())).thenReturn("10.11.12.13");
@@ -164,7 +166,7 @@ public class ImplicitSnitchTest {
   public void testGetTags_withHostNameRequestedTag_ip99999_returns_nothing() throws Exception {
     String node = "serv01.dc01.london.uk.apache.org:8983_solr";
 
-    SnitchContext context = new SnitchContext(null, node, new HashMap<>());
+    SnitchContext context = new ServerSnitchContext(null, node, new HashMap<>(),null);
     //We need mocking here otherwise, we would need proper DNS entry for this test to pass
     ImplicitSnitch mockedSnitch = Mockito.spy(snitch);
     when(mockedSnitch.getHostIp(anyString())).thenReturn("10.11.12.13");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/063d624c/solr/core/src/test/org/apache/solr/cloud/rule/RuleEngineTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/rule/RuleEngineTest.java b/solr/core/src/test/org/apache/solr/cloud/rule/RuleEngineTest.java
index 01dd868..7c33541 100644
--- a/solr/core/src/test/org/apache/solr/cloud/rule/RuleEngineTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/rule/RuleEngineTest.java
@@ -30,7 +30,10 @@ import com.google.common.collect.ImmutableList;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.cloud.rule.ReplicaAssigner.Position;
 import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.cloud.rule.Snitch;
+import org.apache.solr.common.cloud.rule.SnitchContext;
 import org.apache.solr.common.util.Utils;
+import org.apache.solr.core.CoreContainer;
 import org.junit.Test;
 
 import static java.util.Collections.singletonList;
@@ -91,8 +94,8 @@ public class RuleEngineTest extends SolrTestCaseJ4{
         new HashMap(), new ArrayList<>(MockSnitch.nodeVsTags.keySet()), null, null) {
 
       @Override
-      protected SnitchContext getSnitchCtx(String node, SnitchInfoImpl info) {
-        return new SnitchContext(info, node, snitchSession){
+      protected SnitchContext getSnitchCtx(String node, SnitchInfoImpl info, CoreContainer cc) {
+        return new ServerSnitchContext(info, node, snitchSession,cc){
           @Override
           public Map getZkJson(String path) {
             if(ZkStateReader.ROLES.equals(path)){
@@ -100,8 +103,10 @@ public class RuleEngineTest extends SolrTestCaseJ4{
             }
             return null;
           }
+
         };
       }
+
     };
     mapping = replicaAssigner.getNodeMappings();
     assertNotNull(mapping);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/063d624c/solr/solrj/src/java/org/apache/solr/common/cloud/rule/ImplicitSnitch.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/rule/ImplicitSnitch.java b/solr/solrj/src/java/org/apache/solr/common/cloud/rule/ImplicitSnitch.java
new file mode 100644
index 0000000..b177394
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/rule/ImplicitSnitch.java
@@ -0,0 +1,163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.common.cloud.rule;
+
+import java.lang.invoke.MethodHandles;
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+//This is the client-side component of the snitch
+public class ImplicitSnitch extends Snitch {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public static final Pattern hostAndPortPattern = Pattern.compile("(?:https?://)?([^:]+):(\\d+)");
+
+  //well known tags
+  public static final String NODE = "node";
+  public static final String PORT = "port";
+  public static final String HOST = "host";
+  public static final String CORES = "cores";
+  public static final String DISK = "freedisk";
+  public static final String ROLE = "role";
+  public static final String SYSPROP = "sysprop.";
+  public static final List<String> IP_SNITCHES = ImmutableList.of("ip_1", "ip_2", "ip_3", "ip_4");
+
+  public static final Set<String> tags = ImmutableSet.<String>builder().add(NODE, PORT, HOST, CORES, DISK, ROLE).addAll(IP_SNITCHES).build();
+
+
+
+  @Override
+  public void getTags(String solrNode, Set<String> requestedTags, SnitchContext ctx) {
+    if (requestedTags.contains(NODE)) ctx.getTags().put(NODE, solrNode);
+    if (requestedTags.contains(HOST)) {
+      Matcher hostAndPortMatcher = hostAndPortPattern.matcher(solrNode);
+      if (hostAndPortMatcher.find()) ctx.getTags().put(HOST, hostAndPortMatcher.group(1));
+    }
+    if (requestedTags.contains(PORT)) {
+      Matcher hostAndPortMatcher = hostAndPortPattern.matcher(solrNode);
+      if (hostAndPortMatcher.find()) ctx.getTags().put(PORT, hostAndPortMatcher.group(2));
+    }
+    if (requestedTags.contains(ROLE)) fillRole(solrNode, ctx);
+    addIpTags(solrNode, requestedTags, ctx);
+
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    if (requestedTags.contains(CORES)) params.add(CORES, "1");
+    if (requestedTags.contains(DISK)) params.add(DISK, "1");
+    for (String tag : requestedTags) {
+      if (tag.startsWith(SYSPROP)) params.add(SYSPROP, tag.substring(SYSPROP.length()));
+    }
+    if (params.size() > 0) ctx.invokeRemote(solrNode, params, "org.apache.solr.cloud.rule.ImplicitSnitch", null);
+  }
+
+  private void fillRole(String solrNode, SnitchContext ctx) {
+    Map roles = (Map) ctx.retrieve(ZkStateReader.ROLES); // we don't want to hit the ZK for each node
+    // so cache and reuse
+    if(roles == null) roles = ctx.getZkJson(ZkStateReader.ROLES);
+    ctx.store(ZkStateReader.ROLES, roles == null ? Collections.emptyMap() : roles);
+    if (roles != null) {
+      for (Object o : roles.entrySet()) {
+        Map.Entry e = (Map.Entry) o;
+        if (e.getValue() instanceof List) {
+          if(((List) e.getValue()).contains(solrNode)) {
+            ctx.getTags().put(ROLE, e.getKey());
+            break;
+          }
+        }
+      }
+    }
+  }
+
+  private static final String HOST_FRAG_SEPARATOR_REGEX = "\\.";
+
+  @Override
+  public boolean isKnownTag(String tag) {
+    return tags.contains(tag) ||
+        tag.startsWith(SYSPROP);
+  }
+
+  private void addIpTags(String solrNode, Set<String> requestedTags, SnitchContext context) {
+
+    List<String> requestedHostTags = new ArrayList<>();
+    for (String tag : requestedTags) {
+      if (IP_SNITCHES.contains(tag)) {
+        requestedHostTags.add(tag);
+      }
+    }
+
+    if (requestedHostTags.isEmpty()) {
+      return;
+    }
+
+    String[] ipFragments = getIpFragments(solrNode);
+
+    if (ipFragments == null) {
+      return;
+    }
+
+    int ipSnitchCount = IP_SNITCHES.size();
+    for (int i = 0; i < ipSnitchCount; i++) {
+      String currentTagValue = ipFragments[i];
+      String currentTagKey = IP_SNITCHES.get(ipSnitchCount - i - 1);
+
+      if (requestedHostTags.contains(currentTagKey)) {
+        context.getTags().put(currentTagKey, currentTagValue);
+      }
+
+    }
+
+  }
+
+  private String[] getIpFragments(String solrNode) {
+    Matcher hostAndPortMatcher = hostAndPortPattern.matcher(solrNode);
+    if (hostAndPortMatcher.find()) {
+      String host = hostAndPortMatcher.group(1);
+      if (host != null) {
+        String ip = getHostIp(host);
+        if (ip != null) {
+          return ip.split(HOST_FRAG_SEPARATOR_REGEX); //IPv6 support will be provided by SOLR-8523
+        }
+      }
+    }
+
+    log.warn("Failed to match host IP address from node URL [{}] using regex [{}]", solrNode, hostAndPortPattern.pattern());
+    return null;
+  }
+
+  public String getHostIp(String host) {
+    try {
+      InetAddress address = InetAddress.getByName(host);
+      return address.getHostAddress();
+    } catch (Exception e) {
+      log.warn("Failed to get IP address from host [{}], with exception [{}] ", host, e);
+      return null;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/063d624c/solr/solrj/src/java/org/apache/solr/common/cloud/rule/RemoteCallback.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/rule/RemoteCallback.java b/solr/solrj/src/java/org/apache/solr/common/cloud/rule/RemoteCallback.java
new file mode 100644
index 0000000..a5a4e98
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/rule/RemoteCallback.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.
+ */
+package org.apache.solr.common.cloud.rule;
+
+import java.util.Map;
+
+public interface RemoteCallback {
+  void remoteCallback(SnitchContext ctx, Map<String, Object> returnedVal);
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/063d624c/solr/solrj/src/java/org/apache/solr/common/cloud/rule/Snitch.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/rule/Snitch.java b/solr/solrj/src/java/org/apache/solr/common/cloud/rule/Snitch.java
new file mode 100644
index 0000000..1d78efc
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/rule/Snitch.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.common.cloud.rule;
+
+import java.util.Set;
+
+import com.google.common.collect.ImmutableSet;
+
+/**
+ *
+ */
+public abstract class Snitch {
+  public static final Set<Class> WELL_KNOWN_SNITCHES = ImmutableSet.of(ImplicitSnitch.class);
+
+
+  public abstract void getTags(String solrNode, Set<String> requestedTags, SnitchContext ctx);
+
+  public abstract boolean isKnownTag(String tag);
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/063d624c/solr/solrj/src/java/org/apache/solr/common/cloud/rule/SnitchContext.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/rule/SnitchContext.java b/solr/solrj/src/java/org/apache/solr/common/cloud/rule/SnitchContext.java
new file mode 100644
index 0000000..3bb081b
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/rule/SnitchContext.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.common.cloud.rule;
+
+import java.lang.invoke.MethodHandles;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This is the context provided to the snitches to interact with the system. This is a per-node-per-snitch
+ * instance.
+ */
+public abstract class SnitchContext implements RemoteCallback {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  private final Map<String, Object> tags = new HashMap<>();
+  private String node;
+  private Map<String, Object> session;
+  public final SnitchInfo snitchInfo;
+  public Exception exception;
+
+
+  public SnitchContext(SnitchInfo perSnitch, String node, Map<String, Object> session) {
+    this.snitchInfo = perSnitch;
+    this.node = node;
+    this.session = session;
+  }
+
+  public SnitchInfo getSnitchInfo() {
+    return snitchInfo;
+  }
+
+  public Map<String, Object> getTags() {
+    return tags;
+  }
+
+  public void store(String s, Object val) {
+    if (session != null) session.put(s, val);
+
+  }
+
+  public Object retrieve(String s) {
+    return session != null ? session.get(s) : null;
+
+  }
+
+  public abstract Map getZkJson(String path) ;
+
+  public String getNode() {
+    return node;
+  }
+
+  /**
+   * make a call to solrnode/admin/cores with the given params and give a callback. This is designed to be
+   * asynchronous because the system would want to batch the calls made to any given node
+   *
+   * @param node     The node for which this call is made
+   * @param params   The params to be passed to the Snitch counterpart
+   * @param klas     The  name of the class to be invoked in the remote node
+   * @param callback The callback to be called when the response is obtained from remote node.
+   *                 If this is passed as null the entire response map will be added as tags
+   */
+  public abstract void invokeRemote(String node, ModifiableSolrParams params, String klas, RemoteCallback callback) ;
+
+
+  @Override
+  public void remoteCallback(SnitchContext ctx, Map<String, Object> returnedVal) {
+    tags.putAll(returnedVal);
+  }
+
+  public String getErrMsg() {
+    return exception == null ? null : exception.getMessage();
+  }
+
+  public static abstract class SnitchInfo {
+    private final Map<String, Object> conf;
+
+    protected SnitchInfo(Map<String, Object> conf) {
+      this.conf = conf;
+    }
+
+    public abstract Set<String> getTagNames();
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/063d624c/solr/solrj/src/java/org/apache/solr/common/cloud/rule/package-info.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/rule/package-info.java b/solr/solrj/src/java/org/apache/solr/common/cloud/rule/package-info.java
new file mode 100644
index 0000000..c91d889
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/rule/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.
+ */
+ 
+/** 
+ * Classes for managing Replica placement strategy when operating in <a href="http://wiki.apache.org/solr/SolrCloud">SolrCloud</a> mode.
+ */
+package org.apache.solr.common.cloud.rule;
+
+