mirror of https://github.com/apache/lucene.git
merge trunk
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene5969@1627806 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
commit
3edb271c90
|
@ -66,6 +66,10 @@ New Features
|
|||
|
||||
* LUCENE-5949: Add Accountable.getChildResources(). (Robert Muir)
|
||||
|
||||
* SOLR-5986: Added ExitableDirectoryReader that extends FilterDirectoryReader and enables
|
||||
exiting requests that take too long to enumerate over terms. (Anshum Gupta, Steve Rowe,
|
||||
Robert Muir)
|
||||
|
||||
API Changes
|
||||
|
||||
* LUCENE-5900: Deprecated more constructors taking Version in *InfixSuggester and
|
||||
|
|
|
@ -0,0 +1,217 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/*
|
||||
* 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.index.FilterLeafReader.FilterFields;
|
||||
import org.apache.lucene.index.FilterLeafReader.FilterTerms;
|
||||
import org.apache.lucene.index.FilterLeafReader.FilterTermsEnum;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.automaton.CompiledAutomaton;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
|
||||
/**
|
||||
* The {@link ExitableDirectoryReader} wraps a real index {@link DirectoryReader} and
|
||||
* allows for a {@link QueryTimeout} implementation object to be checked periodically
|
||||
* to see if the thread should exit or not. If {@link QueryTimeout#shouldExit()}
|
||||
* returns true, an {@link ExitingReaderException} is thrown.
|
||||
*/
|
||||
public class ExitableDirectoryReader extends FilterDirectoryReader {
|
||||
|
||||
private QueryTimeout queryTimeout;
|
||||
|
||||
/**
|
||||
* Exception that is thrown to prematurely terminate a term enumeration.
|
||||
*/
|
||||
@SuppressWarnings("serial")
|
||||
public static class ExitingReaderException extends RuntimeException {
|
||||
|
||||
/** Constructor **/
|
||||
ExitingReaderException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Wrapper class for a SubReaderWrapper that is used by the ExitableDirectoryReader.
|
||||
*/
|
||||
public static class ExitableSubReaderWrapper extends SubReaderWrapper {
|
||||
private QueryTimeout queryTimeout;
|
||||
|
||||
/** Constructor **/
|
||||
public ExitableSubReaderWrapper(QueryTimeout queryTimeout) {
|
||||
this.queryTimeout = queryTimeout;
|
||||
}
|
||||
|
||||
@Override
|
||||
public LeafReader wrap(LeafReader reader) {
|
||||
return new ExitableFilterAtomicReader(reader, queryTimeout);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Wrapper class for another FilterAtomicReader. This is used by ExitableSubReaderWrapper.
|
||||
*/
|
||||
public static class ExitableFilterAtomicReader extends FilterLeafReader {
|
||||
|
||||
private QueryTimeout queryTimeout;
|
||||
|
||||
/** Constructor **/
|
||||
public ExitableFilterAtomicReader(LeafReader in, QueryTimeout queryTimeout) {
|
||||
super(in);
|
||||
this.queryTimeout = queryTimeout;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Fields fields() throws IOException {
|
||||
Fields fields = super.fields();
|
||||
if (fields == null) {
|
||||
return null;
|
||||
}
|
||||
return new ExitableFields(fields, queryTimeout);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getCoreCacheKey() {
|
||||
return in.getCoreCacheKey();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getCombinedCoreAndDeletesKey() {
|
||||
return in.getCombinedCoreAndDeletesKey();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Wrapper class for another Fields implementation that is used by the ExitableFilterAtomicReader.
|
||||
*/
|
||||
public static class ExitableFields extends FilterFields {
|
||||
|
||||
private QueryTimeout queryTimeout;
|
||||
|
||||
/** Constructor **/
|
||||
public ExitableFields(Fields fields, QueryTimeout queryTimeout) {
|
||||
super(fields);
|
||||
this.queryTimeout = queryTimeout;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Terms terms(String field) throws IOException {
|
||||
Terms terms = in.terms(field);
|
||||
if (terms == null) {
|
||||
return null;
|
||||
}
|
||||
return new ExitableTerms(terms, queryTimeout);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Wrapper class for another Terms implementation that is used by ExitableFields.
|
||||
*/
|
||||
public static class ExitableTerms extends FilterTerms {
|
||||
|
||||
private QueryTimeout queryTimeout;
|
||||
|
||||
/** Constructor **/
|
||||
public ExitableTerms(Terms terms, QueryTimeout queryTimeout) {
|
||||
super(terms);
|
||||
this.queryTimeout = queryTimeout;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermsEnum intersect(CompiledAutomaton compiled, BytesRef startTerm) throws IOException {
|
||||
return new ExitableTermsEnum(in.intersect(compiled, startTerm), queryTimeout);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermsEnum iterator(TermsEnum reuse) throws IOException {
|
||||
return new ExitableTermsEnum(in.iterator(reuse), queryTimeout);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Wrapper class for TermsEnum that is used by ExitableTerms for implementing an
|
||||
* exitable enumeration of terms.
|
||||
*/
|
||||
public static class ExitableTermsEnum extends FilterTermsEnum {
|
||||
|
||||
private QueryTimeout queryTimeout;
|
||||
|
||||
/** Constructor **/
|
||||
public ExitableTermsEnum(TermsEnum termsEnum, QueryTimeout queryTimeout) {
|
||||
super(termsEnum);
|
||||
this.queryTimeout = queryTimeout;
|
||||
checkAndThrow();
|
||||
}
|
||||
|
||||
/**
|
||||
* Throws {@link ExitingReaderException} if {@link QueryTimeout#shouldExit()} returns true,
|
||||
* or if {@link Thread#interrupted()} returns true.
|
||||
*/
|
||||
private void checkAndThrow() {
|
||||
if (queryTimeout.shouldExit()) {
|
||||
throw new ExitingReaderException("The request took too long to iterate over terms. Timeout: "
|
||||
+ queryTimeout.toString()
|
||||
+ ", TermsEnum=" + in
|
||||
);
|
||||
} else if (Thread.interrupted()) {
|
||||
throw new ExitingReaderException("Interrupted while iterating over terms. TermsEnum=" + in);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef next() throws IOException {
|
||||
// Before every iteration, check if the iteration should exit
|
||||
checkAndThrow();
|
||||
return in.next();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructor
|
||||
* @param in DirectoryReader that this ExitableDirectoryReader wraps around to make it Exitable.
|
||||
* @param queryTimeout The object to periodically check if the query should time out.
|
||||
*/
|
||||
public ExitableDirectoryReader(DirectoryReader in, QueryTimeout queryTimeout) {
|
||||
super(in, new ExitableSubReaderWrapper(queryTimeout));
|
||||
this.queryTimeout = queryTimeout;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) {
|
||||
return new ExitableDirectoryReader(in, queryTimeout);
|
||||
}
|
||||
|
||||
/**
|
||||
* Wraps a provided DirectoryReader. Note that for convenience, the returned reader
|
||||
* can be used normally (e.g. passed to {@link DirectoryReader#openIfChanged(DirectoryReader)})
|
||||
* and so on.
|
||||
*/
|
||||
public static DirectoryReader wrap(DirectoryReader in, QueryTimeout queryTimeout) {
|
||||
return new ExitableDirectoryReader(in, queryTimeout);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "ExitableDirectoryReader(" + in.toString() + ")";
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -0,0 +1,33 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
/**
|
||||
* Base for query timeout implementations, which will provide a {@code shouldExit()} method,
|
||||
* used with {@link ExitableDirectoryReader}.
|
||||
*/
|
||||
public interface QueryTimeout {
|
||||
|
||||
/**
|
||||
* Called from {@link ExitableDirectoryReader.ExitableTermsEnum#next()}
|
||||
* to determine whether to stop processing a query.
|
||||
*/
|
||||
public abstract boolean shouldExit();
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,80 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/*
|
||||
* 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 java.util.concurrent.TimeUnit;
|
||||
|
||||
import static java.lang.System.nanoTime;
|
||||
|
||||
/**
|
||||
* An implementation of {@link QueryTimeout} that can be used by
|
||||
* the {@link ExitableDirectoryReader} class to time out and exit out
|
||||
* when a query takes a long time to rewrite.
|
||||
*/
|
||||
public class QueryTimeoutImpl implements QueryTimeout {
|
||||
|
||||
/**
|
||||
* The local variable to store the time beyond which, the processing should exit.
|
||||
*/
|
||||
private Long timeoutAt;
|
||||
|
||||
/**
|
||||
* Sets the time at which to time out by adding the given timeAllowed to the current time.
|
||||
*
|
||||
* @param timeAllowed Number of milliseconds after which to time out. Use {@code Long.MAX_VALUE}
|
||||
* to effectively never time out.
|
||||
*/
|
||||
public QueryTimeoutImpl(long timeAllowed) {
|
||||
if (timeAllowed < 0L) {
|
||||
timeAllowed = Long.MAX_VALUE;
|
||||
}
|
||||
timeoutAt = nanoTime() + TimeUnit.NANOSECONDS.convert(timeAllowed, TimeUnit.MILLISECONDS);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns time at which to time out, in nanoseconds relative to the (JVM-specific)
|
||||
* epoch for {@link System#nanoTime()}, to compare with the value returned by
|
||||
* {@code nanoTime()}.
|
||||
*/
|
||||
public Long getTimeoutAt() {
|
||||
return timeoutAt;
|
||||
}
|
||||
|
||||
/**
|
||||
* Return true if {@link #reset()} has not been called
|
||||
* and the elapsed time has exceeded the time allowed.
|
||||
*/
|
||||
@Override
|
||||
public boolean shouldExit() {
|
||||
return timeoutAt != null && nanoTime() - timeoutAt > 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* Reset the timeout value.
|
||||
*/
|
||||
public void reset() {
|
||||
timeoutAt = null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "timeoutAt: " + timeoutAt + " (System.nanoTime(): " + nanoTime() + ")";
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -39,7 +39,7 @@ public class TimeLimitingCollector implements Collector {
|
|||
private long timeElapsed;
|
||||
private int lastDocCollected;
|
||||
private TimeExceededException(long timeAllowed, long timeElapsed, int lastDocCollected) {
|
||||
super("Elapsed time: " + timeElapsed + "Exceeded allowed search time: " + timeAllowed + " ms.");
|
||||
super("Elapsed time: " + timeElapsed + ". Exceeded allowed search time: " + timeAllowed + " ms.");
|
||||
this.timeAllowed = timeAllowed;
|
||||
this.timeElapsed = timeElapsed;
|
||||
this.lastDocCollected = lastDocCollected;
|
||||
|
@ -142,7 +142,7 @@ public class TimeLimitingCollector implements Collector {
|
|||
@Override
|
||||
public void collect(int doc) throws IOException {
|
||||
final long time = clock.get();
|
||||
if (timeout < time) {
|
||||
if (time - timeout > 0L) {
|
||||
if (greedy) {
|
||||
//System.out.println(this+" greedy: before failing, collecting doc: "+(docBase + doc)+" "+(time-t0));
|
||||
in.collect(doc);
|
||||
|
|
|
@ -0,0 +1,170 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/*
|
||||
* 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.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.index.ExitableDirectoryReader.ExitingReaderException;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.PrefixQuery;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Test that uses a default/lucene Implementation of {@link QueryTimeout}
|
||||
* to exit out long running queries that take too long to iterate over Terms.
|
||||
*/
|
||||
public class TestExitableDirectoryReader extends LuceneTestCase {
|
||||
private static class TestReader extends FilterLeafReader {
|
||||
|
||||
private static class TestFields extends FilterFields {
|
||||
TestFields(Fields in) {
|
||||
super(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Terms terms(String field) throws IOException {
|
||||
return new TestTerms(super.terms(field));
|
||||
}
|
||||
}
|
||||
|
||||
private static class TestTerms extends FilterTerms {
|
||||
TestTerms(Terms in) {
|
||||
super(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermsEnum iterator(TermsEnum reuse) throws IOException {
|
||||
return new TestTermsEnum(super.iterator(reuse));
|
||||
}
|
||||
}
|
||||
|
||||
private static class TestTermsEnum extends FilterTermsEnum {
|
||||
public TestTermsEnum(TermsEnum in) {
|
||||
super(in);
|
||||
}
|
||||
|
||||
/**
|
||||
* Sleep between iterations to timeout things.
|
||||
*/
|
||||
@Override
|
||||
public BytesRef next() throws IOException {
|
||||
try {
|
||||
// Sleep for 1s before each .next() call.
|
||||
Thread.sleep(1000);
|
||||
} catch (InterruptedException e) {
|
||||
}
|
||||
return in.next();
|
||||
}
|
||||
}
|
||||
|
||||
public TestReader(IndexReader reader) throws IOException {
|
||||
super(SlowCompositeReaderWrapper.wrap(reader));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Fields fields() throws IOException {
|
||||
return new TestFields(super.fields());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests timing out of TermsEnum iterations
|
||||
* @throws Exception on error
|
||||
*/
|
||||
public void testExitableFilterIndexReader() throws Exception {
|
||||
Directory directory = newDirectory();
|
||||
IndexWriter writer = new IndexWriter(directory, newIndexWriterConfig(new MockAnalyzer(random())));
|
||||
|
||||
Document d1 = new Document();
|
||||
d1.add(newTextField("default", "one two", Field.Store.YES));
|
||||
writer.addDocument(d1);
|
||||
|
||||
Document d2 = new Document();
|
||||
d2.add(newTextField("default", "one three", Field.Store.YES));
|
||||
writer.addDocument(d2);
|
||||
|
||||
Document d3 = new Document();
|
||||
d3.add(newTextField("default", "ones two four", Field.Store.YES));
|
||||
writer.addDocument(d3);
|
||||
|
||||
writer.commit();
|
||||
writer.close();
|
||||
|
||||
DirectoryReader directoryReader;
|
||||
DirectoryReader exitableDirectoryReader;
|
||||
IndexReader reader;
|
||||
IndexSearcher searcher;
|
||||
|
||||
Query query = new PrefixQuery(new Term("default", "o"));
|
||||
|
||||
// Set a fairly high timeout value (10 seconds) and expect the query to complete in that time frame.
|
||||
// Not checking the validity of the result, all we are bothered about in this test is the timing out.
|
||||
directoryReader = DirectoryReader.open(directory);
|
||||
exitableDirectoryReader = new ExitableDirectoryReader(directoryReader, new QueryTimeoutImpl(10000));
|
||||
reader = new TestReader(exitableDirectoryReader);
|
||||
searcher = new IndexSearcher(reader);
|
||||
searcher.search(query, 10);
|
||||
reader.close();
|
||||
exitableDirectoryReader.close();
|
||||
|
||||
|
||||
// Set a really low timeout value (1 millisecond) and expect an Exception
|
||||
directoryReader = DirectoryReader.open(directory);
|
||||
exitableDirectoryReader = new ExitableDirectoryReader(directoryReader, new QueryTimeoutImpl(1));
|
||||
reader = new TestReader(exitableDirectoryReader);
|
||||
searcher = new IndexSearcher(reader);
|
||||
try {
|
||||
searcher.search(query, 10);
|
||||
fail("This query should have led to an ExitingReaderException!");
|
||||
} catch (ExitingReaderException ex) {
|
||||
// Do nothing, we expect this!
|
||||
} finally {
|
||||
reader.close();
|
||||
exitableDirectoryReader.close();
|
||||
}
|
||||
|
||||
// Set maximum time out and expect the query to complete.
|
||||
// Not checking the validity of the result, all we are bothered about in this test is the timing out.
|
||||
directoryReader = DirectoryReader.open(directory);
|
||||
exitableDirectoryReader = new ExitableDirectoryReader(directoryReader, new QueryTimeoutImpl(Long.MAX_VALUE));
|
||||
reader = new TestReader(exitableDirectoryReader);
|
||||
searcher = new IndexSearcher(reader);
|
||||
searcher.search(query, 10);
|
||||
reader.close();
|
||||
exitableDirectoryReader.close();
|
||||
|
||||
// Set a negative time allowed and expect the query to complete (should disable timeouts)
|
||||
// Not checking the validity of the result, all we are bothered about in this test is the timing out.
|
||||
directoryReader = DirectoryReader.open(directory);
|
||||
exitableDirectoryReader = new ExitableDirectoryReader(directoryReader, new QueryTimeoutImpl(-189034L));
|
||||
reader = new TestReader(exitableDirectoryReader);
|
||||
searcher = new IndexSearcher(reader);
|
||||
searcher.search(query, 10);
|
||||
reader.close();
|
||||
exitableDirectoryReader.close();
|
||||
|
||||
directory.close();
|
||||
}
|
||||
}
|
||||
|
|
@ -33,6 +33,7 @@ import org.apache.lucene.store.Directory;
|
|||
import org.apache.lucene.util.Counter;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
import org.apache.lucene.util.ThreadInterruptedException;
|
||||
|
||||
/**
|
||||
|
@ -143,7 +144,8 @@ public class TestTimeLimitingCollector extends LuceneTestCase {
|
|||
|
||||
myHc = new MyHitCollector();
|
||||
long oneHour = 3600000;
|
||||
Collector tlCollector = createTimedCollector(myHc, oneHour, false);
|
||||
long duration = TestUtil.nextLong(random(), oneHour, Long.MAX_VALUE);
|
||||
Collector tlCollector = createTimedCollector(myHc, duration, false);
|
||||
search(tlCollector);
|
||||
totalTLCResults = myHc.hitCount();
|
||||
} catch (Exception e) {
|
||||
|
|
|
@ -140,6 +140,9 @@ New Features
|
|||
|
||||
* SOLR-6543: Give HttpSolrServer the ability to send PUT requests (Gregory Chanan)
|
||||
|
||||
* SOLR-5986: Don't allow runaway queries from harming Solr cluster health or search
|
||||
performance (Anshum Gupta, Steve Rowe, Robert Muir)
|
||||
|
||||
Bug Fixes
|
||||
----------------------
|
||||
|
||||
|
|
|
@ -17,29 +17,21 @@
|
|||
|
||||
package org.apache.solr.handler;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Reader;
|
||||
import java.net.MalformedURLException;
|
||||
import java.net.URL;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Comparator;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.lucene.index.ExitableDirectoryReader;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.StoredDocument;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.*;
|
||||
import org.apache.lucene.queries.mlt.MoreLikeThis;
|
||||
import org.apache.lucene.search.BooleanClause;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.common.params.CommonParams;
|
||||
import org.apache.solr.common.params.FacetParams;
|
||||
import org.apache.solr.common.params.MoreLikeThisParams;
|
||||
import org.apache.solr.common.params.SolrParams;
|
||||
import org.apache.solr.common.params.MoreLikeThisParams.TermStyle;
|
||||
import org.apache.solr.common.params.SolrParams;
|
||||
import org.apache.solr.common.util.ContentStream;
|
||||
import org.apache.solr.common.util.NamedList;
|
||||
import org.apache.solr.common.util.SimpleOrderedMap;
|
||||
|
@ -49,10 +41,31 @@ import org.apache.solr.request.SolrQueryRequest;
|
|||
import org.apache.solr.response.SolrQueryResponse;
|
||||
import org.apache.solr.schema.IndexSchema;
|
||||
import org.apache.solr.schema.SchemaField;
|
||||
import org.apache.solr.search.*;
|
||||
|
||||
import org.apache.solr.search.DocIterator;
|
||||
import org.apache.solr.search.DocList;
|
||||
import org.apache.solr.search.DocListAndSet;
|
||||
import org.apache.solr.search.QParser;
|
||||
import org.apache.solr.search.QParserPlugin;
|
||||
import org.apache.solr.search.QueryParsing;
|
||||
import org.apache.solr.search.ReturnFields;
|
||||
import org.apache.solr.search.SolrIndexSearcher;
|
||||
import org.apache.solr.search.SolrQueryTimeoutImpl;
|
||||
import org.apache.solr.search.SolrReturnFields;
|
||||
import org.apache.solr.search.SortSpec;
|
||||
import org.apache.solr.search.SyntaxError;
|
||||
import org.apache.solr.util.SolrPluginUtils;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Reader;
|
||||
import java.net.MalformedURLException;
|
||||
import java.net.URL;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Comparator;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
/**
|
||||
* Solr MoreLikeThis --
|
||||
*
|
||||
|
@ -75,180 +88,190 @@ public class MoreLikeThisHandler extends RequestHandlerBase
|
|||
{
|
||||
SolrParams params = req.getParams();
|
||||
|
||||
// Set field flags
|
||||
ReturnFields returnFields = new SolrReturnFields( req );
|
||||
rsp.setReturnFields( returnFields );
|
||||
int flags = 0;
|
||||
if (returnFields.wantsScore()) {
|
||||
flags |= SolrIndexSearcher.GET_SCORES;
|
||||
long timeAllowed = (long)params.getInt( CommonParams.TIME_ALLOWED, -1 );
|
||||
if(timeAllowed > 0) {
|
||||
SolrQueryTimeoutImpl.set(timeAllowed);
|
||||
}
|
||||
|
||||
String defType = params.get(QueryParsing.DEFTYPE, QParserPlugin.DEFAULT_QTYPE);
|
||||
String q = params.get( CommonParams.Q );
|
||||
Query query = null;
|
||||
SortSpec sortSpec = null;
|
||||
List<Query> filters = null;
|
||||
|
||||
try {
|
||||
if (q != null) {
|
||||
QParser parser = QParser.getParser(q, defType, req);
|
||||
query = parser.getQuery();
|
||||
sortSpec = parser.getSort(true);
|
||||
}
|
||||
|
||||
String[] fqs = req.getParams().getParams(CommonParams.FQ);
|
||||
if (fqs!=null && fqs.length!=0) {
|
||||
filters = new ArrayList<>();
|
||||
for (String fq : fqs) {
|
||||
if (fq != null && fq.trim().length()!=0) {
|
||||
QParser fqp = QParser.getParser(fq, null, req);
|
||||
filters.add(fqp.getQuery());
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (SyntaxError e) {
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
|
||||
}
|
||||
|
||||
SolrIndexSearcher searcher = req.getSearcher();
|
||||
|
||||
MoreLikeThisHelper mlt = new MoreLikeThisHelper( params, searcher );
|
||||
|
||||
// Hold on to the interesting terms if relevant
|
||||
TermStyle termStyle = TermStyle.get( params.get( MoreLikeThisParams.INTERESTING_TERMS ) );
|
||||
List<InterestingTerm> interesting = (termStyle == TermStyle.NONE )
|
||||
? null : new ArrayList<InterestingTerm>( mlt.mlt.getMaxQueryTerms() );
|
||||
|
||||
DocListAndSet mltDocs = null;
|
||||
|
||||
// Parse Required Params
|
||||
// This will either have a single Reader or valid query
|
||||
Reader reader = null;
|
||||
try {
|
||||
if (q == null || q.trim().length() < 1) {
|
||||
Iterable<ContentStream> streams = req.getContentStreams();
|
||||
if (streams != null) {
|
||||
Iterator<ContentStream> iter = streams.iterator();
|
||||
if (iter.hasNext()) {
|
||||
reader = iter.next().getReader();
|
||||
}
|
||||
if (iter.hasNext()) {
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
|
||||
"MoreLikeThis does not support multiple ContentStreams");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
int start = params.getInt(CommonParams.START, 0);
|
||||
int rows = params.getInt(CommonParams.ROWS, 10);
|
||||
|
||||
// Find documents MoreLikeThis - either with a reader or a query
|
||||
// --------------------------------------------------------------------------------
|
||||
if (reader != null) {
|
||||
mltDocs = mlt.getMoreLikeThis(reader, start, rows, filters,
|
||||
interesting, flags);
|
||||
} else if (q != null) {
|
||||
// Matching options
|
||||
boolean includeMatch = params.getBool(MoreLikeThisParams.MATCH_INCLUDE,
|
||||
true);
|
||||
int matchOffset = params.getInt(MoreLikeThisParams.MATCH_OFFSET, 0);
|
||||
// Find the base match
|
||||
DocList match = searcher.getDocList(query, null, null, matchOffset, 1,
|
||||
flags); // only get the first one...
|
||||
if (includeMatch) {
|
||||
rsp.add("match", match);
|
||||
}
|
||||
|
||||
// This is an iterator, but we only handle the first match
|
||||
DocIterator iterator = match.iterator();
|
||||
if (iterator.hasNext()) {
|
||||
// do a MoreLikeThis query for each document in results
|
||||
int id = iterator.nextDoc();
|
||||
mltDocs = mlt.getMoreLikeThis(id, start, rows, filters, interesting,
|
||||
flags);
|
||||
}
|
||||
} else {
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
|
||||
"MoreLikeThis requires either a query (?q=) or text to find similar documents.");
|
||||
}
|
||||
|
||||
} finally {
|
||||
if (reader != null) {
|
||||
reader.close();
|
||||
}
|
||||
}
|
||||
|
||||
if( mltDocs == null ) {
|
||||
mltDocs = new DocListAndSet(); // avoid NPE
|
||||
}
|
||||
rsp.add( "response", mltDocs.docList );
|
||||
|
||||
|
||||
if( interesting != null ) {
|
||||
if( termStyle == TermStyle.DETAILS ) {
|
||||
NamedList<Float> it = new NamedList<>();
|
||||
for( InterestingTerm t : interesting ) {
|
||||
it.add( t.term.toString(), t.boost );
|
||||
}
|
||||
rsp.add( "interestingTerms", it );
|
||||
}
|
||||
else {
|
||||
List<String> it = new ArrayList<>( interesting.size() );
|
||||
for( InterestingTerm t : interesting ) {
|
||||
it.add( t.term.text());
|
||||
}
|
||||
rsp.add( "interestingTerms", it );
|
||||
}
|
||||
}
|
||||
|
||||
// maybe facet the results
|
||||
if (params.getBool(FacetParams.FACET,false)) {
|
||||
if( mltDocs.docSet == null ) {
|
||||
rsp.add( "facet_counts", null );
|
||||
}
|
||||
else {
|
||||
SimpleFacets f = new SimpleFacets(req, mltDocs.docSet, params );
|
||||
rsp.add( "facet_counts", f.getFacetCounts() );
|
||||
}
|
||||
}
|
||||
boolean dbg = req.getParams().getBool(CommonParams.DEBUG_QUERY, false);
|
||||
|
||||
boolean dbgQuery = false, dbgResults = false;
|
||||
if (dbg == false){//if it's true, we are doing everything anyway.
|
||||
String[] dbgParams = req.getParams().getParams(CommonParams.DEBUG);
|
||||
if (dbgParams != null) {
|
||||
for (int i = 0; i < dbgParams.length; i++) {
|
||||
if (dbgParams[i].equals(CommonParams.QUERY)){
|
||||
dbgQuery = true;
|
||||
} else if (dbgParams[i].equals(CommonParams.RESULTS)){
|
||||
dbgResults = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
dbgQuery = true;
|
||||
dbgResults = true;
|
||||
}
|
||||
// Copied from StandardRequestHandler... perhaps it should be added to doStandardDebug?
|
||||
if (dbg == true) {
|
||||
try {
|
||||
NamedList<Object> dbgInfo = SolrPluginUtils.doStandardDebug(req, q, mlt.getRawMLTQuery(), mltDocs.docList, dbgQuery, dbgResults);
|
||||
if (null != dbgInfo) {
|
||||
if (null != filters) {
|
||||
dbgInfo.add("filter_queries",req.getParams().getParams(CommonParams.FQ));
|
||||
List<String> fqs = new ArrayList<>(filters.size());
|
||||
for (Query fq : filters) {
|
||||
fqs.add(QueryParsing.toString(fq, req.getSchema()));
|
||||
}
|
||||
dbgInfo.add("parsed_filter_queries",fqs);
|
||||
}
|
||||
rsp.add("debug", dbgInfo);
|
||||
|
||||
// Set field flags
|
||||
ReturnFields returnFields = new SolrReturnFields(req);
|
||||
rsp.setReturnFields(returnFields);
|
||||
int flags = 0;
|
||||
if (returnFields.wantsScore()) {
|
||||
flags |= SolrIndexSearcher.GET_SCORES;
|
||||
}
|
||||
} catch (Exception e) {
|
||||
SolrException.log(SolrCore.log, "Exception during debug", e);
|
||||
rsp.add("exception_during_debug", SolrException.toStr(e));
|
||||
|
||||
String defType = params.get(QueryParsing.DEFTYPE, QParserPlugin.DEFAULT_QTYPE);
|
||||
String q = params.get(CommonParams.Q);
|
||||
Query query = null;
|
||||
SortSpec sortSpec = null;
|
||||
List<Query> filters = null;
|
||||
|
||||
try {
|
||||
if (q != null) {
|
||||
QParser parser = QParser.getParser(q, defType, req);
|
||||
query = parser.getQuery();
|
||||
sortSpec = parser.getSort(true);
|
||||
}
|
||||
|
||||
String[] fqs = req.getParams().getParams(CommonParams.FQ);
|
||||
if (fqs != null && fqs.length != 0) {
|
||||
filters = new ArrayList<>();
|
||||
for (String fq : fqs) {
|
||||
if (fq != null && fq.trim().length() != 0) {
|
||||
QParser fqp = QParser.getParser(fq, null, req);
|
||||
filters.add(fqp.getQuery());
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (SyntaxError e) {
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
|
||||
}
|
||||
|
||||
SolrIndexSearcher searcher = req.getSearcher();
|
||||
|
||||
MoreLikeThisHelper mlt = new MoreLikeThisHelper(params, searcher);
|
||||
|
||||
// Hold on to the interesting terms if relevant
|
||||
TermStyle termStyle = TermStyle.get(params.get(MoreLikeThisParams.INTERESTING_TERMS));
|
||||
List<InterestingTerm> interesting = (termStyle == TermStyle.NONE)
|
||||
? null : new ArrayList<InterestingTerm>(mlt.mlt.getMaxQueryTerms());
|
||||
|
||||
DocListAndSet mltDocs = null;
|
||||
|
||||
// Parse Required Params
|
||||
// This will either have a single Reader or valid query
|
||||
Reader reader = null;
|
||||
try {
|
||||
if (q == null || q.trim().length() < 1) {
|
||||
Iterable<ContentStream> streams = req.getContentStreams();
|
||||
if (streams != null) {
|
||||
Iterator<ContentStream> iter = streams.iterator();
|
||||
if (iter.hasNext()) {
|
||||
reader = iter.next().getReader();
|
||||
}
|
||||
if (iter.hasNext()) {
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
|
||||
"MoreLikeThis does not support multiple ContentStreams");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
int start = params.getInt(CommonParams.START, 0);
|
||||
int rows = params.getInt(CommonParams.ROWS, 10);
|
||||
|
||||
// Find documents MoreLikeThis - either with a reader or a query
|
||||
// --------------------------------------------------------------------------------
|
||||
if (reader != null) {
|
||||
mltDocs = mlt.getMoreLikeThis(reader, start, rows, filters,
|
||||
interesting, flags);
|
||||
} else if (q != null) {
|
||||
// Matching options
|
||||
boolean includeMatch = params.getBool(MoreLikeThisParams.MATCH_INCLUDE,
|
||||
true);
|
||||
int matchOffset = params.getInt(MoreLikeThisParams.MATCH_OFFSET, 0);
|
||||
// Find the base match
|
||||
DocList match = searcher.getDocList(query, null, null, matchOffset, 1,
|
||||
flags); // only get the first one...
|
||||
if (includeMatch) {
|
||||
rsp.add("match", match);
|
||||
}
|
||||
|
||||
// This is an iterator, but we only handle the first match
|
||||
DocIterator iterator = match.iterator();
|
||||
if (iterator.hasNext()) {
|
||||
// do a MoreLikeThis query for each document in results
|
||||
int id = iterator.nextDoc();
|
||||
mltDocs = mlt.getMoreLikeThis(id, start, rows, filters, interesting,
|
||||
flags);
|
||||
}
|
||||
} else {
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
|
||||
"MoreLikeThis requires either a query (?q=) or text to find similar documents.");
|
||||
}
|
||||
|
||||
} finally {
|
||||
if (reader != null) {
|
||||
reader.close();
|
||||
}
|
||||
}
|
||||
|
||||
if (mltDocs == null) {
|
||||
mltDocs = new DocListAndSet(); // avoid NPE
|
||||
}
|
||||
rsp.add("response", mltDocs.docList);
|
||||
|
||||
|
||||
if (interesting != null) {
|
||||
if (termStyle == TermStyle.DETAILS) {
|
||||
NamedList<Float> it = new NamedList<>();
|
||||
for (InterestingTerm t : interesting) {
|
||||
it.add(t.term.toString(), t.boost);
|
||||
}
|
||||
rsp.add("interestingTerms", it);
|
||||
} else {
|
||||
List<String> it = new ArrayList<>(interesting.size());
|
||||
for (InterestingTerm t : interesting) {
|
||||
it.add(t.term.text());
|
||||
}
|
||||
rsp.add("interestingTerms", it);
|
||||
}
|
||||
}
|
||||
|
||||
// maybe facet the results
|
||||
if (params.getBool(FacetParams.FACET, false)) {
|
||||
if (mltDocs.docSet == null) {
|
||||
rsp.add("facet_counts", null);
|
||||
} else {
|
||||
SimpleFacets f = new SimpleFacets(req, mltDocs.docSet, params);
|
||||
rsp.add("facet_counts", f.getFacetCounts());
|
||||
}
|
||||
}
|
||||
boolean dbg = req.getParams().getBool(CommonParams.DEBUG_QUERY, false);
|
||||
|
||||
boolean dbgQuery = false, dbgResults = false;
|
||||
if (dbg == false) {//if it's true, we are doing everything anyway.
|
||||
String[] dbgParams = req.getParams().getParams(CommonParams.DEBUG);
|
||||
if (dbgParams != null) {
|
||||
for (int i = 0; i < dbgParams.length; i++) {
|
||||
if (dbgParams[i].equals(CommonParams.QUERY)) {
|
||||
dbgQuery = true;
|
||||
} else if (dbgParams[i].equals(CommonParams.RESULTS)) {
|
||||
dbgResults = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
dbgQuery = true;
|
||||
dbgResults = true;
|
||||
}
|
||||
// Copied from StandardRequestHandler... perhaps it should be added to doStandardDebug?
|
||||
if (dbg == true) {
|
||||
try {
|
||||
NamedList<Object> dbgInfo = SolrPluginUtils.doStandardDebug(req, q, mlt.getRawMLTQuery(), mltDocs.docList, dbgQuery, dbgResults);
|
||||
if (null != dbgInfo) {
|
||||
if (null != filters) {
|
||||
dbgInfo.add("filter_queries", req.getParams().getParams(CommonParams.FQ));
|
||||
List<String> fqs = new ArrayList<>(filters.size());
|
||||
for (Query fq : filters) {
|
||||
fqs.add(QueryParsing.toString(fq, req.getSchema()));
|
||||
}
|
||||
dbgInfo.add("parsed_filter_queries", fqs);
|
||||
}
|
||||
rsp.add("debug", dbgInfo);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
SolrException.log(SolrCore.log, "Exception during debug", e);
|
||||
rsp.add("exception_during_debug", SolrException.toStr(e));
|
||||
}
|
||||
}
|
||||
} catch (ExitableDirectoryReader.ExitingReaderException ex) {
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
|
||||
"MLTHandler Request took too long during query expansion. Terminating request.");
|
||||
} finally {
|
||||
SolrQueryTimeoutImpl.reset();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static class InterestingTerm
|
||||
|
|
|
@ -285,7 +285,7 @@ public class QueryComponent extends SearchComponent
|
|||
}
|
||||
|
||||
// -1 as flag if not set.
|
||||
long timeAllowed = (long)params.getInt( CommonParams.TIME_ALLOWED, -1 );
|
||||
long timeAllowed = params.getLong(CommonParams.TIME_ALLOWED, -1L);
|
||||
if (null != rb.getCursorMark() && 0 < timeAllowed) {
|
||||
// fundementally incompatible
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Can not search using both " +
|
||||
|
|
|
@ -17,19 +17,13 @@
|
|||
|
||||
package org.apache.solr.handler.component;
|
||||
|
||||
import java.io.PrintWriter;
|
||||
import java.io.StringWriter;
|
||||
import java.util.ArrayList;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.index.ExitableDirectoryReader;
|
||||
import org.apache.solr.client.solrj.SolrServerException;
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.common.SolrException.ErrorCode;
|
||||
import org.apache.solr.common.params.CommonParams;
|
||||
import org.apache.solr.common.params.ModifiableSolrParams;
|
||||
import org.apache.solr.common.params.ShardParams;
|
||||
import org.apache.solr.common.util.ContentStream;
|
||||
import org.apache.solr.common.util.NamedList;
|
||||
import org.apache.solr.common.util.SimpleOrderedMap;
|
||||
import org.apache.solr.core.CloseHook;
|
||||
|
@ -38,6 +32,7 @@ import org.apache.solr.core.SolrCore;
|
|||
import org.apache.solr.handler.RequestHandlerBase;
|
||||
import org.apache.solr.request.SolrQueryRequest;
|
||||
import org.apache.solr.response.SolrQueryResponse;
|
||||
import org.apache.solr.search.SolrQueryTimeoutImpl;
|
||||
import org.apache.solr.util.RTimer;
|
||||
import org.apache.solr.util.SolrPluginUtils;
|
||||
import org.apache.solr.util.plugin.PluginInfoInitialized;
|
||||
|
@ -45,6 +40,12 @@ import org.apache.solr.util.plugin.SolrCoreAware;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.PrintWriter;
|
||||
import java.io.StringWriter;
|
||||
import java.util.ArrayList;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -212,30 +213,42 @@ public class SearchHandler extends RequestHandlerBase implements SolrCoreAware ,
|
|||
if (!rb.isDistrib) {
|
||||
// a normal non-distributed request
|
||||
|
||||
// The semantics of debugging vs not debugging are different enough that
|
||||
// it makes sense to have two control loops
|
||||
if(!rb.isDebug()) {
|
||||
// Process
|
||||
for( SearchComponent c : components ) {
|
||||
c.process(rb);
|
||||
}
|
||||
long timeAllowed = req.getParams().getLong(CommonParams.TIME_ALLOWED, -1L);
|
||||
if (timeAllowed > 0L) {
|
||||
SolrQueryTimeoutImpl.set(timeAllowed);
|
||||
}
|
||||
else {
|
||||
// Process
|
||||
RTimer subt = timer.sub( "process" );
|
||||
for( SearchComponent c : components ) {
|
||||
rb.setTimer( subt.sub( c.getName() ) );
|
||||
c.process(rb);
|
||||
rb.getTimer().stop();
|
||||
try {
|
||||
// The semantics of debugging vs not debugging are different enough that
|
||||
// it makes sense to have two control loops
|
||||
if(!rb.isDebug()) {
|
||||
// Process
|
||||
for( SearchComponent c : components ) {
|
||||
c.process(rb);
|
||||
}
|
||||
}
|
||||
subt.stop();
|
||||
timer.stop();
|
||||
else {
|
||||
// Process
|
||||
RTimer subt = timer.sub( "process" );
|
||||
for( SearchComponent c : components ) {
|
||||
rb.setTimer( subt.sub( c.getName() ) );
|
||||
c.process(rb);
|
||||
rb.getTimer().stop();
|
||||
}
|
||||
subt.stop();
|
||||
timer.stop();
|
||||
|
||||
// add the timing info
|
||||
if (rb.isDebugTimings()) {
|
||||
rb.addDebugInfo("timing", timer.asNamedList() );
|
||||
// add the timing info
|
||||
if (rb.isDebugTimings()) {
|
||||
rb.addDebugInfo("timing", timer.asNamedList() );
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (ExitableDirectoryReader.ExitingReaderException ex) {
|
||||
log.warn( "Query: " + req.getParamString() + "; " + ex.getMessage());
|
||||
throw new SolrException(ErrorCode.BAD_REQUEST,
|
||||
"Request took too long during query expansion. Terminating request.");
|
||||
} finally {
|
||||
SolrQueryTimeoutImpl.reset();
|
||||
}
|
||||
} else {
|
||||
// a distributed request
|
||||
|
||||
|
|
|
@ -48,6 +48,7 @@ import org.apache.lucene.index.LeafReader;
|
|||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.DocsEnum;
|
||||
import org.apache.lucene.index.ExitableDirectoryReader;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.Fields;
|
||||
|
@ -193,7 +194,9 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn
|
|||
// this reader supports reopen
|
||||
private static DirectoryReader wrapReader(SolrCore core, DirectoryReader reader) {
|
||||
assert reader != null;
|
||||
return UninvertingReader.wrap(reader, core.getLatestSchema().getUninversionMap(reader));
|
||||
return ExitableDirectoryReader.wrap
|
||||
(UninvertingReader.wrap(reader, core.getLatestSchema().getUninversionMap(reader)),
|
||||
SolrQueryTimeoutImpl.getInstance());
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -0,0 +1,86 @@
|
|||
package org.apache.solr.search;
|
||||
|
||||
/*
|
||||
* 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.index.QueryTimeout;
|
||||
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static java.lang.System.nanoTime;
|
||||
|
||||
/**
|
||||
* Implementation of {@link QueryTimeout} that is used by Solr.
|
||||
* It uses a ThreadLocal variable to track the timeoutAt value
|
||||
* for each request thread.
|
||||
*/
|
||||
public class SolrQueryTimeoutImpl implements QueryTimeout {
|
||||
/**
|
||||
* The ThreadLocal variable to store the time beyond which, the processing should exit.
|
||||
*/
|
||||
public static ThreadLocal<Long> timeoutAt = new ThreadLocal<Long>() {
|
||||
@Override
|
||||
protected Long initialValue() {
|
||||
return nanoTime() + Long.MAX_VALUE;
|
||||
}
|
||||
};
|
||||
|
||||
private SolrQueryTimeoutImpl() { }
|
||||
private static SolrQueryTimeoutImpl instance = new SolrQueryTimeoutImpl();
|
||||
|
||||
/** Return singleton instance */
|
||||
public static SolrQueryTimeoutImpl getInstance() {
|
||||
return instance;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the current value of timeoutAt.
|
||||
*/
|
||||
public static Long get() {
|
||||
return timeoutAt.get();
|
||||
}
|
||||
|
||||
/**
|
||||
* Return true if a timeoutAt value is set and the current time has exceeded the set timeOut.
|
||||
*/
|
||||
@Override
|
||||
public boolean shouldExit() {
|
||||
return get() - nanoTime() < 0L;
|
||||
}
|
||||
|
||||
/**
|
||||
* Method to set the time at which the timeOut should happen.
|
||||
* @param timeAllowed set the time at which this thread should timeout.
|
||||
*/
|
||||
public static void set(Long timeAllowed) {
|
||||
long time = nanoTime() + TimeUnit.NANOSECONDS.convert(timeAllowed, TimeUnit.MILLISECONDS);
|
||||
timeoutAt.set(time);
|
||||
}
|
||||
|
||||
/**
|
||||
* Cleanup the ThreadLocal timeout value.
|
||||
*/
|
||||
public static void reset() {
|
||||
timeoutAt.remove();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "timeoutAt: " + get() + " (System.nanoTime(): " + nanoTime() + ")";
|
||||
}
|
||||
}
|
||||
|
|
@ -492,6 +492,10 @@ public class TestDistributedSearch extends BaseDistributedSearchTestCase {
|
|||
ShardParams.SHARDS_TOLERANT, "true");
|
||||
|
||||
// test group query
|
||||
// TODO: Remove this? This doesn't make any real sense now that timeAllowed might trigger early
|
||||
// termination of the request during Terms enumeration/Query expansion.
|
||||
// During such an exit, partial results isn't supported as it wouldn't make any sense.
|
||||
// Increasing the timeAllowed from 1 to 100 for now.
|
||||
queryPartialResults(upShards, upClients,
|
||||
"q", "*:*",
|
||||
"rows", 100,
|
||||
|
@ -500,7 +504,7 @@ public class TestDistributedSearch extends BaseDistributedSearchTestCase {
|
|||
"group.query", t1 + ":kings OR " + t1 + ":eggs",
|
||||
"group.limit", 10,
|
||||
"sort", i1 + " asc, id asc",
|
||||
CommonParams.TIME_ALLOWED, 1,
|
||||
CommonParams.TIME_ALLOWED, 100,
|
||||
ShardParams.SHARDS_INFO, "true",
|
||||
ShardParams.SHARDS_TOLERANT, "true");
|
||||
|
||||
|
|
|
@ -278,7 +278,9 @@ public class TestGroupingSearch extends SolrTestCaseJ4 {
|
|||
assertU(commit());
|
||||
|
||||
// Just checking if no errors occur
|
||||
assertJQ(req("q", "*:*", "group", "true", "group.query", "id:1", "group.query", "id:2", "timeAllowed", "1"));
|
||||
// TODO: Check if this makes any sense now that timeAllowed is also used during Terms enumeration.
|
||||
// The query can potentially timeout iterating over terms if this is set to too low.
|
||||
assertJQ(req("q", "*:*", "group", "true", "group.query", "id:1", "group.query", "id:2", "timeAllowed", "100"));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -0,0 +1,128 @@
|
|||
package org.apache.solr.cloud;
|
||||
|
||||
/*
|
||||
* 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.util.TestUtil;
|
||||
import org.apache.solr.client.solrj.SolrServerException;
|
||||
import org.apache.solr.client.solrj.response.QueryResponse;
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.common.params.ModifiableSolrParams;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import static org.apache.solr.common.SolrException.ErrorCode;
|
||||
|
||||
/**
|
||||
* Distributed test for {@link org.apache.lucene.index.ExitableDirectoryReader}
|
||||
*/
|
||||
public class CloudExitableDirectoryReaderTest extends AbstractFullDistribZkTestBase {
|
||||
public static Logger log = LoggerFactory.getLogger(CloudExitableDirectoryReaderTest.class);
|
||||
private static final int NUM_DOCS_PER_TYPE = 2000;
|
||||
|
||||
public CloudExitableDirectoryReaderTest() {
|
||||
configString = "solrconfig-tlog.xml";
|
||||
schemaString = "schema.xml";
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getCloudSolrConfig() {
|
||||
return configString;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doTest() throws Exception {
|
||||
handle.clear();
|
||||
handle.put("timestamp", SKIPVAL);
|
||||
waitForRecoveriesToFinish(false);
|
||||
indexDocs();
|
||||
doTimeoutTests();
|
||||
}
|
||||
|
||||
public void indexDocs() throws Exception {
|
||||
int counter = 1;
|
||||
|
||||
for(; (counter % NUM_DOCS_PER_TYPE) != 0; counter++ )
|
||||
indexDoc(sdoc("id", Integer.toString(counter), "name", "a" + counter));
|
||||
|
||||
counter++;
|
||||
for(; (counter % NUM_DOCS_PER_TYPE) != 0; counter++ )
|
||||
indexDoc(sdoc("id", Integer.toString(counter), "name", "b" + counter));
|
||||
|
||||
counter++;
|
||||
for(; counter % NUM_DOCS_PER_TYPE != 0; counter++ )
|
||||
indexDoc(sdoc("id", Integer.toString(counter), "name", "dummy term doc" + counter));
|
||||
|
||||
commit();
|
||||
}
|
||||
|
||||
public void doTimeoutTests() throws Exception {
|
||||
assertFail(params("q", "name:a*", "timeAllowed", "1"));
|
||||
|
||||
long oneSecond = 1000L; // query rewriting for NUM_DOCS_PER_TYPE terms should take less time than this
|
||||
Long timeAllowed = TestUtil.nextLong(random(), oneSecond, Long.MAX_VALUE);
|
||||
assertSuccess(params("q", "name:a*", "timeAllowed",timeAllowed.toString()));
|
||||
|
||||
assertFail(params("q", "name:a*", "timeAllowed", "1"));
|
||||
|
||||
timeAllowed = TestUtil.nextLong(random(), oneSecond, Long.MAX_VALUE);
|
||||
assertSuccess(params("q", "name:b*", "timeAllowed",timeAllowed.toString()));
|
||||
|
||||
timeAllowed = TestUtil.nextLong(random(), Long.MIN_VALUE, -1L); // negative timeAllowed should disable timeouts
|
||||
assertSuccess(params("q", "name:b*", "timeAllowed",timeAllowed.toString()));
|
||||
|
||||
assertSuccess(params("q","name:b*")); // no time limitation
|
||||
}
|
||||
|
||||
/**
|
||||
* execute a request, verify that we get an expected error
|
||||
*/
|
||||
public void assertFail(ModifiableSolrParams p) throws Exception {
|
||||
String timeoutMessage = "Request took too long during query expansion. Terminating request.";
|
||||
|
||||
try {
|
||||
ignoreException(timeoutMessage);
|
||||
queryServer(p);
|
||||
fail("no exception matching expected: " + ErrorCode.BAD_REQUEST.code + ": " + timeoutMessage);
|
||||
} catch (SolrServerException e) {
|
||||
assertTrue("Exception " + e.getCause() + " is not a SolrException:\n" + prettyStackTrace(e.getCause()),
|
||||
e.getCause() instanceof SolrException);
|
||||
assertEquals(ErrorCode.BAD_REQUEST.code, ((SolrException)e.getCause()).code());
|
||||
assertTrue("Expected error message substr not found: " + timeoutMessage + " <!< " + e.getMessage(),
|
||||
e.getMessage().contains(timeoutMessage));
|
||||
} finally {
|
||||
unIgnoreException(timeoutMessage);
|
||||
}
|
||||
}
|
||||
|
||||
public void assertSuccess(ModifiableSolrParams p) throws Exception {
|
||||
QueryResponse response = queryServer(p);
|
||||
assertEquals("Wrong #docs in response", NUM_DOCS_PER_TYPE - 1, response.getResults().getNumFound());
|
||||
}
|
||||
|
||||
public String prettyStackTrace(Throwable t) {
|
||||
StringBuilder builder = new StringBuilder();
|
||||
for (StackTraceElement elem : t.getStackTrace()) {
|
||||
builder.append(" at ");
|
||||
builder.append(elem.toString());
|
||||
builder.append('\n');
|
||||
}
|
||||
return builder.toString();
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,95 @@
|
|||
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.solr.SolrTestCaseJ4;
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
/**
|
||||
* Test that checks that long running queries are exited by Solr using the
|
||||
* SolrQueryTimeoutImpl implementation.
|
||||
*/
|
||||
public class ExitableDirectoryReaderTest extends SolrTestCaseJ4 {
|
||||
|
||||
static int NUM_DOCS_PER_TYPE = 2000;
|
||||
static final String assertionString = "//result[@numFound='"+ (NUM_DOCS_PER_TYPE - 1) + "']";
|
||||
|
||||
@BeforeClass
|
||||
public static void beforeClass() throws Exception {
|
||||
System.setProperty("enable.update.log", "false"); // schema12 doesn't support _version_
|
||||
initCore("solrconfig-nocache.xml", "schema12.xml");
|
||||
createIndex();
|
||||
}
|
||||
|
||||
public static void createIndex() {
|
||||
int counter = 1;
|
||||
|
||||
for(; (counter % NUM_DOCS_PER_TYPE) != 0; counter++ )
|
||||
assertU(adoc("id", Integer.toString(counter), "name", "a" + counter));
|
||||
|
||||
counter++;
|
||||
for(; (counter % NUM_DOCS_PER_TYPE) != 0; counter++ )
|
||||
assertU(adoc("id", Integer.toString(counter), "name", "b" + counter));
|
||||
|
||||
counter++;
|
||||
for(; counter % NUM_DOCS_PER_TYPE != 0; counter++ )
|
||||
assertU(adoc("id", Integer.toString(counter), "name", "dummy term doc" + counter));
|
||||
|
||||
assertU(commit());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPrefixQuery() {
|
||||
assertQEx("", req("q","name:a*", "indent","true","timeAllowed","1")
|
||||
, SolrException.ErrorCode.BAD_REQUEST
|
||||
);
|
||||
|
||||
assertQ(req("q","name:a*", "indent","true", "timeAllowed","1000"), assertionString);
|
||||
|
||||
assertQEx("", req("q","name:a*", "indent","true", "timeAllowed","1")
|
||||
, SolrException.ErrorCode.BAD_REQUEST
|
||||
);
|
||||
|
||||
assertQ(req("q","name:b*", "indent","true", "timeAllowed","1000"), assertionString);
|
||||
|
||||
assertQ(req("q","name:b*", "indent","true", "timeAllowed",Long.toString(Long.MAX_VALUE)), assertionString);
|
||||
|
||||
assertQ(req("q","name:b*", "indent","true", "timeAllowed","-7")); // negative timeAllowed should disable timeouts
|
||||
|
||||
assertQ(req("q","name:b*", "indent","true"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testQueriesOnDocsWithMultipleTerms() {
|
||||
assertQ(req("q","name:dummy", "indent","true", "timeAllowed","1000"), assertionString);
|
||||
|
||||
// This should pass even though this may take more than the 'timeAllowed' time, it doesn't take long
|
||||
// to iterate over 1 term (dummy).
|
||||
assertQ(req("q","name:dummy", "indent","true", "timeAllowed","1000"), assertionString);
|
||||
|
||||
assertQEx("", req("q","name:doc*", "indent","true", "timeAllowed","1")
|
||||
, SolrException.ErrorCode.BAD_REQUEST
|
||||
);
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -142,7 +142,28 @@ public abstract class SolrParams implements Serializable {
|
|||
throw new SolrException( SolrException.ErrorCode.BAD_REQUEST, ex.getMessage(), ex );
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/** Returns the Long value of the param, or null if not set */
|
||||
public Long getLong(String param) {
|
||||
String val = get(param);
|
||||
try {
|
||||
return val == null ? null : Long.valueOf(val);
|
||||
} catch (Exception ex) {
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, ex.getMessage(), ex);
|
||||
}
|
||||
}
|
||||
|
||||
/** Returns the long value of the param, or def if not set */
|
||||
public long getLong(String param, long def) {
|
||||
String val = get(param);
|
||||
try {
|
||||
return val == null ? def : Long.parseLong(val);
|
||||
} catch (Exception ex) {
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, ex.getMessage(), ex);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @return The int value of the field param, or the value for param
|
||||
* or <code>null</code> if neither is set.
|
||||
|
|
Loading…
Reference in New Issue