You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by cp...@apache.org on 2017/06/02 10:56:40 UTC

[17/38] lucene-solr:jira/solr-8668: SOLR-10773: Add support for replica types in V2 API

SOLR-10773: Add support for replica types in V2 API


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

Branch: refs/heads/jira/solr-8668
Commit: 22c2ed070a821efa3c5a81ca03312d4034fcb878
Parents: 3291ef8
Author: Tomas Fernandez Lobbe <tf...@apache.org>
Authored: Wed May 31 13:53:34 2017 -0700
Committer: Tomas Fernandez Lobbe <tf...@apache.org>
Committed: Wed May 31 13:53:34 2017 -0700

----------------------------------------------------------------------
 .../org/apache/solr/cloud/AddReplicaCmd.java    |  3 +-
 .../resources/apispec/collections.Commands.json | 14 ++-
 .../collections.collection.shards.Commands.json |  5 ++
 .../org/apache/solr/cloud/TestPullReplica.java  | 89 ++++++++++++++++----
 .../org/apache/solr/cloud/TestTlogReplica.java  | 86 +++++++++++++++----
 .../solr/handler/admin/TestCollectionAPIs.java  | 38 ++++++++-
 6 files changed, 197 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22c2ed07/solr/core/src/java/org/apache/solr/cloud/AddReplicaCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/AddReplicaCmd.java b/solr/core/src/java/org/apache/solr/cloud/AddReplicaCmd.java
index 7338d9e..63acdd1 100644
--- a/solr/core/src/java/org/apache/solr/cloud/AddReplicaCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/AddReplicaCmd.java
@@ -22,6 +22,7 @@ import java.lang.invoke.MethodHandles;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.Locale;
 import java.util.Map;
 
 import org.apache.commons.lang.StringUtils;
@@ -72,7 +73,7 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
     String node = message.getStr(CoreAdminParams.NODE);
     String shard = message.getStr(SHARD_ID_PROP);
     String coreName = message.getStr(CoreAdminParams.NAME);
-    Replica.Type replicaType = Replica.Type.valueOf(message.getStr(ZkStateReader.REPLICA_TYPE, Replica.Type.NRT.name()));
+    Replica.Type replicaType = Replica.Type.valueOf(message.getStr(ZkStateReader.REPLICA_TYPE, Replica.Type.NRT.name()).toUpperCase(Locale.ROOT));
     boolean parallel = message.getBool("parallel", false);
     if (StringUtils.isBlank(coreName)) {
       coreName = message.getStr(CoreAdminParams.PROPERTY_PREFIX + CoreAdminParams.NAME);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22c2ed07/solr/core/src/resources/apispec/collections.Commands.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/collections.Commands.json b/solr/core/src/resources/apispec/collections.Commands.json
index 8f5132a..13a75c0 100644
--- a/solr/core/src/resources/apispec/collections.Commands.json
+++ b/solr/core/src/resources/apispec/collections.Commands.json
@@ -51,7 +51,19 @@
         },
         "replicationFactor": {
           "type": "integer",
-          "description": "The number of replicas to be created for each shard. Replicas are physical copies of each shard, acting as failover for the shard."
+          "description": "The number of NRT replicas to be created for each shard. Replicas are physical copies of each shard, acting as failover for the shard."
+        },
+        "nrtReplicas": {
+          "type": "integer",
+          "description": "The number of NRT replicas to be created for each shard. Replicas are physical copies of each shard, acting as failover for the shard. Replicas of type NRT will be updated with each document that is added to the cluster, and can use \"softCommits\" to get a new view of the index in Near Real Time. This parameter works in the same way as 'replicationFactor'"
+        },
+        "tlogReplicas": {
+          "type": "integer",
+          "description": "The number of TLOG replicas to be created for each shard. TLOG replicas update their transaction log for every update to the cluster, but only the shard leader updates the local index, other TLOG replicas will use segment replication and copy the latest index files from the leader."
+        },
+        "pullReplicas": {
+          "type": "integer",
+          "description": "The number of PULL replicas to be created for each shard. PULL replicas don't receive copies of the documents on update requests, they just replicate the latest segments periodically from the shard leader. PULL replicas can't become shard leaders, and need at least one active TLOG(recommended) or NRT replicas in the shard to replicate from."
         },
         "nodeSet": {
           "type": "array",

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22c2ed07/solr/core/src/resources/apispec/collections.collection.shards.Commands.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/collections.collection.shards.Commands.json b/solr/core/src/resources/apispec/collections.collection.shards.Commands.json
index c3bf7bf..4750e2c 100644
--- a/solr/core/src/resources/apispec/collections.collection.shards.Commands.json
+++ b/solr/core/src/resources/apispec/collections.collection.shards.Commands.json
@@ -101,6 +101,11 @@
         "async": {
           "type": "string",
           "description": "Defines a request ID that can be used to track this action after it's submitted. The action will be processed asynchronously when this is defined."
+        },
+        "type": {
+          "type": "string",
+          "enum":["NRT", "TLOG", "PULL"],
+          "description": "The type of replica to add. NRT (default), TLOG or PULL"
         }
       },
       "required":["shard"]

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22c2ed07/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java b/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java
index cb0603d..2156d3b 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java
@@ -28,8 +28,12 @@ import java.util.Map;
 import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 
+import org.apache.http.HttpResponse;
+import org.apache.http.client.ClientProtocolException;
 import org.apache.http.client.HttpClient;
 import org.apache.http.client.methods.HttpGet;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.entity.StringEntity;
 import org.apache.lucene.util.LuceneTestCase.Slow;
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.SolrServerException;
@@ -117,21 +121,40 @@ public class TestPullReplica extends SolrCloudTestCase {
   @Repeat(iterations=2) // 2 times to make sure cleanup is complete and we can create the same collection
   public void testCreateDelete() throws Exception {
     try {
-      if (random().nextBoolean()) {
-        CollectionAdminRequest.createCollection(collectionName, "conf", 2, 1, 0, 3)
-        .setMaxShardsPerNode(100)
-        .process(cluster.getSolrClient());
-      } else {
-        // Sometimes don't use SolrJ.
-        String url = String.format(Locale.ROOT, "%s/admin/collections?action=CREATE&name=%s&numShards=%s&pullReplicas=%s&maxShardsPerNode=%s", 
-            cluster.getRandomJetty(random()).getBaseUrl(), 
-            collectionName,
-            2,    // numShards
-            3,    // pullReplicas 
-            100); // maxShardsPerNode
-        url = url + pickRandom("", "&nrtReplicas=1", "&replicationFactor=1"); // These options should all mean the same
-        HttpGet createCollectionRequest = new HttpGet(url);
-        cluster.getSolrClient().getHttpClient().execute(createCollectionRequest);
+      switch (random().nextInt(3)) {
+        case 0:
+          // Sometimes use SolrJ
+          CollectionAdminRequest.createCollection(collectionName, "conf", 2, 1, 0, 3)
+          .setMaxShardsPerNode(100)
+          .process(cluster.getSolrClient());
+          break;
+        case 1:
+          // Sometimes use v1 API
+          String url = String.format(Locale.ROOT, "%s/admin/collections?action=CREATE&name=%s&numShards=%s&pullReplicas=%s&maxShardsPerNode=%s",
+              cluster.getRandomJetty(random()).getBaseUrl(),
+              collectionName,
+              2,    // numShards
+              3,    // pullReplicas
+              100); // maxShardsPerNode
+          url = url + pickRandom("", "&nrtReplicas=1", "&replicationFactor=1"); // These options should all mean the same
+          HttpGet createCollectionGet = new HttpGet(url);
+          cluster.getSolrClient().getHttpClient().execute(createCollectionGet);
+          break;
+        case 2:
+          // Sometimes use V2 API
+          url = cluster.getRandomJetty(random()).getBaseUrl().toString() + "/____v2/c";
+          String requestBody = String.format(Locale.ROOT, "{create:{name:%s, numShards:%s, pullReplicas:%s, maxShardsPerNode:%s %s}}",
+              collectionName,
+              2,    // numShards
+              3,    // pullReplicas
+              100, // maxShardsPerNode
+              pickRandom("", ", nrtReplicas:1", ", replicationFactor:1")); // These options should all mean the same
+          HttpPost createCollectionPost = new HttpPost(url);
+          createCollectionPost.setHeader("Content-type", "application/json");
+          createCollectionPost.setEntity(new StringEntity(requestBody));
+          HttpResponse httpResponse = cluster.getSolrClient().getHttpClient().execute(createCollectionPost);
+          assertEquals(200, httpResponse.getStatusLine().getStatusCode());
+          break;
       }
       boolean reloaded = false;
       while (true) {
@@ -243,9 +266,9 @@ public class TestPullReplica extends SolrCloudTestCase {
     DocCollection docCollection = assertNumberOfReplicas(2, 0, 0, false, true);
     assertEquals(2, docCollection.getSlices().size());
     
-    CollectionAdminRequest.addReplicaToShard(collectionName, "shard1", Replica.Type.PULL).process(cluster.getSolrClient());
+    addReplicaToShard("shard1", Replica.Type.PULL);
     docCollection = assertNumberOfReplicas(2, 0, 1, true, false);
-    CollectionAdminRequest.addReplicaToShard(collectionName, "shard2", Replica.Type.PULL).process(cluster.getSolrClient());    
+    addReplicaToShard("shard2", Replica.Type.PULL);
     docCollection = assertNumberOfReplicas(2, 0, 2, true, false);
     
     waitForState("Expecting collection to have 2 shards and 2 replica each", collectionName, clusterShape(2, 2));
@@ -587,4 +610,36 @@ public class TestPullReplica extends SolrCloudTestCase {
     cluster.getSolrClient().add(collectionName, docs);
     cluster.getSolrClient().commit(collectionName);
   }
+  
+  private void addReplicaToShard(String shardName, Replica.Type type) throws ClientProtocolException, IOException, SolrServerException {
+    switch (random().nextInt(3)) {
+      case 0: // Add replica with SolrJ
+        CollectionAdminResponse response = CollectionAdminRequest.addReplicaToShard(collectionName, shardName, type).process(cluster.getSolrClient());
+        assertEquals("Unexpected response status: " + response.getStatus(), 0, response.getStatus());
+        break;
+      case 1: // Add replica with V1 API
+        String url = String.format(Locale.ROOT, "%s/admin/collections?action=ADDREPLICA&collection=%s&shard=%s&type=%s",
+            cluster.getRandomJetty(random()).getBaseUrl(),
+            collectionName,
+            shardName,
+            type);
+        HttpGet addReplicaGet = new HttpGet(url);
+        HttpResponse httpResponse = cluster.getSolrClient().getHttpClient().execute(addReplicaGet);
+        assertEquals(200, httpResponse.getStatusLine().getStatusCode());
+        break;
+      case 2:// Add replica with V2 API
+        url = String.format(Locale.ROOT, "%s/____v2/c/%s/shards",
+            cluster.getRandomJetty(random()).getBaseUrl(),
+            collectionName);
+        String requestBody = String.format(Locale.ROOT, "{add-replica:{shard:%s, type:%s}}",
+            shardName,
+            type);
+        HttpPost addReplicaPost = new HttpPost(url);
+        addReplicaPost.setHeader("Content-type", "application/json");
+        addReplicaPost.setEntity(new StringEntity(requestBody));
+        httpResponse = cluster.getSolrClient().getHttpClient().execute(addReplicaPost);
+        assertEquals(200, httpResponse.getStatusLine().getStatusCode());
+        break;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22c2ed07/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java b/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java
index 1c2e7aa..9990601 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java
@@ -29,8 +29,12 @@ import java.util.concurrent.Semaphore;
 import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 
+import org.apache.http.HttpResponse;
+import org.apache.http.client.ClientProtocolException;
 import org.apache.http.client.HttpClient;
 import org.apache.http.client.methods.HttpGet;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.entity.StringEntity;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.util.LuceneTestCase.Slow;
 import org.apache.solr.client.solrj.SolrClient;
@@ -145,20 +149,38 @@ public class TestTlogReplica extends SolrCloudTestCase {
   @Repeat(iterations=2) // 2 times to make sure cleanup is complete and we can create the same collection
   public void testCreateDelete() throws Exception {
     try {
-      if (random().nextBoolean()) {
-        CollectionAdminRequest.createCollection(collectionName, "conf", 2, 0, 4, 0)
-        .setMaxShardsPerNode(100)
-        .process(cluster.getSolrClient());
-      } else {
-        // Sometimes don't use SolrJ
-        String url = String.format(Locale.ROOT, "%s/admin/collections?action=CREATE&name=%s&numShards=%s&tlogReplicas=%s&maxShardsPerNode=%s", 
-            cluster.getRandomJetty(random()).getBaseUrl(), 
-            collectionName,
-            2,    // numShards
-            4,    // tlogReplicas 
-            100); // maxShardsPerNode
-        HttpGet createCollectionRequest = new HttpGet(url);
-        cluster.getSolrClient().getHttpClient().execute(createCollectionRequest);
+      switch (random().nextInt(3)) {
+        case 0:
+          CollectionAdminRequest.createCollection(collectionName, "conf", 2, 0, 4, 0)
+          .setMaxShardsPerNode(100)
+          .process(cluster.getSolrClient());
+          break;
+        case 1:
+          // Sometimes don't use SolrJ
+          String url = String.format(Locale.ROOT, "%s/admin/collections?action=CREATE&name=%s&numShards=%s&tlogReplicas=%s&maxShardsPerNode=%s",
+              cluster.getRandomJetty(random()).getBaseUrl(),
+              collectionName,
+              2,    // numShards
+              4,    // tlogReplicas
+              100); // maxShardsPerNode
+          HttpGet createCollectionGet = new HttpGet(url);
+          HttpResponse httpResponse = cluster.getSolrClient().getHttpClient().execute(createCollectionGet);
+          assertEquals(200, httpResponse.getStatusLine().getStatusCode());
+          break;
+        case 2:
+          // Sometimes use V2 API
+          url = cluster.getRandomJetty(random()).getBaseUrl().toString() + "/____v2/c";
+          String requestBody = String.format(Locale.ROOT, "{create:{name:%s, numShards:%s, tlogReplicas:%s, maxShardsPerNode:%s}}",
+              collectionName,
+              2,    // numShards
+              4,    // tlogReplicas
+              100); // maxShardsPerNode
+          HttpPost createCollectionPost = new HttpPost(url);
+          createCollectionPost.setHeader("Content-type", "application/json");
+          createCollectionPost.setEntity(new StringEntity(requestBody));
+          httpResponse = cluster.getSolrClient().getHttpClient().execute(createCollectionPost);
+          assertEquals(200, httpResponse.getStatusLine().getStatusCode());
+          break;
       }
       
       boolean reloaded = false;
@@ -244,9 +266,9 @@ public class TestTlogReplica extends SolrCloudTestCase {
     DocCollection docCollection = createAndWaitForCollection(2, 0, 1, 0);
     assertEquals(2, docCollection.getSlices().size());
     
-    CollectionAdminRequest.addReplicaToShard(collectionName, "shard1", Replica.Type.TLOG).process(cluster.getSolrClient());
+    addReplicaToShard("shard1", Replica.Type.TLOG);
     docCollection = assertNumberOfReplicas(0, 3, 0, true, false);
-    CollectionAdminRequest.addReplicaToShard(collectionName, "shard2", Replica.Type.TLOG).process(cluster.getSolrClient());    
+    addReplicaToShard("shard2", Replica.Type.TLOG);
     docCollection = assertNumberOfReplicas(0, 4, 0, true, false);
     
     waitForState("Expecting collection to have 2 shards and 2 replica each", collectionName, clusterShape(2, 2));
@@ -260,6 +282,38 @@ public class TestTlogReplica extends SolrCloudTestCase {
     assertNumberOfReplicas(0, 3, 0, true, true);
   }
   
+  private void addReplicaToShard(String shardName, Replica.Type type) throws ClientProtocolException, IOException, SolrServerException {
+    switch (random().nextInt(3)) {
+      case 0: // Add replica with SolrJ
+        CollectionAdminResponse response = CollectionAdminRequest.addReplicaToShard(collectionName, shardName, type).process(cluster.getSolrClient());
+        assertEquals("Unexpected response status: " + response.getStatus(), 0, response.getStatus());
+        break;
+      case 1: // Add replica with V1 API
+        String url = String.format(Locale.ROOT, "%s/admin/collections?action=ADDREPLICA&collection=%s&shard=%s&type=%s", 
+            cluster.getRandomJetty(random()).getBaseUrl(), 
+            collectionName,
+            shardName,
+            type);
+        HttpGet addReplicaGet = new HttpGet(url);
+        HttpResponse httpResponse = cluster.getSolrClient().getHttpClient().execute(addReplicaGet);
+        assertEquals(200, httpResponse.getStatusLine().getStatusCode());
+        break;
+      case 2:// Add replica with V2 API
+        url = String.format(Locale.ROOT, "%s/____v2/c/%s/shards", 
+            cluster.getRandomJetty(random()).getBaseUrl(), 
+            collectionName);
+        String requestBody = String.format(Locale.ROOT, "{add-replica:{shard:%s, type:%s}}", 
+            shardName,
+            type);
+        HttpPost addReplicaPost = new HttpPost(url);
+        addReplicaPost.setHeader("Content-type", "application/json");
+        addReplicaPost.setEntity(new StringEntity(requestBody));
+        httpResponse = cluster.getSolrClient().getHttpClient().execute(addReplicaPost);
+        assertEquals(200, httpResponse.getStatusLine().getStatusCode());
+        break;
+    }
+  }
+  
   public void testRemoveLeader() throws Exception {
     doReplaceLeader(true);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22c2ed07/solr/core/src/test/org/apache/solr/handler/admin/TestCollectionAPIs.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/TestCollectionAPIs.java b/solr/core/src/test/org/apache/solr/handler/admin/TestCollectionAPIs.java
index 9f5a246..e7dbf3e 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/TestCollectionAPIs.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/TestCollectionAPIs.java
@@ -58,11 +58,19 @@ public class TestCollectionAPIs extends SolrTestCaseJ4 {
     MockCollectionsHandler collectionsHandler = new MockCollectionsHandler();
     ApiBag apiBag = new ApiBag(false);
     Collection<Api> apis = collectionsHandler.getApis();
-    for (Api api : apis) apiBag.register(api, Collections.EMPTY_MAP);
+    for (Api api : apis) apiBag.register(api, Collections.emptyMap());
     //test a simple create collection call
     compareOutput(apiBag, "/collections", POST,
         "{create:{name:'newcoll', config:'schemaless', numShards:2, replicationFactor:2 }}", null,
         "{name:newcoll, fromApi:'true', replicationFactor:'2', collection.configName:schemaless, numShards:'2', stateFormat:'2', operation:create}");
+    
+    compareOutput(apiBag, "/collections", POST,
+        "{create:{name:'newcoll', config:'schemaless', numShards:2, nrtReplicas:2 }}", null,
+        "{name:newcoll, fromApi:'true', nrtReplicas:'2', collection.configName:schemaless, numShards:'2', stateFormat:'2', operation:create}");
+    
+    compareOutput(apiBag, "/collections", POST,
+        "{create:{name:'newcoll', config:'schemaless', numShards:2, nrtReplicas:2, tlogReplicas:2, pullReplicas:2 }}", null,
+        "{name:newcoll, fromApi:'true', nrtReplicas:'2', tlogReplicas:'2', pullReplicas:'2', collection.configName:schemaless, numShards:'2', stateFormat:'2', operation:create}");
 
     //test a create collection with custom properties
     compareOutput(apiBag, "/collections", POST,
@@ -106,6 +114,21 @@ public class TestCollectionAPIs extends SolrTestCaseJ4 {
         "{split:{ splitKey:id12345, coreProperties : {prop1:prop1Val, prop2:prop2Val} }}", null,
         "{collection: collName , split.key : id12345 , operation : splitshard, property.prop1:prop1Val, property.prop2: prop2Val}"
     );
+    
+    compareOutput(apiBag, "/collections/collName/shards", POST,
+        "{add-replica:{shard: shard1, node: 'localhost_8978' , type:'TLOG' }}", null,
+        "{collection: collName , shard : shard1, node :'localhost_8978', operation : addreplica, type: TLOG}"
+    );
+    
+    compareOutput(apiBag, "/collections/collName/shards", POST,
+        "{add-replica:{shard: shard1, node: 'localhost_8978' , type:'PULL' }}", null,
+        "{collection: collName , shard : shard1, node :'localhost_8978', operation : addreplica, type: PULL}"
+    );
+    
+    assertErrorContains(apiBag, "/collections/collName/shards", POST,
+        "{add-replica:{shard: shard1, node: 'localhost_8978' , type:'foo' }}", null,
+        "Value of enum must be one of"
+    );
 
     compareOutput(apiBag, "/collections/collName", POST,
         "{add-replica-property : {name:propA , value: VALA, shard: shard1, replica:replica1}}", null,
@@ -150,13 +173,22 @@ public class TestCollectionAPIs extends SolrTestCaseJ4 {
     Map expected = (Map) fromJSONString(expectedOutputMapJson);
     assertMapEqual(expected, output);
     return output;
-
+  }
+  
+  static void assertErrorContains(final ApiBag apiBag, final String path, final SolrRequest.METHOD method,
+      final String payload, final CoreContainer cc, String expectedErrorMsg) throws Exception {
+    try {
+      makeCall(apiBag, path, method, payload, cc);
+      fail("Expected exception");
+    } catch (RuntimeException e) {
+      assertTrue("Expected exception with error message '" + expectedErrorMsg + "' but got: " + e.getMessage(), e.getMessage().contains(expectedErrorMsg));
+    }
   }
 
   public static Pair<SolrQueryRequest, SolrQueryResponse> makeCall(final ApiBag apiBag, String path,
                                                                    final SolrRequest.METHOD method,
                                                                    final String payload, final CoreContainer cc) throws Exception {
-    SolrParams queryParams = new MultiMapSolrParams(Collections.EMPTY_MAP);
+    SolrParams queryParams = new MultiMapSolrParams(Collections.emptyMap());
     if (path.indexOf('?') > 0) {
       String queryStr = path.substring(path.indexOf('?') + 1);
       path = path.substring(0, path.indexOf('?'));