LUCENE-8766: Add monitor subproject

This commit is contained in:
Alan Woodward 2019-06-13 09:40:57 +01:00
parent 9a796117fe
commit 251dbe7cea
80 changed files with 8859 additions and 0 deletions

View File

@ -25,6 +25,7 @@
<buildFile url="file://$PROJECT_DIR$/lucene/highlighter/build.xml" />
<buildFile url="file://$PROJECT_DIR$/lucene/join/build.xml" />
<buildFile url="file://$PROJECT_DIR$/lucene/luke/build.xml" />
<buildFile url="file://$PROJECT_DIR$/lucene/luwak/build.xml" />
<buildFile url="file://$PROJECT_DIR$/lucene/memory/build.xml" />
<buildFile url="file://$PROJECT_DIR$/lucene/misc/build.xml" />
<buildFile url="file://$PROJECT_DIR$/lucene/queries/build.xml" />

View File

@ -31,6 +31,7 @@
<module group="Lucene/Other" filepath="$PROJECT_DIR$/lucene/highlighter/highlighter.iml" />
<module group="Lucene/Other" filepath="$PROJECT_DIR$/lucene/join/join.iml" />
<module group="Lucene/Other" filepath="$PROJECT_DIR$/lucene/luke/luke.iml" />
<module group="Lucene/Other" filepath="$PROJECT_DIR$/lucene/monitor/monitor.iml" />
<module group="Lucene/Other" filepath="$PROJECT_DIR$/lucene/memory/memory.iml" />
<module group="Lucene/Other" filepath="$PROJECT_DIR$/lucene/misc/misc.iml" />
<module group="Lucene/Other" filepath="$PROJECT_DIR$/lucene/queries/queries.iml" />

View File

@ -172,6 +172,14 @@
<option name="TEST_SEARCH_SCOPE"><value defaultName="singleModule" /></option>
<patterns><pattern testClass=".*\.Test[^.]*|.*\.[^.]*Test" /></patterns>
</configuration>
<configuration default="false" name="Module monitor" type="JUnit" factoryName="JUnit">
<module name="monitor" />
<option name="TEST_OBJECT" value="pattern" />
<option name="WORKING_DIRECTORY" value="file://$PROJECT_DIR$/idea-build/lucene/monitor" />
<option name="VM_PARAMETERS" value="-ea -DtempDir=temp" />
<option name="TEST_SEARCH_SCOPE"><value defaultName="singleModule" /></option>
<patterns><pattern testClass=".*\.Test[^.]*|.*\.[^.]*Test" /></patterns>
</configuration>
<configuration default="false" name="Module queries" type="JUnit" factoryName="JUnit">
<module name="queries" />
<option name="TEST_OBJECT" value="pattern" />

View File

@ -0,0 +1,32 @@
<?xml version="1.0" encoding="UTF-8"?>
<module type="JAVA_MODULE" version="4">
<component name="NewModuleRootManager" inherit-compiler-output="false">
<output url="file://$MODULE_DIR$/../../idea-build/lucene/monitor/classes/java" />
<output-test url="file://$MODULE_DIR$/../../idea-build/lucene/monitor/classes/test" />
<exclude-output />
<content url="file://$MODULE_DIR$">
<sourceFolder url="file://$MODULE_DIR$/src/java" isTestSource="false" />
<sourceFolder url="file://$MODULE_DIR$/src/resources" isTestSource="false" />
<sourceFolder url="file://$MODULE_DIR$/src/test" isTestSource="true" />
<excludeFolder url="file://$MODULE_DIR$/work" />
</content>
<orderEntry type="inheritedJdk" />
<orderEntry type="sourceFolder" forTests="false" />
<orderEntry type="module-library">
<library>
<CLASSES>
<root url="file://$MODULE_DIR$/lib" />
</CLASSES>
<JAVADOC />
<SOURCES />
<jarDirectory url="file://$MODULE_DIR$/lib" recursive="false" />
</library>
</orderEntry>
<orderEntry type="library" scope="TEST" name="JUnit" level="project" />
<orderEntry type="module" scope="TEST" module-name="lucene-test-framework" />
<orderEntry type="module" module-name="lucene-core" />
<orderEntry type="module" module-name="analysis-common" />
<orderEntry type="module" module-name="memory" />
<orderEntry type="module" module-name="queryparser" />
</component>
</module>

View File

@ -0,0 +1,70 @@
<!--
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.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>org.apache.lucene</groupId>
<artifactId>lucene-parent</artifactId>
<version>@version@</version>
<relativePath>../pom.xml</relativePath>
</parent>
<groupId>org.apache.lucene</groupId>
<artifactId>lucene-monitor</artifactId>
<packaging>jar</packaging>
<name>Lucene Monitor</name>
<description>
High-performance single-document index to compare against Query
</description>
<properties>
<module-directory>lucene/monitor</module-directory>
<relative-top-level>../../..</relative-top-level>
<module-path>${relative-top-level}/${module-directory}</module-path>
</properties>
<scm>
<connection>scm:git:${vc-anonymous-base-url}</connection>
<developerConnection>scm:git:${vc-dev-base-url}</developerConnection>
<url>${vc-browse-base-url};f=${module-directory}</url>
</scm>
<dependencies>
<dependency>
<!-- lucene-test-framework dependency must be declared before lucene-core -->
<groupId>org.apache.lucene</groupId>
<artifactId>lucene-test-framework</artifactId>
<scope>test</scope>
</dependency>
@lucene-monitor.internal.dependencies@
@lucene-monitor.external.dependencies@
@lucene-monitor.internal.test.dependencies@
@lucene-monitor.external.test.dependencies@
</dependencies>
<build>
<sourceDirectory>${module-path}/src/java</sourceDirectory>
<testSourceDirectory>${module-path}/src/test</testSourceDirectory>
<testResources>
<testResource>
<directory>${project.build.testSourceDirectory}</directory>
<excludes>
<exclude>**/*.java</exclude>
</excludes>
</testResource>
</testResources>
</build>
</project>

View File

@ -55,6 +55,7 @@
<module>join</module>
<module>memory</module>
<module>misc</module>
<module>monitor</module>
<module>queries</module>
<module>queryparser</module>
<module>replicator</module>

53
lucene/monitor/build.xml Normal file
View File

@ -0,0 +1,53 @@
<?xml version="1.0"?>
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<project name="monitor" default="default">
<description>
Reverse-search implementation for monitoring and classification
</description>
<import file="../module-build.xml"/>
<path id="test.classpath">
<path refid="test.base.classpath"/>
<pathelement path="${memory.jar}"/>
</path>
<path id="classpath">
<pathelement path="${memory.jar}"/>
<pathelement path="${analyzers-common.jar}"/>
<pathelement path="${queryparser.jar}"/>
<path refid="base.classpath"/>
</path>
<target name="init" depends="module-build.init,jar-analyzers-common,jar-queryparser,jar-memory"/>
<target name="javadocs" depends="javadocs-memory,compile-core,check-javadocs-uptodate"
unless="javadocs-uptodate-${name}">
<invoke-module-javadoc>
<links>
<link href="../memory"/>
<link href="../analyzers-common"/>
<link href="../queryparser"/>
</links>
</invoke-module-javadoc>
</target>
</project>

21
lucene/monitor/ivy.xml Normal file
View File

@ -0,0 +1,21 @@
<!--
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.
-->
<ivy-module version="2.0">
<info organisation="org.apache.lucene" module="luwak"/>
</ivy-module>

View File

@ -0,0 +1,133 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
/**
* Class used to match candidate queries selected by a Presearcher from a Monitor
* query index.
*/
public abstract class CandidateMatcher<T extends QueryMatch> {
/**
* The searcher to run candidate queries against
*/
protected final IndexSearcher searcher;
private final Map<String, Exception> errors = new HashMap<>();
private final List<MatchHolder<T>> matches;
private long searchTime = System.nanoTime();
private static class MatchHolder<T> {
Map<String, T> matches = new HashMap<>();
}
/**
* Creates a new CandidateMatcher for the supplied DocumentBatch
*
* @param searcher the IndexSearcher to run queries against
*/
public CandidateMatcher(IndexSearcher searcher) {
this.searcher = searcher;
int docCount = searcher.getIndexReader().maxDoc();
this.matches = new ArrayList<>(docCount);
for (int i = 0; i < docCount; i++) {
this.matches.add(new MatchHolder<>());
}
}
/**
* Runs the supplied query against this CandidateMatcher's set of documents, storing any
* resulting match, and recording the query in the presearcher hits
*
* @param queryId the query id
* @param matchQuery the query to run
* @param metadata the query metadata
* @throws IOException on IO errors
*/
protected abstract void matchQuery(String queryId, Query matchQuery, Map<String, String> metadata) throws IOException;
/**
* Record a match
*
* @param match a QueryMatch object
*/
protected final void addMatch(T match, int doc) {
MatchHolder<T> docMatches = matches.get(doc);
docMatches.matches.compute(match.getQueryId(), (key, oldValue) -> {
if (oldValue != null) {
return resolve(match, oldValue);
}
return match;
});
}
/**
* If two matches from the same query are found (for example, two branches of a disjunction),
* combine them.
*
* @param match1 the first match found
* @param match2 the second match found
* @return a Match object that combines the two
*/
public abstract T resolve(T match1, T match2);
/**
* Called by the Monitor if running a query throws an Exception
*/
void reportError(String queryId, Exception e) {
this.errors.put(queryId, e);
}
/**
* @return the matches from this matcher
*/
final MultiMatchingQueries<T> finish(long buildTime, int queryCount) {
doFinish();
this.searchTime = TimeUnit.MILLISECONDS.convert(System.nanoTime() - searchTime, TimeUnit.NANOSECONDS);
List<Map<String, T>> results = new ArrayList<>();
for (MatchHolder<T> matchHolder : matches) {
results.add(matchHolder.matches);
}
return new MultiMatchingQueries<>(results, errors, buildTime, searchTime, queryCount, matches.size());
}
/**
* Called when all monitoring of a batch of documents is complete
*/
protected void doFinish() { }
/**
* Copy all matches from another CandidateMatcher
*/
protected void copyMatches(CandidateMatcher<T> other) {
this.matches.clear();
this.matches.addAll(other.matches);
}
}

View File

@ -0,0 +1,84 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.IOException;
import java.util.Map;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.Scorable;
import org.apache.lucene.search.ScoreMode;
import org.apache.lucene.search.SimpleCollector;
abstract class CollectingMatcher<T extends QueryMatch> extends CandidateMatcher<T> {
private final ScoreMode scoreMode;
CollectingMatcher(IndexSearcher searcher, ScoreMode scoreMode) {
super(searcher);
this.scoreMode = scoreMode;
}
@Override
protected void matchQuery(final String queryId, Query matchQuery, Map<String, String> metadata) throws IOException {
searcher.search(matchQuery, new MatchCollector(queryId, scoreMode));
}
/**
* Called when a query matches a Document
*
* @param queryId the query ID
* @param doc the index of the document in the DocumentBatch
* @param scorer the Scorer for this query
* @return a match object
* @throws IOException on IO error
*/
protected abstract T doMatch(String queryId, int doc, Scorable scorer) throws IOException;
private class MatchCollector extends SimpleCollector {
private final String queryId;
private final ScoreMode scoreMode;
private Scorable scorer;
MatchCollector(String queryId, ScoreMode scoreMode) {
this.queryId = queryId;
this.scoreMode = scoreMode;
}
@Override
public void collect(int doc) throws IOException {
T match = doMatch(queryId, doc, scorer);
if (match != null) {
addMatch(match, doc);
}
}
@Override
public void setScorer(Scorable scorer) {
this.scorer = scorer;
}
@Override
public ScoreMode scoreMode() {
return scoreMode;
}
}
}

View File

@ -0,0 +1,196 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.Objects;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
import org.apache.lucene.util.NamedThreadFactory;
/**
* Utility class for concurrently loading queries into a Monitor.
* <p>
* This is useful to speed up startup times for a Monitor. You can use multiple
* threads to parse and index queries before starting matches.
* <p>
* Use as follows:
* <pre class="prettyprint">
* List&lt;QueryError&gt; errors = new ArrayList&lt;&gt;();
* try (ConcurrentQueryLoader loader = new ConcurrentQueryLoader(monitor, errors)) {
* for (MonitorQuery mq : getQueries()) {
* loader.add(mq);
* }
* }
* </pre>
* <p>
* The Monitor's MonitorQueryParser must be thread-safe for this to work correctly.
*/
public class ConcurrentQueryLoader implements Closeable {
private final Monitor monitor;
private final ExecutorService executor;
private final CountDownLatch shutdownLatch;
private final BlockingQueue<MonitorQuery> queue;
private boolean shutdown = false;
private List<IOException> errors = new ArrayList<>();
public static final int DEFAULT_QUEUE_SIZE = 2000;
/**
* Create a new ConcurrentQueryLoader for a {@link Monitor}
*
* @param monitor Monitor
*/
public ConcurrentQueryLoader(Monitor monitor) {
this(monitor, Runtime.getRuntime().availableProcessors(), DEFAULT_QUEUE_SIZE);
}
/**
* Create a new ConcurrentQueryLoader
*
* @param monitor the Monitor to load queries to
* @param threads the number of threads to use
* @param queueSize the size of the buffer to hold queries in
*/
public ConcurrentQueryLoader(Monitor monitor, int threads, int queueSize) {
this.monitor = monitor;
this.queue = new LinkedBlockingQueue<>(queueSize);
this.executor = Executors.newFixedThreadPool(threads, new NamedThreadFactory("loader"));
this.shutdownLatch = new CountDownLatch(threads);
for (int i = 0; i < threads; i++) {
this.executor.submit(new Worker(queueSize / threads));
}
}
/**
* Add a MonitorQuery to the loader's internal buffer
* <p>
* If the buffer is full, this will block until there is room to add
* the MonitorQuery
*
* @param mq the monitor query
* @throws InterruptedException if interrupted while waiting
*/
public void add(MonitorQuery mq) throws InterruptedException {
if (shutdown)
throw new IllegalStateException("ConcurrentQueryLoader has been shutdown, cannot add new queries");
this.queue.put(mq);
}
@Override
public void close() throws IOException {
this.shutdown = true;
this.executor.shutdown();
try {
this.shutdownLatch.await();
} catch (InterruptedException e) {
// fine
}
if (errors.size() > 0) {
IOException e = new IOException();
errors.forEach(e::addSuppressed);
throw e;
}
}
private class Worker implements Runnable {
final List<MonitorQuery> workerQueue;
final int queueSize;
boolean running = true;
Worker(int queueSize) {
workerQueue = new ArrayList<>(queueSize);
this.queueSize = queueSize;
}
@Override
public void run() {
try {
while (running) {
workerQueue.clear();
drain(queue, workerQueue, queueSize, 100, TimeUnit.MILLISECONDS);
if (workerQueue.size() == 0 && shutdown)
running = false;
if (workerQueue.size() > 0) {
monitor.register(workerQueue);
}
}
} catch (IOException e) {
errors.add(e);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
} finally {
shutdownLatch.countDown();
}
}
}
/**
* Drains the queue as {@link BlockingQueue#drainTo(Collection, int)}, but if the requested
* {@code numElements} elements are not available, it will wait for them up to the specified
* timeout.
* <p>
* Taken from Google Guava 18.0 Queues
*
* @param q the blocking queue to be drained
* @param buffer where to add the transferred elements
* @param numElements the number of elements to be waited for
* @param timeout how long to wait before giving up, in units of {@code unit}
* @param unit a {@code TimeUnit} determining how to interpret the timeout parameter
* @param <E> the type of the queue
* @return the number of elements transferred
* @throws InterruptedException if interrupted while waiting
*/
private static <E> int drain(BlockingQueue<E> q, Collection<? super E> buffer, int numElements,
long timeout, TimeUnit unit) throws InterruptedException {
buffer = Objects.requireNonNull(buffer);
/*
* This code performs one System.nanoTime() more than necessary, and in return, the time to
* execute Queue#drainTo is not added *on top* of waiting for the timeout (which could make
* the timeout arbitrarily inaccurate, given a queue that is slow to drain).
*/
long deadline = System.nanoTime() + unit.toNanos(timeout);
int added = 0;
while (added < numElements) {
// we could rely solely on #poll, but #drainTo might be more efficient when there are multiple
// elements already available (e.g. LinkedBlockingQueue#drainTo locks only once)
added += q.drainTo(buffer, numElements - added);
if (added < numElements) { // not enough elements immediately available; will have to poll
E e = q.poll(deadline - System.nanoTime(), TimeUnit.NANOSECONDS);
if (e == null) {
break; // we already waited enough, and there are no more elements in sight
}
buffer.add(e);
added++;
}
}
return added;
}
}

View File

@ -0,0 +1,45 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.search.Query;
/**
* Builds a {@link QueryTree} for a query that needs custom treatment
*
* The default query analyzers will use the QueryVisitor API to extract
* terms from queries. If different handling is needed, implement a
* CustomQueryHandler and pass it to the presearcher
*/
public interface CustomQueryHandler {
/**
* Builds a {@link QueryTree} node from a query
*/
QueryTree handleQuery(Query query, TermWeightor termWeightor);
/**
* Adds additional processing to the {@link TokenStream} over a document's
* terms index
*/
default TokenStream wrapTermStream(String field, TokenStream in) {
return in;
}
}

View File

@ -0,0 +1,121 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.Closeable;
import java.io.IOException;
import java.util.Arrays;
import java.util.function.Supplier;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.memory.MemoryIndex;
import org.apache.lucene.store.ByteBuffersDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.IOUtils;
abstract class DocumentBatch implements Closeable, Supplier<LeafReader> {
/**
* Create a DocumentBatch containing a single InputDocument
*
* @param doc the document to add
* @return the batch containing the input document
*/
public static DocumentBatch of(Analyzer analyzer, Document doc) {
return new SingletonDocumentBatch(analyzer, doc);
}
/**
* Create a DocumentBatch containing a set of InputDocuments
*
* @param docs Collection of documents to add
* @return the batch containing the input documents
*/
public static DocumentBatch of(Analyzer analyzer, Document... docs) {
return new MultiDocumentBatch(analyzer, docs);
}
// Implementation of DocumentBatch for collections of documents
private static class MultiDocumentBatch extends DocumentBatch {
private final Directory directory = new ByteBuffersDirectory();
private final LeafReader reader;
MultiDocumentBatch(Analyzer analyzer, Document... docs) {
IndexWriterConfig iwc = new IndexWriterConfig(analyzer);
try (IndexWriter writer = new IndexWriter(directory, iwc)) {
this.reader = build(writer, docs);
} catch (IOException e) {
throw new RuntimeException(e); // This is a RAMDirectory, so should never happen...
}
}
@Override
public LeafReader get() {
return reader;
}
private LeafReader build(IndexWriter writer, Document... docs) throws IOException {
writer.addDocuments(Arrays.asList(docs));
writer.commit();
writer.forceMerge(1);
LeafReader reader = DirectoryReader.open(directory).leaves().get(0).reader();
assert reader != null;
return reader;
}
@Override
public void close() throws IOException {
IOUtils.close(reader, directory);
}
}
// Specialized class for batches containing a single object - MemoryIndex benchmarks as
// better performing than RAMDirectory for this case
private static class SingletonDocumentBatch extends DocumentBatch {
private final LeafReader reader;
private SingletonDocumentBatch(Analyzer analyzer, Document doc) {
MemoryIndex memoryindex = new MemoryIndex(true, true);
for (IndexableField field : doc) {
memoryindex.addField(field, analyzer);
}
memoryindex.freeze();
reader = (LeafReader) memoryindex.createSearcher().getIndexReader();
}
@Override
public LeafReader get() {
return reader;
}
@Override
public void close() throws IOException {
reader.close();
}
}
}

View File

@ -0,0 +1,81 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.IOException;
import java.util.Map;
import java.util.Objects;
import org.apache.lucene.search.Explanation;
import org.apache.lucene.search.Query;
/**
* A query match containing the score explanation of the match
*/
public class ExplainingMatch extends QueryMatch {
/**
* A MatcherFactory for producing ExplainingMatches
*/
public static final MatcherFactory<ExplainingMatch> MATCHER = searcher -> new CandidateMatcher<ExplainingMatch>(searcher) {
@Override
protected void matchQuery(String queryId, Query matchQuery, Map<String, String> metadata) throws IOException {
int maxDocs = searcher.getIndexReader().maxDoc();
for (int i = 0; i < maxDocs; i++) {
Explanation explanation = searcher.explain(matchQuery, i);
if (explanation.isMatch())
addMatch(new ExplainingMatch(queryId, explanation), i);
}
}
@Override
public ExplainingMatch resolve(ExplainingMatch match1, ExplainingMatch match2) {
return new ExplainingMatch(match1.getQueryId(),
Explanation.match(match1.getExplanation().getValue().doubleValue() + match2.getExplanation().getValue().doubleValue(),
"sum of:", match1.getExplanation(), match2.getExplanation()));
}
};
private final Explanation explanation;
ExplainingMatch(String queryId, Explanation explanation) {
super(queryId);
this.explanation = explanation;
}
/**
* @return the Explanation
*/
public Explanation getExplanation() {
return explanation;
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
if (!super.equals(o)) return false;
ExplainingMatch that = (ExplainingMatch) o;
return Objects.equals(explanation, that.explanation);
}
@Override
public int hashCode() {
return Objects.hash(super.hashCode(), explanation);
}
}

View File

@ -0,0 +1,103 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.IOException;
import java.util.Objects;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.*;
import org.apache.lucene.search.Matches;
/**
* Query wrapper that forces its wrapped Query to use the default doc-by-doc
* BulkScorer.
*/
class ForceNoBulkScoringQuery extends Query {
private final Query inner;
public ForceNoBulkScoringQuery(Query inner) {
this.inner = inner;
}
@Override
public Query rewrite(IndexReader reader) throws IOException {
Query rewritten = inner.rewrite(reader);
if (rewritten != inner)
return new ForceNoBulkScoringQuery(rewritten);
return super.rewrite(reader);
}
@Override
public void visit(QueryVisitor visitor) {
inner.visit(visitor);
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
ForceNoBulkScoringQuery that = (ForceNoBulkScoringQuery) o;
return Objects.equals(inner, that.inner);
}
@Override
public int hashCode() {
return Objects.hash(inner);
}
public Query getWrappedQuery() {
return inner;
}
@Override
public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
final Weight innerWeight = inner.createWeight(searcher, scoreMode, boost);
return new Weight(ForceNoBulkScoringQuery.this) {
@Override
public boolean isCacheable(LeafReaderContext ctx) {
return innerWeight.isCacheable(ctx);
}
@Override
public Explanation explain(LeafReaderContext leafReaderContext, int i) throws IOException {
return innerWeight.explain(leafReaderContext, i);
}
@Override
public Scorer scorer(LeafReaderContext leafReaderContext) throws IOException {
return innerWeight.scorer(leafReaderContext);
}
@Override
public Matches matches(LeafReaderContext context, int doc) throws IOException {
return innerWeight.matches(context, doc);
}
};
}
@Override
public String toString(String s) {
return "NoBulkScorer(" + inner.toString(s) + ")";
}
}

View File

@ -0,0 +1,230 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.IOException;
import java.util.*;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.Matches;
import org.apache.lucene.search.MatchesIterator;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreMode;
import org.apache.lucene.search.Weight;
/**
* QueryMatch object that contains the hit positions of a matching Query
* <p>
* If the Query does not support interval iteration (eg, if it gets re-written to
* a Filter), then no hits will be reported, but an IntervalsQueryMatch will still
* be returned from an IntervalsMatcher to indicate a match.
*/
public class HighlightsMatch extends QueryMatch {
public static final MatcherFactory<HighlightsMatch> MATCHER = searcher -> new CandidateMatcher<HighlightsMatch>(searcher) {
@Override
protected void matchQuery(String queryId, Query matchQuery, Map<String, String> metadata) throws IOException {
Weight w = searcher.createWeight(searcher.rewrite(matchQuery), ScoreMode.COMPLETE_NO_SCORES, 1);
for (LeafReaderContext ctx : searcher.getIndexReader().leaves()) {
for (int i = 0; i < ctx.reader().maxDoc(); i++) {
Matches matches = w.matches(ctx, i);
if (matches != null) {
addMatch(buildMatch(matches, queryId), i);
}
}
}
}
@Override
public HighlightsMatch resolve(HighlightsMatch match1, HighlightsMatch match2) {
return HighlightsMatch.merge(match1.getQueryId(), match1, match2);
}
private HighlightsMatch buildMatch(Matches matches, String queryId) throws IOException {
HighlightsMatch m = new HighlightsMatch(queryId);
for (String field : matches) {
MatchesIterator mi = matches.getMatches(field);
while (mi.next()) {
MatchesIterator sub = mi.getSubMatches();
if (sub != null) {
while (sub.next()) {
m.addHit(field, sub.startPosition(), sub.endPosition(), sub.startOffset(), sub.endOffset());
}
}
else {
m.addHit(field, mi.startPosition(), mi.endPosition(), mi.startOffset(), mi.endOffset());
}
}
}
return m;
}
};
private final Map<String, Set<Hit>> hits;
HighlightsMatch(String queryId) {
super(queryId);
this.hits = new TreeMap<>();
}
/**
* @return a map of hits per field
*/
public Map<String, Set<Hit>> getHits() {
return Collections.unmodifiableMap(this.hits);
}
/**
* @return the fields in which matches have been found
*/
public Set<String> getFields() {
return Collections.unmodifiableSet(hits.keySet());
}
/**
* Get the hits for a specific field
*
* @param field the field
* @return the Hits found in this field
*/
public Collection<Hit> getHits(String field) {
Collection<Hit> found = hits.get(field);
if (found != null)
return Collections.unmodifiableCollection(found);
return Collections.emptyList();
}
/**
* @return the total number of hits for the query
*/
public int getHitCount() {
int c = 0;
for (Set<Hit> fieldhits : hits.values()) {
c += fieldhits.size();
}
return c;
}
static HighlightsMatch merge(String queryId, HighlightsMatch... matches) {
HighlightsMatch newMatch = new HighlightsMatch(queryId);
for (HighlightsMatch match : matches) {
for (String field : match.getFields()) {
Set<Hit> hitSet = newMatch.hits.computeIfAbsent(field, f -> new TreeSet<>());
hitSet.addAll(match.getHits(field));
}
}
return newMatch;
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (!(o instanceof HighlightsMatch)) return false;
if (!super.equals(o)) return false;
HighlightsMatch that = (HighlightsMatch) o;
if (hits != null ? !hits.equals(that.hits) : that.hits != null) return false;
return true;
}
@Override
public int hashCode() {
int result = super.hashCode();
result = 31 * result + (hits != null ? hits.hashCode() : 0);
return result;
}
@Override
public String toString() {
return super.toString() + "{hits=" + hits + "}";
}
void addHit(String field, int startPos, int endPos, int startOffset, int endOffset) {
Set<Hit> hitSet = hits.computeIfAbsent(field, f -> new TreeSet<>());
hitSet.add(new Hit(startPos, startOffset, endPos, endOffset));
}
/**
* Represents an individual hit
*/
public static class Hit implements Comparable<Hit> {
/**
* The start position
*/
public final int startPosition;
/**
* The start offset
*/
public final int startOffset;
/**
* The end positions
*/
public final int endPosition;
/**
* The end offset
*/
public final int endOffset;
public Hit(int startPosition, int startOffset, int endPosition, int endOffset) {
this.startPosition = startPosition;
this.startOffset = startOffset;
this.endPosition = endPosition;
this.endOffset = endOffset;
}
@Override
public boolean equals(Object obj) {
if (this == obj) return true;
if (!(obj instanceof Hit))
return false;
Hit other = (Hit) obj;
return this.startOffset == other.startOffset &&
this.endOffset == other.endOffset &&
this.startPosition == other.startPosition &&
this.endPosition == other.endPosition;
}
@Override
public int hashCode() {
int result = startPosition;
result = 31 * result + startOffset;
result = 31 * result + endPosition;
result = 31 * result + endOffset;
return result;
}
@Override
public String toString() {
return String.format(Locale.ROOT, "%d(%d)->%d(%d)", startPosition, startOffset, endPosition, endOffset);
}
@Override
public int compareTo(Hit other) {
if (this.startPosition != other.startPosition)
return Integer.compare(this.startPosition, other.startPosition);
return Integer.compare(this.endPosition, other.endPosition);
}
}
}

View File

@ -0,0 +1,35 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import org.apache.lucene.search.IndexSearcher;
/**
* Interface for the creation of new CandidateMatcher objects
*
* @param <T> a subclass of {@link CandidateMatcher}
*/
public interface MatcherFactory<T extends QueryMatch> {
/**
* Create a new {@link CandidateMatcher} object, to select
* queries to match against the passed-in IndexSearcher
*/
CandidateMatcher<T> createMatcher(IndexSearcher searcher);
}

View File

@ -0,0 +1,97 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.util.*;
/**
* Class to hold the results of matching a single {@link org.apache.lucene.document.Document}
* against queries held in the Monitor
*
* @param <T> the type of QueryMatch returned
*/
public class MatchingQueries<T extends QueryMatch> {
private final Map<String, T> matches;
private final Map<String, Exception> errors;
private final long queryBuildTime;
private final long searchTime;
private final int queriesRun;
MatchingQueries(Map<String, T> matches, Map<String, Exception> errors,
long queryBuildTime, long searchTime, int queriesRun) {
this.matches = Collections.unmodifiableMap(matches);
this.errors = Collections.unmodifiableMap(errors);
this.queryBuildTime = queryBuildTime;
this.searchTime = searchTime;
this.queriesRun = queriesRun;
}
/**
* Returns the QueryMatch for the given query, or null if it did not match
*
* @param queryId the query id
*/
public T matches(String queryId) {
return matches.get(queryId);
}
/**
* @return all matches
*/
public Collection<T> getMatches() {
return matches.values();
}
/**
* @return the number of queries that matched
*/
public int getMatchCount() {
return matches.size();
}
/**
* @return how long (in ms) it took to build the Presearcher query for the matcher run
*/
public long getQueryBuildTime() {
return queryBuildTime;
}
/**
* @return how long (in ms) it took to run the selected queries
*/
public long getSearchTime() {
return searchTime;
}
/**
* @return the number of queries passed to this CandidateMatcher during the matcher run
*/
public int getQueriesRun() {
return queriesRun;
}
/**
* @return a List of any MatchErrors created during the matcher run
*/
public Map<String, Exception> getErrors() {
return errors;
}
}

View File

@ -0,0 +1,409 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Matches;
import org.apache.lucene.search.MatchesIterator;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.ScoreMode;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.Weight;
import org.apache.lucene.util.NamedThreadFactory;
/**
* A Monitor contains a set of {@link Query} objects with associated IDs, and efficiently
* matches them against sets of {@link Document} objects.
*/
public class Monitor implements Closeable {
protected final Presearcher presearcher;
private final Analyzer analyzer;
private final QueryIndex queryIndex;
private final List<MonitorUpdateListener> listeners = new ArrayList<>();
private final long commitBatchSize;
private final ScheduledExecutorService purgeExecutor;
private long lastPurged = -1;
/**
* Create a non-persistent Monitor instance with the default term-filtering Presearcher
*
* @param analyzer to analyze {@link Document}s at match time
*/
public Monitor(Analyzer analyzer) throws IOException {
this(analyzer, new TermFilteredPresearcher());
}
/**
* Create a new non-persistent Monitor instance
*
* @param analyzer to analyze {@link Document}s at match time
* @param presearcher the presearcher to use
*/
public Monitor(Analyzer analyzer, Presearcher presearcher) throws IOException {
this(analyzer, presearcher, new MonitorConfiguration());
}
/**
* Create a new Monitor instance with a specific configuration
*
* @param analyzer to analyze {@link Document}s at match time
* @param config the configuration
*/
public Monitor(Analyzer analyzer, MonitorConfiguration config) throws IOException {
this(analyzer, new TermFilteredPresearcher(), config);
}
/**
* Create a new Monitor instance
*
* @param analyzer to analyze {@link Document}s at match time
* @param presearcher the presearcher to use
* @param configuration the configuration
*/
public Monitor(Analyzer analyzer, Presearcher presearcher,
MonitorConfiguration configuration) throws IOException {
this.analyzer = analyzer;
this.presearcher = presearcher;
this.queryIndex = new QueryIndex(configuration, presearcher);
long purgeFrequency = configuration.getPurgeFrequency();
this.purgeExecutor = Executors.newSingleThreadScheduledExecutor(new NamedThreadFactory("cache-purge"));
this.purgeExecutor.scheduleAtFixedRate(() -> {
try {
purgeCache();
} catch (Throwable e) {
listeners.forEach(l -> l.onPurgeError(e));
}
}, purgeFrequency, purgeFrequency, configuration.getPurgeFrequencyUnits());
this.commitBatchSize = configuration.getQueryUpdateBufferSize();
}
/**
* Register a {@link MonitorUpdateListener} that will be notified whenever changes
* are made to the Monitor's queryindex
*
* @param listener listener to register
*/
public void addQueryIndexUpdateListener(MonitorUpdateListener listener) {
listeners.add(listener);
}
/**
* @return Statistics for the internal query index and cache
*/
public QueryCacheStats getQueryCacheStats() {
return new QueryCacheStats(queryIndex.numDocs(), queryIndex.cacheSize(), lastPurged);
}
/**
* Statistics for the query cache and query index
*/
public static class QueryCacheStats {
/**
* Total number of queries in the query index
*/
public final int queries;
/**
* Total number of queries int the query cache
*/
public final int cachedQueries;
/**
* Time the query cache was last purged
*/
public final long lastPurged;
public QueryCacheStats(int queries, int cachedQueries, long lastPurged) {
this.queries = queries;
this.cachedQueries = cachedQueries;
this.lastPurged = lastPurged;
}
}
/**
* Remove unused queries from the query cache.
* <p>
* This is normally called from a background thread at a rate set by configurePurgeFrequency().
*
* @throws IOException on IO errors
*/
public void purgeCache() throws IOException {
queryIndex.purgeCache();
lastPurged = System.nanoTime();
listeners.forEach(MonitorUpdateListener::onPurge);
}
@Override
public void close() throws IOException {
purgeExecutor.shutdown();
queryIndex.close();
}
/**
* Add new queries to the monitor
*
* @param queries the MonitorQueries to add
*/
public void register(Iterable<MonitorQuery> queries) throws IOException {
List<MonitorQuery> updates = new ArrayList<>();
for (MonitorQuery query : queries) {
updates.add(query);
if (updates.size() > commitBatchSize) {
commit(updates);
updates.clear();
}
}
commit(updates);
}
private void commit(List<MonitorQuery> updates) throws IOException {
queryIndex.commit(updates);
listeners.forEach(l -> l.afterUpdate(updates));
}
/**
* Add new queries to the monitor
*
* @param queries the MonitorQueries to add
* @throws IOException on IO errors
*/
public void register(MonitorQuery... queries) throws IOException {
register(Arrays.asList(queries));
}
/**
* Delete queries from the monitor by ID
*
* @param queryIds the IDs to delete
* @throws IOException on IO errors
*/
public void deleteById(List<String> queryIds) throws IOException {
queryIndex.deleteQueries(queryIds);
listeners.forEach(l -> l.afterDelete(queryIds));
}
/**
* Delete queries from the monitor by ID
*
* @param queryIds the IDs to delete
* @throws IOException on IO errors
*/
public void deleteById(String... queryIds) throws IOException {
deleteById(Arrays.asList(queryIds));
}
/**
* Delete all queries from the monitor
*
* @throws IOException on IO errors
*/
public void clear() throws IOException {
queryIndex.clear();
listeners.forEach(MonitorUpdateListener::afterClear);
}
/**
* Match an array of {@link Document}s against the queryindex, calling a {@link CandidateMatcher} produced by the
* supplied {@link MatcherFactory} for each possible matching query.
*
* @param docs the DocumentBatch to match
* @param factory a {@link MatcherFactory} to use to create a {@link CandidateMatcher} for the match run
* @param <T> the type of {@link QueryMatch} to return
* @return a {@link MatchingQueries} object summarizing the match run.
* @throws IOException on IO errors
*/
public <T extends QueryMatch> MultiMatchingQueries<T> match(Document[] docs, MatcherFactory<T> factory) throws IOException {
try (DocumentBatch batch = DocumentBatch.of(analyzer, docs)) {
LeafReader reader = batch.get();
CandidateMatcher<T> matcher = factory.createMatcher(new IndexSearcher(batch.get()));
StandardQueryCollector<T> collector = new StandardQueryCollector<>(matcher);
long buildTime = queryIndex.search(t -> presearcher.buildQuery(reader, t), collector);
return matcher.finish(buildTime, collector.queryCount);
}
}
/**
* Match a single {@link Document} against the queryindex, calling a {@link CandidateMatcher} produced by the
* supplied {@link MatcherFactory} for each possible matching query.
*
* @param doc the InputDocument to match
* @param factory a {@link MatcherFactory} to use to create a {@link CandidateMatcher} for the match run
* @param <T> the type of {@link QueryMatch} to return
* @return a {@link MatchingQueries} object summarizing the match run.
* @throws IOException on IO errors
*/
public <T extends QueryMatch> MatchingQueries<T> match(Document doc, MatcherFactory<T> factory) throws IOException {
return match(new Document[]{ doc }, factory).singleton();
}
/**
* Get the MonitorQuery for a given query id
*
* @param queryId the id of the query to get
* @return the MonitorQuery stored for this id, or null if not found
* @throws IOException on IO errors
* @throws IllegalStateException if queries are not stored in the queryindex
*/
public MonitorQuery getQuery(final String queryId) throws IOException {
return queryIndex.getQuery(queryId);
}
/**
* @return the number of queries (after decomposition) stored in this Monitor
*/
public int getDisjunctCount() {
return queryIndex.numDocs();
}
/**
* @return the number of queries stored in this Monitor
* @throws IOException on IO errors
*/
public int getQueryCount() throws IOException {
return getQueryIds().size();
}
/**
* @return the set of query ids of the queries stored in this Monitor
* @throws IOException on IO errors
*/
public Set<String> getQueryIds() throws IOException {
final Set<String> ids = new HashSet<>();
queryIndex.scan((id, query, dataValues) -> ids.add(id));
return ids;
}
// For each query selected by the presearcher, pass on to a CandidateMatcher
private static class StandardQueryCollector<T extends QueryMatch> implements QueryIndex.QueryCollector {
final CandidateMatcher<T> matcher;
int queryCount = 0;
private StandardQueryCollector(CandidateMatcher<T> matcher) {
this.matcher = matcher;
}
@Override
public void matchQuery(String id, QueryCacheEntry query, QueryIndex.DataValues dataValues) throws IOException {
if (query == null)
return;
try {
queryCount++;
matcher.matchQuery(id, query.matchQuery, query.metadata);
} catch (Exception e) {
matcher.reportError(id, e);
}
}
}
/**
* Match a DocumentBatch against the queries stored in the Monitor, also returning information
* about which queries were selected by the presearcher, and why.
*
* @param docs a DocumentBatch to match against the index
* @param factory a {@link MatcherFactory} to use to create a {@link CandidateMatcher} for the match run
* @param <T> the type of QueryMatch produced by the CandidateMatcher
* @return a {@link PresearcherMatches} object containing debug information
* @throws IOException on IO errors
*/
public <T extends QueryMatch> PresearcherMatches<T> debug(Document[] docs, MatcherFactory<T> factory)
throws IOException {
try (DocumentBatch batch = DocumentBatch.of(analyzer, docs)) {
LeafReader reader = batch.get();
IndexSearcher searcher = new IndexSearcher(reader);
searcher.setQueryCache(null);
PresearcherQueryCollector<T> collector = new PresearcherQueryCollector<>(factory.createMatcher(searcher));
long buildTime = queryIndex.search(t -> new ForceNoBulkScoringQuery(presearcher.buildQuery(reader, t)), collector);
return collector.getMatches(buildTime);
}
}
/**
* Match a single {@link Document} against the queries stored in the Monitor, also returning information
* about which queries were selected by the presearcher, and why.
*
* @param doc an InputDocument to match against the index
* @param factory a {@link MatcherFactory} to use to create a {@link CandidateMatcher} for the match run
* @param <T> the type of QueryMatch produced by the CandidateMatcher
* @return a {@link PresearcherMatches} object containing debug information
* @throws IOException on IO errors
*/
public <T extends QueryMatch> PresearcherMatches<T> debug(Document doc, MatcherFactory<T> factory) throws IOException {
return debug(new Document[]{doc}, factory);
}
private class PresearcherQueryCollector<T extends QueryMatch> extends StandardQueryCollector<T> {
final Map<String, StringBuilder> matchingTerms = new HashMap<>();
private PresearcherQueryCollector(CandidateMatcher<T> matcher) {
super(matcher);
}
public PresearcherMatches<T> getMatches(long buildTime) {
return new PresearcherMatches<>(matchingTerms, matcher.finish(buildTime, queryCount));
}
@Override
public ScoreMode scoreMode() {
return ScoreMode.COMPLETE;
}
@Override
public void matchQuery(final String id, QueryCacheEntry query, QueryIndex.DataValues dataValues) throws IOException {
Weight w = ((Scorer)dataValues.scorer).getWeight();
Matches matches = w.matches(dataValues.ctx, dataValues.scorer.docID());
for (String field : matches) {
MatchesIterator mi = matches.getMatches(field);
while (mi.next()) {
matchingTerms.computeIfAbsent(id, i -> new StringBuilder())
.append(" ").append(mi.getQuery());
}
}
super.matchQuery(id, query, dataValues);
}
}
}

View File

@ -0,0 +1,135 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.IOException;
import java.nio.file.Path;
import java.util.concurrent.TimeUnit;
import org.apache.lucene.analysis.core.KeywordAnalyzer;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.TieredMergePolicy;
import org.apache.lucene.store.ByteBuffersDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FSDirectory;
/**
* Encapsulates various configuration settings for a Monitor's query index
*/
public class MonitorConfiguration {
private int queryUpdateBufferSize = 5000;
private long purgeFrequency = 5;
private TimeUnit purgeFrequencyUnits = TimeUnit.MINUTES;
private QueryDecomposer queryDecomposer = new QueryDecomposer();
private Path indexPath = null;
private MonitorQuerySerializer serializer;
private static IndexWriterConfig defaultIndexWriterConfig() {
IndexWriterConfig iwc = new IndexWriterConfig(new KeywordAnalyzer());
TieredMergePolicy mergePolicy = new TieredMergePolicy();
mergePolicy.setSegmentsPerTier(4);
iwc.setMergePolicy(mergePolicy);
iwc.setOpenMode(IndexWriterConfig.OpenMode.CREATE_OR_APPEND);
return iwc;
}
public MonitorConfiguration setIndexPath(Path indexPath, MonitorQuerySerializer serializer) {
this.indexPath = indexPath;
this.serializer = serializer;
return this;
}
public IndexWriter buildIndexWriter() throws IOException {
Directory directory = indexPath == null ? new ByteBuffersDirectory() : FSDirectory.open(indexPath);
return new IndexWriter(directory, getIndexWriterConfig());
}
protected IndexWriterConfig getIndexWriterConfig() {
return defaultIndexWriterConfig();
}
public MonitorQuerySerializer getQuerySerializer() {
return serializer;
}
/**
* Set the QueryDecomposer to be used by the Monitor
*
* @param queryDecomposer the QueryDecomposer to be used by the Monitor
* @return the current configuration
*/
public MonitorConfiguration setQueryDecomposer(QueryDecomposer queryDecomposer) {
this.queryDecomposer = queryDecomposer;
return this;
}
/**
* @return the QueryDecomposer used by the Monitor
*/
public QueryDecomposer getQueryDecomposer() {
return queryDecomposer;
}
/**
* Set the frequency with with the Monitor's querycache will be garbage-collected
*
* @param frequency the frequency value
* @param units the frequency units
* @return the current configuration
*/
public MonitorConfiguration setPurgeFrequency(long frequency, TimeUnit units) {
this.purgeFrequency = frequency;
this.purgeFrequencyUnits = units;
return this;
}
/**
* @return the value of Monitor's querycache garbage-collection frequency
*/
public long getPurgeFrequency() {
return purgeFrequency;
}
/**
* @return Get the units of the Monitor's querycache garbage-collection frequency
*/
public TimeUnit getPurgeFrequencyUnits() {
return purgeFrequencyUnits;
}
/**
* Set how many queries will be buffered in memory before being committed to the queryindex
*
* @param size how many queries will be buffered in memory before being committed to the queryindex
* @return the current configuration
*/
public MonitorConfiguration setQueryUpdateBufferSize(int size) {
this.queryUpdateBufferSize = size;
return this;
}
/**
* @return the size of the queryindex's in-memory buffer
*/
public int getQueryUpdateBufferSize() {
return queryUpdateBufferSize;
}
}

View File

@ -0,0 +1,135 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.util.Collections;
import java.util.Map;
import java.util.Objects;
import java.util.TreeMap;
import org.apache.lucene.search.Query;
/**
* Defines a query to be stored in a Monitor
*/
public class MonitorQuery {
private final String id;
private final Query query;
private final String queryString;
private final Map<String, String> metadata;
/**
* Creates a new MonitorQuery
*
* @param id the query ID
* @param query the query to store
* @param queryString an optional string representation of the query, for persistent Monitors
* @param metadata metadata passed to {@link Presearcher#indexQuery(Query, Map)}. Must not
* have any null values
*/
public MonitorQuery(String id, Query query, String queryString, Map<String, String> metadata) {
this.id = id;
this.query = query;
this.queryString = queryString;
this.metadata = Collections.unmodifiableMap(new TreeMap<>(metadata));
checkNullEntries(this.metadata);
}
/**
* Creates a new MonitorQuery with empty metadata and no string representation
*
* @param id the ID
* @param query the query
*/
public MonitorQuery(String id, Query query) {
this(id, query, null, Collections.emptyMap());
}
private static void checkNullEntries(Map<String, String> metadata) {
for (Map.Entry<String, String> entry : metadata.entrySet()) {
if (entry.getValue() == null)
throw new IllegalArgumentException("Null value for key " + entry.getKey() + " in metadata map");
}
}
/**
* @return this MonitorQuery's ID
*/
public String getId() {
return id;
}
/**
* @return this MonitorQuery's query
*/
public Query getQuery() {
return query;
}
/**
* @return this MonitorQuery's string representation
*/
public String getQueryString() {
return queryString;
}
/**
* @return this MonitorQuery's metadata
*/
public Map<String, String> getMetadata() {
return metadata;
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
MonitorQuery that = (MonitorQuery) o;
return Objects.equals(id, that.id) && Objects.equals(query, that.query) && Objects.equals(metadata, that.metadata);
}
@Override
public int hashCode() {
return Objects.hash(id, query, metadata);
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder(id);
sb.append(": ");
if (queryString == null) {
sb.append(query.toString());
}
else {
sb.append(queryString);
}
if (metadata.size() != 0) {
sb.append(" { ");
int n = metadata.size();
for (Map.Entry<String, String> entry : metadata.entrySet()) {
n--;
sb.append(entry.getKey()).append(": ").append(entry.getValue());
if (n > 0)
sb.append(", ");
}
sb.append(" }");
}
return sb.toString();
}
}

View File

@ -0,0 +1,92 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.util.function.Function;
import org.apache.lucene.search.Query;
import org.apache.lucene.store.InputStreamDataInput;
import org.apache.lucene.store.OutputStreamDataOutput;
import org.apache.lucene.util.BytesRef;
/**
* Serializes and deserializes MonitorQuery objects into byte streams
*
* Use this for persistent query indexes
*/
public interface MonitorQuerySerializer {
/**
* Builds a MonitorQuery from a byte representation
*/
MonitorQuery deserialize(BytesRef binaryValue);
/**
* Converts a MonitorQuery into a byte representation
*/
BytesRef serialize(MonitorQuery query);
/**
* Build a serializer from a query parser
*
* @param parser a parser to convert a String representation of a query into a lucene query object
*/
static MonitorQuerySerializer fromParser(Function<String, Query> parser) {
return new MonitorQuerySerializer() {
@Override
public MonitorQuery deserialize(BytesRef binaryValue) {
ByteArrayInputStream is = new ByteArrayInputStream(binaryValue.bytes);
try (InputStreamDataInput data = new InputStreamDataInput(is)) {
String id = data.readString();
String query = data.readString();
Map<String, String> metadata = new HashMap<>();
for (int i = data.readInt(); i > 0; i--) {
metadata.put(data.readString(), data.readString());
}
return new MonitorQuery(id, parser.apply(query), query, metadata);
} catch (IOException e) {
throw new RuntimeException(e); // shouldn't happen, we're reading from a bytearray!
}
}
@Override
public BytesRef serialize(MonitorQuery query) {
ByteArrayOutputStream os = new ByteArrayOutputStream();
try (OutputStreamDataOutput data = new OutputStreamDataOutput(os)) {
data.writeString(query.getId());
data.writeString(query.getQueryString());
data.writeInt(query.getMetadata().size());
for (Map.Entry<String, String> entry : query.getMetadata().entrySet()) {
data.writeString(entry.getKey());
data.writeString(entry.getValue());
}
return new BytesRef(os.toByteArray());
}
catch (IOException e) {
throw new RuntimeException(e); // All in memory, so no IOException should be thrown
}
}
};
}
}

View File

@ -0,0 +1,52 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.util.List;
/**
* For reporting events on a Monitor's query index
*/
public interface MonitorUpdateListener {
/**
* Called after a set of queries have been added to the Monitor's query index
*/
default void afterUpdate(List<MonitorQuery> updates) {};
/**
* Called after a set of queries have been deleted from the Monitor's query index
*/
default void afterDelete(List<String> queryIds) {};
/**
* Called after all queries have been removed from the Monitor's query index
*/
default void afterClear() {};
/**
* Called after the Monitor's query cache has been purged of deleted queries
*/
default void onPurge() {};
/**
* Called if there was an error removing deleted queries from the Monitor's query cache
*/
default void onPurgeError(Throwable t) {};
}

View File

@ -0,0 +1,123 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Map;
/**
* Class to hold the results of matching a batch of {@link org.apache.lucene.document.Document}s
* against queries held in the Monitor
*
* @param <T> the type of QueryMatch returned
*/
public class MultiMatchingQueries<T extends QueryMatch> {
private final List<Map<String, T>> matches;
private final Map<String, Exception> errors;
private final long queryBuildTime;
private final long searchTime;
private final int queriesRun;
private final int batchSize;
MultiMatchingQueries(List<Map<String, T>> matches, Map<String, Exception> errors,
long queryBuildTime, long searchTime, int queriesRun, int batchSize) {
this.matches = Collections.unmodifiableList(matches);
this.errors = Collections.unmodifiableMap(errors);
this.queryBuildTime = queryBuildTime;
this.searchTime = searchTime;
this.queriesRun = queriesRun;
this.batchSize = batchSize;
}
/**
* Returns the QueryMatch for the given query and document, or null if it did not match
*
* @param queryId the query id
* @param docId the doc id
* @return the QueryMatch for the given query and document, or null if it did not match
*/
public T matches(String queryId, int docId) {
Map<String, T> docMatches = matches.get(docId);
if (docMatches == null)
return null;
return docMatches.get(queryId);
}
/**
* @param docId document id to check
* @return all matches for a particular document
*/
public Collection<T> getMatches(int docId) {
return matches.get(docId).values();
}
/**
* @param docId document id to check
* @return the number of queries that matched for a given document
*/
public int getMatchCount(int docId) {
Map<String, T> docMatches = matches.get(docId);
if (docMatches == null)
return 0;
return docMatches.size();
}
/**
* @return how long (in ms) it took to build the Presearcher query for the matcher run
*/
public long getQueryBuildTime() {
return queryBuildTime;
}
/**
* @return how long (in ms) it took to run the selected queries
*/
public long getSearchTime() {
return searchTime;
}
/**
* @return the number of queries passed to this CandidateMatcher during the matcher run
*/
public int getQueriesRun() {
return queriesRun;
}
/**
* @return the number of documents in the batch
*/
public int getBatchSize() {
return batchSize;
}
/**
* @return a List of any MatchErrors created during the matcher run
*/
public Map<String, Exception> getErrors() {
return errors;
}
MatchingQueries<T> singleton() {
assert matches.size() == 1;
return new MatchingQueries<>(matches.get(0), errors, queryBuildTime, searchTime, queriesRun);
}
}

View File

@ -0,0 +1,161 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.TermInSetQuery;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefHash;
/**
* A TermFilteredPresearcher that indexes queries multiple times, with terms collected
* from different routes through a querytree. Each route will produce a set of terms
* that are *sufficient* to select the query, and are indexed into a separate, suffixed field.
* <p>
* Incoming documents are then converted to a set of Disjunction queries over each
* suffixed field, and these queries are combined into a conjunction query, such that the
* document's set of terms must match a term from each route.
* <p>
* This allows filtering out of documents that contain one half of a two-term phrase query, for
* example. The query {@code "hello world"} will be indexed twice, once under 'hello' and once
* under 'world'. A document containing the terms "hello there" would match the first field,
* but not the second, and so would not be selected for matching.
* <p>
* The number of passes the presearcher makes is configurable. More passes will improve the
* selected/matched ratio, but will take longer to index and will use more RAM.
* <p>
* A minimum weight can we set for terms to be chosen for the second and subsequent passes. This
* allows users to avoid indexing stopwords, for example.
*/
public class MultipassTermFilteredPresearcher extends TermFilteredPresearcher {
private final int passes;
private final float minWeight;
/**
* Construct a new MultipassTermFilteredPresearcher
*
* @param passes the number of times a query should be indexed
* @param minWeight the minimum weight a querytree should be advanced over
* @param weightor the TreeWeightor to use
* @param queryHandlers a list of custom query handlers
* @param filterFields a set of fields to use as filters
*/
public MultipassTermFilteredPresearcher(int passes, float minWeight, TermWeightor weightor,
List<CustomQueryHandler> queryHandlers, Set<String> filterFields) {
super(weightor, queryHandlers, filterFields);
this.passes = passes;
this.minWeight = minWeight;
}
/**
* Construct a new MultipassTermFilteredPresearcher using {@link TermFilteredPresearcher#DEFAULT_WEIGHTOR}
* <p>
* Note that this will be constructed with a minimum advance weight of zero
*
* @param passes the number of times a query should be indexed
*/
public MultipassTermFilteredPresearcher(int passes) {
this(passes, 0, DEFAULT_WEIGHTOR, Collections.emptyList(), Collections.emptySet());
}
@Override
protected DocumentQueryBuilder getQueryBuilder() {
return new MultipassDocumentQueryBuilder();
}
private static String field(String field, int pass) {
return field + "_" + pass;
}
private class MultipassDocumentQueryBuilder implements DocumentQueryBuilder {
BooleanQuery.Builder[] queries = new BooleanQuery.Builder[passes];
Map<String, BytesRefHash> terms = new HashMap<>();
MultipassDocumentQueryBuilder() {
for (int i = 0; i < queries.length; i++) {
queries[i] = new BooleanQuery.Builder();
}
}
@Override
public void addTerm(String field, BytesRef term) {
BytesRefHash t = terms.computeIfAbsent(field, f -> new BytesRefHash());
t.add(term);
}
@Override
public Query build() {
Map<String, BytesRef[]> collectedTerms = new HashMap<>();
for (String field : terms.keySet()) {
collectedTerms.put(field, convertHash(terms.get(field)));
}
BooleanQuery.Builder parent = new BooleanQuery.Builder();
for (int i = 0; i < passes; i++) {
BooleanQuery.Builder child = new BooleanQuery.Builder();
for (String field : terms.keySet()) {
child.add(new TermInSetQuery(field(field, i), collectedTerms.get(field)), BooleanClause.Occur.SHOULD);
}
parent.add(child.build(), BooleanClause.Occur.MUST);
}
return parent.build();
}
}
@Override
public Document buildQueryDocument(QueryTree querytree) {
Document doc = new Document();
for (int i = 0; i < passes; i++) {
Map<String, BytesRefHash> fieldTerms = collectTerms(querytree);
for (Map.Entry<String, BytesRefHash> entry : fieldTerms.entrySet()) {
// we add the index terms once under a suffixed field for the multipass query, and
// once under the plan field name for the TermsEnumTokenFilter
doc.add(new Field(field(entry.getKey(), i),
new TermsEnumTokenStream(new BytesRefHashIterator(entry.getValue())), QUERYFIELDTYPE));
doc.add(new Field(entry.getKey(),
new TermsEnumTokenStream(new BytesRefHashIterator(entry.getValue())), QUERYFIELDTYPE));
}
querytree.advancePhase(minWeight);
}
return doc;
}
private static BytesRef[] convertHash(BytesRefHash hash) {
BytesRef[] terms = new BytesRef[hash.size()];
for (int i = 0; i < terms.length; i++) {
BytesRef t = new BytesRef();
terms[i] = hash.get(i, t);
}
return terms;
}
}

View File

@ -0,0 +1,205 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.concurrent.LinkedBlockingQueue;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
/**
* Matcher class that runs matching queries in parallel.
* <p>
* This class delegates the actual matching to separate CandidateMatcher classes,
* built from a passed in MatcherFactory.
* <p>
* Use this when individual queries can take a long time to run, and you want
* to minimize latency. The matcher distributes queries amongst its worker
* threads using a BlockingQueue, and synchronization overhead may affect performance
* if the individual queries are very fast.
*
* @param <T> the QueryMatch type returned
* @see PartitionMatcher
*/
public class ParallelMatcher<T extends QueryMatch> extends CandidateMatcher<T> {
private final BlockingQueue<MatcherTask> queue = new LinkedBlockingQueue<>(1024);
private final List<Future<CandidateMatcher<T>>> futures = new ArrayList<>();
private final CandidateMatcher<T> collectorMatcher;
/**
* Create a new ParallelMatcher
*
* @param searcher the IndexSearcher to match against
* @param executor an ExecutorService to use for parallel execution
* @param matcherFactory MatcherFactory to use to create CandidateMatchers
* @param threads the number of threads to execute on
*/
private ParallelMatcher(IndexSearcher searcher, ExecutorService executor,
MatcherFactory<T> matcherFactory, int threads) {
super(searcher);
for (int i = 0; i < threads; i++) {
MatcherWorker mw = new MatcherWorker(matcherFactory);
futures.add(executor.submit(mw));
}
collectorMatcher = matcherFactory.createMatcher(searcher);
}
@Override
protected void matchQuery(String queryId, Query matchQuery, Map<String, String> metadata) throws IOException {
try {
queue.put(new MatcherTask(queryId, matchQuery, metadata));
} catch (InterruptedException e) {
throw new IOException("Interrupted during match", e);
}
}
@Override
public T resolve(T match1, T match2) {
return collectorMatcher.resolve(match1, match2);
}
@Override
protected void doFinish() {
try {
for (int i = 0; i < futures.size(); i++) {
queue.put(END);
}
for (Future<CandidateMatcher<T>> future : futures) {
MultiMatchingQueries<T> matches = future.get().finish(0, 0);
for (int doc = 0; doc < matches.getBatchSize(); doc++) {
for (T match : matches.getMatches(doc)) {
this.addMatch(match, doc);
}
}
for (Map.Entry<String, Exception> error : matches.getErrors().entrySet()) {
this.reportError(error.getKey(), error.getValue());
}
}
} catch (InterruptedException | ExecutionException e) {
throw new RuntimeException("Interrupted during match", e);
}
}
private class MatcherWorker implements Callable<CandidateMatcher<T>> {
final CandidateMatcher<T> matcher;
private MatcherWorker(MatcherFactory<T> matcherFactory) {
this.matcher = matcherFactory.createMatcher(searcher);
}
@Override
public CandidateMatcher<T> call() {
MatcherTask task;
try {
while ((task = queue.take()) != END) {
try {
matcher.matchQuery(task.id, task.matchQuery, task.metadata);
} catch (IOException e) {
matcher.reportError(task.id, e);
}
}
} catch (InterruptedException e) {
throw new RuntimeException("Interrupted during match", e);
}
return matcher;
}
}
private static class MatcherTask {
final String id;
final Query matchQuery;
final Map<String, String> metadata;
private MatcherTask(String id, Query matchQuery, Map<String, String> metadata) {
this.id = id;
this.matchQuery = matchQuery;
this.metadata = metadata;
}
}
/* Marker object placed on the queue after all matches are done, to indicate to the
worker threads that they should finish */
private static final MatcherTask END = new MatcherTask("", null, Collections.emptyMap());
private static class ParallelMatcherFactory<T extends QueryMatch> implements MatcherFactory<T> {
private final ExecutorService executor;
private final MatcherFactory<T> matcherFactory;
private final int threads;
ParallelMatcherFactory(ExecutorService executor, MatcherFactory<T> matcherFactory,
int threads) {
this.executor = executor;
this.matcherFactory = matcherFactory;
this.threads = threads;
}
@Override
public ParallelMatcher<T> createMatcher(IndexSearcher searcher) {
return new ParallelMatcher<>(searcher, executor, matcherFactory, threads);
}
}
/**
* Create a new MatcherFactory for a ParallelMatcher
*
* @param executor the ExecutorService to use
* @param matcherFactory the MatcherFactory to use to create submatchers
* @param threads the number of threads to use
* @param <T> the type of QueryMatch generated
*/
public static <T extends QueryMatch> MatcherFactory<T> factory(ExecutorService executor,
MatcherFactory<T> matcherFactory, int threads) {
return new ParallelMatcherFactory<>(executor, matcherFactory, threads);
}
/**
* Create a new MatcherFactory for a ParallelMatcher
* <p>
* This factory will create a ParallelMatcher that uses as many threads as there are cores available
* to the JVM (as determined by {@code Runtime.getRuntime().availableProcessors()}).
*
* @param executor the ExecutorService to use
* @param matcherFactory the MatcherFactory to use to create submatchers
* @param <T> the type of QueryMatch generated
*/
public static <T extends QueryMatch> MatcherFactory<T> factory(ExecutorService executor,
MatcherFactory<T> matcherFactory) {
int threads = Runtime.getRuntime().availableProcessors();
return new ParallelMatcherFactory<>(executor, matcherFactory, threads);
}
}

View File

@ -0,0 +1,202 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
/**
* A multi-threaded matcher that collects all possible matches in one pass, and
* then partitions them amongst a number of worker threads to perform the actual
* matching.
* <p>
* This class delegates the matching to separate CandidateMatcher classes,
* built from a passed in MatcherFactory.
* <p>
* Use this if your query sets contain large numbers of very fast queries, where
* the synchronization overhead of {@link ParallelMatcher}
* can outweigh the benefit of multithreading.
*
* @param <T> the type of QueryMatch to return
* @see ParallelMatcher
*/
public class PartitionMatcher<T extends QueryMatch> extends CandidateMatcher<T> {
private final ExecutorService executor;
private final MatcherFactory<T> matcherFactory;
private final int threads;
private final CandidateMatcher<T> resolvingMatcher;
private static class MatchTask {
final String queryId;
final Query matchQuery;
final Map<String, String> metadata;
private MatchTask(String queryId, Query matchQuery, Map<String, String> metadata) {
this.queryId = queryId;
this.matchQuery = matchQuery;
this.metadata = metadata;
}
}
private final List<MatchTask> tasks = new ArrayList<>();
private PartitionMatcher(IndexSearcher searcher, ExecutorService executor, MatcherFactory<T> matcherFactory, int threads) {
super(searcher);
this.executor = executor;
this.matcherFactory = matcherFactory;
this.threads = threads;
this.resolvingMatcher = matcherFactory.createMatcher(searcher);
}
@Override
protected void matchQuery(String queryId, Query matchQuery, Map<String, String> metadata) {
tasks.add(new MatchTask(queryId, matchQuery, metadata));
}
@Override
public T resolve(T match1, T match2) {
return resolvingMatcher.resolve(match1, match2);
}
@Override
protected void doFinish() {
List<Callable<MultiMatchingQueries<T>>> workers = new ArrayList<>(threads);
for (List<MatchTask> taskset : partition(tasks, threads)) {
CandidateMatcher<T> matcher = matcherFactory.createMatcher(searcher);
workers.add(new MatcherWorker(taskset, matcher));
}
try {
for (Future<MultiMatchingQueries<T>> future : executor.invokeAll(workers)) {
MultiMatchingQueries<T> matches = future.get();
for (int doc = 0; doc < matches.getBatchSize(); doc++) {
for (T match : matches.getMatches(doc)) {
addMatch(match, doc);
}
}
}
} catch (InterruptedException | ExecutionException e) {
throw new RuntimeException("Interrupted during match", e);
}
}
private class MatcherWorker implements Callable<MultiMatchingQueries<T>> {
final List<MatchTask> tasks;
final CandidateMatcher<T> matcher;
private MatcherWorker(List<MatchTask> tasks, CandidateMatcher<T> matcher) {
this.tasks = tasks;
this.matcher = matcher;
}
@Override
public MultiMatchingQueries<T> call() {
for (MatchTask task : tasks) {
try {
matcher.matchQuery(task.queryId, task.matchQuery, task.metadata);
} catch (IOException e) {
PartitionMatcher.this.reportError(task.queryId, e);
}
}
return matcher.finish(0, 0);
}
}
private static class PartitionMatcherFactory<T extends QueryMatch> implements MatcherFactory<T> {
private final ExecutorService executor;
private final MatcherFactory<T> matcherFactory;
private final int threads;
PartitionMatcherFactory(ExecutorService executor, MatcherFactory<T> matcherFactory,
int threads) {
this.executor = executor;
this.matcherFactory = matcherFactory;
this.threads = threads;
}
@Override
public PartitionMatcher<T> createMatcher(IndexSearcher searcher) {
return new PartitionMatcher<>(searcher, executor, matcherFactory, threads);
}
}
/**
* Create a new MatcherFactory for a PartitionMatcher
*
* @param executor the ExecutorService to use
* @param matcherFactory the MatcherFactory to use to create submatchers
* @param threads the number of threads to use
* @param <T> the type of QueryMatch generated
*/
public static <T extends QueryMatch> MatcherFactory<T> factory(ExecutorService executor,
MatcherFactory<T> matcherFactory, int threads) {
return new PartitionMatcherFactory<>(executor, matcherFactory, threads);
}
/**
* Create a new MatcherFactory for a PartitionMatcher
* <p>
* This factory will create a PartitionMatcher that uses as many threads as there are cores available
* to the JVM (as determined by {@code Runtime.getRuntime().availableProcessors()}).
*
* @param executor the ExecutorService to use
* @param matcherFactory the MatcherFactory to use to create submatchers
* @param <T> the type of QueryMatch generated
*/
public static <T extends QueryMatch> MatcherFactory<T> factory(ExecutorService executor,
MatcherFactory<T> matcherFactory) {
int threads = Runtime.getRuntime().availableProcessors();
return new PartitionMatcherFactory<>(executor, matcherFactory, threads);
}
static <T> List<List<T>> partition(List<T> items, int slices) {
double size = items.size() / (double) slices;
double accum = 0;
int start = 0;
List<List<T>> list = new ArrayList<>(slices);
for (int i = 0; i < slices; i++) {
int end = (int) Math.floor(accum + size);
if (i == slices - 1)
end = items.size();
list.add(items.subList(start, end));
accum += size;
start = (int) Math.floor(accum);
}
return list;
}
}

View File

@ -0,0 +1,71 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.util.Map;
import java.util.function.BiPredicate;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.BytesRef;
/**
* A Presearcher is used by the Monitor to reduce the number of queries actually
* run against a Document. It defines how queries are stored in the monitor's
* internal index, and how a Document is converted to a query against that
* index.
*/
public abstract class Presearcher {
/**
* A Presearcher implementation that does no query filtering, and runs all
* registered queries
*/
public static final Presearcher NO_FILTERING = new Presearcher() {
@Override
public Query buildQuery(LeafReader reader, BiPredicate<String, BytesRef> termAcceptor) {
return new MatchAllDocsQuery();
}
@Override
public Document indexQuery(Query query, Map<String, String> metadata) {
return new Document();
}
};
/**
* Build a query for a Monitor's queryindex from a LeafReader over a set of documents to monitor.
*
* @param reader a {@link LeafReader} over the input documents
* @param termAcceptor a predicate indicating if a term should be added to the query
* @return a Query to run over a Monitor's queryindex
*/
public abstract Query buildQuery(LeafReader reader, BiPredicate<String, BytesRef> termAcceptor);
/**
* Build a lucene Document to index the query in a Monitor's queryindex
*
* @param query the Query to index
* @param metadata a Map of arbitrary query metadata
* @return a lucene Document to add to the queryindex
*/
public abstract Document indexQuery(Query query, Map<String, String> metadata);
}

View File

@ -0,0 +1,45 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
/**
* Wraps a {@link QueryMatch} with information about which queries were selected by the presearcher
*/
public class PresearcherMatch<T extends QueryMatch> {
/**
* The presearcher hits
*/
public final String presearcherMatches;
/**
* The QueryMatch
*/
public final T queryMatch;
/**
* The query id
*/
public final String queryId;
PresearcherMatch(String id, String presearcherMatches, T queryMatch) {
this.presearcherMatches = presearcherMatches;
this.queryMatch = queryMatch;
this.queryId = id;
}
}

View File

@ -0,0 +1,50 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.util.Map;
/**
* Wraps a {@link MultiMatchingQueries} with information on which presearcher queries were selected
*/
public class PresearcherMatches<T extends QueryMatch> {
private final Map<String, StringBuilder> matchingTerms;
/** The wrapped Matches */
public final MultiMatchingQueries<T> matcher;
/**
* Builds a new PresearcherMatches
*/
public PresearcherMatches(Map<String, StringBuilder> matchingTerms, MultiMatchingQueries<T> matcher) {
this.matcher = matcher;
this.matchingTerms = matchingTerms;
}
/**
* Returns match information for a given query
*/
public PresearcherMatch<T> match(String queryId, int doc) {
StringBuilder found = matchingTerms.get(queryId);
if (found != null)
return new PresearcherMatch<>(queryId, found.toString(), matcher.matches(queryId, doc));
return null;
}
}

View File

@ -0,0 +1,143 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.util.ArrayList;
import java.util.List;
import java.util.function.BiFunction;
import java.util.function.Function;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryVisitor;
/**
* Class to analyze and extract terms from a lucene query, to be used by
* a {@link Presearcher} in indexing.
*/
class QueryAnalyzer {
private final BiFunction<Query, TermWeightor, QueryTree> unknownQueryMapper;
QueryAnalyzer(List<CustomQueryHandler> queryTreeBuilders) {
this.unknownQueryMapper = buildMapper(queryTreeBuilders);
}
QueryAnalyzer() {
this.unknownQueryMapper = (q, w) -> null;
}
private static BiFunction<Query, TermWeightor, QueryTree> buildMapper(List<CustomQueryHandler> mappers) {
return (q, w) -> {
for (CustomQueryHandler mapper : mappers) {
QueryTree qt = mapper.handleQuery(q, w);
if (qt != null) {
return qt;
}
}
return null;
};
}
/**
* Create a {@link QueryTree} from a passed in Query or Filter
*
* @param luceneQuery the query to analyze
* @return a QueryTree describing the analyzed query
*/
QueryTree buildTree(Query luceneQuery, TermWeightor weightor) {
QueryBuilder builder = new QueryBuilder();
luceneQuery.visit(builder);
return builder.apply(weightor);
}
private class QueryBuilder extends QueryVisitor implements Function<TermWeightor, QueryTree> {
final List<Function<TermWeightor, QueryTree>> children = new ArrayList<>();
@Override
public QueryVisitor getSubVisitor(BooleanClause.Occur occur, Query parent) {
if (occur == BooleanClause.Occur.MUST || occur == BooleanClause.Occur.FILTER) {
QueryBuilder n = new QueryBuilder();
children.add(n);
return n;
}
if (occur == BooleanClause.Occur.MUST_NOT) {
// Check if we're in a pure negative disjunction
if (parent instanceof BooleanQuery) {
BooleanQuery bq = (BooleanQuery) parent;
long positiveCount = bq.clauses().stream()
.filter(c -> c.getOccur() != BooleanClause.Occur.MUST_NOT)
.count();
if (positiveCount == 0) {
children.add(w -> QueryTree.anyTerm("PURE NEGATIVE QUERY[" + parent + "]"));
}
}
return QueryVisitor.EMPTY_VISITOR;
}
// It's a disjunction clause. If the parent has MUST or FILTER clauses, we can
// ignore it
if (parent instanceof BooleanQuery) {
BooleanQuery bq = (BooleanQuery) parent;
long requiredCount = bq.clauses().stream()
.filter(c -> c.getOccur() == BooleanClause.Occur.MUST || c.getOccur() == BooleanClause.Occur.FILTER)
.count();
if (requiredCount > 0) {
return QueryVisitor.EMPTY_VISITOR;
}
}
Disjunction n = new Disjunction();
children.add(n);
return n;
}
@Override
public void consumeTerms(Query query, Term... terms) {
for (Term term : terms) {
children.add(w -> QueryTree.term(term, w));
}
}
@Override
public void visitLeaf(Query query) {
children.add(w -> {
QueryTree q = unknownQueryMapper.apply(query, w);
if (q == null) {
return QueryTree.anyTerm(query.toString());
}
return q;
});
}
@Override
public QueryTree apply(TermWeightor termWeightor) {
return QueryTree.conjunction(children, termWeightor);
}
}
private class Disjunction extends QueryBuilder {
@Override
public QueryTree apply(TermWeightor termWeightor) {
return QueryTree.disjunction(children, termWeightor);
}
}
}

View File

@ -0,0 +1,71 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import org.apache.lucene.search.Query;
class QueryCacheEntry {
/**
* The (possibly partial due to decomposition) query
*/
final Query matchQuery;
/**
* The id of this query
*/
final String cacheId;
/**
* The id of the MonitorQuery that produced this entry
*
* Note that this may be different to {@link #cacheId} due to decomposition
*/
final String queryId;
/**
* The metadata from the entry's parent {@link MonitorQuery}
*/
final Map<String, String> metadata;
private QueryCacheEntry(String cacheId, String queryId, Query matchQuery, Map<String, String> metadata) {
this.cacheId = cacheId;
this.queryId = queryId;
this.matchQuery = matchQuery;
this.metadata = metadata;
}
static List<QueryCacheEntry> decompose(MonitorQuery mq, QueryDecomposer decomposer) {
int upto = 0;
List<QueryCacheEntry> cacheEntries = new ArrayList<>();
for (Query subquery : decomposer.decompose(mq.getQuery())) {
cacheEntries.add(new QueryCacheEntry(mq.getId() + "_" + upto, mq.getId(), subquery, mq.getMetadata()));
upto++;
}
return cacheEntries;
}
@Override
public String toString() {
return queryId + "/" + cacheId + "/" + matchQuery;
}
}

View File

@ -0,0 +1,123 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.util.Collections;
import java.util.HashSet;
import java.util.Set;
import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.BoostQuery;
import org.apache.lucene.search.DisjunctionMaxQuery;
import org.apache.lucene.search.Query;
/**
* Split a disjunction query into its consituent parts, so that they can be indexed
* and run separately in the Monitor.
*/
public class QueryDecomposer {
/**
* Split a query up into individual parts that can be indexed and run separately
*
* @param q the query
* @return a collection of subqueries
*/
public Set<Query> decompose(Query q) {
if (q instanceof BooleanQuery)
return decomposeBoolean((BooleanQuery) q);
if (q instanceof DisjunctionMaxQuery) {
Set<Query> subqueries = new HashSet<>();
for (Query subq : ((DisjunctionMaxQuery) q).getDisjuncts()) {
subqueries.addAll(decompose(subq));
}
return subqueries;
}
if (q instanceof BoostQuery) {
return decomposeBoostQuery((BoostQuery) q);
}
return Collections.singleton(q);
}
public Set<Query> decomposeBoostQuery(BoostQuery q) {
if (q.getBoost() == 1.0)
return decompose(q.getQuery());
Set<Query> boostedDecomposedQueries = new HashSet<>();
for (Query subq : decompose(q.getQuery())) {
boostedDecomposedQueries.add(new BoostQuery(subq, q.getBoost()));
}
return boostedDecomposedQueries;
}
/**
* Decompose a {@link org.apache.lucene.search.BooleanQuery}
*
* @param q the boolean query
* @return a collection of subqueries
*/
public Set<Query> decomposeBoolean(BooleanQuery q) {
if (q.getMinimumNumberShouldMatch() > 1)
return Collections.singleton(q);
Set<Query> subqueries = new HashSet<>();
Set<Query> exclusions = new HashSet<>();
Set<Query> mandatory = new HashSet<>();
for (BooleanClause clause : q) {
if (clause.getOccur() == BooleanClause.Occur.MUST || clause.getOccur() == BooleanClause.Occur.FILTER)
mandatory.add(clause.getQuery());
else if (clause.getOccur() == BooleanClause.Occur.MUST_NOT)
exclusions.add(clause.getQuery());
else {
subqueries.addAll(decompose(clause.getQuery()));
}
}
// More than one MUST clause, or a single MUST clause with disjunctions
if (mandatory.size() > 1 || (mandatory.size() == 1 && subqueries.size() > 0))
return Collections.singleton(q);
// If we only have a single MUST clause and no SHOULD clauses, then we can
// decompose the MUST clause instead
if (mandatory.size() == 1) {
subqueries.addAll(decompose(mandatory.iterator().next()));
}
if (exclusions.size() == 0)
return subqueries;
// If there are exclusions, then we need to add them to all the decomposed
// queries
Set<Query> rewrittenSubqueries = new HashSet<>(subqueries.size());
for (Query subquery : subqueries) {
BooleanQuery.Builder bq = new BooleanQuery.Builder();
bq.add(subquery, BooleanClause.Occur.MUST);
for (Query ex : exclusions) {
bq.add(ex, BooleanClause.Occur.MUST_NOT);
}
rewrittenSubqueries.add(bq.build());
}
return rewrittenSubqueries;
}
}

View File

@ -0,0 +1,449 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.function.BiPredicate;
import org.apache.lucene.document.BinaryDocValuesField;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.SortedDocValuesField;
import org.apache.lucene.document.StringField;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.Scorable;
import org.apache.lucene.search.ScoreMode;
import org.apache.lucene.search.SearcherFactory;
import org.apache.lucene.search.SearcherManager;
import org.apache.lucene.search.SimpleCollector;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefHash;
import org.apache.lucene.util.IOUtils;
class QueryIndex implements Closeable {
static final class FIELDS {
static final String query_id = "_query_id";
static final String cache_id = "_cache_id";
static final String mq = "_mq";
}
private final IndexWriter writer;
private final SearcherManager manager;
private final QueryDecomposer decomposer;
private final MonitorQuerySerializer serializer;
private final Presearcher presearcher;
/* Used to cache updates while a purge is ongoing */
private volatile Map<String, QueryCacheEntry> purgeCache = null;
/* Used to lock around the creation of the purgeCache */
private final ReadWriteLock purgeLock = new ReentrantReadWriteLock();
private final Object commitLock = new Object();
/* The current query cache */
private volatile ConcurrentMap<String, QueryCacheEntry> queries = new ConcurrentHashMap<>();
// NB this is not final because it can be replaced by purgeCache()
// package-private for testing
final Map<IndexReader.CacheKey, QueryTermFilter> termFilters = new HashMap<>();
QueryIndex(MonitorConfiguration config, Presearcher presearcher) throws IOException {
this.writer = config.buildIndexWriter();
this.manager = new SearcherManager(writer, true, true, new TermsHashBuilder());
this.decomposer = config.getQueryDecomposer();
this.serializer = config.getQuerySerializer();
this.presearcher = presearcher;
populateQueryCache(serializer, decomposer);
}
private void populateQueryCache(MonitorQuerySerializer serializer, QueryDecomposer decomposer) throws IOException {
if (serializer == null) {
// No query serialization happening here - check that the cache is empty
IndexSearcher searcher = manager.acquire();
try {
if (searcher.count(new MatchAllDocsQuery()) != 0) {
throw new IllegalStateException("Attempting to open a non-empty monitor query index with no MonitorQuerySerializer");
}
}
finally {
manager.release(searcher);
}
return;
}
Set<String> ids = new HashSet<>();
List<Exception> errors = new ArrayList<>();
purgeCache(newCache -> scan((id, cacheEntry, dataValues) -> {
if (ids.contains(id)) {
// this is a branch of a query that has already been reconstructed, but
// then split by decomposition - we don't need to parse it again
return;
}
ids.add(id);
try {
MonitorQuery mq = serializer.deserialize(dataValues.mq.binaryValue());
for (QueryCacheEntry entry : QueryCacheEntry.decompose(mq, decomposer)) {
newCache.put(entry.cacheId, entry);
}
}
catch (Exception e) {
errors.add(e);
}
}));
if (errors.size() > 0) {
IllegalStateException e = new IllegalStateException("Couldn't parse some queries from the index");
for (Exception parseError : errors) {
e.addSuppressed(parseError);
}
throw e;
}
}
private class TermsHashBuilder extends SearcherFactory {
@Override
public IndexSearcher newSearcher(IndexReader reader, IndexReader previousReader) throws IOException {
IndexSearcher searcher = super.newSearcher(reader, previousReader);
searcher.setQueryCache(null);
termFilters.put(reader.getReaderCacheHelper().getKey(), new QueryTermFilter(reader));
reader.getReaderCacheHelper().addClosedListener(termFilters::remove);
return searcher;
}
}
void commit(List<MonitorQuery> updates) throws IOException {
List<Indexable> indexables = buildIndexables(updates);
synchronized (commitLock) {
purgeLock.readLock().lock();
try {
if (indexables.size() > 0) {
Set<String> ids = new HashSet<>();
for (Indexable update : indexables) {
ids.add(update.queryCacheEntry.queryId);
}
for (String id : ids) {
writer.deleteDocuments(new Term(FIELDS.query_id, id));
}
for (Indexable update : indexables) {
this.queries.put(update.queryCacheEntry.cacheId, update.queryCacheEntry);
writer.addDocument(update.document);
if (purgeCache != null)
purgeCache.put(update.queryCacheEntry.cacheId, update.queryCacheEntry);
}
}
writer.commit();
manager.maybeRefresh();
} finally {
purgeLock.readLock().unlock();
}
}
}
private static class Indexable {
final QueryCacheEntry queryCacheEntry;
final Document document;
private Indexable(QueryCacheEntry queryCacheEntry, Document document) {
this.queryCacheEntry = queryCacheEntry;
this.document = document;
}
}
private static final BytesRef EMPTY = new BytesRef();
private List<Indexable> buildIndexables(List<MonitorQuery> updates) {
List<Indexable> indexables = new ArrayList<>();
for (MonitorQuery mq : updates) {
if (serializer != null && mq.getQueryString() == null) {
throw new IllegalArgumentException("Cannot add a MonitorQuery with a null string representation to a non-ephemeral Monitor");
}
BytesRef serialized = serializer == null ? EMPTY : serializer.serialize(mq);
for (QueryCacheEntry qce : QueryCacheEntry.decompose(mq, decomposer)) {
Document doc = presearcher.indexQuery(qce.matchQuery, mq.getMetadata());
doc.add(new StringField(FIELDS.query_id, qce.queryId, Field.Store.NO));
doc.add(new SortedDocValuesField(FIELDS.cache_id, new BytesRef(qce.cacheId)));
doc.add(new SortedDocValuesField(FIELDS.query_id, new BytesRef(qce.queryId)));
doc.add(new BinaryDocValuesField(FIELDS.mq, serialized));
indexables.add(new Indexable(qce, doc));
}
}
return indexables;
}
interface QueryBuilder {
Query buildQuery(BiPredicate<String, BytesRef> termAcceptor) throws IOException;
}
static class QueryTermFilter implements BiPredicate<String, BytesRef> {
private final Map<String, BytesRefHash> termsHash = new HashMap<>();
QueryTermFilter(IndexReader reader) throws IOException {
for (LeafReaderContext ctx : reader.leaves()) {
for (FieldInfo fi : ctx.reader().getFieldInfos()) {
BytesRefHash terms = termsHash.computeIfAbsent(fi.name, f -> new BytesRefHash());
Terms t = ctx.reader().terms(fi.name);
if (t != null) {
TermsEnum te = t.iterator();
BytesRef term;
while ((term = te.next()) != null) {
terms.add(term);
}
}
}
}
}
@Override
public boolean test(String field, BytesRef term) {
BytesRefHash bytes = termsHash.get(field);
if (bytes == null) {
return false;
}
return bytes.find(term) != -1;
}
}
MonitorQuery getQuery(String queryId) throws IOException {
if (serializer == null) {
throw new IllegalStateException("Cannot get queries from an index with no MonitorQuerySerializer");
}
BytesRef[] bytesHolder = new BytesRef[1];
search(new TermQuery(new Term(FIELDS.query_id, queryId)),
(id, query, dataValues) -> bytesHolder[0] = dataValues.mq.binaryValue());
return serializer.deserialize(bytesHolder[0]);
}
void scan(QueryCollector matcher) throws IOException {
search(new MatchAllDocsQuery(), matcher);
}
long search(final Query query, QueryCollector matcher) throws IOException {
QueryBuilder builder = termFilter -> query;
return search(builder, matcher);
}
long search(QueryBuilder queryBuilder, QueryCollector matcher) throws IOException {
IndexSearcher searcher = null;
try {
Map<String, QueryCacheEntry> queries;
purgeLock.readLock().lock();
try {
searcher = manager.acquire();
queries = this.queries;
} finally {
purgeLock.readLock().unlock();
}
MonitorQueryCollector collector = new MonitorQueryCollector(queries, matcher);
long buildTime = System.nanoTime();
Query query = queryBuilder.buildQuery(termFilters.get(searcher.getIndexReader().getReaderCacheHelper().getKey()));
buildTime = System.nanoTime() - buildTime;
searcher.search(query, collector);
return buildTime;
} finally {
if (searcher != null) {
manager.release(searcher);
}
}
}
interface CachePopulator {
void populateCacheWithIndex(Map<String, QueryCacheEntry> newCache) throws IOException;
}
void purgeCache() throws IOException {
purgeCache(newCache -> scan((id, query, dataValues) -> {
if (query != null)
newCache.put(query.cacheId, query);
}));
}
/**
* Remove unused queries from the query cache.
* <p>
* This is normally called from a background thread at a rate set by configurePurgeFrequency().
*
* @throws IOException on IO errors
*/
private synchronized void purgeCache(CachePopulator populator) throws IOException {
// Note on implementation
// The purge works by scanning the query index and creating a new query cache populated
// for each query in the index. When the scan is complete, the old query cache is swapped
// for the new, allowing it to be garbage-collected.
// In order to not drop cached queries that have been added while a purge is ongoing,
// we use a ReadWriteLock to guard the creation and removal of an register log. Commits take
// the read lock. If the register log has been created, then a purge is ongoing, and queries
// are added to the register log within the read lock guard.
// The purge takes the write lock when creating the register log, and then when swapping out
// the old query cache. Within the second write lock guard, the contents of the register log
// are added to the new query cache, and the register log itself is removed.
final ConcurrentMap<String, QueryCacheEntry> newCache = new ConcurrentHashMap<>();
purgeLock.writeLock().lock();
try {
purgeCache = new ConcurrentHashMap<>();
} finally {
purgeLock.writeLock().unlock();
}
populator.populateCacheWithIndex(newCache);
purgeLock.writeLock().lock();
try {
newCache.putAll(purgeCache);
purgeCache = null;
queries = newCache;
} finally {
purgeLock.writeLock().unlock();
}
}
// ---------------------------------------------
// Proxy trivial operations...
// ---------------------------------------------
@Override
public void close() throws IOException {
IOUtils.close(manager, writer, writer.getDirectory());
}
int numDocs() {
return writer.getDocStats().numDocs;
}
int cacheSize() {
return queries.size();
}
void deleteQueries(Iterable<String> ids) throws IOException {
for (String id : ids) {
writer.deleteDocuments(new Term(FIELDS.query_id, id));
}
commit(Collections.emptyList());
}
void clear() throws IOException {
writer.deleteAll();
commit(Collections.emptyList());
}
interface QueryCollector {
void matchQuery(String id, QueryCacheEntry query, DataValues dataValues) throws IOException;
default ScoreMode scoreMode() {
return ScoreMode.COMPLETE_NO_SCORES;
}
}
// ---------------------------------------------
// Helper classes...
// ---------------------------------------------
static final class DataValues {
SortedDocValues queryId;
SortedDocValues cacheId;
BinaryDocValues mq;
Scorable scorer;
LeafReaderContext ctx;
void advanceTo(int doc) throws IOException {
assert scorer.docID() == doc;
queryId.advanceExact(doc);
cacheId.advanceExact(doc);
if (mq != null) {
mq.advanceExact(doc);
}
}
}
/**
* A Collector that decodes the stored query for each document hit.
*/
static final class MonitorQueryCollector extends SimpleCollector {
private final Map<String, QueryCacheEntry> queries;
private final QueryCollector matcher;
private final DataValues dataValues = new DataValues();
MonitorQueryCollector(Map<String, QueryCacheEntry> queries, QueryCollector matcher) {
this.queries = queries;
this.matcher = matcher;
}
@Override
public void setScorer(Scorable scorer) {
this.dataValues.scorer = scorer;
}
@Override
public void collect(int doc) throws IOException {
dataValues.advanceTo(doc);
BytesRef cache_id = dataValues.cacheId.binaryValue();
BytesRef query_id = dataValues.queryId.binaryValue();
QueryCacheEntry query = queries.get(cache_id.utf8ToString());
matcher.matchQuery(query_id.utf8ToString(), query, dataValues);
}
@Override
public void doSetNextReader(LeafReaderContext context) throws IOException {
this.dataValues.cacheId = context.reader().getSortedDocValues(FIELDS.cache_id);
this.dataValues.queryId = context.reader().getSortedDocValues(FIELDS.query_id);
this.dataValues.mq = context.reader().getBinaryDocValues(FIELDS.mq);
this.dataValues.ctx = context;
}
@Override
public ScoreMode scoreMode() {
return matcher.scoreMode();
}
}
}

View File

@ -0,0 +1,84 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.util.Objects;
import org.apache.lucene.search.Scorable;
import org.apache.lucene.search.ScoreMode;
/**
* Represents a match for a specific query and document
* <p>
* Derived classes may contain more information (such as scores, highlights, etc)
*
* @see ExplainingMatch
* @see ScoringMatch
* @see HighlightsMatch
*/
public class QueryMatch {
private final String queryId;
public static final MatcherFactory<QueryMatch> SIMPLE_MATCHER =
searcher -> new CollectingMatcher<QueryMatch>(searcher, ScoreMode.COMPLETE_NO_SCORES) {
@Override
public QueryMatch resolve(QueryMatch match1, QueryMatch match2) {
return match1;
}
@Override
protected QueryMatch doMatch(String queryId, int doc, Scorable scorer) {
return new QueryMatch(queryId);
}
};
/**
* Creates a new QueryMatch for a specific query and document
*
* @param queryId the query id
*/
public QueryMatch(String queryId) {
this.queryId = Objects.requireNonNull(queryId);
}
/**
* @return the queryid of this match
*/
public String getQueryId() {
return queryId;
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (!(o instanceof QueryMatch)) return false;
QueryMatch that = (QueryMatch) o;
return Objects.equals(queryId, that.queryId);
}
@Override
public int hashCode() {
return Objects.hash(queryId);
}
@Override
public String toString() {
return "Match(query=" + queryId + ")";
}
}

View File

@ -0,0 +1,64 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.IOException;
import java.util.Map;
import org.apache.lucene.search.Query;
/**
* Notified of the time it takes to run individual queries against a set of documents
*/
public interface QueryTimeListener {
/**
* How long it took to run a particular query
*/
void logQueryTime(String queryId, long timeInNanos);
/**
* A wrapping matcher factory to log query times to a QueryTimeListener
* @param factory a matcher factory to use for the actual matching
* @param listener the QueryTimeListener
*/
static <T extends QueryMatch> MatcherFactory<T> timingMatcher(MatcherFactory<T> factory, QueryTimeListener listener) {
return searcher -> {
CandidateMatcher<T> matcher = factory.createMatcher(searcher);
return new CandidateMatcher<T>(searcher) {
@Override
protected void matchQuery(String queryId, Query matchQuery, Map<String, String> metadata) throws IOException {
long t = System.nanoTime();
matcher.matchQuery(queryId, matchQuery, metadata);
t = System.nanoTime() - t;
listener.logQueryTime(queryId, t);
}
@Override
public T resolve(T match1, T match2) {
return matcher.resolve(match1, match2);
}
@Override
protected void doFinish() {
copyMatches(matcher);
}
};
};
}
}

View File

@ -0,0 +1,299 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Comparator;
import java.util.List;
import java.util.Optional;
import java.util.function.BiConsumer;
import java.util.function.Function;
import java.util.stream.Collectors;
import org.apache.lucene.index.Term;
import org.apache.lucene.util.BytesRef;
/**
* A representation of a node in a query tree
*
* Queries are analyzed and converted into an abstract tree, consisting
* of conjunction and disjunction nodes, and leaf nodes containing terms.
*
* Terms may be collected from a node, which will use the weights of its
* sub-nodes to determine which paths are followed. The path may be changed
* by calling {@link #advancePhase(double)}
*/
public abstract class QueryTree {
/**
* The weight of this node
*/
public abstract double weight();
/**
* Collect terms from the most highly-weighted path below this node
*/
public abstract void collectTerms(BiConsumer<String, BytesRef> termCollector);
/**
* Find the next-most highly-weighted path below this node
* @param minWeight do not advance if the next path has a weight below this value
* @return {@code false} if there are no more paths above the minimum weight
*/
public abstract boolean advancePhase(double minWeight);
/**
* Returns a string representation of the node
* @param depth the current depth of this node in the overall query tree
*/
public abstract String toString(int depth);
@Override
public String toString() {
return toString(0);
}
/**
* Returns a string of {@code width} spaces
*/
protected String space(int width) {
StringBuilder sb = new StringBuilder();
for (int i = 0; i < width; i++) {
sb.append(" ");
}
return sb.toString();
}
/**
* Returns a leaf node for a particular term
*/
public static QueryTree term(Term term, TermWeightor weightor) {
return term(term.field(), term.bytes(), weightor.applyAsDouble(term));
}
/**
* Returns a leaf node for a particular term and weight
*
* The weight must be greater than 0
*/
public static QueryTree term(Term term, double weight) {
return term(term.field(), term.bytes(), weight);
}
/**
* Returns a leaf node for a particular term and weight
*
* The weight must be greater than 0
*/
public static QueryTree term(String field, BytesRef term, double weight) {
return new QueryTree() {
@Override
public double weight() {
if (weight <= 0) {
throw new IllegalArgumentException("Term weights must be greater than 0");
}
return weight;
}
@Override
public void collectTerms(BiConsumer<String, BytesRef> termCollector) {
termCollector.accept(field, term);
}
@Override
public boolean advancePhase(double minWeight) {
return false;
}
@Override
public String toString(int depth) {
return space(depth) + field + ":" + term.utf8ToString() + "^" + weight;
}
};
}
/**
* Returns a leaf node that will match any document
*/
public static QueryTree anyTerm(String reason) {
return new QueryTree() {
@Override
public double weight() {
return 0;
}
@Override
public void collectTerms(BiConsumer<String, BytesRef> termCollector) {
termCollector.accept(TermFilteredPresearcher.ANYTOKEN_FIELD, new BytesRef(TermFilteredPresearcher.ANYTOKEN));
}
@Override
public boolean advancePhase(double minWeight) {
return false;
}
@Override
public String toString(int depth) {
return space(depth) + "ANY[" + reason + "]";
}
};
}
/**
* Returns a conjunction of a set of child nodes
*/
public static QueryTree conjunction(List<Function<TermWeightor, QueryTree>> children, TermWeightor weightor) {
if (children.size() == 0) {
throw new IllegalArgumentException("Cannot build a conjunction with no children");
}
if (children.size() == 1) {
return children.get(0).apply(weightor);
}
List<QueryTree> qt = children.stream()
.map(f -> f.apply(weightor)).collect(Collectors.toList());
List<QueryTree> restricted = qt.stream().filter(t -> t.weight() > 0).collect(Collectors.toList());
if (restricted.size() == 0) {
// all children are ANY, so just return the first one
return qt.get(0);
}
return new ConjunctionQueryTree(qt);
}
static QueryTree conjunction(QueryTree... children) {
return new ConjunctionQueryTree(Arrays.asList(children));
}
/**
* Returns a disjunction of a set of child nodes
*/
public static QueryTree disjunction(List<Function<TermWeightor, QueryTree>> children, TermWeightor weightor) {
if (children.size() == 0) {
throw new IllegalArgumentException("Cannot build a disjunction with no children");
}
if (children.size() == 1) {
return children.get(0).apply(weightor);
}
List<QueryTree> qt = children.stream()
.map(f -> f.apply(weightor)).collect(Collectors.toList());
Optional<QueryTree> firstAnyChild = qt.stream().filter(q -> q.weight() == 0).findAny();
// if any of the children is an ANY node, just return that, otherwise build the disjunction
return firstAnyChild.orElseGet(() -> new DisjunctionQueryTree(qt));
}
static QueryTree disjunction(QueryTree... children) {
return new DisjunctionQueryTree(Arrays.asList(children));
}
private static class ConjunctionQueryTree extends QueryTree {
private static final Comparator<QueryTree> COMPARATOR = Comparator.comparingDouble(QueryTree::weight).reversed();
final List<QueryTree> children = new ArrayList<>();
ConjunctionQueryTree(List<QueryTree> children) {
this.children.addAll(children);
this.children.sort(COMPARATOR);
}
@Override
public double weight() {
return children.get(0).weight();
}
@Override
public void collectTerms(BiConsumer<String, BytesRef> termCollector) {
children.get(0).collectTerms(termCollector);
}
@Override
public boolean advancePhase(double minWeight) {
if (children.get(0).advancePhase(minWeight)) {
this.children.sort(COMPARATOR);
return true;
}
if (children.size() == 1) {
return false;
}
if (children.get(1).weight() <= minWeight) {
return false;
}
children.remove(0);
return true;
}
@Override
public String toString(int depth) {
StringBuilder sb = new StringBuilder(space(depth)).append("Conjunction[")
.append(children.size())
.append("]^")
.append(weight())
.append("\n");
for (QueryTree child : children) {
sb.append(child.toString(depth + 2)).append("\n");
}
return sb.toString();
}
}
private static class DisjunctionQueryTree extends QueryTree {
final List<QueryTree> children = new ArrayList<>();
private DisjunctionQueryTree(List<QueryTree> children) {
this.children.addAll(children);
this.children.sort(Comparator.comparingDouble(QueryTree::weight));
}
@Override
public double weight() {
return children.get(0).weight();
}
@Override
public void collectTerms(BiConsumer<String, BytesRef> termCollector) {
for (QueryTree child : children) {
child.collectTerms(termCollector);
}
}
@Override
public boolean advancePhase(double minWeight) {
boolean changed = false;
for (QueryTree child : children) {
changed |= child.advancePhase(minWeight);
}
if (changed == false) {
return false;
}
children.sort(Comparator.comparingDouble(QueryTree::weight));
return true;
}
@Override
public String toString(int depth) {
StringBuilder sb = new StringBuilder(space(depth)).append("Disjunction[");
sb.append(children.size()).append("]^");
sb.append(weight()).append("\n");
for (QueryTree child : children) {
sb.append(child.toString(depth + 2)).append("\n");
}
return sb.toString();
}
}
}

View File

@ -0,0 +1,155 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.util.Collections;
import java.util.Set;
import java.util.function.BiConsumer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.RegexpQuery;
import org.apache.lucene.util.BytesRef;
/**
* A query handler implementation that matches Regexp queries by indexing regex
* terms by their longest static substring, and generates ngrams from Document
* tokens to match them.
* <p>
* This implementation will filter out more wildcard queries than TermFilteredPresearcher,
* at the expense of longer document build times. Which one is more performant will depend
* on the type and number of queries registered in the Monitor, and the size of documents
* to be monitored. Profiling is recommended.
*/
public class RegexpQueryHandler implements CustomQueryHandler {
/**
* The default suffix with which to mark ngrams
*/
public static final String DEFAULT_NGRAM_SUFFIX = "XX";
/**
* The default maximum length of an input token before ANYTOKENS are generated
*/
public static final int DEFAULT_MAX_TOKEN_SIZE = 30;
/**
* The default token to emit if a term is longer than MAX_TOKEN_SIZE
*/
public static final String DEFAULT_WILDCARD_TOKEN = "__WILDCARD__";
private final String ngramSuffix;
private final String wildcardToken;
private final BytesRef wildcardTokenBytes;
private final int maxTokenSize;
private final Set<String> excludedFields;
/**
* Creates a new RegexpQueryHandler
*
* @param ngramSuffix the suffix with which to mark ngrams
* @param maxTokenSize the maximum length of an input token before WILDCARD tokens are generated
* @param wildcardToken the token to emit if a token is longer than maxTokenSize in length
* @param excludedFields a Set of fields to ignore when generating ngrams
*/
public RegexpQueryHandler(String ngramSuffix, int maxTokenSize, String wildcardToken, Set<String> excludedFields) {
this.ngramSuffix = ngramSuffix;
this.maxTokenSize = maxTokenSize;
this.wildcardTokenBytes = new BytesRef(wildcardToken);
this.wildcardToken = wildcardToken;
this.excludedFields = excludedFields == null ? Collections.emptySet() : excludedFields;
}
/**
* Creates a new RegexpQueryHandler using default settings
*/
public RegexpQueryHandler() {
this(DEFAULT_NGRAM_SUFFIX, DEFAULT_MAX_TOKEN_SIZE, DEFAULT_WILDCARD_TOKEN, null);
}
/**
* Creates a new RegexpQueryHandler with a maximum token size
*
* @param maxTokenSize the maximum length of an input token before WILDCARD tokens are generated
*/
public RegexpQueryHandler(int maxTokenSize) {
this(DEFAULT_NGRAM_SUFFIX, maxTokenSize, DEFAULT_WILDCARD_TOKEN, null);
}
@Override
public TokenStream wrapTermStream(String field, TokenStream ts) {
if (excludedFields.contains(field))
return ts;
return new SuffixingNGramTokenFilter(ts, ngramSuffix, wildcardToken, maxTokenSize);
}
@Override
public QueryTree handleQuery(Query q, TermWeightor termWeightor) {
if (q instanceof RegexpQuery == false) {
return null;
}
RegexpQuery query = (RegexpQuery) q;
String regexp = parseOutRegexp(query.toString(""));
String selected = selectLongestSubstring(regexp);
Term term = new Term(query.getField(), selected + ngramSuffix);
double weight = termWeightor.applyAsDouble(term);
return new QueryTree() {
@Override
public double weight() {
return weight;
}
@Override
public void collectTerms(BiConsumer<String, BytesRef> termCollector) {
termCollector.accept(term.field(), term.bytes());
termCollector.accept(term.field(), wildcardTokenBytes);
}
@Override
public boolean advancePhase(double minWeight) {
return false;
}
@Override
public String toString(int depth) {
return space(depth) + "WILDCARD_NGRAM[" + term.toString() + "]^" + weight;
}
};
}
private static String parseOutRegexp(String rep) {
int fieldSepPos = rep.indexOf(":");
int firstSlash = rep.indexOf("/", fieldSepPos);
int lastSlash = rep.lastIndexOf("/");
return rep.substring(firstSlash + 1, lastSlash);
}
private static String selectLongestSubstring(String regexp) {
String selected = "";
for (String substr : regexp.split("\\.|\\*|.\\?")) {
if (substr.length() > selected.length()) {
selected = substr;
}
}
return selected;
}
}

View File

@ -0,0 +1,81 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.IOException;
import org.apache.lucene.search.Scorable;
import org.apache.lucene.search.ScoreMode;
import org.apache.lucene.search.similarities.BM25Similarity;
import org.apache.lucene.search.similarities.Similarity;
/**
* A QueryMatch that reports scores for each match
*/
public class ScoringMatch extends QueryMatch {
public static final MatcherFactory<ScoringMatch> matchWithSimilarity(Similarity similarity) {
return searcher -> {
searcher.setSimilarity(similarity);
return new CollectingMatcher<ScoringMatch>(searcher, ScoreMode.COMPLETE) {
@Override
protected ScoringMatch doMatch(String queryId, int doc, Scorable scorer) throws IOException {
float score = scorer.score();
if (score > 0)
return new ScoringMatch(queryId, score);
return null;
}
@Override
public ScoringMatch resolve(ScoringMatch match1, ScoringMatch match2) {
return new ScoringMatch(match1.getQueryId(), match1.getScore() + match2.getScore());
}
};
};
}
public static final MatcherFactory<ScoringMatch> DEFAULT_MATCHER = matchWithSimilarity(new BM25Similarity());
private final float score;
private ScoringMatch(String queryId, float score) {
super(queryId);
this.score = score;
}
public float getScore() {
return score;
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (!(o instanceof ScoringMatch)) return false;
if (!super.equals(o)) return false;
ScoringMatch that = (ScoringMatch) o;
return Float.compare(that.score, score) == 0;
}
@Override
public int hashCode() {
int result = super.hashCode();
result = 31 * result + (score != +0.0f ? Float.floatToIntBits(score) : 0);
return result;
}
}

View File

@ -0,0 +1,88 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
/**
* Reports on slow queries in a given match run
*/
public class SlowLog implements Iterable<SlowLog.Entry> {
private final List<Entry> slowQueries = new ArrayList<>();
/**
* Add a query and time taken to the slow log.
* <p>
* The query will only be recorded if the time is above the configured limit
*
* @param query the query id
* @param time the time taken by the query in ns
*/
void addQuery(String query, long time) {
slowQueries.add(new Entry(query, time));
}
/**
* Add all entries to this slow log
*
* @param queries the entries to add
*/
void addAll(Iterable<SlowLog.Entry> queries) {
for (SlowLog.Entry query : queries) {
slowQueries.add(query);
}
}
@Override
public Iterator<Entry> iterator() {
return slowQueries.iterator();
}
/**
* An individual entry in the slow log
*/
public static class Entry {
/**
* The query id
*/
final String queryId;
/**
* The time taken to execute the query in ms
*/
final long time;
Entry(String queryId, long time) {
this.queryId = queryId;
this.time = time;
}
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder();
for (Entry entry : slowQueries) {
sb.append(entry.queryId).append(" [").append(entry.time).append("ns]\n");
}
return sb.toString();
}
}

View File

@ -0,0 +1,144 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.IOException;
import org.apache.lucene.analysis.TokenFilter;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.tokenattributes.*;
import org.apache.lucene.analysis.CharArraySet;
final class SuffixingNGramTokenFilter extends TokenFilter {
private final String suffix;
private final int maxTokenLength;
private final String anyToken;
private char[] curTermBuffer;
private int curTermLength;
private int curCodePointCount;
private int curGramSize;
private int curPos;
private int curPosInc, curPosLen;
private int tokStart;
private int tokEnd;
private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
private final PositionIncrementAttribute posIncAtt;
private final PositionLengthAttribute posLenAtt;
private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
private final KeywordAttribute keywordAtt = addAttribute(KeywordAttribute.class);
private final CharArraySet seenSuffixes = new CharArraySet(1024, false);
private final CharArraySet seenInfixes = new CharArraySet(1024, false);
/**
* Creates SuffixingNGramTokenFilter.
*
* @param input {@link org.apache.lucene.analysis.TokenStream} holding the input to be tokenized
* @param suffix a string to suffix to all ngrams
* @param wildcardToken a token to emit if the input token is longer than maxTokenLength
* @param maxTokenLength tokens longer than this will not be ngrammed
*/
public SuffixingNGramTokenFilter(TokenStream input, String suffix, String wildcardToken, int maxTokenLength) {
super(input);
this.suffix = suffix;
this.anyToken = wildcardToken;
this.maxTokenLength = maxTokenLength;
posIncAtt = addAttribute(PositionIncrementAttribute.class);
posLenAtt = addAttribute(PositionLengthAttribute.class);
}
/**
* Returns the next token in the stream, or null at EOS.
*/
@Override
public final boolean incrementToken() throws IOException {
while (true) {
if (curTermBuffer == null) {
if (!input.incrementToken()) {
return false;
}
if (keywordAtt.isKeyword())
return true;
curTermBuffer = termAtt.buffer().clone();
curTermLength = termAtt.length();
curCodePointCount = Character.codePointCount(termAtt, 0, termAtt.length());
curGramSize = curTermLength;
curPos = 0;
curPosInc = posIncAtt.getPositionIncrement();
curPosLen = posLenAtt.getPositionLength();
tokStart = offsetAtt.startOffset();
tokEnd = offsetAtt.endOffset();
//termAtt.setEmpty().append(suffix);
return true;
}
if (curTermLength > maxTokenLength) {
clearAttributes();
termAtt.append(anyToken);
curTermBuffer = null;
return true;
}
if (curGramSize == 0) {
++curPos;
curGramSize = curTermLength - curPos;
}
if (curGramSize >= 0 && (curPos + curGramSize) <= curCodePointCount) {
clearAttributes();
final int start = Character.offsetByCodePoints(curTermBuffer, 0, curTermLength, 0, curPos);
final int end = Character.offsetByCodePoints(curTermBuffer, 0, curTermLength, start, curGramSize);
termAtt.copyBuffer(curTermBuffer, start, end - start);
termAtt.append(suffix);
if ((curGramSize == curTermLength - curPos) && !seenSuffixes.add(termAtt.subSequence(0, termAtt.length()))) {
curTermBuffer = null;
continue;
}
if (!seenInfixes.add(termAtt.subSequence(0, termAtt.length()))) {
curGramSize = 0;
continue;
}
posIncAtt.setPositionIncrement(curPosInc);
curPosInc = 0;
posLenAtt.setPositionLength(curPosLen);
offsetAtt.setOffset(tokStart, tokEnd);
curGramSize--;
return true;
}
curTermBuffer = null;
}
}
@Override
public void reset() throws IOException {
super.reset();
curTermBuffer = null;
seenInfixes.clear();
seenSuffixes.clear();
}
}

View File

@ -0,0 +1,309 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.function.BiPredicate;
import org.apache.lucene.analysis.FilteringTokenFilter;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.TermInSetQuery;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefHash;
import org.apache.lucene.util.BytesRefIterator;
/**
* Presearcher implementation that uses terms extracted from queries to index
* them in the Monitor, and builds a disjunction from terms in a document to match
* them.
*
* Handling of queries that do not support term extraction through the
* {@link org.apache.lucene.search.QueryVisitor} API can be configured by passing
* a list of {@link CustomQueryHandler} implementations.
*
* Filtering by additional fields can be configured by passing a set of field names.
* Documents that contain values in those fields will only be checked against
* {@link MonitorQuery} instances that have the same fieldname-value mapping in
* their metadata.
*/
public class TermFilteredPresearcher extends Presearcher {
/**
* The default TermWeightor, weighting by token length
*/
public static final TermWeightor DEFAULT_WEIGHTOR = TermWeightor.DEFAULT;
private final QueryAnalyzer extractor;
private final TermWeightor weightor;
private final Set<String> filterFields;
private final List<CustomQueryHandler> queryHandlers = new ArrayList<>();
static final String ANYTOKEN_FIELD = "__anytokenfield";
static final String ANYTOKEN = "__ANYTOKEN__";
/**
* Creates a new TermFilteredPresearcher using the default term weighting
*/
public TermFilteredPresearcher() {
this(DEFAULT_WEIGHTOR, Collections.emptyList(), Collections.emptySet());
}
/**
* Creates a new TermFilteredPresearcher
*
* @param weightor the TermWeightor
* @param customQueryHandlers A list of custom query handlers to extract terms from non-core queries
* @param filterFields A set of fields to filter on
*/
public TermFilteredPresearcher(TermWeightor weightor, List<CustomQueryHandler> customQueryHandlers, Set<String> filterFields) {
this.extractor = new QueryAnalyzer(customQueryHandlers);
this.filterFields = filterFields;
this.queryHandlers.addAll(customQueryHandlers);
this.weightor = weightor;
}
@Override
public final Query buildQuery(LeafReader reader, BiPredicate<String, BytesRef> termAcceptor) {
try {
DocumentQueryBuilder queryBuilder = getQueryBuilder();
for (FieldInfo field : reader.getFieldInfos()) {
TokenStream ts = new TermsEnumTokenStream(reader.terms(field.name).iterator());
for (CustomQueryHandler handler : queryHandlers) {
ts = handler.wrapTermStream(field.name, ts);
}
ts = new FilteringTokenFilter(ts) {
TermToBytesRefAttribute termAtt = addAttribute(TermToBytesRefAttribute.class);
@Override
protected boolean accept() {
return filterFields.contains(field.name) == false && termAcceptor.test(field.name, termAtt.getBytesRef());
}
};
TermToBytesRefAttribute termAtt = ts.addAttribute(TermToBytesRefAttribute.class);
while (ts.incrementToken()) {
queryBuilder.addTerm(field.name, BytesRef.deepCopyOf(termAtt.getBytesRef()));
}
ts.close();
}
Query presearcherQuery = queryBuilder.build();
BooleanQuery.Builder bq = new BooleanQuery.Builder();
bq.add(presearcherQuery, BooleanClause.Occur.SHOULD);
bq.add(new TermQuery(new Term(ANYTOKEN_FIELD, ANYTOKEN)), BooleanClause.Occur.SHOULD);
presearcherQuery = bq.build();
if (filterFields.isEmpty() == false) {
bq = new BooleanQuery.Builder();
bq.add(presearcherQuery, BooleanClause.Occur.MUST);
Query filterQuery = buildFilterFields(reader);
if (filterQuery != null) {
bq.add(filterQuery, BooleanClause.Occur.FILTER);
presearcherQuery = bq.build();
}
}
return presearcherQuery;
} catch (IOException e) {
// We're a MemoryIndex, so this shouldn't happen...
throw new RuntimeException(e);
}
}
private Query buildFilterFields(LeafReader reader) throws IOException {
BooleanQuery.Builder builder = new BooleanQuery.Builder();
for (String field : filterFields) {
Query q = buildFilterClause(reader, field);
if (q != null) {
builder.add(q, BooleanClause.Occur.MUST);
}
}
BooleanQuery bq = builder.build();
if (bq.clauses().size() == 0) {
return null;
}
return bq;
}
private Query buildFilterClause(LeafReader reader, String field) throws IOException {
Terms terms = reader.terms(field);
if (terms == null)
return null;
BooleanQuery.Builder bq = new BooleanQuery.Builder();
int docsInBatch = reader.maxDoc();
BytesRef term;
TermsEnum te = terms.iterator();
while ((term = te.next()) != null) {
// we need to check that every document in the batch has the same field values, otherwise
// this filtering will not work
if (te.docFreq() != docsInBatch)
throw new IllegalArgumentException("Some documents in this batch do not have a term value of "
+ field + ":" + Term.toString(term));
bq.add(new TermQuery(new Term(field, BytesRef.deepCopyOf(term))), BooleanClause.Occur.SHOULD);
}
BooleanQuery built = bq.build();
if (built.clauses().size() == 0)
return null;
return built;
}
/**
* Constructs a document disjunction from a set of terms
*/
protected interface DocumentQueryBuilder {
/**
* Add a term from this document
*/
void addTerm(String field, BytesRef term) throws IOException;
/**
* @return the final Query
*/
Query build();
}
/**
* Returns a {@link DocumentQueryBuilder} for this presearcher
*/
protected DocumentQueryBuilder getQueryBuilder() {
return new DocumentQueryBuilder() {
Map<String, List<BytesRef>> terms = new HashMap<>();
@Override
public void addTerm(String field, BytesRef term) {
List<BytesRef> t = terms.computeIfAbsent(field, f -> new ArrayList<>());
t.add(term);
}
@Override
public Query build() {
BooleanQuery.Builder builder = new BooleanQuery.Builder();
for (String field : terms.keySet()) {
builder.add(new TermInSetQuery(field, terms.get(field)), BooleanClause.Occur.SHOULD);
}
return builder.build();
}
};
}
static final FieldType QUERYFIELDTYPE;
static {
QUERYFIELDTYPE = new FieldType(TextField.TYPE_NOT_STORED);
QUERYFIELDTYPE.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
QUERYFIELDTYPE.freeze();
}
@Override
public final Document indexQuery(Query query, Map<String, String> metadata) {
QueryTree querytree = extractor.buildTree(query, weightor);
Document doc = buildQueryDocument(querytree);
for (String field : filterFields) {
if (metadata != null && metadata.containsKey(field)) {
doc.add(new TextField(field, metadata.get(field), Field.Store.YES));
}
}
return doc;
}
/**
* Builds a {@link Document} from the terms extracted from a query
*/
protected Document buildQueryDocument(QueryTree querytree) {
Map<String, BytesRefHash> fieldTerms = collectTerms(querytree);
Document doc = new Document();
for (Map.Entry<String, BytesRefHash> entry : fieldTerms.entrySet()) {
doc.add(new Field(entry.getKey(),
new TermsEnumTokenStream(new BytesRefHashIterator(entry.getValue())), QUERYFIELDTYPE));
}
return doc;
}
/**
* Collects terms from a {@link QueryTree} and maps them per-field
*/
protected Map<String, BytesRefHash> collectTerms(QueryTree querytree) {
Map<String, BytesRefHash> fieldTerms = new HashMap<>();
querytree.collectTerms((field, term) -> {
BytesRefHash tt = fieldTerms.computeIfAbsent(field, f -> new BytesRefHash());
tt.add(term);
});
return fieldTerms;
}
/**
* Implements a {@link BytesRefIterator} over a {@link BytesRefHash}
*/
protected class BytesRefHashIterator implements BytesRefIterator {
final BytesRef scratch = new BytesRef();
final BytesRefHash terms;
final int[] sortedTerms;
int upto = -1;
BytesRefHashIterator(BytesRefHash terms) {
this.terms = terms;
this.sortedTerms = terms.sort();
}
@Override
public BytesRef next() {
if (upto >= sortedTerms.length)
return null;
upto++;
if (sortedTerms[upto] == -1)
return null;
this.terms.get(sortedTerms[upto], scratch);
return scratch;
}
}
}

View File

@ -0,0 +1,152 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.function.ToDoubleFunction;
import org.apache.lucene.index.Term;
import org.apache.lucene.util.BytesRef;
/**
* Calculates the weight of a {@link Term}
*/
public interface TermWeightor extends ToDoubleFunction<Term> {
/**
* A default TermWeightor based on token length
*/
TermWeightor DEFAULT = lengthWeightor(3, 0.3f);
/**
* Combine weightors by multiplication
*/
static TermWeightor combine(TermWeightor... weightors) {
return value -> {
double r = 1;
for (TermWeightor w : weightors) {
r *= w.applyAsDouble(value);
}
return r;
};
}
/**
* QueryTerms with a field from the selected set will be assigned the given weight
*/
static TermWeightor fieldWeightor(double weight, Set<String> fields) {
return value -> {
if (fields.contains(value.field())) {
return weight;
}
return 1;
};
}
/**
* QueryTerms with a field from the selected set will be assigned the given weight
*/
static TermWeightor fieldWeightor(double weight, String... fields) {
return fieldWeightor(weight, new HashSet<>(Arrays.asList(fields)));
}
/**
* QueryTerms with a term value from the selected set will be assigned the given weight
*/
static TermWeightor termWeightor(double weight, Set<BytesRef> terms) {
return value -> {
if (terms.contains(value.bytes())) {
return weight;
}
return 1;
};
}
/**
* QueryTerms with a term value from the selected set will be assigned the given weight
*/
static TermWeightor termWeightor(double weight, BytesRef... terms) {
return termWeightor(weight, new HashSet<>(Arrays.asList(terms)));
}
/**
* QueryTerms with a term and field value from the selected set will be assigned the given weight
*/
static TermWeightor termAndFieldWeightor(double weight, Set<Term> terms) {
return value -> {
if (terms.contains(value)) {
return weight;
}
return 1;
};
}
/**
* QueryTerms with a term and field value from the selected set will be assigned the given weight
*/
static TermWeightor termAndFieldWeightor(double weight, Term... terms) {
return termAndFieldWeightor(weight, new HashSet<>(Arrays.asList(terms)));
}
/**
* QueryTerms will be assigned a weight based on their term frequency
*
* More infrequent terms are weighted higher. Terms are weighted according
* to the function {@code w = (n / freq) + k}. Terms with no associated
* frequency receive a weight of value {@code 1}
*
* @param frequencies a map of terms to frequencies
* @param n a scaling factor
* @param k the minimum weight to scale to
*/
static TermWeightor termFreqWeightor(Map<String, Integer> frequencies, double n, double k) {
return value -> {
Integer mapVal = frequencies.get(value.text());
if (mapVal != null)
return (n / mapVal) + k;
return 1;
};
}
/**
* QueryTerms will be assigned a weight based on their term length
*
* Weights are assigned by the function {@code a * e ^ (-k * length)}. Longer
* terms are weighted higher. Terms of length greater than 32 all receive the
* same weight.
*
* @param a a
* @param k k
*/
static TermWeightor lengthWeightor(double a, double k) {
final double[] lengthNorms = new double[32];
for (int i = 0; i < 32; i++) {
lengthNorms[i] = (float) (a * (Math.exp(-k * i)));
}
return value -> {
if (value.bytes().length >= 32) {
return 4 - lengthNorms[31];
}
return 4 - lengthNorms[value.bytes().length];
};
}
}

View File

@ -0,0 +1,54 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.IOException;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefIterator;
/**
* A TokenStream created from a {@link org.apache.lucene.index.TermsEnum}
*/
class TermsEnumTokenStream extends TokenStream {
private final BytesRefIterator termsEnum;
private final CharTermAttribute charTerm = addAttribute(CharTermAttribute.class);
/**
* Create a new TermsEnumTokenStream using a TermsEnum
*
* @param termsEnum the TermsEnum to convert
*/
public TermsEnumTokenStream(BytesRefIterator termsEnum) {
this.termsEnum = termsEnum;
}
@Override
public final boolean incrementToken() throws IOException {
clearAttributes();
BytesRef bytes = termsEnum.next();
if (bytes == null)
return false;
charTerm.setEmpty();
charTerm.append(bytes.utf8ToString());
return true;
}
}

View File

@ -0,0 +1,98 @@
/*
* 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.
*/
/**
* <h2>Monitoring framework</h2>
*
* This package contains classes to allow the monitoring of a stream of
* documents with a set of queries.
*
* To use, instantiate a {@link org.apache.lucene.monitor.Monitor} object,
* register queries with it via
* {@link org.apache.lucene.monitor.Monitor#register(org.apache.lucene.monitor.MonitorQuery...)},
* and then match documents against it either invidually via
* {@link org.apache.lucene.monitor.Monitor#match(org.apache.lucene.document.Document, org.apache.lucene.monitor.MatcherFactory)}
* or in batches via
* {@link org.apache.lucene.monitor.Monitor#match(org.apache.lucene.document.Document[], org.apache.lucene.monitor.MatcherFactory)}
*
* <h3>Matcher types</h3>
*
* A number of matcher types are included:
* <ul>
* <li>{@link org.apache.lucene.monitor.QueryMatch#SIMPLE_MATCHER} &mdash; just returns the set of query ids that a Document has matched</li>
* <li>{@link org.apache.lucene.monitor.ScoringMatch#matchWithSimilarity(org.apache.lucene.search.similarities.Similarity)}
* &mdash; returns the set of matching queries, with the score that each one records against a Document</li>
* <li>{@link org.apache.lucene.monitor.ExplainingMatch#MATCHER &mdash; similar to ScoringMatch, but include the full Explanation}</li>
* <li>{@link org.apache.lucene.monitor.HighlightsMatch#MATCHER &mdash; return the matching queries along with the matching terms for each query}</li>
* </ul>
*
* Matchers can be wrapped in {@link org.apache.lucene.monitor.PartitionMatcher} or {@link org.apache.lucene.monitor.ParallelMatcher} to increase
* performance in low-concurrency systems.
*
* <h3>Pre-filtering of queries</h3>
*
* Monitoring is done efficiently by extracting minimal sets of terms from queries, and using these
* to build a query index. When a document is passed to
* {@link org.apache.lucene.monitor.Monitor#match(org.apache.lucene.document.Document, org.apache.lucene.monitor.MatcherFactory)},
* it is converted into a small index, and the terms dictionary from that index is then used to build
* a disjunction query to run against the query index. Queries that match this disjunction are then run
* against the document. In this way, the Monitor can avoid running queries that have no chance of
* matching. The process of extracting terms and building document disjunctions is handled by a
* {@link org.apache.lucene.monitor.Presearcher}
*
* In addition, extra per-field filtering can be specified by passing a set of keyword fields to
* filter on. When queries are registered with the monitor, field-value pairs can be added as
* optional metadata for each query, and these can then be used to restrict which queries a
* document is checked against. For example, you can specify a language that each query should
* apply to, and documents containing a value in their language field would only be checked against
* queries that have that same value in their language metadata. Note that when matching documents
* in batches, all documents in the batch must have the same values in their filter fields.
*
* Query analysis uses the {@link org.apache.lucene.search.QueryVisitor} API to extract terms, which will work
* for all basic term-based queries shipped with Lucene. The analyzer builds a representation of the query
* called a {@link org.apache.lucene.monitor.QueryTree}, and then selects a minimal set of terms, one of which
* must be present in a document for that document to match. Individual terms are weighted using a
* {@link org.apache.lucene.monitor.TermWeightor}, which allows some selectivity when building the term set.
* For example, given a conjunction of terms (a boolean query with several MUST clauses, or a phrase, span or interval
* query), we need only extract one term. The TermWeightor can be configured in a number of ways; by default
* it will weight longer terms more highly.
*
* For query sets that contain many conjunctions, it can be useful to extract and index different
* minimal term combinations. For example, a phrase query on 'the quick brown fox' could index
* both 'quick' and 'brown', and avoid being run againt documents that contain only one of these
* terms. The {@link org.apache.lucene.monitor.MultipassTermFilteredPresearcher} allows this sort
* of indexing, taking a minimum term weight so that very common terms such as 'the' can be avoided.
*
* Custom Query implementations that are based on term matching, and that implement
* {@link org.apache.lucene.search.Query#visit(org.apache.lucene.search.QueryVisitor)} will work with no
* extra configuration; for more complicated custom queries, you can register a
* {@link org.apache.lucene.monitor.CustomQueryHandler} with the presearcher. Included in this package
* is a {@link org.apache.lucene.monitor.RegexpQueryHandler}, which gives an example of a different method
* of indexing automaton-based queries by extracting fixed substrings from a regular expression, and then
* using ngram filtering to build the document disjunction.
*
* <h3>Persistent query sets</h3>
*
* By default, {@link org.apache.lucene.monitor.Monitor} instances are ephemeral, storing their query
* indexes in memory. To make a persistent monitor, build a {@link org.apache.lucene.monitor.MonitorConfiguration}
* object and call {@link org.apache.lucene.monitor.MonitorConfiguration#setIndexPath(java.nio.file.Path, org.apache.lucene.monitor.MonitorQuerySerializer)}
* to tell the Monitor to store its query index on disk. All queries registered with this Monitor will
* need to have a string representation that is also stored, and can be re-parsed by the associated
* {@link org.apache.lucene.monitor.MonitorQuerySerializer} when the index is loaded by a new Monitor
* instance.
*/
package org.apache.lucene.monitor;

View File

@ -0,0 +1,26 @@
<!--
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.
-->
<html>
<head>
<title>
monitor
</title>
</head>
<body>
Monitoring framework for lucene queries
</body>
</html>

View File

@ -0,0 +1,96 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.standard.StandardAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.NamedThreadFactory;
public abstract class ConcurrentMatcherTestBase extends LuceneTestCase {
private static final Analyzer ANALYZER = new StandardAnalyzer();
protected abstract <T extends QueryMatch> MatcherFactory<T> matcherFactory(ExecutorService executor,
MatcherFactory<T> factory, int threads);
public void testAllMatchesAreCollected() throws Exception {
ExecutorService executor = Executors.newFixedThreadPool(10, new NamedThreadFactory("matchers"));
try (Monitor monitor = new Monitor(ANALYZER)) {
List<MonitorQuery> queries = new ArrayList<>();
for (int i = 0; i < 1000; i++) {
queries.add(new MonitorQuery(Integer.toString(i), MonitorTestBase.parse("+test " + i)));
}
monitor.register(queries);
Document doc = new Document();
doc.add(newTextField("field", "test", Field.Store.NO));
MatchingQueries<QueryMatch> matches
= monitor.match(doc, matcherFactory(executor, QueryMatch.SIMPLE_MATCHER, 10));
assertEquals(1000, matches.getMatchCount());
}
finally {
executor.shutdown();
}
}
public void testMatchesAreDisambiguated() throws Exception {
ExecutorService executor = Executors.newFixedThreadPool(4, new NamedThreadFactory("matchers"));
try (Monitor monitor = new Monitor(ANALYZER)) {
List<MonitorQuery> queries = new ArrayList<>();
for (int i = 0; i < 10; i++) {
queries.add(new MonitorQuery(Integer.toString(i), MonitorTestBase.parse("test^10 doc " + i)));
}
monitor.register(queries);
assertEquals(30, monitor.getDisjunctCount());
Document doc = new Document();
doc.add(newTextField("field", "test doc doc", Field.Store.NO));
MatchingQueries<ScoringMatch> matches
= monitor.match(doc, matcherFactory(executor, ScoringMatch.DEFAULT_MATCHER, 10));
assertEquals(20, matches.getQueriesRun());
assertEquals(10, matches.getMatchCount());
assertTrue(matches.getErrors().isEmpty());
for (ScoringMatch match : matches.getMatches()) {
// The queries are all split into three by the QueryDecomposer, and the
// 'test' and 'doc' parts will match. 'test' will have a higher score,
// because of it's lower termfreq. We need to check that each query ends
// up with the sum of the scores for the 'test' and 'doc' parts
assertEquals(1.4874471f, match.getScore(), 0);
}
}
finally {
executor.shutdown();
}
}
}

View File

@ -0,0 +1,141 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.IOException;
import java.util.Collections;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.search.MatchAllDocsQuery;
import static org.hamcrest.CoreMatchers.containsString;
public abstract class FieldFilterPresearcherComponentTestBase extends PresearcherTestBase {
public void testBatchFiltering() throws IOException {
try (Monitor monitor = newMonitor()) {
monitor.register(
mq("1", "test", "language", "en"),
mq("2", "wahl", "language", "de"),
mq("3", "wibble", "language", "en"),
mq("4", "*:*", "language", "de"),
mq("5", "*:*", "language", "es"));
Document doc1 = new Document();
doc1.add(newTextField(TEXTFIELD, "this is a test", Field.Store.NO));
doc1.add(newTextField("language", "en", Field.Store.NO));
Document doc2 = new Document();
doc2.add(newTextField(TEXTFIELD, "this is a wibble", Field.Store.NO));
doc2.add(newTextField("language", "en", Field.Store.NO));
Document doc3 = new Document();
doc3.add(newTextField(TEXTFIELD, "wahl is a misspelling of whale", Field.Store.NO));
doc3.add(newTextField("language", "en", Field.Store.NO));
MultiMatchingQueries<QueryMatch> matches = monitor.match(new Document[]{ doc1, doc2, doc3 }, QueryMatch.SIMPLE_MATCHER);
assertEquals(1, matches.getMatchCount(0));
assertNotNull(matches.matches("1", 0));
assertEquals(1, matches.getMatchCount(1));
assertNotNull(matches.matches("3", 1));
assertEquals(0, matches.getMatchCount(2));
assertEquals(2, matches.getQueriesRun());
}
}
public void testBatchesWithDissimilarFieldValuesThrowExceptions() throws IOException {
Document doc1 = new Document();
doc1.add(newTextField(TEXTFIELD, "test", Field.Store.NO));
doc1.add(newTextField("language", "en", Field.Store.NO));
Document doc2 = new Document();
doc2.add(newTextField(TEXTFIELD, "test", Field.Store.NO));
doc2.add(newTextField("language", "de", Field.Store.NO));
try (Monitor monitor = newMonitor()) {
IllegalArgumentException e
= expectThrows(IllegalArgumentException.class, () -> monitor.match(new Document[]{ doc1, doc2 }, QueryMatch.SIMPLE_MATCHER));
assertThat(e.getMessage(), containsString("language:"));
}
}
public void testFieldFiltering() throws IOException {
try (Monitor monitor = newMonitor()) {
monitor.register(
new MonitorQuery("1", parse("test"), null, Collections.singletonMap("language", "en")),
new MonitorQuery("2", parse("test"), null, Collections.singletonMap("language", "de")),
new MonitorQuery("3", parse("wibble"), null, Collections.singletonMap("language", "en")),
new MonitorQuery("4", parse("*:*"), null, Collections.singletonMap("language", "de")));
Document enDoc = new Document();
enDoc.add(newTextField(TEXTFIELD, "this is a test", Field.Store.NO));
enDoc.add(newTextField("language", "en", Field.Store.NO));
MatchingQueries<QueryMatch> en = monitor.match(enDoc, QueryMatch.SIMPLE_MATCHER);
assertEquals(1, en.getMatchCount());
assertNotNull(en.matches("1"));
assertEquals(1, en.getQueriesRun());
Document deDoc = new Document();
deDoc.add(newTextField(TEXTFIELD, "das ist ein test", Field.Store.NO));
deDoc.add(newTextField("language", "de", Field.Store.NO));
MatchingQueries<QueryMatch> de = monitor.match(deDoc, QueryMatch.SIMPLE_MATCHER);
assertEquals(2, de.getMatchCount());
assertEquals(2, de.getQueriesRun());
assertNotNull(de.matches("2"));
assertNotNull(de.matches("4"));
Document bothDoc = new Document();
bothDoc.add(newTextField(TEXTFIELD, "this is ein test", Field.Store.NO));
bothDoc.add(newTextField("language", "en", Field.Store.NO));
bothDoc.add(newTextField("language", "de", Field.Store.NO));
MatchingQueries<QueryMatch> both = monitor.match(bothDoc, QueryMatch.SIMPLE_MATCHER);
assertEquals(3, both.getMatchCount());
assertEquals(3, both.getQueriesRun());
}
}
public void testFilteringOnMatchAllQueries() throws IOException {
try (Monitor monitor = newMonitor()) {
monitor.register(new MonitorQuery("1", new MatchAllDocsQuery(), null, Collections.singletonMap("language", "de")));
Document enDoc = new Document();
enDoc.add(newTextField(TEXTFIELD, "this is a test", Field.Store.NO));
enDoc.add(newTextField("language", "en", Field.Store.NO));
MatchingQueries<QueryMatch> matches = monitor.match(enDoc, QueryMatch.SIMPLE_MATCHER);
assertEquals(0, matches.getMatchCount());
assertEquals(0, matches.getQueriesRun());
}
}
public void testDebugQueries() throws Exception {
try (Monitor monitor = newMonitor()) {
monitor.register(new MonitorQuery("1", parse("test"), null, Collections.singletonMap("language", "en")));
Document enDoc = new Document();
enDoc.add(newTextField(TEXTFIELD, "this is a test", Field.Store.NO));
enDoc.add(newTextField("language", "en", Field.Store.NO));
PresearcherMatches<QueryMatch> matches = monitor.debug(enDoc, QueryMatch.SIMPLE_MATCHER);
assertFalse(matches.match("1", 0).presearcherMatches.isEmpty());
}
}
}

View File

@ -0,0 +1,93 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.standard.StandardAnalyzer;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.queryparser.classic.ParseException;
import org.apache.lucene.queryparser.classic.QueryParser;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryVisitor;
import org.apache.lucene.util.LuceneTestCase;
public abstract class MonitorTestBase extends LuceneTestCase {
public static final String FIELD = "field";
public static final Analyzer ANALYZER = new StandardAnalyzer();
public static Query parse(String query) {
QueryParser parser = new QueryParser(FIELD, ANALYZER);
try {
return parser.parse(query);
} catch (ParseException e) {
throw new IllegalArgumentException(e);
}
}
public static MonitorQuery mq(String id, String query, String... metadata) {
Query q = parse(query);
assert metadata.length % 2 == 0;
Map<String, String> mm = new HashMap<>();
for (int i = 0; i < metadata.length / 2; i += 2) {
mm.put(metadata[i], metadata[i + 1]);
}
return new MonitorQuery(id, q, query, mm);
}
protected Monitor newMonitor() throws IOException {
return newMonitor(new StandardAnalyzer());
}
protected Monitor newMonitor(Analyzer analyzer) throws IOException {
// TODO: randomize presearcher
return new Monitor(analyzer);
}
public static class ThrowOnRewriteQuery extends Query {
@Override
public Query rewrite(IndexReader reader) throws IOException {
throw new IOException("Error rewriting");
}
@Override
public String toString(String field) {
return "ThrowOnRewriteQuery";
}
@Override
public void visit(QueryVisitor visitor) {
visitor.visitLeaf(this);
}
@Override
public boolean equals(Object obj) {
return false;
}
@Override
public int hashCode() {
return 0;
}
}
}

View File

@ -0,0 +1,178 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.IOException;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.util.AttributeImpl;
import org.apache.lucene.util.AttributeReflector;
import org.apache.lucene.util.BytesRef;
public abstract class PresearcherTestBase extends MonitorTestBase {
public Monitor newMonitor() throws IOException {
return new Monitor(WHITESPACE, createPresearcher());
}
protected abstract Presearcher createPresearcher();
static final String TEXTFIELD = FIELD;
static final Analyzer WHITESPACE = new WhitespaceAnalyzer();
public static Document buildDoc(String field, String text) {
Document doc = new Document();
doc.add(newTextField(field, text, Field.Store.NO));
return doc;
}
public void testNullFieldHandling() throws IOException {
try (Monitor monitor = newMonitor()) {
monitor.register(new MonitorQuery("1", parse("field_1:test")));
assertEquals(0,
monitor.match(buildDoc("field_2", "test"), QueryMatch.SIMPLE_MATCHER).getMatchCount());
}
}
public void testEmptyMonitorHandling() throws IOException {
try (Monitor monitor = newMonitor()) {
MatchingQueries<QueryMatch> matches = monitor.match(buildDoc("field_2", "test"), QueryMatch.SIMPLE_MATCHER);
assertEquals(0, matches.getMatchCount());
assertEquals(0, matches.getQueriesRun());
}
}
public void testMatchAllQueryHandling() throws IOException {
try (Monitor monitor = newMonitor()) {
monitor.register(new MonitorQuery("1", new MatchAllDocsQuery()));
assertEquals(1,
monitor.match(buildDoc("f", "wibble"), QueryMatch.SIMPLE_MATCHER).getMatchCount());
}
}
public void testNegativeQueryHandling() throws IOException {
Query q = new BooleanQuery.Builder()
.add(new MatchAllDocsQuery(), BooleanClause.Occur.SHOULD)
.add(new TermQuery(new Term("f", "foo")), BooleanClause.Occur.MUST_NOT)
.build();
try (Monitor monitor = newMonitor()) {
monitor.register(new MonitorQuery("1", q));
MultiMatchingQueries<QueryMatch> matches = monitor.match(new Document[]{
buildDoc("f", "bar"), buildDoc("f", "foo")
}, QueryMatch.SIMPLE_MATCHER);
assertEquals(1, matches.getMatchCount(0));
assertEquals(0, matches.getMatchCount(1));
}
}
public void testAnyTokenHandling() throws IOException {
try (Monitor monitor = newMonitor()) {
monitor.register(new MonitorQuery("1", new MatchAllDocsQuery()));
MatchingQueries<QueryMatch> matches = monitor.match(buildDoc("f", "wibble"), QueryMatch.SIMPLE_MATCHER);
assertEquals(1, matches.getMatchCount());
assertEquals(1, matches.getQueriesRun());
}
}
private static final BytesRef NON_STRING_TERM = new BytesRef(new byte[]{60, 8, 0, 0, 0, 9});
static class BytesRefAttribute extends AttributeImpl implements TermToBytesRefAttribute {
@Override
public BytesRef getBytesRef() {
return NON_STRING_TERM;
}
@Override
public void clear() {
}
@Override
public void reflectWith(AttributeReflector attributeReflector) {
}
@Override
public void copyTo(AttributeImpl attribute) {
}
}
static final class NonStringTokenStream extends TokenStream {
final TermToBytesRefAttribute att;
boolean done = false;
NonStringTokenStream() {
addAttributeImpl(new BytesRefAttribute());
this.att = addAttribute(TermToBytesRefAttribute.class);
}
@Override
public boolean incrementToken() {
if (done)
return false;
return done = true;
}
}
public void testNonStringTermHandling() throws IOException {
FieldType ft = new FieldType();
ft.setTokenized(true);
ft.setIndexOptions(IndexOptions.DOCS_AND_FREQS);
try (Monitor monitor = newMonitor()) {
monitor.register(new MonitorQuery("1", new TermQuery(new Term("f", NON_STRING_TERM))));
Document doc = new Document();
doc.add(new Field("f", new NonStringTokenStream(), ft));
MatchingQueries<QueryMatch> m = monitor.match(doc, QueryMatch.SIMPLE_MATCHER);
assertEquals(1, m.getMatchCount());
assertEquals(1, m.getQueriesRun());
}
}
public static BooleanClause must(Query q) {
return new BooleanClause(q, BooleanClause.Occur.MUST);
}
public static BooleanClause should(Query q) {
return new BooleanClause(q, BooleanClause.Occur.SHOULD);
}
}

View File

@ -0,0 +1,64 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.util.Collections;
import java.util.HashSet;
import java.util.Set;
import org.apache.lucene.document.LongPoint;
import org.apache.lucene.index.Term;
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.lucene.util.LuceneTestCase;
public class TestBooleanClauseWeightings extends LuceneTestCase {
private static QueryAnalyzer treeBuilder = new QueryAnalyzer();
public void testExactClausesPreferred() {
Query bq = new BooleanQuery.Builder()
.add(LongPoint.newRangeQuery("field2", 1, 2), BooleanClause.Occur.MUST)
.add(new BooleanQuery.Builder()
.add(new TermQuery(new Term("field1", "term1")), BooleanClause.Occur.SHOULD)
.add(new TermQuery(new Term("field1", "term2")), BooleanClause.Occur.SHOULD)
.build(), BooleanClause.Occur.MUST)
.build();
QueryTree tree = treeBuilder.buildTree(bq, TermWeightor.DEFAULT);
Set<Term> terms = new HashSet<>();
tree.collectTerms((f, b) -> terms.add(new Term(f, b)));
assertEquals(2, terms.size());
}
public void testLongerTermsPreferred() {
Query q = new BooleanQuery.Builder()
.add(new TermQuery(new Term("field1", "a")), BooleanClause.Occur.MUST)
.add(new TermQuery(new Term("field1", "supercalifragilisticexpialidocious")), BooleanClause.Occur.MUST)
.add(new TermQuery(new Term("field1", "b")), BooleanClause.Occur.MUST)
.build();
Set<Term> expected
= Collections.singleton(new Term("field1", "supercalifragilisticexpialidocious"));
QueryTree tree = treeBuilder.buildTree(q, TermWeightor.DEFAULT);
Set<Term> terms = new HashSet<>();
tree.collectTerms((f, b) -> terms.add(new Term(f, b)));
assertEquals(expected, terms);
}
}

View File

@ -0,0 +1,135 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.Set;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.LuceneTestCase;
public class TestBooleanTermExtractor extends LuceneTestCase {
private static final QueryAnalyzer treeBuilder = new QueryAnalyzer();
private static final TermWeightor WEIGHTOR = TermWeightor.DEFAULT;
private Set<Term> collectTerms(Query query) {
Set<Term> terms = new HashSet<>();
QueryTree tree = treeBuilder.buildTree(query, TermWeightor.DEFAULT);
tree.collectTerms((f, b) -> terms.add(new Term(f, b)));
return terms;
}
public void testAllDisjunctionQueriesAreIncluded() {
Query bq = MonitorTestBase.parse("field1:term1 field1:term2");
Set<Term> terms = collectTerms(bq);
Set<Term> expected = new HashSet<>(Arrays.asList(
new Term("field1", "term1"),
new Term("field1", "term2")));
assertEquals(expected, terms);
}
public void testAllNestedDisjunctionClausesAreIncluded() {
Query q = MonitorTestBase.parse("field1:term3 (field1:term1 field1:term2)");
assertEquals(3, collectTerms(q).size());
}
public void testAllDisjunctionClausesOfAConjunctionAreExtracted() {
Query q = MonitorTestBase.parse("+(field1:term1 field1:term2) field1:term3");
assertEquals(2, collectTerms(q).size());
}
public void testConjunctionsOutweighDisjunctions() {
Query bq = MonitorTestBase.parse("field1:term1 +field1:term2");
Set<Term> expected = Collections.singleton(new Term("field1", "term2"));
assertEquals(expected, collectTerms(bq));
}
public void testDisjunctionsWithPureNegativeClausesReturnANYTOKEN() {
Query q = MonitorTestBase.parse("+field1:term1 +(field2:term22 (-field2:notterm))");
Set<Term> expected = Collections.singleton(new Term("field1", "term1"));
assertEquals(expected, collectTerms(q));
}
public void testDisjunctionsWithMatchAllNegativeClausesReturnANYTOKEN() {
Query q = MonitorTestBase.parse("+field1:term1 +(field2:term22 (*:* -field2:notterm))");
Set<Term> expected = Collections.singleton(new Term("field1", "term1"));
assertEquals(expected, collectTerms(q));
}
public void testMatchAllDocsIsOnlyQuery() {
// Set up - single MatchAllDocsQuery clause in a BooleanQuery
Query q = MonitorTestBase.parse("+*:*");
assertTrue(q instanceof BooleanQuery);
BooleanClause clause = ((BooleanQuery)q).iterator().next();
assertTrue(clause.getQuery() instanceof MatchAllDocsQuery);
assertEquals(BooleanClause.Occur.MUST, clause.getOccur());
Set<Term> terms = collectTerms(q);
assertEquals(1, terms.size());
Term t = terms.iterator().next();
assertEquals(TermFilteredPresearcher.ANYTOKEN_FIELD, t.field());
}
public void testMatchAllDocsMustWithKeywordShould() {
Query q = MonitorTestBase.parse("+*:* field1:term1");
// Because field1:term1 is optional, only the MatchAllDocsQuery is collected.
Set<Term> terms = collectTerms(q);
assertEquals(1, terms.size());
Term t = terms.iterator().next();
assertEquals(TermFilteredPresearcher.ANYTOKEN_FIELD, t.field());
}
public void testMatchAllDocsMustWithKeywordNot() throws Exception {
Query q = MonitorTestBase.parse("+*:* -field1:notterm");
// Because field1:notterm is negated, only the mandatory MatchAllDocsQuery is collected.
Set<Term> terms = collectTerms(q);
assertEquals(1, terms.size());
Term t = terms.iterator().next();
assertEquals(TermFilteredPresearcher.ANYTOKEN_FIELD, t.field());
}
public void testMatchAllDocsMustWithKeywordShouldAndKeywordNot() throws Exception {
Query q = MonitorTestBase.parse("+*:* field1:term1 -field2:notterm");
// Because field1:notterm is negated and field1:term1 is optional, only the mandatory MatchAllDocsQuery is collected.
Set<Term> terms = collectTerms(q);
assertEquals(1, terms.size());
Term t = terms.iterator().next();
assertEquals(TermFilteredPresearcher.ANYTOKEN_FIELD, t.field());
}
public void testMatchAllDocsMustAndOtherMustWithKeywordShouldAndKeywordNot() throws Exception {
Query q = MonitorTestBase.parse("+*:* +field9:term9 field1:term1 -field2:notterm");
// The queryterm collected by weight is the non-anynode, so field9:term9 shows up before MatchAllDocsQuery.
Set<Term> terms = collectTerms(q);
Set<Term> expected = Collections.singleton(new Term("field9", "term9"));
assertEquals(expected, terms);
}
}

View File

@ -0,0 +1,153 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.IOException;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.util.NamedThreadFactory;
import static org.hamcrest.core.Is.is;
public class TestCachePurging extends MonitorTestBase {
public void testQueryCacheCanBePurged() throws IOException {
final AtomicInteger purgeCount = new AtomicInteger();
MonitorUpdateListener listener = new MonitorUpdateListener() {
@Override
public void onPurge() {
purgeCount.incrementAndGet();
}
};
try (Monitor monitor = new Monitor(ANALYZER)) {
MonitorQuery[] queries = new MonitorQuery[]{
new MonitorQuery("1", parse("test1 test4")),
new MonitorQuery("2", parse("test2")),
new MonitorQuery("3", parse("test3"))
};
monitor.addQueryIndexUpdateListener(listener);
monitor.register(queries);
assertThat(monitor.getQueryCount(), is(3));
assertThat(monitor.getDisjunctCount(), is(4));
assertThat(monitor.getQueryCacheStats().cachedQueries, is(4));
Document doc = new Document();
doc.add(newTextField("field", "test1 test2 test3", Field.Store.NO));
assertThat(monitor.match(doc, QueryMatch.SIMPLE_MATCHER).getMatchCount(), is(3));
monitor.deleteById("1");
assertThat(monitor.getQueryCount(), is(2));
assertThat(monitor.getQueryCacheStats().cachedQueries, is(4));
assertThat(monitor.match(doc, QueryMatch.SIMPLE_MATCHER).getMatchCount(), is(2));
monitor.purgeCache();
assertThat(monitor.getQueryCacheStats().cachedQueries, is(2));
MatchingQueries<QueryMatch> result = monitor.match(doc, QueryMatch.SIMPLE_MATCHER);
assertThat(result.getMatchCount(), is(2));
assertTrue(purgeCount.get() > 0);
}
}
public void testConcurrentPurges() throws Exception {
int iters = Integer.getInteger("purgeIters", 2);
for (int i = 0; i < iters; i++) {
doConcurrentPurgesAndUpdatesTest();
}
}
private static void doConcurrentPurgesAndUpdatesTest() throws Exception {
final CountDownLatch startUpdating = new CountDownLatch(1);
final CountDownLatch finishUpdating = new CountDownLatch(1);
try (final Monitor monitor = new Monitor(ANALYZER)) {
Runnable updaterThread = () -> {
try {
startUpdating.await();
for (int i = 200; i < 400; i++) {
monitor.register(newMonitorQuery(i));
}
finishUpdating.countDown();
} catch (Exception e) {
throw new RuntimeException(e);
}
};
ExecutorService executor = Executors.newFixedThreadPool(1, new NamedThreadFactory("updaters"));
try {
executor.submit(updaterThread);
for (int i = 0; i < 200; i++) {
monitor.register(newMonitorQuery(i));
}
for (int i = 20; i < 80; i++) {
monitor.deleteById(Integer.toString(i));
}
assertEquals(200, monitor.getQueryCacheStats().cachedQueries);
startUpdating.countDown();
monitor.purgeCache();
finishUpdating.await();
assertEquals(340, monitor.getQueryCacheStats().cachedQueries);
Document doc = new Document();
doc.add(newTextField("field", "test", Field.Store.NO));
MatchingQueries<QueryMatch> matcher = monitor.match(doc, QueryMatch.SIMPLE_MATCHER);
assertEquals(0, matcher.getErrors().size());
assertEquals(340, matcher.getMatchCount());
} finally {
executor.shutdownNow();
}
}
}
private static MonitorQuery newMonitorQuery(int id) {
return new MonitorQuery(Integer.toString(id), parse("+test " + id));
}
public void testBackgroundPurges() throws IOException, InterruptedException {
MonitorConfiguration config = new MonitorConfiguration().setPurgeFrequency(1, TimeUnit.SECONDS);
try (Monitor monitor = new Monitor(ANALYZER, Presearcher.NO_FILTERING, config)) {
assertEquals(-1, monitor.getQueryCacheStats().lastPurged);
for (int i = 0; i < 100; i++) {
monitor.register(newMonitorQuery(i));
}
monitor.deleteById("5");
assertEquals(99, monitor.getQueryCacheStats().queries);
assertEquals(100, monitor.getQueryCacheStats().cachedQueries);
TimeUnit.SECONDS.sleep(2);
assertEquals(99, monitor.getQueryCacheStats().queries);
assertEquals(99, monitor.getQueryCacheStats().cachedQueries);
assertTrue(monitor.getQueryCacheStats().lastPurged > 0);
}
}
}

View File

@ -0,0 +1,33 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
public class TestConcurrentQueryLoader extends MonitorTestBase {
public void testLoading() throws Exception {
try (Monitor monitor = newMonitor()) {
try (ConcurrentQueryLoader loader = new ConcurrentQueryLoader(monitor)) {
for (int i = 0; i < 2000; i++) {
loader.add(new MonitorQuery(Integer.toString(i), parse("\"test " + i + "\"")));
}
}
assertEquals(2000, monitor.getQueryCount());
}
}
}

View File

@ -0,0 +1,53 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.IOException;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.search.Explanation;
public class TestExplainingMatcher extends MonitorTestBase {
public void testExplainingMatcher() throws IOException {
try (Monitor monitor = newMonitor()) {
monitor.register(new MonitorQuery("1", parse("test")), new MonitorQuery("2", parse("wibble")));
Document doc = new Document();
doc.add(newTextField("field", "test", Field.Store.NO));
MatchingQueries<ExplainingMatch> matches = monitor.match(doc, ExplainingMatch.MATCHER);
assertNotNull(matches.matches("1"));
assertNotNull(matches.matches("1").getExplanation());
}
}
public void testHashcodeAndEquals() {
ExplainingMatch m1 = new ExplainingMatch("1", Explanation.match(0.1f, "an explanation"));
ExplainingMatch m3 = new ExplainingMatch("1", Explanation.match(0.1f, "another explanation"));
ExplainingMatch m4 = new ExplainingMatch("1", Explanation.match(0.1f, "an explanation"));
assertEquals(m1, m4);
assertEquals(m1.hashCode(), m4.hashCode());
assertNotEquals(m1, m3);
assertNotEquals(m3, m4);
}
}

View File

@ -0,0 +1,103 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.Set;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.BoostQuery;
import org.apache.lucene.search.ConstantScoreQuery;
import org.apache.lucene.search.DisjunctionMaxQuery;
import org.apache.lucene.search.PhraseQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.util.LuceneTestCase;
public class TestExtractors extends LuceneTestCase {
private static final QueryAnalyzer treeBuilder = new QueryAnalyzer();
private Set<Term> collectTerms(Query query) {
Set<Term> terms = new HashSet<>();
QueryTree tree = treeBuilder.buildTree(query, TermWeightor.DEFAULT);
tree.collectTerms((f, b) -> terms.add(new Term(f, b)));
return terms;
}
public void testConstantScoreQueryExtractor() {
BooleanQuery.Builder bq = new BooleanQuery.Builder();
bq.add(new TermQuery(new Term("f", "q1")), BooleanClause.Occur.MUST);
bq.add(new TermQuery(new Term("f", "q2")), BooleanClause.Occur.SHOULD);
Query csqWithQuery = new ConstantScoreQuery(bq.build());
Set<Term> expected = Collections.singleton(new Term("f", "q1"));
assertEquals(expected, collectTerms(csqWithQuery));
}
public void testPhraseQueryExtractor() {
PhraseQuery.Builder pq = new PhraseQuery.Builder();
pq.add(new Term("f", "hello"));
pq.add(new Term("f", "encyclopedia"));
Set<Term> expected = Collections.singleton(new Term("f", "encyclopedia"));
assertEquals(expected, collectTerms(pq.build()));
}
public void testBoostQueryExtractor() {
BooleanQuery.Builder bq = new BooleanQuery.Builder();
bq.add(new TermQuery(new Term("f", "q1")), BooleanClause.Occur.MUST);
bq.add(new TermQuery(new Term("f", "q2")), BooleanClause.Occur.SHOULD);
Query boostQuery = new BoostQuery(bq.build(), 0.5f);
Set<Term> expected = Collections.singleton(new Term("f", "q1"));
assertEquals(expected, collectTerms(boostQuery));
}
public void testDisjunctionMaxExtractor() {
Query query = new DisjunctionMaxQuery(
Arrays.asList(new TermQuery(new Term("f", "t1")), new TermQuery(new Term("f", "t2"))), 0.1f
);
Set<Term> expected = new HashSet<>(Arrays.asList(
new Term("f", "t1"),
new Term("f", "t2")
));
assertEquals(expected, collectTerms(query));
}
public void testBooleanExtractsFilter() {
Query q = new BooleanQuery.Builder()
.add(new TermQuery(new Term("f", "must")), BooleanClause.Occur.MUST)
.add(new TermQuery(new Term("f", "filter")), BooleanClause.Occur.FILTER)
.build();
Set<Term> expected = Collections.singleton(new Term("f", "filter")); // it's longer, so it wins
assertEquals(expected, collectTerms(q));
}
}

View File

@ -0,0 +1,29 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.util.Collections;
public class TestFieldFilteredMultipassPresearcher extends FieldFilterPresearcherComponentTestBase {
@Override
protected Presearcher createPresearcher() {
return new MultipassTermFilteredPresearcher(2, 0, TermWeightor.DEFAULT,
Collections.emptyList(), Collections.singleton("language"));
}
}

View File

@ -0,0 +1,28 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.util.Collections;
public class TestFieldTermFilteredPresearcher extends FieldFilterPresearcherComponentTestBase {
@Override
protected Presearcher createPresearcher() {
return new TermFilteredPresearcher(TermWeightor.DEFAULT, Collections.emptyList(), Collections.singleton("language"));
}
}

View File

@ -0,0 +1,82 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.IOException;
import org.apache.lucene.analysis.standard.StandardAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.PrefixQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.store.ByteBuffersDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.LuceneTestCase;
public class TestForceNoBulkScoringQuery extends LuceneTestCase {
public void testEquality() {
TermQuery tq1 = new TermQuery(new Term("f", "t"));
TermQuery tq2 = new TermQuery(new Term("f", "t2"));
TermQuery tq3 = new TermQuery(new Term("f", "t2"));
assertEquals(new ForceNoBulkScoringQuery(tq1), new ForceNoBulkScoringQuery(tq1));
assertNotEquals(new ForceNoBulkScoringQuery(tq1), new ForceNoBulkScoringQuery(tq2));
assertEquals(new ForceNoBulkScoringQuery(tq2), new ForceNoBulkScoringQuery(tq3));
assertEquals(new ForceNoBulkScoringQuery(tq2).hashCode(), new ForceNoBulkScoringQuery(tq3).hashCode());
}
public void testRewrite() throws IOException {
try (Directory dir = new ByteBuffersDirectory();
IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig(new StandardAnalyzer()))) {
Document doc = new Document();
doc.add(new TextField("field", "term1 term2 term3 term4", Field.Store.NO));
iw.addDocument(doc);
iw.commit();
IndexReader reader = DirectoryReader.open(dir);
PrefixQuery pq = new PrefixQuery(new Term("field", "term"));
ForceNoBulkScoringQuery q = new ForceNoBulkScoringQuery(pq);
assertEquals(q.getWrappedQuery(), pq);
Query rewritten = q.rewrite(reader);
assertTrue(rewritten instanceof ForceNoBulkScoringQuery);
Query inner = ((ForceNoBulkScoringQuery) rewritten).getWrappedQuery();
assertNotEquals(inner, pq);
}
}
}

View File

@ -0,0 +1,527 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
import org.apache.lucene.analysis.core.WhitespaceTokenizer;
import org.apache.lucene.analysis.standard.StandardAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.Term;
import org.apache.lucene.queryparser.complexPhrase.ComplexPhraseQueryParser;
import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.DisjunctionMaxQuery;
import org.apache.lucene.search.PrefixQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.RegexpQuery;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.WildcardQuery;
import org.apache.lucene.search.spans.SpanMultiTermQueryWrapper;
import org.apache.lucene.search.spans.SpanNearQuery;
import org.apache.lucene.search.spans.SpanQuery;
import org.apache.lucene.search.spans.SpanTermQuery;
public class TestHighlightingMatcher extends MonitorTestBase {
private static final Analyzer WHITESPACE = new WhitespaceAnalyzer();
public static Document buildDoc(String text) {
Document doc = new Document();
doc.add(newTextField(FIELD, text, Field.Store.NO));
return doc;
}
public void testSingleTermQueryMatchesSingleDocument() throws IOException {
try (Monitor monitor = newMonitor()) {
MonitorQuery mq = new MonitorQuery("query1", parse("test"));
monitor.register(mq);
MatchingQueries<HighlightsMatch> matches = monitor.match(buildDoc("this is a test document"),
HighlightsMatch.MATCHER);
assertEquals(1, matches.getMatchCount());
HighlightsMatch match = matches.matches("query1");
assertTrue(match.getHits(FIELD).contains(new HighlightsMatch.Hit(3, 10, 3, 14)));
}
}
public void testSinglePhraseQueryMatchesSingleDocument() throws IOException {
try (Monitor monitor = newMonitor()) {
MonitorQuery mq = new MonitorQuery("query1", parse("\"test document\""));
monitor.register(mq);
MatchingQueries<HighlightsMatch> matches = monitor.match(buildDoc("this is a test document"),
HighlightsMatch.MATCHER);
assertEquals(1, matches.getMatchCount());
HighlightsMatch m = matches.matches("query1");
assertTrue(m.getHits(FIELD).contains(new HighlightsMatch.Hit(3, 10, 4, 23)));
}
}
public void testToString() {
HighlightsMatch match = new HighlightsMatch("1");
match.addHit("field", 2, 3, -1, -1);
match.addHit("field", 0, 1, -1, -1);
match.addHit("afield", 0, 1, 0, 4);
assertEquals("Match(query=1){hits={afield=[0(0)->1(4)], field=[0(-1)->1(-1), 2(-1)->3(-1)]}}", match.toString());
}
public void testMultiFieldQueryMatches() throws IOException {
Document doc = new Document();
doc.add(newTextField("field1", "this is a test of field one", Field.Store.NO));
doc.add(newTextField("field2", "and this is an additional test", Field.Store.NO));
try (Monitor monitor = newMonitor()) {
monitor.register(new MonitorQuery("query1", parse("field1:test field2:test")));
MatchingQueries<HighlightsMatch> matches = monitor.match(doc, HighlightsMatch.MATCHER);
assertEquals(1, matches.getMatchCount());
HighlightsMatch m = matches.matches("query1");
assertNotNull(m);
assertTrue(m.getFields().contains("field1"));
assertTrue(m.getHits("field1").contains(new HighlightsMatch.Hit(3, 10, 3, 14)));
assertTrue(m.getHits("field2").contains(new HighlightsMatch.Hit(5, 26, 5, 30)));
}
}
public void testQueryErrors() throws IOException {
try (Monitor monitor = new Monitor(ANALYZER, Presearcher.NO_FILTERING)) {
monitor.register(new MonitorQuery("1", parse("test")),
new MonitorQuery("2", new ThrowOnRewriteQuery()),
new MonitorQuery("3", parse("document")),
new MonitorQuery("4", parse("foo")));
MatchingQueries<HighlightsMatch> matches = monitor.match(buildDoc("this is a test document"), HighlightsMatch.MATCHER);
assertEquals(4, matches.getQueriesRun());
assertEquals(2, matches.getMatchCount());
assertEquals(1, matches.getErrors().size());
}
}
public void testWildcards() throws IOException {
try (Monitor monitor = newMonitor()) {
monitor.register(new MonitorQuery("1", new RegexpQuery(new Term(FIELD, "he.*"))));
MatchingQueries<HighlightsMatch> matches = monitor.match(buildDoc("hello world"), HighlightsMatch.MATCHER);
assertEquals(1, matches.getQueriesRun());
assertEquals(1, matches.getMatchCount());
assertEquals(1, matches.matches("1").getHitCount());
}
}
public void testWildcardCombinations() throws Exception {
final BooleanQuery bq = new BooleanQuery.Builder()
.add(new TermQuery(new Term(FIELD, "term1")), BooleanClause.Occur.MUST)
.add(new PrefixQuery(new Term(FIELD, "term2")), BooleanClause.Occur.MUST)
.add(new TermQuery(new Term(FIELD, "term3")), BooleanClause.Occur.MUST_NOT)
.build();
try (Monitor monitor = newMonitor()) {
monitor.register(new MonitorQuery("1", bq));
MatchingQueries<HighlightsMatch> matches = monitor.match(buildDoc("term1 term22 term4"), HighlightsMatch.MATCHER);
HighlightsMatch m = matches.matches("1");
assertNotNull(m);
assertEquals(2, m.getHitCount());
}
}
public void testDisjunctionMaxQuery() throws IOException {
final DisjunctionMaxQuery query = new DisjunctionMaxQuery(Arrays.asList(
new TermQuery(new Term(FIELD, "term1")), new PrefixQuery(new Term(FIELD, "term2"))
), 1.0f);
try (Monitor monitor = newMonitor()) {
monitor.register(new MonitorQuery("1", query));
MatchingQueries<HighlightsMatch> matches = monitor.match(buildDoc("term1 term2 term3"), HighlightsMatch.MATCHER);
HighlightsMatch m = matches.matches("1");
assertNotNull(m);
assertEquals(2, m.getHitCount());
}
}
public void testIdenticalMatches() throws Exception {
final BooleanQuery bq = new BooleanQuery.Builder()
.add(new TermQuery(new Term(FIELD, "term1")), BooleanClause.Occur.MUST)
.add(new TermQuery(new Term(FIELD, "term1")), BooleanClause.Occur.SHOULD)
.build();
try (Monitor monitor = new Monitor(ANALYZER)) {
monitor.register(new MonitorQuery("1", bq));
MatchingQueries<HighlightsMatch> matches = monitor.match(buildDoc("term1 term2"), HighlightsMatch.MATCHER);
HighlightsMatch m = matches.matches("1");
assertNotNull(m);
assertEquals(1, m.getHitCount());
}
}
public void testWildcardBooleanRewrites() throws Exception {
final Query wc = new PrefixQuery(new Term(FIELD, "term1"));
final Query wrapper = new BooleanQuery.Builder()
.add(wc, BooleanClause.Occur.MUST)
.build();
final Query wrapper2 = new BooleanQuery.Builder()
.add(wrapper, BooleanClause.Occur.MUST)
.build();
final BooleanQuery bq = new BooleanQuery.Builder()
.add(new PrefixQuery(new Term(FIELD, "term2")), BooleanClause.Occur.MUST)
.add(wrapper2, BooleanClause.Occur.MUST_NOT)
.build();
try (Monitor monitor = new Monitor(ANALYZER)) {
monitor.register(new MonitorQuery("1", bq));
MatchingQueries<HighlightsMatch> matches = monitor.match(buildDoc("term2 term"), HighlightsMatch.MATCHER);
HighlightsMatch m = matches.matches("1");
assertNotNull(m);
assertEquals(1, m.getHitCount());
matches = monitor.match(buildDoc("term2 term"), HighlightsMatch.MATCHER);
m = matches.matches("1");
assertNotNull(m);
assertEquals(1, m.getHitCount());
}
}
public void testWildcardProximityRewrites() throws Exception {
final SpanNearQuery snq = SpanNearQuery.newOrderedNearQuery(FIELD)
.addClause(new SpanMultiTermQueryWrapper<>(new WildcardQuery(new Term(FIELD, "term*"))))
.addClause(new SpanTermQuery(new Term(FIELD, "foo")))
.build();
try (Monitor monitor = newMonitor()) {
monitor.register(new MonitorQuery("1", snq));
MatchingQueries<HighlightsMatch> matches = monitor.match(buildDoc("term1 foo"), HighlightsMatch.MATCHER);
HighlightsMatch m = matches.matches("1");
assertNotNull(m);
assertEquals(2, m.getHitCount());
}
}
public void testDisjunctionWithOrderedNearSpans() throws Exception {
final Query bq = new BooleanQuery.Builder()
.add(new TermQuery(new Term(FIELD, "a")), BooleanClause.Occur.SHOULD)
.add(SpanNearQuery.newOrderedNearQuery(FIELD)
.addClause(new SpanTermQuery(new Term(FIELD, "b")))
.addClause(new SpanTermQuery(new Term(FIELD, "c")))
.setSlop(1)
.build(), BooleanClause.Occur.SHOULD)
.build();
final Query parent = new BooleanQuery.Builder()
.add(new TermQuery(new Term(FIELD, "a")), BooleanClause.Occur.MUST)
.add(bq, BooleanClause.Occur.MUST)
.build();
try (Monitor monitor = new Monitor(ANALYZER)) {
monitor.register(new MonitorQuery("1", parent));
Document doc = buildDoc("a b x x x x c");
MatchingQueries<HighlightsMatch> matches = monitor.match(doc, HighlightsMatch.MATCHER);
HighlightsMatch m = matches.matches("1");
assertNotNull(m);
assertEquals(1, m.getHitCount());
}
}
public void testDisjunctionWithUnorderedNearSpans() throws Exception {
final Query bq = new BooleanQuery.Builder()
.add(new TermQuery(new Term(FIELD, "a")), BooleanClause.Occur.SHOULD)
.add(SpanNearQuery.newUnorderedNearQuery(FIELD)
.addClause(new SpanTermQuery(new Term(FIELD, "b")))
.addClause(new SpanTermQuery(new Term(FIELD, "c")))
.setSlop(1)
.build(), BooleanClause.Occur.SHOULD)
.build();
final Query parent = new BooleanQuery.Builder()
.add(new TermQuery(new Term(FIELD, "a")), BooleanClause.Occur.MUST)
.add(bq, BooleanClause.Occur.MUST)
.build();
try (Monitor monitor = new Monitor(ANALYZER)) {
monitor.register(new MonitorQuery("1", parent));
Document doc = buildDoc("a b x x x x c");
MatchingQueries<HighlightsMatch> matches = monitor.match(doc, HighlightsMatch.MATCHER);
HighlightsMatch m = matches.matches("1");
assertNotNull(m);
assertEquals(1, m.getHitCount());
}
}
public void testEquality() {
HighlightsMatch m1 = new HighlightsMatch("1");
m1.addHit("field", 0, 1, 0, 1);
HighlightsMatch m2 = new HighlightsMatch("1");
m2.addHit("field", 0, 1, 0, 1);
HighlightsMatch m3 = new HighlightsMatch("1");
m3.addHit("field", 0, 2, 0, 1);
HighlightsMatch m4 = new HighlightsMatch("2");
m4.addHit("field", 0, 1, 0, 1);
assertEquals(m1, m2);
assertEquals(m1.hashCode(), m2.hashCode());
assertNotEquals(m1, m3);
assertNotEquals(m1, m4);
}
public void testMutliValuedFieldWithNonDefaultGaps() throws IOException {
Analyzer analyzer = new Analyzer() {
@Override
public int getPositionIncrementGap(String fieldName) {
return 1000;
}
@Override
public int getOffsetGap(String fieldName) {
return 2000;
}
@Override
protected TokenStreamComponents createComponents(String fieldName) {
return new TokenStreamComponents(new WhitespaceTokenizer());
}
};
MonitorQuery mq = new MonitorQuery("query", parse(FIELD + ":\"hello world\"~5"));
try (Monitor monitor = newMonitor(analyzer)) {
monitor.register(mq);
Document doc1 = new Document();
doc1.add(newTextField(FIELD, "hello world", Field.Store.NO));
doc1.add(newTextField(FIELD, "goodbye", Field.Store.NO));
MatchingQueries<HighlightsMatch> matcher1 = monitor.match(doc1, HighlightsMatch.MATCHER);
assertEquals(1, matcher1.getMatchCount());
HighlightsMatch m1 = matcher1.matches("query");
assertNotNull(m1);
assertTrue(m1.getFields().contains(FIELD));
assertTrue(m1.getHits(FIELD).contains(new HighlightsMatch.Hit(0, 0, 1, 11)));
Document doc2 = new Document();
doc1.add(newTextField(FIELD, "hello", Field.Store.NO));
doc1.add(newTextField(FIELD, "world", Field.Store.NO));
MatchingQueries<HighlightsMatch> matcher2 = monitor.match(doc2, HighlightsMatch.MATCHER);
assertNull(matcher2.matches("query"));
assertEquals(0, matcher2.getMatchCount());
Document doc3 = new Document();
doc3.add(newTextField(FIELD, "hello world", Field.Store.NO));
doc3.add(newTextField(FIELD, "hello goodbye world", Field.Store.NO));
MatchingQueries<HighlightsMatch> matcher3 = monitor.match(doc3, HighlightsMatch.MATCHER);
assertEquals(1, matcher3.getMatchCount());
HighlightsMatch m3 = matcher3.matches("query");
assertNotNull(m3);
assertTrue(m3.getFields().contains(FIELD));
assertTrue(m3.getHits(FIELD).contains(new HighlightsMatch.Hit(0, 0, 1, 11)));
assertTrue(m3.getHits(FIELD).contains(new HighlightsMatch.Hit(1002, 2011, 1004, 2030)));
}
}
public void testDisjunctionWithOrderedNearMatch() throws Exception {
final Query bq = new BooleanQuery.Builder()
.add(new TermQuery(new Term(FIELD, "a")), BooleanClause.Occur.SHOULD)
.add(SpanNearQuery.newOrderedNearQuery(FIELD)
.addClause(new SpanTermQuery(new Term(FIELD, "b")))
.addClause(new SpanTermQuery(new Term(FIELD, "c")))
.setSlop(1)
.build(), BooleanClause.Occur.SHOULD)
.build();
final Query parent = new BooleanQuery.Builder()
.add(new TermQuery(new Term(FIELD, "a")), BooleanClause.Occur.MUST)
.add(bq, BooleanClause.Occur.MUST)
.build();
try (Monitor monitor = newMonitor()) {
monitor.register(new MonitorQuery("1", parent));
Document doc = buildDoc("a b c");
MatchingQueries<HighlightsMatch> matches = monitor.match(doc, HighlightsMatch.MATCHER);
HighlightsMatch m = matches.matches("1");
assertNotNull(m);
assertEquals(3, m.getHitCount());
assertTrue(m.getFields().contains(FIELD));
assertTrue(m.getHits(FIELD).contains(new HighlightsMatch.Hit(0, 0, 0, 1)));
assertTrue(m.getHits(FIELD).contains(new HighlightsMatch.Hit(1, 2, 1, 3)));
assertTrue(m.getHits(FIELD).contains(new HighlightsMatch.Hit(2, 4, 2, 5)));
}
}
public void testUnorderedNearWithinOrderedNear() throws Exception {
final SpanQuery spanPhrase = SpanNearQuery.newOrderedNearQuery(FIELD)
.addClause(new SpanTermQuery(new Term(FIELD, "time")))
.addClause(new SpanTermQuery(new Term(FIELD, "men")))
.setSlop(1)
.build();
final SpanQuery unorderedNear = SpanNearQuery.newUnorderedNearQuery(FIELD)
.addClause(spanPhrase)
.addClause(new SpanTermQuery(new Term(FIELD, "all")))
.setSlop(5)
.build();
final SpanQuery orderedNear = SpanNearQuery.newOrderedNearQuery(FIELD)
.addClause(new SpanTermQuery(new Term(FIELD, "the")))
.addClause(unorderedNear)
.setSlop(10)
.build();
final Query innerConjunct = new BooleanQuery.Builder()
.add(new TermQuery(new Term(FIELD, "is")), BooleanClause.Occur.MUST)
.add(orderedNear, BooleanClause.Occur.MUST)
.build();
final Query disjunct = new BooleanQuery.Builder()
.add(new TermQuery(new Term(FIELD, "now")), BooleanClause.Occur.SHOULD)
.add(innerConjunct, BooleanClause.Occur.SHOULD)
.build();
final Query outerConjunct = new BooleanQuery.Builder()
.add(disjunct, BooleanClause.Occur.MUST)
.add(new TermQuery(new Term(FIELD, "good")), BooleanClause.Occur.MUST)
.build();
try (Monitor monitor = newMonitor()) {
monitor.register(new MonitorQuery("1", outerConjunct));
Document doc = buildDoc("now is the time for all good men");
MatchingQueries<HighlightsMatch> matches = monitor.match(doc, HighlightsMatch.MATCHER);
HighlightsMatch m = matches.matches("1");
assertEquals(2, m.getHitCount());
assertTrue(m.getFields().contains(FIELD));
assertTrue(m.getHits(FIELD).contains(new HighlightsMatch.Hit(0, 0, 0, 3)));
assertTrue(m.getHits(FIELD).contains(new HighlightsMatch.Hit(6, 24, 6, 28)));
}
}
public void testMinShouldMatchQuery() throws Exception {
final Query minq = new BooleanQuery.Builder()
.add(new TermQuery(new Term(FIELD, "x")), BooleanClause.Occur.SHOULD)
.add(new TermQuery(new Term(FIELD, "y")), BooleanClause.Occur.SHOULD)
.add(new TermQuery(new Term(FIELD, "z")), BooleanClause.Occur.SHOULD)
.setMinimumNumberShouldMatch(2)
.build();
final Query bq = new BooleanQuery.Builder()
.add(new TermQuery(new Term(FIELD, "a")), BooleanClause.Occur.MUST)
.add(new TermQuery(new Term(FIELD, "b")), BooleanClause.Occur.MUST)
.add(minq, BooleanClause.Occur.SHOULD)
.build();
try (Monitor monitor = newMonitor()) {
monitor.register(new MonitorQuery("1", bq));
Document doc = buildDoc("a b x");
MatchingQueries<HighlightsMatch> matches = monitor.match(doc, HighlightsMatch.MATCHER);
HighlightsMatch m = matches.matches("1");
assertNotNull(m);
assertEquals(2, m.getHitCount());
assertTrue(m.getFields().contains(FIELD));
assertTrue(m.getHits(FIELD).contains(new HighlightsMatch.Hit(0, 0, 0, 1)));
assertTrue(m.getHits(FIELD).contains(new HighlightsMatch.Hit(1, 2, 1, 3)));
}
}
public void testComplexPhraseQueryParser() throws Exception {
ComplexPhraseQueryParser cpqp = new ComplexPhraseQueryParser(FIELD, new StandardAnalyzer());
Query query = cpqp.parse("\"x b\"");
try (Monitor monitor = newMonitor()) {
monitor.register(new MonitorQuery("1", query));
Document doc = buildDoc("x b c");
MatchingQueries<HighlightsMatch> matches = monitor.match(doc, HighlightsMatch.MATCHER);
HighlightsMatch m = matches.matches("1");
assertNotNull(m);
assertEquals(2, m.getHitCount());
assertTrue(m.getFields().contains(FIELD));
}
}
public void testHighlightBatches() throws Exception {
String query = "\"cell biology\"";
try (Monitor monitor = newMonitor(WHITESPACE)) {
monitor.register(new MonitorQuery("query0", parse("non matching query")));
monitor.register(new MonitorQuery("query1", parse(query)));
monitor.register(new MonitorQuery("query2", parse("biology")));
Document doc1 = new Document();
doc1.add(newTextField(FIELD, "the cell biology count", Field.Store.NO)); // matches
Document doc2 = new Document();
doc2.add(newTextField(FIELD, "nope", Field.Store.NO));
Document doc3 = new Document();
doc3.add(newTextField(FIELD, "biology text", Field.Store.NO));
MultiMatchingQueries<HighlightsMatch> matches = monitor.match(new Document[]{doc1, doc2, doc3}, HighlightsMatch.MATCHER);
assertEquals(2, matches.getMatchCount(0));
assertEquals(0, matches.getMatchCount(1));
assertEquals(1, matches.getMatchCount(2));
HighlightsMatch m1 = matches.matches("query1", 0);
assertTrue(m1.getHits(FIELD).contains(new HighlightsMatch.Hit(1, 4, 2, 16)));
HighlightsMatch m2 = matches.matches("query2", 2);
assertTrue(m2.getHits(FIELD).contains(new HighlightsMatch.Hit(0, 0, 0, 7)));
}
}
}

View File

@ -0,0 +1,27 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
public class TestMatchAllPresearcher extends PresearcherTestBase {
@Override
protected Presearcher createPresearcher() {
return Presearcher.NO_FILTERING;
}
}

View File

@ -0,0 +1,245 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
import org.apache.lucene.analysis.core.WhitespaceTokenizer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.TermQuery;
public class TestMonitor extends MonitorTestBase {
private static final Analyzer ANALYZER = new WhitespaceAnalyzer();
public void testSingleTermQueryMatchesSingleDocument() throws IOException {
Document doc = new Document();
doc.add(newTextField(FIELD, "This is a test document", Field.Store.NO));
try (Monitor monitor = newMonitor()) {
monitor.register(new MonitorQuery("query1", new TermQuery(new Term(FIELD, "test"))));
MatchingQueries<QueryMatch> matches = monitor.match(doc, QueryMatch.SIMPLE_MATCHER);
assertNotNull(matches.getMatches());
assertEquals(1, matches.getMatchCount());
assertNotNull(matches.matches("query1"));
}
}
public void testMatchStatisticsAreReported() throws IOException {
Document doc = new Document();
doc.add(newTextField(FIELD, "This is a test document", Field.Store.NO));
try (Monitor monitor = newMonitor()) {
monitor.register(new MonitorQuery("query1", new TermQuery(new Term(MonitorTestBase.FIELD, "test"))));
MatchingQueries<QueryMatch> matches = monitor.match(doc, QueryMatch.SIMPLE_MATCHER);
assertEquals(1, matches.getQueriesRun());
assertTrue(matches.getQueryBuildTime() > -1);
assertTrue(matches.getSearchTime() > -1);
}
}
public void testUpdatesOverwriteOldQueries() throws IOException {
Document doc = new Document();
doc.add(newTextField(FIELD, "that", Field.Store.NO));
try (Monitor monitor = newMonitor()) {
monitor.register(new MonitorQuery("query1", new TermQuery(new Term(MonitorTestBase.FIELD, "this"))));
monitor.register(new MonitorQuery("query1", new TermQuery(new Term(MonitorTestBase.FIELD, "that"))));
MatchingQueries<QueryMatch> matches = monitor.match(doc, QueryMatch.SIMPLE_MATCHER);
assertNotNull(matches.matches("query1"));
assertEquals(1, matches.getQueriesRun());
}
}
public void testCanDeleteById() throws IOException {
Document doc = new Document();
doc.add(newTextField(FIELD, "other things", Field.Store.NO));
try (Monitor monitor = newMonitor()) {
monitor.register(new MonitorQuery("query1", new TermQuery(new Term(MonitorTestBase.FIELD, "this"))));
monitor.register(
new MonitorQuery("query2", new TermQuery(new Term(MonitorTestBase.FIELD, "that"))),
new MonitorQuery("query3", new TermQuery(new Term(MonitorTestBase.FIELD, "other"))));
assertEquals(3, monitor.getQueryCount());
monitor.deleteById("query2", "query1");
assertEquals(1, monitor.getQueryCount());
MatchingQueries<QueryMatch> matches = monitor.match(doc, QueryMatch.SIMPLE_MATCHER);
assertEquals(1, matches.getQueriesRun());
assertNotNull(matches.matches("query3"));
}
}
public void testCanClearTheMonitor() throws IOException {
try (Monitor monitor = newMonitor()) {
monitor.register(
new MonitorQuery("query1", new MatchAllDocsQuery()),
new MonitorQuery("query2", new MatchAllDocsQuery()),
new MonitorQuery("query3", new MatchAllDocsQuery()));
assertEquals(3, monitor.getQueryCount());
monitor.clear();
assertEquals(0, monitor.getQueryCount());
}
}
public void testMatchesAgainstAnEmptyMonitor() throws IOException {
try (Monitor monitor = newMonitor()) {
assertEquals(0, monitor.getQueryCount());
Document doc = new Document();
doc.add(newTextField(FIELD, "This is a test document", Field.Store.NO));
MatchingQueries<QueryMatch> matches = monitor.match(doc, QueryMatch.SIMPLE_MATCHER);
assertEquals(0, matches.getQueriesRun());
}
}
public void testUpdateReporting() throws IOException {
List<MonitorQuery> queries = new ArrayList<>(10400);
for (int i = 0; i < 10355; i++) {
queries.add(new MonitorQuery(Integer.toString(i), MonitorTestBase.parse("test")));
}
final int[] expectedSizes = new int[]{5001, 5001, 353};
final AtomicInteger callCount = new AtomicInteger();
final AtomicInteger updateCount = new AtomicInteger();
MonitorUpdateListener listener = new MonitorUpdateListener() {
@Override
public void afterUpdate(List<MonitorQuery> updates) {
int calls = callCount.getAndIncrement();
updateCount.addAndGet(updates.size());
assertEquals(expectedSizes[calls], updates.size());
}
};
try (Monitor monitor = new Monitor(ANALYZER)) {
monitor.addQueryIndexUpdateListener(listener);
monitor.register(queries);
assertEquals(10355, updateCount.get());
}
}
public void testMatcherMetadata() throws IOException {
try (Monitor monitor = newMonitor()) {
HashMap<String, String> metadataMap = new HashMap<>();
metadataMap.put("key", "value");
monitor.register(new MonitorQuery(Integer.toString(1), MonitorTestBase.parse("+test " + 1), null, metadataMap));
Document doc = new Document();
doc.add(newTextField(FIELD, "This is a test document", Field.Store.NO));
MatcherFactory<QueryMatch> testMatcherFactory = docs -> new CandidateMatcher<QueryMatch>(docs) {
@Override
protected void matchQuery(String queryId, Query matchQuery, Map<String, String> metadata) {
assertEquals("value", metadata.get("key"));
}
@Override
public QueryMatch resolve(QueryMatch match1, QueryMatch match2) {
return null;
}
};
monitor.match(doc, testMatcherFactory);
}
}
public void testDocumentBatching() throws IOException {
Document doc1 = new Document();
doc1.add(newTextField(FIELD, "This is a test document", Field.Store.NO));
Document doc2 = new Document();
doc2.add(newTextField(FIELD, "This is a kangaroo document", Field.Store.NO));
try (Monitor monitor = new Monitor(ANALYZER)) {
monitor.register(new MonitorQuery("1", new TermQuery(new Term(MonitorTestBase.FIELD, "kangaroo"))));
MultiMatchingQueries<QueryMatch> response = monitor.match(new Document[]{ doc1, doc2 }, QueryMatch.SIMPLE_MATCHER);
assertEquals(2, response.getBatchSize());
}
}
public void testMutliValuedFieldWithNonDefaultGaps() throws IOException {
Analyzer analyzer = new Analyzer() {
@Override
public int getPositionIncrementGap(String fieldName) {
return 1000;
}
@Override
public int getOffsetGap(String fieldName) {
return 2000;
}
@Override
protected TokenStreamComponents createComponents(String fieldName) {
return new TokenStreamComponents(new WhitespaceTokenizer());
}
};
MonitorQuery mq = new MonitorQuery("query", MonitorTestBase.parse(MonitorTestBase.FIELD + ":\"hello world\"~5"));
try (Monitor monitor = new Monitor(analyzer)) {
monitor.register(mq);
Document doc1 = new Document();
doc1.add(newTextField(FIELD, "hello world", Field.Store.NO));
doc1.add(newTextField(FIELD, "goodbye", Field.Store.NO));
MatchingQueries<QueryMatch> matches = monitor.match(doc1, QueryMatch.SIMPLE_MATCHER);
assertNotNull(matches.getMatches());
assertEquals(1, matches.getMatchCount());
assertNotNull(matches.matches("query"));
Document doc2 = new Document();
doc2.add(newTextField(FIELD, "hello", Field.Store.NO));
doc2.add(newTextField(FIELD, "world", Field.Store.NO));
matches = monitor.match(doc2, QueryMatch.SIMPLE_MATCHER);
assertEquals(0, matches.getMatchCount());
}
}
}

View File

@ -0,0 +1,57 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.util.HashMap;
import java.util.Map;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.search.MatchAllDocsQuery;
public class TestMonitorErrorHandling extends MonitorTestBase {
public void testMonitorErrors() throws Exception {
try (Monitor monitor = newMonitor()) {
monitor.register(
MonitorTestBase.mq("1", "test"),
new MonitorQuery("2", MonitorTestBase.parse("test")),
new MonitorQuery("3", new ThrowOnRewriteQuery()));
Document doc = new Document();
doc.add(newTextField(FIELD, "test", Field.Store.NO));
MatchingQueries<QueryMatch> matches = monitor.match(doc, QueryMatch.SIMPLE_MATCHER);
assertEquals(1, matches.getErrors().size());
assertEquals("Error rewriting", matches.getErrors().get("3").getMessage());
assertEquals(2, matches.getMatchCount());
assertEquals(3, matches.getQueriesRun());
}
}
public void testMonitorQueryNullValues() {
IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> {
Map<String, String> metadata2 = new HashMap<>();
metadata2.put("key", null);
new MonitorQuery("id", new MatchAllDocsQuery(), null, metadata2);
});
assertEquals("Null value for key key in metadata map", e.getMessage());
}
}

View File

@ -0,0 +1,72 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.IOException;
import java.nio.file.Path;
import java.util.Collections;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.search.MatchAllDocsQuery;
public class TestMonitorPersistence extends MonitorTestBase {
private Path indexDirectory = createTempDir();
public void testCacheIsRepopulated() throws IOException {
Document doc = new Document();
doc.add(newTextField(FIELD, "test", Field.Store.NO));
MonitorConfiguration config = new MonitorConfiguration()
.setIndexPath(indexDirectory, MonitorQuerySerializer.fromParser(MonitorTestBase::parse));
try (Monitor monitor = new Monitor(ANALYZER, config)) {
monitor.register(
mq("1", "test"),
mq("2", "test"),
mq("3", "test", "language", "en"),
mq("4", "test", "wibble", "quack"));
assertEquals(4, monitor.match(doc, QueryMatch.SIMPLE_MATCHER).getMatchCount());
IllegalArgumentException e = expectThrows(IllegalArgumentException.class,
() -> monitor.register(new MonitorQuery("5", new MatchAllDocsQuery(), null, Collections.emptyMap())));
assertEquals("Cannot add a MonitorQuery with a null string representation to a non-ephemeral Monitor", e.getMessage());
}
try (Monitor monitor2 = new Monitor(ANALYZER, config)) {
assertEquals(4, monitor2.getQueryCount());
assertEquals(4, monitor2.match(doc, QueryMatch.SIMPLE_MATCHER).getMatchCount());
MonitorQuery mq = monitor2.getQuery("4");
assertEquals("quack", mq.getMetadata().get("wibble"));
}
}
public void testEphemeralMonitorDoesNotStoreQueries() throws IOException {
try (Monitor monitor2 = new Monitor(ANALYZER)) {
IllegalStateException e = expectThrows(IllegalStateException.class, () -> monitor2.getQuery("query"));
assertEquals("Cannot get queries from an index with no MonitorQuerySerializer", e.getMessage());
}
}
}

View File

@ -0,0 +1,123 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.IOException;
import org.apache.lucene.analysis.core.KeywordAnalyzer;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.memory.MemoryIndex;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.TermInSetQuery;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.store.ByteBuffersDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
public class TestMultipassPresearcher extends PresearcherTestBase {
@Override
protected Presearcher createPresearcher() {
return new MultipassTermFilteredPresearcher(4);
}
public void testSimpleBoolean() throws IOException {
try (Monitor monitor = newMonitor()) {
monitor.register(
new MonitorQuery("1", parse("field:\"hello world\"")),
new MonitorQuery("2", parse("field:world")),
new MonitorQuery("3", parse("field:\"hello there world\"")),
new MonitorQuery("4", parse("field:\"this and that\"")));
MatchingQueries<QueryMatch> matches = monitor.match(buildDoc("field", "hello world and goodbye"),
QueryMatch.SIMPLE_MATCHER);
assertEquals(2, matches.getQueriesRun());
assertNotNull(matches.matches("1"));
}
}
public void testComplexBoolean() throws IOException {
try (Monitor monitor = newMonitor()) {
monitor.register(new MonitorQuery("1", parse("field:(+foo +bar +(badger cormorant))")));
MatchingQueries<QueryMatch> matches
= monitor.match(buildDoc("field", "a badger walked into a bar"), QueryMatch.SIMPLE_MATCHER);
assertEquals(0, matches.getMatchCount());
assertEquals(0, matches.getQueriesRun());
matches = monitor.match(buildDoc("field", "foo badger cormorant"), QueryMatch.SIMPLE_MATCHER);
assertEquals(0, matches.getMatchCount());
assertEquals(0, matches.getQueriesRun());
matches = monitor.match(buildDoc("field", "bar badger foo"), QueryMatch.SIMPLE_MATCHER);
assertEquals(1, matches.getMatchCount());
}
}
public void testQueryBuilder() throws IOException {
IndexWriterConfig iwc = new IndexWriterConfig(new KeywordAnalyzer());
Presearcher presearcher = createPresearcher();
Directory dir = new ByteBuffersDirectory();
IndexWriter writer = new IndexWriter(dir, iwc);
MonitorConfiguration config = new MonitorConfiguration(){
@Override
public IndexWriter buildIndexWriter() {
return writer;
}
};
try (Monitor monitor = new Monitor(ANALYZER, presearcher, config)) {
monitor.register(new MonitorQuery("1", parse("f:test")));
try (IndexReader reader = DirectoryReader.open(writer, false, false)) {
MemoryIndex mindex = new MemoryIndex();
mindex.addField("f", "this is a test document", WHITESPACE);
LeafReader docsReader = (LeafReader) mindex.createSearcher().getIndexReader();
QueryIndex.QueryTermFilter termFilter = new QueryIndex.QueryTermFilter(reader);
BooleanQuery q = (BooleanQuery) presearcher.buildQuery(docsReader, termFilter);
BooleanQuery expected = new BooleanQuery.Builder()
.add(should(new BooleanQuery.Builder()
.add(must(new BooleanQuery.Builder().add(should(new TermInSetQuery("f_0", new BytesRef("test")))).build()))
.add(must(new BooleanQuery.Builder().add(should(new TermInSetQuery("f_1", new BytesRef("test")))).build()))
.add(must(new BooleanQuery.Builder().add(should(new TermInSetQuery("f_2", new BytesRef("test")))).build()))
.add(must(new BooleanQuery.Builder().add(should(new TermInSetQuery("f_3", new BytesRef("test")))).build()))
.build()))
.add(should(new TermQuery(new Term("__anytokenfield", "__ANYTOKEN__"))))
.build();
assertEquals(expected, q);
}
}
}
}

View File

@ -0,0 +1,28 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.util.concurrent.ExecutorService;
public class TestParallelMatcher extends ConcurrentMatcherTestBase {
@Override
protected <T extends QueryMatch> MatcherFactory<T> matcherFactory(ExecutorService executor, MatcherFactory<T> factory, int threads) {
return ParallelMatcher.factory(executor, factory, threads);
}
}

View File

@ -0,0 +1,60 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.ExecutorService;
public class TestPartitionMatcher extends ConcurrentMatcherTestBase {
@Override
protected <T extends QueryMatch> MatcherFactory<T> matcherFactory(ExecutorService executor, MatcherFactory<T> factory, int threads) {
return PartitionMatcher.factory(executor, factory, threads);
}
public void testPartitions() {
List<String> terms = Arrays.asList("1", "2", "3", "4", "5", "6", "7", "8", "9", "10");
List<List<String>> partitions = PartitionMatcher.partition(terms, 2);
assertTrue(partitions.contains(Arrays.asList("1", "2", "3", "4", "5")));
assertTrue(partitions.contains(Arrays.asList("6", "7", "8", "9", "10")));
partitions = PartitionMatcher.partition(terms, 3);
assertTrue(partitions.contains(Arrays.asList("1", "2", "3")));
assertTrue(partitions.contains(Arrays.asList("4", "5", "6")));
assertTrue(partitions.contains(Arrays.asList("7", "8", "9", "10")));
partitions = PartitionMatcher.partition(terms, 4);
assertTrue(partitions.contains(Arrays.asList("1", "2")));
assertTrue(partitions.contains(Arrays.asList("3", "4", "5")));
assertTrue(partitions.contains(Arrays.asList("6", "7")));
assertTrue(partitions.contains(Arrays.asList("8", "9", "10")));
partitions = PartitionMatcher.partition(terms, 6);
assertTrue(partitions.contains(Collections.singletonList("1")));
assertTrue(partitions.contains(Arrays.asList("2", "3")));
assertTrue(partitions.contains(Arrays.asList("4", "5")));
assertTrue(partitions.contains(Collections.singletonList("6")));
assertTrue(partitions.contains(Arrays.asList("7", "8")));
assertTrue(partitions.contains(Arrays.asList("9", "10")));
}
}

View File

@ -0,0 +1,61 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.IOException;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import static org.hamcrest.CoreMatchers.containsString;
public class TestPresearcherMatchCollector extends MonitorTestBase {
public void testMatchCollectorShowMatches() throws IOException {
try (Monitor monitor = new Monitor(ANALYZER, new TermFilteredPresearcher())) {
monitor.register(new MonitorQuery("1", parse("test")));
monitor.register(new MonitorQuery("2", parse("foo bar -baz f2:quuz")));
monitor.register(new MonitorQuery("3", parse("foo -test")));
monitor.register(new MonitorQuery("4", parse("baz")));
assertEquals(4, monitor.getQueryCount());
Document doc = new Document();
doc.add(newTextField(FIELD, "this is a foo test", Field.Store.NO));
doc.add(newTextField("f2", "quuz", Field.Store.NO));
PresearcherMatches<QueryMatch> matches = monitor.debug(doc, QueryMatch.SIMPLE_MATCHER);
assertNotNull(matches.match("1", 0));
assertEquals(" field:test", matches.match("1", 0).presearcherMatches);
assertNotNull(matches.match("1", 0).queryMatch);
assertNotNull(matches.match("2", 0));
String pm = matches.match("2", 0).presearcherMatches;
assertThat(pm, containsString("field:foo"));
assertThat(pm, containsString("f2:quuz"));
assertNotNull(matches.match("3", 0));
assertEquals(" field:foo", matches.match("3", 0).presearcherMatches);
assertNull(matches.match("3", 0).queryMatch);
assertNull(matches.match("4", 0));
}
}
}

View File

@ -0,0 +1,194 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.Set;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
public class TestQueryAnalyzer extends LuceneTestCase {
public static final QueryAnalyzer analyzer = new QueryAnalyzer();
private Set<Term> collectTerms(QueryTree tree) {
Set<Term> terms = new HashSet<>();
tree.collectTerms((f, b) -> terms.add(new Term(f, b)));
return terms;
}
private Set<Term> collectTerms(Query query) {
return collectTerms(analyzer.buildTree(query, TermWeightor.DEFAULT));
}
public void testAdvancesCollectDifferentTerms() {
Query q = MonitorTestBase.parse("field:(+hello +goodbye)");
QueryTree querytree = analyzer.buildTree(q, TermWeightor.DEFAULT);
Set<Term> expected = Collections.singleton(new Term("field", "goodbye"));
assertEquals(expected, collectTerms(querytree));
assertTrue(querytree.advancePhase(0));
expected = Collections.singleton(new Term("field", "hello"));
assertEquals(expected, collectTerms(querytree));
assertFalse(querytree.advancePhase(0));
assertEquals(expected, collectTerms(querytree));
}
public void testDisjunctionsWithAnyClausesOnlyReturnANYTOKEN() {
// disjunction containing a pure negative - we can't narrow this down
Query q = MonitorTestBase.parse("hello goodbye (*:* -term)");
Set<Term> terms = collectTerms(q);
assertEquals(1, terms.size());
assertEquals(TermFilteredPresearcher.ANYTOKEN_FIELD, terms.iterator().next().field());
}
public void testConjunctionsDoNotAdvanceOverANYTOKENs() {
Query q = MonitorTestBase.parse("+hello +howdyedo +(goodbye (*:* -whatever))");
QueryTree tree = analyzer.buildTree(q, TermWeightor.DEFAULT);
Set<Term> expected = Collections.singleton(new Term("field", "howdyedo"));
assertEquals(expected, collectTerms(tree));
assertTrue(tree.advancePhase(0));
expected = Collections.singleton(new Term("field", "hello"));
assertEquals(expected, collectTerms(tree));
assertFalse(tree.advancePhase(0));
assertEquals(expected, collectTerms(tree));
}
public void testConjunctionsCannotAdvanceOverMinWeightedTokens() {
TermWeightor weightor = TermWeightor.combine(
TermWeightor.termWeightor(0.1, new BytesRef("startterm")),
TermWeightor.lengthWeightor(1, 1));
QueryAnalyzer analyzer = new QueryAnalyzer();
Query q = MonitorTestBase.parse("+startterm +hello +goodbye");
QueryTree tree = analyzer.buildTree(q, weightor);
Set<Term> expected = Collections.singleton(new Term("field", "goodbye"));
assertEquals(expected, collectTerms(tree));
assertTrue(tree.advancePhase(0.5));
expected = Collections.singleton(new Term("field", "hello"));
assertEquals(expected, collectTerms(tree));
assertFalse(tree.advancePhase(0.5));
}
public void testNestedConjunctions() {
Query q = MonitorTestBase.parse("+(+(+(+aaaa +cc) +(+d +bbb)))");
QueryTree tree = analyzer.buildTree(q, TermWeightor.DEFAULT);
Set<Term> expected = Collections.singleton(new Term("field", "aaaa"));
assertEquals(expected, collectTerms(tree));
assertTrue(tree.advancePhase(0));
expected = Collections.singleton(new Term("field", "bbb"));
assertEquals(expected, collectTerms(tree));
assertTrue(tree.advancePhase(0));
expected = Collections.singleton(new Term("field", "cc"));
assertEquals(expected, collectTerms(tree));
assertTrue(tree.advancePhase(0));
expected = Collections.singleton(new Term("field", "d"));
assertEquals(expected, collectTerms(tree));
assertFalse(tree.advancePhase(0));
}
public void testNestedDisjunctions() {
Query q = MonitorTestBase.parse("+(+((+aaaa +cc) (+dd +bbb +f)))");
QueryTree tree = analyzer.buildTree(q, TermWeightor.DEFAULT);
Set<Term> expected = new HashSet<>(Arrays.asList(
new Term("field", "aaaa"),
new Term("field", "bbb"
)));
assertEquals(expected, collectTerms(tree));
assertTrue(tree.advancePhase(0));
expected = new HashSet<>(Arrays.asList(
new Term("field", "cc"),
new Term("field", "dd")
));
assertEquals(expected, collectTerms(tree));
assertTrue(tree.advancePhase(0));
expected = new HashSet<>(Arrays.asList(
new Term("field", "cc"),
new Term("field", "f")
));
assertEquals(expected, collectTerms(tree));
assertFalse(tree.advancePhase(0));
}
public void testMinWeightAdvances() {
QueryTree tree = QueryTree.disjunction(
QueryTree.conjunction(
QueryTree.term(new Term("field", "term1"), 1),
QueryTree.term(new Term("field", "term2"), 0.1),
QueryTree.anyTerm("*:*")
),
QueryTree.conjunction(
QueryTree.disjunction(
QueryTree.term(new Term("field", "term4"), 0.2),
QueryTree.term(new Term("field", "term5"), 1)
),
QueryTree.term(new Term("field", "term3"), 0.5)
)
);
Set<Term> expected = new HashSet<>(Arrays.asList(
new Term("field", "term1"),
new Term("field", "term3")
));
assertEquals(expected, collectTerms(tree));
assertTrue(tree.advancePhase(0.1f));
expected = new HashSet<>(Arrays.asList(
new Term("field", "term1"),
new Term("field", "term4"),
new Term("field", "term5")
));
assertEquals(expected, collectTerms(tree));
assertFalse(tree.advancePhase(0.1f));
}
}

View File

@ -0,0 +1,110 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.Set;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.DisjunctionMaxQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.TermQuery;
public class TestQueryDecomposer extends MonitorTestBase {
private static final QueryDecomposer decomposer = new QueryDecomposer();
public void testConjunctionsAreNotDecomposed() {
Query q = parse("+hello world");
Set<Query> expected = Collections.singleton(parse("+hello world"));
assertEquals(expected, decomposer.decompose(q));
}
public void testSimpleDisjunctions() {
Query q = parse("hello world");
Set<Query> expected = new HashSet<>(Arrays.asList(parse("hello"), parse("world")));
assertEquals(expected, decomposer.decompose(q));
}
public void testNestedDisjunctions() {
Query q = parse("(hello goodbye) world");
Set<Query> expected = new HashSet<>(Arrays.asList(parse("hello"), parse("goodbye"), parse("world")));
assertEquals(expected, decomposer.decompose(q));
}
public void testExclusions() {
Set<Query> expected = new HashSet<>(Arrays.asList(parse("+hello -goodbye"), parse("+world -goodbye")));
assertEquals(expected, decomposer.decompose(parse("hello world -goodbye")));
}
public void testNestedExclusions() {
Set<Query> expected
= new HashSet<>(Arrays.asList(parse("+(+hello -goodbye) -greeting"), parse("+(+world -goodbye) -greeting")));
assertEquals(expected, decomposer.decompose(parse("((hello world) -goodbye) -greeting")));
}
public void testSingleValuedConjunctions() {
Set<Query> expected = new HashSet<>(Arrays.asList(parse("hello"), parse("world")));
assertEquals(expected, decomposer.decompose(parse("+(hello world)")));
}
public void testSingleValuedConjunctWithExclusions() {
Set<Query> expected = new HashSet<>(Arrays.asList(parse("+hello -goodbye"), parse("+world -goodbye")));
assertEquals(expected, decomposer.decompose(parse("+(hello world) -goodbye")));
}
public void testBoostsArePreserved() {
Set<Query> expected = new HashSet<>(Arrays.asList(parse("hello^0.7"), parse("world^0.7")));
assertEquals(expected, decomposer.decompose(parse("+(hello world)^0.7")));
expected = new HashSet<>(Arrays.asList(parse("+hello^0.7 -goodbye"), parse("+world^0.7 -goodbye")));
assertEquals(expected, decomposer.decompose(parse("+(hello world)^0.7 -goodbye")));
expected = new HashSet<>(Arrays.asList(parse("(hello^0.5)^0.8"), parse("world^0.8")));
assertEquals(expected, decomposer.decompose(parse("+(hello^0.5 world)^0.8")));
}
public void testDisjunctionMaxDecomposition() {
Query q = new DisjunctionMaxQuery(
Arrays.asList(new TermQuery(new Term("f", "t1")), new TermQuery(new Term("f", "t2"))), 0.1f
);
Set<Query> expected = new HashSet<>(Arrays.asList(parse("f:t1"), parse("f:t2")));
assertEquals(expected, decomposer.decompose(q));
}
public void testNestedDisjunctionMaxDecomposition() {
Query q = new DisjunctionMaxQuery(
Arrays.asList(parse("hello goodbye"), parse("world")), 0.1f
);
Set<Query> expected = new HashSet<>(Arrays.asList(parse("hello"), parse("goodbye"), parse("world")));
assertEquals(expected, decomposer.decompose(q));
}
public void testFilterAndShouldClause() {
final Query shouldTermQuery = new TermQuery(new Term("f", "should"));
final Query filterTermQuery = new TermQuery(new Term("f", "filter"));
Query q = new BooleanQuery.Builder()
.add(shouldTermQuery, BooleanClause.Occur.SHOULD)
.add(filterTermQuery, BooleanClause.Occur.FILTER)
.build();
assertEquals(Collections.singleton(q), decomposer.decompose(q));
}
}

View File

@ -0,0 +1,103 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import org.apache.lucene.index.Term;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
public class TestQueryTermComparators extends LuceneTestCase {
public void testAnyTokensAreNotPreferred() {
QueryTree node1 = QueryTree.term("f", new BytesRef("foo"), 1.0);
QueryTree node2 = QueryTree.anyTerm("*:*");
QueryTree conjunction = QueryTree.conjunction(node1, node2);
Set<Term> terms = new HashSet<>();
conjunction.collectTerms((f, b) -> terms.add(new Term(f, b)));
Set<Term> expected = Collections.singleton(new Term("f", "foo"));
assertEquals(expected, terms);
}
public void testHigherWeightsArePreferred() {
QueryTree node1 = QueryTree.term(new Term("f", "foo"), 1);
QueryTree node2 = QueryTree.term(new Term("f", "foobar"), 1.5);
QueryTree conjunction = QueryTree.conjunction(node1, node2);
Set<Term> terms = new HashSet<>();
conjunction.collectTerms((f, b) -> terms.add(new Term(f, b)));
Set<Term> expected = Collections.singleton(new Term("f", "foobar"));
assertEquals(expected, terms);
}
public void testShorterTermListsArePreferred() {
Term term = new Term("f", "foobar");
QueryTree node1 = QueryTree.term(term, 1);
QueryTree node2 = QueryTree.disjunction(
QueryTree.term(term, 1),
QueryTree.term(term, 1));
QueryTree conjunction = QueryTree.conjunction(node1, node2);
Set<Term> terms = new HashSet<>();
conjunction.collectTerms((f, b) -> terms.add(new Term(f, b)));
assertEquals(1, terms.size());
}
public void testFieldWeights() {
TermWeightor weightor = TermWeightor.fieldWeightor(1.5, "g");
assertEquals(1, weightor.applyAsDouble(new Term("f", "foo")), 0);
assertEquals(1.5f, weightor.applyAsDouble(new Term("g", "foo")), 0);
}
public void testTermWeights() {
TermWeightor weight = TermWeightor.termWeightor(0.01f, new BytesRef("START"));
assertEquals(0.01f, weight.applyAsDouble(new Term("f", "START")), 0);
}
public void testTermFrequencyNorms() {
Map<String, Integer> termfreqs = new HashMap<>();
termfreqs.put("france", 31635);
termfreqs.put("s", 47088);
TermWeightor weight = TermWeightor.termFreqWeightor(termfreqs, 100, 0.8);
assertTrue(weight.applyAsDouble(new Term("f", "france")) >
weight.applyAsDouble(new Term("f", "s")));
}
public void testFieldSpecificTermWeightNorms() {
TermWeightor weight = TermWeightor.termAndFieldWeightor(0.1,
new Term("field1", "f"),
new Term("field1", "g"));
assertEquals(0.1, weight.applyAsDouble(new Term("field1", "f")), 0);
assertEquals(1, weight.applyAsDouble(new Term("field2", "f")), 0);
}
}

View File

@ -0,0 +1,52 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.IOException;
import java.util.Collections;
import java.util.function.BiPredicate;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
public class TestQueryTermFilter extends LuceneTestCase {
private static final String FIELD = "f";
public void testFiltersAreRemoved() throws IOException {
try (QueryIndex qi = new QueryIndex(new MonitorConfiguration(), new TermFilteredPresearcher())) {
qi.commit(Collections.singletonList(new MonitorQuery("1", new TermQuery(new Term(FIELD, "term")))));
assertEquals(1, qi.termFilters.size());
BiPredicate<String, BytesRef> filter = qi.termFilters.values().iterator().next();
assertTrue(filter.test(FIELD, new BytesRef("term")));
assertFalse(filter.test(FIELD, new BytesRef("term2")));
qi.commit(Collections.singletonList(new MonitorQuery("2", new TermQuery(new Term(FIELD, "term2")))));
assertEquals(1, qi.termFilters.size());
filter = qi.termFilters.values().iterator().next();
assertTrue(filter.test(FIELD, new BytesRef("term")));
assertTrue(filter.test(FIELD, new BytesRef("term2")));
assertFalse(filter.test(FIELD, new BytesRef("term3")));
}
}
}

View File

@ -0,0 +1,84 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.Set;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.RegexpQuery;
public class TestRegexpQueryHandler extends BaseTokenStreamTestCase {
public void testTermStreamWrapping() throws IOException {
CustomQueryHandler handler
= new RegexpQueryHandler("FOO", 10, "__wibble__", Collections.singleton("field1"));
try (Analyzer input = new WhitespaceAnalyzer()) {
// field1 is in the excluded set, so nothing should happen
assertTokenStreamContents(handler.wrapTermStream("field1", input.tokenStream("field1", "hello world")),
new String[]{ "hello", "world" });
// field2 is not excluded
assertTokenStreamContents(handler.wrapTermStream("field2", input.tokenStream("field2", "harm alarm asdasasdasdasd")),
new String[]{
"harm", "harmFOO", "harFOO", "haFOO", "hFOO", "armFOO", "arFOO", "aFOO", "rmFOO", "rFOO", "mFOO", "FOO",
"alarm", "alarmFOO", "alarFOO", "alaFOO", "alFOO", "larmFOO", "larFOO", "laFOO", "lFOO",
"asdasasdasdasd", "__wibble__"
});
}
}
private Set<Term> collectTerms(Query q) {
QueryAnalyzer builder = new QueryAnalyzer(Collections.singletonList(
new RegexpQueryHandler("XX", 30, "WILDCARD", null)));
QueryTree tree = builder.buildTree(q, TermWeightor.DEFAULT);
Set<Term> terms = new HashSet<>();
tree.collectTerms((f, b) -> terms.add(new Term(f, b)));
return terms;
}
public void testRegexpExtractor() {
Set<Term> expected = new HashSet<>(Arrays.asList(
new Term("field", "califragilisticXX"),
new Term("field", "WILDCARD")));
assertEquals(expected, collectTerms(new RegexpQuery(new Term("field", "super.*califragilistic"))));
expected = new HashSet<>(Arrays.asList(
new Term("field", "hellXX"),
new Term("field", "WILDCARD")));
assertEquals(expected, collectTerms(new RegexpQuery(new Term("field", "hell."))));
expected = new HashSet<>(Arrays.asList(
new Term("field", "heXX"),
new Term("field", "WILDCARD")));
assertEquals(expected, collectTerms(new RegexpQuery(new Term("field", "hel?o"))));
}
}

View File

@ -0,0 +1,50 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.search.similarities.ClassicSimilarity;
import org.apache.lucene.search.similarities.Similarity;
public class TestSimilarities extends MonitorTestBase {
public void testNonStandardSimilarity() throws Exception {
try (Monitor monitor = newMonitor()) {
monitor.register(new MonitorQuery("1", MonitorTestBase.parse("test")));
Similarity similarity = new ClassicSimilarity() {
@Override
public float tf(float freq) {
return 1000f;
}
};
Document doc = new Document();
doc.add(newTextField("field", "this is a test", Field.Store.NO));
MatchingQueries<ScoringMatch> standard = monitor.match(doc, ScoringMatch.matchWithSimilarity(new ClassicSimilarity()));
MatchingQueries<ScoringMatch> withSim = monitor.match(doc, ScoringMatch.matchWithSimilarity(similarity));
float standScore = standard.getMatches().iterator().next().getScore();
float simScore = withSim.getMatches().iterator().next().getScore();
assertEquals(standScore, simScore / 1000, 0.1f);
}
}
}

View File

@ -0,0 +1,40 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.IOException;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
public class TestSimpleMatcher extends MonitorTestBase {
public void testSimpleMatcher() throws IOException {
try (Monitor monitor = newMonitor()) {
monitor.register(
new MonitorQuery("1", parse("test")),
new MonitorQuery("2", parse("wibble")));
Document doc = new Document();
doc.add(newTextField(FIELD, "test", Field.Store.NO));
MatchingQueries<QueryMatch> matches = monitor.match(doc, QueryMatch.SIMPLE_MATCHER);
assertNotNull(matches.matches("1"));
}
}
}

View File

@ -0,0 +1,134 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.Set;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.RegexpQuery;
import org.apache.lucene.search.spans.FieldMaskingSpanQuery;
import org.apache.lucene.search.spans.SpanBoostQuery;
import org.apache.lucene.search.spans.SpanContainingQuery;
import org.apache.lucene.search.spans.SpanFirstQuery;
import org.apache.lucene.search.spans.SpanMultiTermQueryWrapper;
import org.apache.lucene.search.spans.SpanNearQuery;
import org.apache.lucene.search.spans.SpanOrQuery;
import org.apache.lucene.search.spans.SpanQuery;
import org.apache.lucene.search.spans.SpanTermQuery;
import org.apache.lucene.search.spans.SpanWithinQuery;
import org.apache.lucene.util.LuceneTestCase;
public class TestSpanExtractors extends LuceneTestCase {
private static final QueryAnalyzer treeBuilder = new QueryAnalyzer();
private Set<Term> collectTerms(Query q) {
QueryTree tree = treeBuilder.buildTree(q, TermWeightor.DEFAULT);
Set<Term> terms = new HashSet<>();
tree.collectTerms((f, b) -> terms.add(new Term(f, b)));
return terms;
}
public void testOrderedNearExtractor() {
SpanNearQuery q = new SpanNearQuery(new SpanQuery[]{
new SpanTermQuery(new Term("field1", "term1")),
new SpanTermQuery(new Term("field1", "term"))
}, 0, true);
Set<Term> expected = Collections.singleton(new Term("field1", "term1"));
assertEquals(expected, collectTerms(q));
}
public void testOrderedNearWithWildcardExtractor() {
SpanNearQuery q = new SpanNearQuery(new SpanQuery[]{
new SpanMultiTermQueryWrapper<>(new RegexpQuery(new Term("field", "super.*cali.*"))),
new SpanTermQuery(new Term("field", "is"))
}, 0, true);
Set<Term> expected = Collections.singleton(new Term("field", "is"));
assertEquals(expected, collectTerms(q));
}
public void testSpanOrExtractor() {
SpanOrQuery or = new SpanOrQuery(new SpanTermQuery(new Term("field", "term1")),
new SpanTermQuery(new Term("field", "term2")));
Set<Term> expected = new HashSet<>(Arrays.asList(
new Term("field", "term1"),
new Term("field", "term2")
));
assertEquals(expected, collectTerms(or));
}
public void testSpanMultiTerms() {
SpanQuery q = new SpanMultiTermQueryWrapper<>(new RegexpQuery(new Term("field", "term.*")));
Set<Term> terms = collectTerms(q);
assertEquals(1, terms.size());
assertEquals(TermFilteredPresearcher.ANYTOKEN_FIELD, terms.iterator().next().field());
}
public void testSpanWithin() {
Term t1 = new Term("field", "term1");
Term t2 = new Term("field", "term22");
Term t3 = new Term("field", "term333");
SpanWithinQuery swq = new SpanWithinQuery(
SpanNearQuery.newOrderedNearQuery("field")
.addClause(new SpanTermQuery(t1))
.addClause(new SpanTermQuery(t2))
.build(),
new SpanTermQuery(t3));
assertEquals(Collections.singleton(t3), collectTerms(swq));
}
public void testSpanContains() {
Term t1 = new Term("field", "term1");
Term t2 = new Term("field", "term22");
Term t3 = new Term("field", "term333");
SpanContainingQuery swq = new SpanContainingQuery(
SpanNearQuery.newOrderedNearQuery("field")
.addClause(new SpanTermQuery(t1))
.addClause(new SpanTermQuery(t2))
.build(),
new SpanTermQuery(t3));
assertEquals(Collections.singleton(t3), collectTerms(swq));
}
public void testSpanBoost() {
Term t1 = new Term("field", "term1");
SpanBoostQuery q = new SpanBoostQuery(new SpanTermQuery(t1), 0.1f);
assertEquals(Collections.singleton(t1), collectTerms(q));
}
public void testFieldMaskingSpanQuery() {
Term t1 = new Term("field", "term1");
FieldMaskingSpanQuery q = new FieldMaskingSpanQuery(new SpanTermQuery(t1), "field2");
assertEquals(Collections.singleton(t1), collectTerms(q));
}
public void testSpanPositionQuery() {
Term t1 = new Term("field", "term");
Query q = new SpanFirstQuery(new SpanTermQuery(t1), 10);
assertEquals(Collections.singleton(t1), collectTerms(q));
}
}

View File

@ -0,0 +1,67 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.IOException;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.Tokenizer;
import org.apache.lucene.analysis.core.WhitespaceTokenizer;
public class TestSuffixingNGramTokenizer extends BaseTokenStreamTestCase {
private Analyzer analyzer = new Analyzer() {
@Override
protected TokenStreamComponents createComponents(String fieldName) {
Tokenizer source = new WhitespaceTokenizer();
TokenStream sink = new SuffixingNGramTokenFilter(source, "XX", "ANY", 10);
return new TokenStreamComponents(source, sink);
}
};
public void testTokensAreSuffixed() throws IOException {
assertAnalyzesTo(analyzer, "term", new String[]{
"term", "termXX", "terXX", "teXX", "tXX", "ermXX", "erXX", "eXX", "rmXX", "rXX", "mXX", "XX"
});
}
public void testRepeatedSuffixesAreNotEmitted() throws IOException {
assertAnalyzesTo(analyzer, "arm harm term", new String[]{
"arm", "armXX", "arXX", "aXX", "rmXX", "rXX", "mXX", "XX",
"harm", "harmXX", "harXX", "haXX", "hXX",
"term", "termXX", "terXX", "teXX", "tXX", "ermXX", "erXX", "eXX"
});
}
public void testRepeatedInfixesAreNotEmitted() throws IOException {
assertAnalyzesTo(analyzer, "alarm alas harm", new String[]{
"alarm", "alarmXX", "alarXX", "alaXX", "alXX", "aXX",
"larmXX", "larXX", "laXX", "lXX", "armXX", "arXX", "rmXX", "rXX", "mXX", "XX",
"alas", "alasXX", "lasXX", "asXX", "sXX", "harm", "harmXX", "harXX", "haXX", "hXX"
});
}
public void testLengthyTokensAreNotNgrammed() throws IOException {
assertAnalyzesTo(analyzer, "alongtermthatshouldntbengrammed", new String[]{
"alongtermthatshouldntbengrammed", "ANY"
});
}
}

View File

@ -0,0 +1,152 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import org.apache.lucene.analysis.core.KeywordAnalyzer;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.memory.MemoryIndex;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.TermInSetQuery;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.store.ByteBuffersDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefHash;
public class TestTermPresearcher extends PresearcherTestBase {
public void testFiltersOnTermQueries() throws IOException {
MonitorQuery query1
= new MonitorQuery("1", parse("furble"));
MonitorQuery query2
= new MonitorQuery("2", parse("document"));
MonitorQuery query3 = new MonitorQuery("3", parse("\"a document\"")); // will be selected but not match
try (Monitor monitor = newMonitor()) {
monitor.register(query1, query2, query3);
Map<String, Long> timings = new HashMap<>();
QueryTimeListener timeListener =
(queryId, timeInNanos) -> timings.compute(queryId, (q, t) -> t == null ? timeInNanos : t + timeInNanos);
MatchingQueries<QueryMatch> matches = monitor.match(buildDoc(TEXTFIELD, "this is a test document"),
QueryTimeListener.timingMatcher(QueryMatch.SIMPLE_MATCHER, timeListener));
assertEquals(1, matches.getMatchCount());
assertNotNull(matches.matches("2"));
assertEquals(2, matches.getQueriesRun());
assertEquals(2, timings.size());
assertTrue(timings.keySet().contains("2"));
assertTrue(timings.keySet().contains("3"));
}
}
public void testIgnoresTermsOnNotQueries() throws IOException {
try (Monitor monitor = newMonitor()) {
monitor.register(new MonitorQuery("1", parse("document -test")));
MatchingQueries<QueryMatch> matches = monitor.match(buildDoc(TEXTFIELD, "this is a test document"), QueryMatch.SIMPLE_MATCHER);
assertEquals(0, matches.getMatchCount());
assertEquals(1, matches.getQueriesRun());
matches = monitor.match(buildDoc(TEXTFIELD, "weeble sclup test"), QueryMatch.SIMPLE_MATCHER);
assertEquals(0, matches.getMatchCount());
assertEquals(0, matches.getQueriesRun());
}
}
public void testMatchesAnyQueries() throws IOException {
try (Monitor monitor = newMonitor()) {
monitor.register(new MonitorQuery("1", parse("/hell./")));
MatchingQueries<QueryMatch> matches = monitor.match(buildDoc(TEXTFIELD, "hello"), QueryMatch.SIMPLE_MATCHER);
assertEquals(1, matches.getMatchCount());
assertEquals(1, matches.getQueriesRun());
}
}
@Override
protected Presearcher createPresearcher() {
return new TermFilteredPresearcher();
}
public void testAnyTermsAreCorrectlyAnalyzed() {
QueryAnalyzer analyzer = new QueryAnalyzer();
QueryTree qt = analyzer.buildTree(new MatchAllDocsQuery(), TermFilteredPresearcher.DEFAULT_WEIGHTOR);
TermFilteredPresearcher presearcher = new TermFilteredPresearcher();
Map<String, BytesRefHash> extractedTerms = presearcher.collectTerms(qt);
assertEquals(1, extractedTerms.size());
}
public void testQueryBuilder() throws IOException {
Presearcher presearcher = createPresearcher();
IndexWriterConfig iwc = new IndexWriterConfig(new KeywordAnalyzer());
Directory dir = new ByteBuffersDirectory();
IndexWriter writer = new IndexWriter(dir, iwc);
MonitorConfiguration config = new MonitorConfiguration(){
@Override
public IndexWriter buildIndexWriter() {
return writer;
}
};
try (Monitor monitor = new Monitor(ANALYZER, presearcher, config)) {
monitor.register(new MonitorQuery("1", parse("f:test")));
try (IndexReader reader = DirectoryReader.open(writer, false, false)) {
MemoryIndex mindex = new MemoryIndex();
mindex.addField("f", "this is a test document", WHITESPACE);
LeafReader docsReader = (LeafReader) mindex.createSearcher().getIndexReader();
QueryIndex.QueryTermFilter termFilter = new QueryIndex.QueryTermFilter(reader);
BooleanQuery q = (BooleanQuery) presearcher.buildQuery(docsReader, termFilter);
BooleanQuery expected = new BooleanQuery.Builder()
.add(should(new BooleanQuery.Builder()
.add(should(new TermInSetQuery("f", new BytesRef("test")))).build()))
.add(should(new TermQuery(new Term("__anytokenfield", "__ANYTOKEN__"))))
.build();
assertEquals(expected, q);
}
}
}
}

View File

@ -0,0 +1,73 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.IOException;
import org.apache.lucene.analysis.TokenFilter;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefIterator;
import org.apache.lucene.util.LuceneTestCase;
public class TestTermsEnumTokenFilter extends LuceneTestCase {
final class LeapfrogTokenFilter extends TokenFilter {
final PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class);
LeapfrogTokenFilter(TokenStream input) {
super(input);
}
@Override
public boolean incrementToken() throws IOException {
posIncAtt.setPositionIncrement(100000000);
return input.incrementToken();
}
}
public void testPosIncAttributeOverflow() throws IOException {
final BytesRef foo = new BytesRef("foo");
final BytesRef bar = new BytesRef("bar");
BytesRefIterator terms = new BytesRefIterator() {
long count = 1000;
@Override
public BytesRef next() throws IOException {
if (count-- > 100)
return foo;
if (count-- > 0)
return bar;
return null;
}
};
try (TokenStream ts = new LeapfrogTokenFilter(new TermsEnumTokenStream(terms))) {
while (ts.incrementToken()) {
// This tight loop will throw an exception if clearAttributes() is not called
// by TermsEnumTokenStream. See issue #46
}
}
}
}

View File

@ -0,0 +1,80 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.lucene.monitor;
import java.io.IOException;
import java.util.Collections;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
public class TestWildcardTermPresearcher extends PresearcherTestBase {
public void testFiltersWildcards() throws IOException {
try (Monitor monitor = newMonitor()) {
monitor.register(new MonitorQuery("1", parse("/hell.*/")));
assertEquals(1,
monitor.match(buildDoc(TEXTFIELD, "well hello there"), QueryMatch.SIMPLE_MATCHER).getMatchCount());
assertEquals(0, monitor.match(buildDoc(TEXTFIELD, "hi there"), QueryMatch.SIMPLE_MATCHER).getQueriesRun());
}
}
public void testNgramsOnlyMatchWildcards() throws IOException {
try (Monitor monitor = newMonitor()) {
monitor.register(new MonitorQuery("1", parse("hello")));
assertEquals(0, monitor.match(buildDoc(TEXTFIELD, "hellopolis"), QueryMatch.SIMPLE_MATCHER).getQueriesRun());
}
}
private static String repeat(String input, int size) {
StringBuilder sb = new StringBuilder();
for (int i = 0; i < size; i++) {
sb.append(input);
}
return sb.toString();
}
public void testLongTermsStillMatchWildcards() throws IOException {
try (Monitor monitor = newMonitor()) {
monitor.register(new MonitorQuery("1", parse("/a.*/")));
Document doc = new Document();
doc.add(newTextField(TEXTFIELD, repeat("a", RegexpQueryHandler.DEFAULT_MAX_TOKEN_SIZE + 1), Field.Store.NO));
MatchingQueries<QueryMatch> matches = monitor.match(doc, QueryMatch.SIMPLE_MATCHER);
assertEquals(1, matches.getQueriesRun());
assertNotNull(matches.matches("1"));
}
}
public void testCaseSensitivity() throws IOException {
try (Monitor monitor = newMonitor()) {
monitor.register(new MonitorQuery("1", parse("foo")));
assertEquals(1,
monitor.match(buildDoc(TEXTFIELD, "Foo foo"), QueryMatch.SIMPLE_MATCHER).getMatchCount());
}
}
@Override
protected Presearcher createPresearcher() {
return new TermFilteredPresearcher(TermWeightor.DEFAULT, Collections.singletonList(new RegexpQueryHandler()), Collections.emptySet());
}
}