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 2017/04/04 12:55:21 UTC

lucene-solr:feature/autoscaling: SOLR-10278: added classes to use Metrics to fetch node values such as cores freedisk

Repository: lucene-solr
Updated Branches:
  refs/heads/feature/autoscaling 3eb2321c8 -> 69acd5f98


SOLR-10278: added classes to use Metrics to fetch node values such as cores freedisk


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

Branch: refs/heads/feature/autoscaling
Commit: 69acd5f98b09753737745a5e2ac56a9e3e703424
Parents: 3eb2321
Author: Noble Paul <no...@apache.org>
Authored: Tue Apr 4 22:25:08 2017 +0930
Committer: Noble Paul <no...@apache.org>
Committed: Tue Apr 4 22:25:08 2017 +0930

----------------------------------------------------------------------
 .../cloud/autoscaling/AutoScalingSnitch.java    | 80 ++++++++++++++++++++
 .../autoscaling/ServerClusterDataProvider.java  | 80 ++++++++++++++++++++
 .../solr/cloud/rule/ServerSnitchContext.java    |  7 +-
 .../solr/client/solrj/impl/CloudSolrClient.java |  9 +--
 .../impl/ZkClientClusterStateProvider.java      |  9 +--
 .../solr/common/cloud/rule/ImplicitSnitch.java  |  8 +-
 .../solr/common/cloud/rule/SnitchContext.java   |  4 -
 .../src/java/org/apache/solr/recipe/Policy.java |  6 +-
 .../recipe/SolrClientClusterDataProvider.java   | 71 -----------------
 .../org/apache/solr/recipe/package-info.java    |  2 +-
 .../solrj/impl/CloudSolrClientCacheTest.java    |  5 --
 11 files changed, 179 insertions(+), 102 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/69acd5f9/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScalingSnitch.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScalingSnitch.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScalingSnitch.java
new file mode 100644
index 0000000..fb489b1
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/AutoScalingSnitch.java
@@ -0,0 +1,80 @@
+/*
+ * 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.autoscaling;
+
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.solr.client.solrj.response.SimpleSolrResponse;
+import org.apache.solr.cloud.rule.ServerSnitchContext;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.rule.ImplicitSnitch;
+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.util.StrUtils;
+import org.apache.solr.common.util.Utils;
+
+//uses metrics API to get node information
+public class AutoScalingSnitch extends ImplicitSnitch {
+
+
+  @Override
+  protected void getRemoteInfo(String solrNode, Set<String> requestedTags, SnitchContext ctx) {
+    ServerSnitchContext snitchContext = (ServerSnitchContext) ctx;
+    List<String> groups = new ArrayList<>();
+    List<String> prefixes = new ArrayList<>();
+    if (requestedTags.contains(DISK)) {
+      groups.add("solr.node");
+      prefixes.add("CONTAINER.fs.usableSpace");
+    }
+    if (requestedTags.contains(CORES)) {
+      groups.add("solr.core");
+      prefixes.add("CORE.coreName");
+    }
+    if(groups.isEmpty() || prefixes.isEmpty()) return;
+
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.add("group", StrUtils.join(groups, ','));
+    params.add("prefix", StrUtils.join(prefixes,','));
+
+    try {
+      SimpleSolrResponse rsp = snitchContext.invoke(solrNode, CommonParams.METRICS_PATH, params);
+      Map m = rsp.nl.asMap(4);
+      if(requestedTags.contains(DISK)){
+        Number n = (Number) Utils.getObjectByPath(m,true, "metrics/solr.node/CONTAINER.fs.usableSpace");
+        if(n != null) ctx.getTags().put(DISK, n.longValue());
+      }
+      if(requestedTags.contains(CORES)){
+        int count = 0;
+        Map cores  = (Map) m.get("metrics");
+        for (Object o : cores.keySet()) {
+          if(o.toString().startsWith("solr.core.")) count++;
+        }
+        ctx.getTags().put(CORES, count);
+      }
+
+    } catch (Exception e) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "", e);
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/69acd5f9/solr/core/src/java/org/apache/solr/cloud/autoscaling/ServerClusterDataProvider.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ServerClusterDataProvider.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ServerClusterDataProvider.java
new file mode 100644
index 0000000..1267f8f
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ServerClusterDataProvider.java
@@ -0,0 +1,80 @@
+/*
+ * 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.autoscaling;
+
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.solr.cloud.rule.ServerSnitchContext;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.recipe.Policy.ClusterDataProvider;
+import org.apache.solr.recipe.Policy.ReplicaInfo;
+
+public class ServerClusterDataProvider implements ClusterDataProvider {
+
+  private final CoreContainer coreContainer;
+  private Set<String> liveNodes;
+  private Map<String,Object> snitchSession = new HashMap<>();
+  private final Map<String, Map<String, Map<String, List<ReplicaInfo>>>> data = new HashMap<>();
+
+  public ServerClusterDataProvider(CoreContainer coreContainer) {
+    this.coreContainer = coreContainer;
+    ClusterState clusterState = coreContainer.getZkController().getZkStateReader().getClusterState();
+    this.liveNodes = clusterState.getLiveNodes();
+    Map<String, ClusterState.CollectionRef> all = clusterState.getCollectionStates();
+    all.forEach((collName, ref) -> {
+      DocCollection coll = ref.get();
+      if (coll == null) return;
+      coll.forEachReplica((shard, replica) -> {
+        Map<String, Map<String, List<ReplicaInfo>>> nodeData = data.get(replica.getNodeName());
+        if (nodeData == null) data.put(replica.getNodeName(), nodeData = new HashMap<>());
+        Map<String, List<ReplicaInfo>> collData = nodeData.get(collName);
+        if (collData == null) nodeData.put(collName, collData = new HashMap<>());
+        List<ReplicaInfo> replicas = collData.get(shard);
+        if (replicas == null) collData.put(shard, replicas = new ArrayList<>());
+        replicas.add(new ReplicaInfo(replica.getName(), new HashMap<>()));
+      });
+    });
+  }
+
+  @Override
+  public Map<String, Object> getNodeValues(String node, Collection<String> keys) {
+    AutoScalingSnitch  snitch = new AutoScalingSnitch();
+    ServerSnitchContext ctx = new ServerSnitchContext(null, node, snitchSession, coreContainer);
+    snitch.getRemoteInfo(node, new HashSet<>(keys), ctx);
+    return ctx.getTags();
+  }
+
+  @Override
+  public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
+    return data.get(node);//todo fill other details
+  }
+
+  @Override
+  public Collection<String> getNodes() {
+    return liveNodes;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/69acd5f9/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
index 2d526fd..446c80f 100644
--- a/solr/core/src/java/org/apache/solr/cloud/rule/ServerSnitchContext.java
+++ b/solr/core/src/java/org/apache/solr/cloud/rule/ServerSnitchContext.java
@@ -74,13 +74,12 @@ public class ServerSnitchContext extends SnitchContext {
 
   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);
+      SimpleSolrResponse rsp = invoke(node, CommonParams.CORES_HANDLER_PATH, params);
       Map<String, Object> returnedVal = (Map<String, Object>) rsp.getResponse().get(klas);
       if(exception == null){
 //        log this
@@ -94,8 +93,10 @@ public class ServerSnitchContext extends SnitchContext {
     }
   }
 
-  public SimpleSolrResponse invoke(UpdateShardHandler shardHandler,  final String url, String path, SolrParams params)
+  public SimpleSolrResponse invoke(String solrNode, String path, SolrParams params)
       throws IOException, SolrServerException {
+    String url = coreContainer.getZkController().getZkStateReader().getBaseUrlForNodeName(solrNode);
+    UpdateShardHandler shardHandler = coreContainer.getUpdateShardHandler();
     GenericSolrRequest request = new GenericSolrRequest(SolrRequest.METHOD.GET, path, params);
     try (HttpSolrClient client = new HttpSolrClient.Builder(url).withHttpClient(shardHandler.getHttpClient())
         .withResponseParser(new BinaryResponseParser()).build()) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/69acd5f9/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
index 4172f65..b219fac 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
@@ -60,7 +60,6 @@ import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.ToleratedUpdateError;
 import org.apache.solr.common.cloud.ClusterState;
-import org.apache.solr.common.cloud.ClusterState.CollectionRef;
 import org.apache.solr.common.cloud.CollectionStatePredicate;
 import org.apache.solr.common.cloud.CollectionStateWatcher;
 import org.apache.solr.common.cloud.DocCollection;
@@ -1446,7 +1445,7 @@ public class CloudSolrClient extends SolrClient {
           && !cacheEntry.shoulRetry()) return col;
     }
 
-    CollectionRef ref = getCollectionRef(collection);
+    ClusterState.CollectionRef ref = getCollectionRef(collection);
     if (ref == null) {
       //no such collection exists
       return null;
@@ -1481,7 +1480,7 @@ public class CloudSolrClient extends SolrClient {
     }
   }
 
-  CollectionRef getCollectionRef(String collection) {
+  ClusterState.CollectionRef getCollectionRef(String collection) {
     return stateProvider.getState(collection);
   }
 
@@ -1732,7 +1731,7 @@ public class CloudSolrClient extends SolrClient {
 
   public interface ClusterStateProvider extends Closeable {
 
-    CollectionRef getState(String collection);
+    ClusterState.CollectionRef getState(String collection);
 
     Set<String> liveNodes();
 
@@ -1742,8 +1741,6 @@ public class CloudSolrClient extends SolrClient {
 
     Map<String, Object> getClusterProperties();
 
-    Map<String,CollectionRef> getCollections();
-
     void connect();
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/69acd5f9/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ZkClientClusterStateProvider.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ZkClientClusterStateProvider.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ZkClientClusterStateProvider.java
index f0783e2..8ed1b5c 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ZkClientClusterStateProvider.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ZkClientClusterStateProvider.java
@@ -27,8 +27,6 @@ import java.util.Set;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.Aliases;
 import org.apache.solr.common.cloud.ClusterState;
-import org.apache.solr.common.cloud.ClusterState.CollectionRef;
-import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.cloud.ZooKeeperException;
 import org.apache.zookeeper.KeeperException;
@@ -54,7 +52,7 @@ public class ZkClientClusterStateProvider implements CloudSolrClient.ClusterStat
   }
 
   @Override
-  public CollectionRef getState(String collection) {
+  public ClusterState.CollectionRef getState(String collection) {
     return zkStateReader.getClusterState().getCollectionRef(collection);
   }
 
@@ -184,11 +182,6 @@ public class ZkClientClusterStateProvider implements CloudSolrClient.ClusterStat
   }
 
   @Override
-  public Map<String, CollectionRef> getCollections() {
-    return zkStateReader.getClusterState().getCollectionStates();
-  }
-
-  @Override
   public String toString() {
     return zkHost;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/69acd5f9/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
index 1ae618d..3cae572 100644
--- 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
@@ -46,6 +46,7 @@ public class ImplicitSnitch extends Snitch {
   public static final String CORES = "cores";
   public static final String DISK = "freedisk";
   public static final String ROLE = "role";
+  public static final String NODEROLE = "noderole";
   public static final String SYSPROP = "sysprop.";
   public static final List<String> IP_SNITCHES = Collections.unmodifiableList(Arrays.asList("ip_1", "ip_2", "ip_3", "ip_4"));
   public static final Set<String> tags = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(NODE, PORT, HOST, CORES, DISK, ROLE, "ip_1", "ip_2", "ip_3", "ip_4")));
@@ -61,9 +62,14 @@ public class ImplicitSnitch extends Snitch {
       Matcher hostAndPortMatcher = hostAndPortPattern.matcher(solrNode);
       if (hostAndPortMatcher.find()) ctx.getTags().put(PORT, hostAndPortMatcher.group(2));
     }
-    if (requestedTags.contains(ROLE)) fillRole(solrNode, ctx);
+    if (requestedTags.contains(ROLE) || requestedTags.contains(NODEROLE)) fillRole(solrNode, ctx);
+
     addIpTags(solrNode, requestedTags, ctx);
 
+    getRemoteInfo(solrNode, requestedTags, ctx);
+  }
+
+  protected void getRemoteInfo(String solrNode, Set<String> requestedTags, SnitchContext ctx) {
     ModifiableSolrParams params = new ModifiableSolrParams();
     if (requestedTags.contains(CORES)) params.add(CORES, "1");
     if (requestedTags.contains(DISK)) params.add(DISK, "1");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/69acd5f9/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
index 3bb081b..69a353e 100644
--- 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
@@ -44,10 +44,6 @@ public abstract class SnitchContext implements RemoteCallback {
     this.session = session;
   }
 
-  public SnitchInfo getSnitchInfo() {
-    return snitchInfo;
-  }
-
   public Map<String, Object> getTags() {
     return tags;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/69acd5f9/solr/solrj/src/java/org/apache/solr/recipe/Policy.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/recipe/Policy.java b/solr/solrj/src/java/org/apache/solr/recipe/Policy.java
index 0325378..0f6affa 100644
--- a/solr/solrj/src/java/org/apache/solr/recipe/Policy.java
+++ b/solr/solrj/src/java/org/apache/solr/recipe/Policy.java
@@ -201,11 +201,11 @@ public class Policy {
   }
 
 
-  static class ReplicaInfo implements MapWriter {
+  public static class ReplicaInfo implements MapWriter {
     final String name;
     Map<String, Object> variables;
 
-    ReplicaInfo(String name, Map<String, Object> vals) {
+    public ReplicaInfo(String name, Map<String, Object> vals) {
       this.name = name;
       this.variables = vals;
     }
@@ -217,7 +217,7 @@ public class Policy {
   }
 
 
-  interface ClusterDataProvider {
+  public interface ClusterDataProvider {
     Map<String, Object> getNodeValues(String node, Collection<String> keys);
 
     /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/69acd5f9/solr/solrj/src/java/org/apache/solr/recipe/SolrClientClusterDataProvider.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/recipe/SolrClientClusterDataProvider.java b/solr/solrj/src/java/org/apache/solr/recipe/SolrClientClusterDataProvider.java
deleted file mode 100644
index 295d051..0000000
--- a/solr/solrj/src/java/org/apache/solr/recipe/SolrClientClusterDataProvider.java
+++ /dev/null
@@ -1,71 +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.recipe;
-
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.solr.client.solrj.impl.CloudSolrClient.ClusterStateProvider;
-import org.apache.solr.common.cloud.ClusterState;
-import org.apache.solr.common.cloud.DocCollection;
-import org.apache.solr.recipe.Policy.ClusterDataProvider;
-import org.apache.solr.recipe.Policy.ReplicaInfo;
-
-public class SolrClientClusterDataProvider implements ClusterDataProvider {
-
-  private final ClusterStateProvider clusterStateProvider;
-  private final Map<String, Map<String, Map<String, List<ReplicaInfo>>>> data = new HashMap<>();
-
-  public SolrClientClusterDataProvider(ClusterStateProvider csp) {
-    this.clusterStateProvider = csp;
-    Map<String, ClusterState.CollectionRef> all = clusterStateProvider.getCollections();
-    all.forEach((collName, ref) -> {
-      DocCollection coll = ref.get();
-      if (coll == null) return;
-      coll.forEachReplica((shard, replica) -> {
-        Map<String, Map<String, List<ReplicaInfo>>> nodeData = data.get(replica.getNodeName());
-        if (nodeData == null) data.put(replica.getNodeName(), nodeData = new HashMap<>());
-        Map<String, List<ReplicaInfo>> collData = nodeData.get(collName);
-        if (collData == null) nodeData.put(collName, collData = new HashMap<>());
-        List<ReplicaInfo> replicas = collData.get(shard);
-        if (replicas == null) collData.put(shard, replicas = new ArrayList<>());
-        replicas.add(new ReplicaInfo(replica.getName(), new HashMap<>()));
-      });
-    });
-  }
-
-  @Override
-  public Map<String, Object> getNodeValues(String node, Collection<String> keys) {
-    //todo
-    return new HashMap<>();
-  }
-
-  @Override
-  public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
-    return data.get(node);//todo fill other details
-  }
-
-  @Override
-  public Collection<String> getNodes() {
-    return clusterStateProvider.liveNodes();
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/69acd5f9/solr/solrj/src/java/org/apache/solr/recipe/package-info.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/recipe/package-info.java b/solr/solrj/src/java/org/apache/solr/recipe/package-info.java
index d973015..7133e0f 100644
--- a/solr/solrj/src/java/org/apache/solr/recipe/package-info.java
+++ b/solr/solrj/src/java/org/apache/solr/recipe/package-info.java
@@ -16,7 +16,7 @@
  */
 
 /**
- * Common classes for recipe parsing filtering nodes & sorting
+ * Common classes for recipe parsing filtering nodes and sorting
  */
 
 package org.apache.solr.recipe;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/69acd5f9/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientCacheTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientCacheTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientCacheTest.java
index 51b24f8..6adbaae 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientCacheTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/CloudSolrClientCacheTest.java
@@ -150,11 +150,6 @@ public class CloudSolrClientCacheTest extends SolrTestCaseJ4 {
       public void connect() { }
 
       @Override
-      public Map<String, CollectionRef> getCollections() {
-        return Collections.emptyMap();
-      }
-
-      @Override
       public void close() throws IOException {
 
       }