You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by an...@apache.org on 2014/12/22 08:05:35 UTC

svn commit: r1647253 [2/2] - in /lucene/dev/trunk/solr: ./ core/src/java/org/apache/solr/core/ core/src/java/org/apache/solr/handler/component/ core/src/java/org/apache/solr/search/ core/src/java/org/apache/solr/search/stats/ core/src/java/org/apache/s...

Added: lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/stats/TestDefaultStatsCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/stats/TestDefaultStatsCache.java?rev=1647253&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/stats/TestDefaultStatsCache.java (added)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/stats/TestDefaultStatsCache.java Mon Dec 22 07:05:34 2014
@@ -0,0 +1,104 @@
+package org.apache.solr.search.stats;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.solr.BaseDistributedSearchTestCase;
+import org.apache.solr.client.solrj.SolrServer;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.params.ModifiableSolrParams;
+
+public class TestDefaultStatsCache extends BaseDistributedSearchTestCase {
+  private int docId = 0;
+  
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    System.setProperty("solr.statsCache", LocalStatsCache.class.getName());
+  }
+  
+  public void tearDown() throws Exception {
+    super.tearDown();
+    System.clearProperty("solr.statsCache");
+  }
+  
+  @Override
+  public void doTest() throws Exception {
+    del("*:*");
+    for (int i = 0; i < clients.size(); i++) {
+      int shard = i + 1;
+      for (int j = 0; j <= i; j++) {
+        index_specific(i, id, docId++, "a_t", "one two three",
+            "shard_i", shard);
+      }
+    }
+    commit();
+    handle.clear();
+    handle.put("QTime", SKIPVAL);   
+    handle.put("timestamp", SKIPVAL);
+    
+    dfQuery("q", "a_t:one", "debugQuery", "true", "fl", "*,score");
+    
+    // add another document
+    for (int i = 0; i < clients.size(); i++) {
+      int shard = i + 1;
+      for (int j = 0; j <= i; j++) {
+        index_specific(i, id, docId++, "a_t", "one two three four five",
+            "shard_i", shard);
+      }
+    }
+    commit();
+
+    dfQuery("q", "a_t:one a_t:four", "debugQuery", "true", "fl", "*,score");
+  }
+  
+  // in this case, as the number of shards increases, per-shard scores begin to
+  // diverge due to the different docFreq-s per shard.
+  protected void checkResponse(QueryResponse controlRsp, QueryResponse shardRsp) {
+    SolrDocumentList shardList = shardRsp.getResults();
+    SolrDocumentList controlList = controlRsp.getResults();
+    assertEquals(controlList.getNumFound(), shardList.getNumFound());
+    Float shardScore = (Float) shardList.get(0).getFieldValue("score");
+    Float controlScore = (Float) controlList.get(0).getFieldValue("score");
+    if (clients.size() == 1) {
+      // only one shard
+      assertEquals(controlScore, shardScore);
+    } else {
+      assertTrue("control:" + controlScore.floatValue() + " shard:"
+          + shardScore.floatValue(),
+          controlScore.floatValue() > shardScore.floatValue());
+    }
+  }
+  
+  protected void dfQuery(Object... q) throws Exception {
+    final ModifiableSolrParams params = new ModifiableSolrParams();
+    
+    for (int i = 0; i < q.length; i += 2) {
+      params.add(q[i].toString(), q[i + 1].toString());
+    }
+    
+    final QueryResponse controlRsp = controlClient.query(params);
+    
+    // query a random server
+    params.set("shards", shards);
+    int which = r.nextInt(clients.size());
+    SolrServer client = clients.get(which);
+    QueryResponse rsp = client.query(params);
+    checkResponse(controlRsp, rsp);
+  }
+}

Added: lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/stats/TestExactSharedStatsCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/stats/TestExactSharedStatsCache.java?rev=1647253&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/stats/TestExactSharedStatsCache.java (added)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/stats/TestExactSharedStatsCache.java Mon Dec 22 07:05:34 2014
@@ -0,0 +1,25 @@
+package org.apache.solr.search.stats;
+
+/**
+ * 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.
+ */
+public class TestExactSharedStatsCache extends TestBaseStatsCache {
+
+  @Override
+  protected String getStatsCacheClassName() {
+    return ExactSharedStatsCache.class.getName();
+  }
+}

Added: lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/stats/TestExactStatsCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/stats/TestExactStatsCache.java?rev=1647253&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/stats/TestExactStatsCache.java (added)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/stats/TestExactStatsCache.java Mon Dec 22 07:05:34 2014
@@ -0,0 +1,24 @@
+package org.apache.solr.search.stats;
+
+/**
+ * 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.
+ */
+public class TestExactStatsCache extends TestBaseStatsCache {
+  @Override
+  protected String getStatsCacheClassName() {
+    return ExactStatsCache.class.getName();
+  }
+}

Added: lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/stats/TestLRUStatsCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/stats/TestLRUStatsCache.java?rev=1647253&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/stats/TestLRUStatsCache.java (added)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/stats/TestLRUStatsCache.java Mon Dec 22 07:05:34 2014
@@ -0,0 +1,24 @@
+package org.apache.solr.search.stats;
+
+/**
+ * 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.
+ */
+public class TestLRUStatsCache extends TestBaseStatsCache {
+  @Override
+  protected String getStatsCacheClassName() {
+    return LRUStatsCache.class.getName();
+  }
+}

Modified: lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/params/ShardParams.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/params/ShardParams.java?rev=1647253&r1=1647252&r2=1647253&view=diff
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/params/ShardParams.java (original)
+++ lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/params/ShardParams.java Mon Dec 22 07:05:34 2014
@@ -49,6 +49,9 @@ public interface ShardParams {
   /** Should things fail if there is an error? (true/false) */
   @Deprecated
   public static final String SHARD_KEYS = "shard.keys";
+  
+  /** query purpose for shard requests */
+  public static final String SHARDS_PURPOSE = "shards.purpose";
 
   public static final String _ROUTE_ = "_route_";