mirror of https://github.com/apache/lucene.git
test to help verify that functions wrapping queries (or filtered by queries) are only evaluated for docs that match those queries
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1548955 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
37a067efde
commit
101f0b4978
|
@ -24,6 +24,11 @@
|
|||
<queryParser name="foo" class="FooQParserPlugin"/>
|
||||
<!-- override the default "lucene" qparser -->
|
||||
<queryParser name="lucene" class="org.apache.solr.search.FooQParserPlugin"/>
|
||||
<queryParser name="notfoo" class="solr.LuceneQParserPlugin"/>
|
||||
|
||||
<valueSourceParser name="boost" class="org.apache.solr.core.DummyValueSourceParser"/>
|
||||
<valueSourceParser name="countUsage" class="org.apache.solr.core.CountUsageValueSourceParser"/>
|
||||
|
||||
<requestHandler default="true" name="/select" class="solr.SearchHandler" />
|
||||
|
||||
</config>
|
||||
|
|
|
@ -0,0 +1,88 @@
|
|||
package org.apache.solr.core;
|
||||
/*
|
||||
* 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.lucene.queries.function.FunctionValues;
|
||||
import org.apache.lucene.queries.function.ValueSource;
|
||||
import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
|
||||
import org.apache.lucene.queries.function.valuesource.DoubleConstValueSource;
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.solr.common.util.NamedList;
|
||||
import org.apache.solr.search.FunctionQParser;
|
||||
import org.apache.solr.search.SyntaxError;
|
||||
import org.apache.solr.search.ValueSourceParser;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
/**
|
||||
* A Mock ValueSource parser that produces ValueSources that returns a constant
|
||||
* value butalso keeps track of how many times it was asked for a value for any
|
||||
* document via a static map and a user defined key.
|
||||
**/
|
||||
public class CountUsageValueSourceParser extends ValueSourceParser {
|
||||
|
||||
private static final ConcurrentMap<String,AtomicInteger> counters
|
||||
= new ConcurrentHashMap<String,AtomicInteger>();
|
||||
|
||||
public static void clearCounters() {
|
||||
counters.clear();
|
||||
}
|
||||
public static int getAndClearCount(String key) {
|
||||
AtomicInteger counter = counters.remove(key);
|
||||
if (null == counter) {
|
||||
throw new IllegalArgumentException("Key has never been used in function: " + key);
|
||||
}
|
||||
return counter.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueSource parse(FunctionQParser fp) throws SyntaxError {
|
||||
String key = fp.parseArg();
|
||||
double val = fp.parseDouble();
|
||||
|
||||
AtomicInteger counter = new AtomicInteger();
|
||||
if (null != counters.putIfAbsent(key, counter)) {
|
||||
throw new IllegalArgumentException("Key has already been used: " + key);
|
||||
}
|
||||
return new CountDocsValueSource(counter, val);
|
||||
}
|
||||
|
||||
static final private class CountDocsValueSource extends DoubleConstValueSource {
|
||||
private final AtomicInteger counter;
|
||||
private final double value;
|
||||
public CountDocsValueSource(AtomicInteger counter, double value) {
|
||||
super(value);
|
||||
this.value = value;
|
||||
this.counter = counter;
|
||||
}
|
||||
@Override
|
||||
public FunctionValues getValues(Map context, AtomicReaderContext readerContext) throws IOException {
|
||||
return new DoubleDocValues(this) {
|
||||
@Override
|
||||
public double doubleVal(int doc) {
|
||||
counter.incrementAndGet();
|
||||
return value;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -44,4 +44,39 @@ public class SOLR749Test extends SolrTestCaseJ4 {
|
|||
assertTrue("vsp is null and it shouldn't be", vsp != null);
|
||||
assertTrue("vsp is not an instanceof " + DummyValueSourceParser.class, vsp instanceof DummyValueSourceParser);
|
||||
}
|
||||
|
||||
public void testHowManyDocsHaveBoostFunctionComputed() throws Exception {
|
||||
for (int i = 0; i < 100; i++) {
|
||||
assertU(adoc("id",""+i));
|
||||
}
|
||||
assertU(commit());
|
||||
|
||||
// NOTE: we can't rely on the default lucene syntax because "FooQParser" is registered as "lucene"
|
||||
assertQ(req("q","{!notfoo}*:*"), "//result[@numFound=100]");
|
||||
assertQ(req("q","{!notfoo}id:[* TO 49]"), "//result[@numFound=50]");
|
||||
try {
|
||||
assertQ("query wrapped in boost func should only eval func for query matches",
|
||||
req("q","{!boost b=$boostFunc defType=notfoo}id:[* TO 49]",
|
||||
"boostFunc", "countUsage('boost_func',3.4)"),
|
||||
"//result[@numFound=50]");
|
||||
assertEquals(50, CountUsageValueSourceParser.getAndClearCount("boost_func"));
|
||||
|
||||
assertQ("func query that is filtered should be evaled only for filtered docs",
|
||||
req("q","{!func}product(id,countUsage('func_q',4.5))",
|
||||
"fq", "{!notfoo}id:[30 TO 59]"),
|
||||
"//result[@numFound=30]");
|
||||
assertEquals(30, CountUsageValueSourceParser.getAndClearCount("func_q"));
|
||||
|
||||
assertQ("func query that wraps a query which is also used as a should be evaled only for filtered docs",
|
||||
req("q","{!func}product(query($qq),countUsage('func_q_wrapping_fq',4.5))",
|
||||
"qq", "{!notfoo}id:[20 TO 39]",
|
||||
"fq", "{!query v=$qq}"),
|
||||
"//result[@numFound=20]");
|
||||
assertEquals(20, CountUsageValueSourceParser.getAndClearCount("func_q_wrapping_fq"));
|
||||
|
||||
} finally {
|
||||
CountUsageValueSourceParser.clearCounters();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue