mirror of https://github.com/apache/lucene.git
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/lucene-solr
This commit is contained in:
commit
b4fa0e785b
|
@ -1,20 +1,19 @@
|
|||
# .
|
||||
/eclipse-build
|
||||
/classes
|
||||
**/build
|
||||
build
|
||||
/idea-build
|
||||
**/dist
|
||||
**/lib
|
||||
**/test-lib
|
||||
dist
|
||||
lib
|
||||
test-lib
|
||||
/*~
|
||||
/velocity.log
|
||||
/build.properties
|
||||
/.idea
|
||||
lucene/**/*.iml
|
||||
solr/**/*.iml
|
||||
parent.iml
|
||||
**/*.ipr
|
||||
**/*.iws
|
||||
*.ipr
|
||||
*.iws
|
||||
/.project
|
||||
/.classpath
|
||||
/.settings
|
||||
|
@ -22,33 +21,7 @@ parent.iml
|
|||
/prj.el
|
||||
/bin
|
||||
/bin.*
|
||||
**/pom.xml
|
||||
pom.xml
|
||||
/nbproject
|
||||
/nb-build
|
||||
.pydevproject
|
||||
|
||||
/solr/package
|
||||
|
||||
# can this be minimized?
|
||||
/solr/example/start.jar
|
||||
/solr/example/webapps/*
|
||||
/solr/example/logs/*.log
|
||||
/solr/example/**/data
|
||||
/solr/example/solr/lib
|
||||
/solr/example/solr/logs
|
||||
/solr/example/solr/zoo_data
|
||||
/solr/example/work/*
|
||||
/solr/example/exampledocs/post.jar
|
||||
|
||||
/solr/example/example-DIH/**/data
|
||||
/solr/example/example-DIH/**/dataimport.properties
|
||||
/solr/example/example-DIH/solr/mail/lib/*.jar
|
||||
|
||||
solr/contrib/dataimporthandler/test-lib/
|
||||
|
||||
solr/core/test-lib/
|
||||
|
||||
solr/server/logs/
|
||||
solr/server/solr/zoo_data/
|
||||
solr/server/solr-webapp
|
||||
solr/server/start.jar
|
||||
|
|
37
build.xml
37
build.xml
|
@ -138,7 +138,7 @@
|
|||
'java', 'jflex', 'py', 'pl', 'g4', 'jj', 'html', 'js',
|
||||
'css', 'xml', 'xsl', 'vm', 'sh', 'cmd', 'bat', 'policy',
|
||||
'properties', 'mdtext',
|
||||
'template',
|
||||
'template', 'adoc',
|
||||
];
|
||||
def invalidPatterns = [
|
||||
(~$/@author\b/$) : '@author javadoc tag',
|
||||
|
@ -170,10 +170,15 @@
|
|||
def javaCommentPattern = ~$/(?sm)^\Q/*\E(.*?)\Q*/\E/$;
|
||||
def xmlCommentPattern = ~$/(?sm)\Q<!--\E(.*?)\Q-->\E/$;
|
||||
def lineSplitter = ~$/[\r\n]+/$;
|
||||
def singleLineSplitter = ~$/\n\r?/$;
|
||||
def licenseMatcher = Defaults.createDefaultMatcher();
|
||||
def validLoggerPattern = ~$/(?s)\b(private\s|static\s|final\s){3}+\s*Logger\s+\p{javaJavaIdentifierStart}+\s+=\s+\QLoggerFactory.getLogger(MethodHandles.lookup().lookupClass());\E/$;
|
||||
def packagePattern = ~$/(?m)^\s*package\s+org\.apache.*;/$;
|
||||
def xmlTagPattern = ~$/(?m)\s*<[a-zA-Z].*/$;
|
||||
def sourceHeaderPattern = ~$/\[source\b.*/$;
|
||||
def blockBoundaryPattern = ~$/----\s*/$;
|
||||
def blockTitlePattern = ~$/\..*/$;
|
||||
def unescapedSymbolPattern = ~$/(?<=[^\\]|^)([-=]>|<[-=])/$; // SOLR-10883
|
||||
|
||||
def isLicense = { matcher, ratDocument ->
|
||||
licenseMatcher.reset();
|
||||
|
@ -197,6 +202,33 @@
|
|||
}
|
||||
}
|
||||
|
||||
def checkForUnescapedSymbolSubstitutions = { f, text ->
|
||||
def inCodeBlock = false;
|
||||
def underSourceHeader = false;
|
||||
def lineNumber = 0;
|
||||
singleLineSplitter.split(text).each {
|
||||
++lineNumber;
|
||||
if (underSourceHeader) { // This line is either a single source line, or the boundary of a code block
|
||||
inCodeBlock = blockBoundaryPattern.matcher(it).matches();
|
||||
if ( ! blockTitlePattern.matcher(it).matches()) { // Keep underSourceHeader=true
|
||||
underSourceHeader = false;
|
||||
}
|
||||
} else {
|
||||
if (inCodeBlock) {
|
||||
inCodeBlock = ! blockBoundaryPattern.matcher(it).matches();
|
||||
} else {
|
||||
underSourceHeader = sourceHeaderPattern.matcher(it).matches();
|
||||
if ( ! underSourceHeader) {
|
||||
def unescapedSymbolMatcher = unescapedSymbolPattern.matcher(it);
|
||||
if (unescapedSymbolMatcher.find()) {
|
||||
reportViolation(f, 'Unescaped symbol "' + unescapedSymbolMatcher.group(1) + '" on line #' + lineNumber);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ant.fileScanner{
|
||||
fileset(dir: baseDir){
|
||||
extensions.each{
|
||||
|
@ -244,6 +276,9 @@
|
|||
if (f.toString().endsWith('.xml') || f.toString().endsWith('.xml.template')) {
|
||||
checkLicenseHeaderPrecedes(f, '<tag>', xmlTagPattern, xmlCommentPattern, text, ratDocument);
|
||||
}
|
||||
if (f.toString().endsWith('.adoc')) {
|
||||
checkForUnescapedSymbolSubstitutions(f, text);
|
||||
}
|
||||
};
|
||||
|
||||
if (found) {
|
||||
|
|
|
@ -66,6 +66,13 @@
|
|||
</foaf:Person>
|
||||
</maintainer>
|
||||
|
||||
<release>
|
||||
<Version>
|
||||
<name>lucene-6.6.0</name>
|
||||
<created>2017-06-06</created>
|
||||
<revision>6.6.0</revision>
|
||||
</Version>
|
||||
</release>
|
||||
<release>
|
||||
<Version>
|
||||
<name>lucene-6.5.1</name>
|
||||
|
|
|
@ -66,6 +66,13 @@
|
|||
</foaf:Person>
|
||||
</maintainer>
|
||||
|
||||
<release>
|
||||
<Version>
|
||||
<name>solr-6.6.0</name>
|
||||
<created>2017-06-06</created>
|
||||
<revision>6.6.0</revision>
|
||||
</Version>
|
||||
</release>
|
||||
<release>
|
||||
<Version>
|
||||
<name>solr-6.5.1</name>
|
||||
|
|
|
@ -19,6 +19,5 @@
|
|||
<orderEntry type="module" module-name="analysis-common" />
|
||||
<orderEntry type="module" module-name="grouping" />
|
||||
<orderEntry type="module" module-name="misc" />
|
||||
<orderEntry type="module" module-name="sandbox" />
|
||||
</component>
|
||||
</module>
|
||||
|
|
|
@ -24,10 +24,7 @@
|
|||
<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="queries" />
|
||||
<orderEntry type="module" module-name="misc" />
|
||||
<orderEntry type="module" module-name="spatial3d" />
|
||||
<orderEntry type="module" module-name="backward-codecs" />
|
||||
<orderEntry type="module" module-name="analysis-common" scope="TEST"/>
|
||||
</component>
|
||||
</module>
|
||||
</module>
|
||||
|
|
|
@ -6,6 +6,9 @@
|
|||
<content url="file://$MODULE_DIR$">
|
||||
<sourceFolder url="file://$MODULE_DIR$" isTestSource="false" />
|
||||
</content>
|
||||
<content url="file://$MODULE_DIR$/../resources">
|
||||
<sourceFolder url="file://$MODULE_DIR$/../resources" type="java-resource" />
|
||||
</content>
|
||||
<orderEntry type="inheritedJdk" />
|
||||
<orderEntry type="sourceFolder" forTests="false" />
|
||||
<orderEntry type="library" name="Solrj library" level="project" />
|
||||
|
|
|
@ -266,7 +266,10 @@ def checkAll(dirName):
|
|||
if __name__ == '__main__':
|
||||
if checkAll(sys.argv[1]):
|
||||
print()
|
||||
print('Broken javadocs links were found!')
|
||||
print('Broken javadocs links were found! Common root causes:')
|
||||
# please feel free to add to this list
|
||||
print('* A typo of some sort for manually created links.')
|
||||
print('* Public methods referencing non-public classes in their signature.')
|
||||
sys.exit(1)
|
||||
sys.exit(0)
|
||||
|
||||
|
|
|
@ -14,6 +14,20 @@ New Features
|
|||
well as the oldest Lucene version that contributed to the segment.
|
||||
(Adrien Grand)
|
||||
|
||||
* LUCENE-7854: The new TermFrequencyAttribute used during analysis
|
||||
with a custom token stream allows indexing custom term frequencies
|
||||
(Mike McCandless)
|
||||
|
||||
* LUCENE-7866: Add a new DelimitedTermFrequencyTokenFilter that allows to
|
||||
mark tokens with a custom term frequency (LUCENE-7854). It parses a numeric
|
||||
value after a separator char ('|') at the end of each token and changes
|
||||
the term frequency to this value. (Uwe Schindler, Robert Muir, Mike
|
||||
McCandless)
|
||||
|
||||
* LUCENE-7868: Multiple threads can now resolve deletes and doc values
|
||||
updates concurrently, giving sizable speedups in update-heavy
|
||||
indexing use cases (Simon Willnauer, Mike McCandless)
|
||||
|
||||
API Changes
|
||||
|
||||
* LUCENE-2605: Classic QueryParser no longer splits on whitespace by default.
|
||||
|
@ -59,11 +73,38 @@ API Changes
|
|||
|
||||
* LUCENE-7850: Removed support for legacy numerics. (Adrien Grand)
|
||||
|
||||
* LUCENE-7500: Removed abstract LeafReader.fields(); instead terms(fieldName)
|
||||
has been made abstract, fomerly was final. Also, MultiFields.getTerms
|
||||
was optimized to work directly instead of being implemented on getFields.
|
||||
(David Smiley)
|
||||
|
||||
* LUCENE-7872: TopDocs.totalHits is now a long. (Adrien Grand, hossman)
|
||||
|
||||
* LUCENE-7868: IndexWriterConfig.setMaxBufferedDeleteTerms is
|
||||
removed. (Simon Willnauer, Mike McCandless)
|
||||
|
||||
* LUCENE-7877: PrefixAwareTokenStream is replaced with ConcatenatingTokenStream
|
||||
(Alan Woodward, Uwe Schindler, Adrien Grand)
|
||||
|
||||
* LUCENE-7867: The deprecated Token class is now only available in the test
|
||||
framework (Alan Woodward, Adrien Grand)
|
||||
|
||||
* LUCENE-7723: DoubleValuesSource enforces implementation of equals() and
|
||||
hashCode() (Alan Woodward)
|
||||
|
||||
* LUCENE-7737: The spatial-extras module no longer has a dependency on the
|
||||
queries module. All uses of ValueSource are either replaced with core
|
||||
DoubleValuesSource extensions, or with the new ShapeValuesSource and
|
||||
ShapeValuesPredicate classes (Alan Woodward, David Smiley)
|
||||
|
||||
Bug Fixes
|
||||
|
||||
* LUCENE-7626: IndexWriter will no longer accept broken token offsets
|
||||
(Mike McCandless)
|
||||
|
||||
* LUCENE-7859: Spatial-extras PackedQuadPrefixTree bug that only revealed itself
|
||||
with the new pointsOnly optimizations in LUCENE-7845. (David Smiley)
|
||||
|
||||
Improvements
|
||||
|
||||
* LUCENE-7489: Better storage of sparse doc-values fields with the default
|
||||
|
@ -94,6 +135,8 @@ Optimizations
|
|||
query is a point (for 2D) or a is a simple date interval (e.g. 1 month). When
|
||||
the strategy is marked as pointsOnly, the results is a TermQuery. (David Smiley)
|
||||
|
||||
* LUCENE-7874: DisjunctionMaxQuery rewrites to a BooleanQuery when tiebreaker is set to 1. (Jim Ferenczi)
|
||||
|
||||
Other
|
||||
|
||||
* LUCENE-7328: Remove LegacyNumericEncoding from GeoPointField. (Nick Knize)
|
||||
|
@ -107,18 +150,61 @@ Other
|
|||
|
||||
* LUCENE-7540: Upgrade ICU to 59.1 (Mike McCandless, Jim Ferenczi)
|
||||
|
||||
* LUCENE-7852: Correct copyright year(s) in lucene/LICENSE.txt file.
|
||||
(Christine Poerschke, Steve Rowe)
|
||||
|
||||
* LUCENE-7719: Generalized the UnifiedHighlighter's support for AutomatonQuery
|
||||
for character & binary automata. Added AutomatonQuery.isBinary. (David Smiley)
|
||||
|
||||
* LUCENE-7873: Due to serious problems with context class loaders in several
|
||||
frameworks (OSGI, Java 9 Jigsaw), the lookup of Codecs, PostingsFormats,
|
||||
DocValuesFormats and all analysis factories was changed to only inspect the
|
||||
current classloader that defined the interface class (lucene-core.jar).
|
||||
See MIGRATE.txt for more information! (Uwe Schindler, Dawid Weiss)
|
||||
|
||||
* LUCENE-7883: Lucene no longer uses the context class loader when resolving
|
||||
resources in CustomAnalyzer or ClassPathResourceLoader. Resources are only
|
||||
resolved against Lucene's class loader by default. Please use another builder
|
||||
method to change to a custom classloader. (Uwe Schindler)
|
||||
|
||||
======================= Lucene 6.7.0 =======================
|
||||
|
||||
New Features
|
||||
|
||||
* LUCENE-7855: Added advanced options of the Wikipedia tokenizer to its factory.
|
||||
(Juan Pedro via Adrien Grand)
|
||||
|
||||
Bug Fixes
|
||||
|
||||
* LUCENE-7864: IndexMergeTool is not using intermediate hard links (even
|
||||
if possible). (Dawid Weiss)
|
||||
|
||||
* LUCENE-7869: Changed MemoryIndex to sort 1d points. In case of 1d points, the PointInSetQuery.MergePointVisitor expects
|
||||
that these points are visited in ascending order. The memory index doesn't do this and this can result in document
|
||||
with multiple points that should match to not match. (Martijn van Groningen)
|
||||
|
||||
* LUCENE-7878: Fix query builder to keep the SHOULD clause that wraps multi-word synonyms. (Jim Ferenczi)
|
||||
|
||||
Other
|
||||
|
||||
* LUCENE-7800: Remove code that potentially rethrows checked exceptions
|
||||
from methods that don't declare them ("sneaky throw" hack). (Robert Muir,
|
||||
Uwe Schindler, Dawid Weiss)
|
||||
|
||||
* LUCENE-7876: Avoid calls to LeafReader.fields() and MultiFields.getFields()
|
||||
that are trivially replaced by LeafReader.terms() and MultiFields.getTerms()
|
||||
(David Smiley)
|
||||
|
||||
Improvements
|
||||
|
||||
* LUCENE-7841: Normalize ґ to г in Ukrainian analyzer. (Andriy Rysin via Dawid Weiss)
|
||||
|
||||
Optimizations
|
||||
|
||||
* LUCENE-7828: Speed up range queries on range fields by improving how we
|
||||
compute the relation between the query and inner nodes of the BKD tree.
|
||||
(Adrien Grand)
|
||||
|
||||
======================= Lucene 6.6.0 =======================
|
||||
|
||||
New Features
|
||||
|
|
|
@ -1,5 +1,53 @@
|
|||
# Apache Lucene Migration Guide
|
||||
|
||||
## Changed SPI lookups for codecs and analysis changed (LUCENE-7873) ##
|
||||
|
||||
Due to serious problems with context class loaders in several frameworks
|
||||
(OSGI, Java 9 Jigsaw), the lookup of Codecs, PostingsFormats, DocValuesFormats
|
||||
and all analysis factories was changed to only inspect the current classloader
|
||||
that defined the interface class (`lucene-core.jar`). Normal applications
|
||||
should not encounter any issues with that change, because the application
|
||||
classloader (unnamed module in Java 9) can load all SPIs from all JARs
|
||||
from classpath.
|
||||
|
||||
For any code that relies on the old behaviour (e.g., certain web applications
|
||||
or components in application servers) one can manually instruct the Lucene
|
||||
SPI implementation to also inspect the context classloader. To do this,
|
||||
add this code to the early startup phase of your application before any
|
||||
Apache Lucene component is used:
|
||||
|
||||
ClassLoader cl = Thread.currentThread().getContextClassLoader();
|
||||
// Codecs:
|
||||
PostingsFormat.reloadPostingsFormats(cl);
|
||||
DocValuesFormat.reloadDocValuesFormats(cl);
|
||||
Codec.reloadCodecs(cl);
|
||||
// Analysis:
|
||||
CharFilterFactory.reloadCharFilters(cl);
|
||||
TokenFilterFactory.reloadTokenFilters(cl);
|
||||
TokenizerFactory.reloadTokenizers(cl);
|
||||
|
||||
This code will reload all service providers from the given class loader
|
||||
(in our case the context class loader). Of course, instead of specifying
|
||||
the context class loader, it is receommended to use the application's main
|
||||
class loader or the module class loader.
|
||||
|
||||
If you are migrating your project to Java 9 Jigsaw module system, keep in mind
|
||||
that Lucene currently does not yet support `module-info.java` declarations of
|
||||
service provider impls (`provides` statement). It is therefore recommended
|
||||
to keep all of Lucene in one Uber-Module and not try to split Lucene into
|
||||
several modules. As soon as Lucene will migrate to Java 9 as minimum
|
||||
requirement, we will work on improving that.
|
||||
|
||||
For OSGI, the same applies. You have to create a bundle with all of Lucene for
|
||||
SPI to work correctly.
|
||||
|
||||
## CustomAnalyzer resources (LUCENE-7883)##
|
||||
|
||||
Lucene no longer uses the context class loader when resolving resources in
|
||||
CustomAnalyzer or ClassPathResourceLoader. Resources are only resolved
|
||||
against Lucene's class loader by default. Please use another builder method
|
||||
to change to a custom classloader.
|
||||
|
||||
## Query.hashCode and Query.equals are now abstract methods (LUCENE-7277)
|
||||
|
||||
Any custom query subclasses should redeclare equivalence relationship according
|
||||
|
@ -80,3 +128,19 @@ use TopGroupsCollector.
|
|||
Support for legacy numerics has been removed since legacy numerics had been
|
||||
deprecated since Lucene 6.0. Points should be used instead, see
|
||||
org.apache.lucene.index.PointValues for an introduction.
|
||||
|
||||
## TopDocs.totalHits is now a long (LUCENE-7872)
|
||||
|
||||
TopDocs.totalHits is now a long so that TopDocs instances can be used to
|
||||
represent top hits that have more than 2B matches. This is necessary for the
|
||||
case that multiple TopDocs instances are merged together with TopDocs#merge as
|
||||
they might have more than 2B matches in total. However TopDocs instances
|
||||
returned by IndexSearcher will still have a total number of hits which is less
|
||||
than 2B since Lucene indexes are still bound to at most 2B documents, so it
|
||||
can safely be casted to an int in that case.
|
||||
|
||||
## PrefixAwareTokenFilter and PrefixAndSuffixAwareTokenFilter removed
|
||||
(LUCENE-7877)
|
||||
|
||||
Instead use ConcatentingTokenStream, which will allow for the use of custom
|
||||
attributes.
|
||||
|
|
|
@ -1,5 +1,5 @@
|
|||
Apache Lucene
|
||||
Copyright 2014 The Apache Software Foundation
|
||||
Copyright 2001-2017 The Apache Software Foundation
|
||||
|
||||
This product includes software developed at
|
||||
The Apache Software Foundation (http://www.apache.org/).
|
||||
|
|
|
@ -76,17 +76,21 @@ import org.apache.lucene.util.Version;
|
|||
*/
|
||||
public final class CustomAnalyzer extends Analyzer {
|
||||
|
||||
/** Returns a builder for custom analyzers that loads all resources from classpath.
|
||||
* All path names given must be absolute with package prefixes. */
|
||||
/**
|
||||
* Returns a builder for custom analyzers that loads all resources from
|
||||
* Lucene's classloader. All path names given must be absolute with package prefixes.
|
||||
*/
|
||||
public static Builder builder() {
|
||||
return builder(new ClasspathResourceLoader());
|
||||
return builder(new ClasspathResourceLoader(CustomAnalyzer.class.getClassLoader()));
|
||||
}
|
||||
|
||||
/** Returns a builder for custom analyzers that loads all resources from the given
|
||||
/**
|
||||
* Returns a builder for custom analyzers that loads all resources from the given
|
||||
* file system base directory. Place, e.g., stop word files there.
|
||||
* Files that are not in the given directory are loaded from classpath. */
|
||||
* Files that are not in the given directory are loaded from Lucene's classloader.
|
||||
*/
|
||||
public static Builder builder(Path configDir) {
|
||||
return builder(new FilesystemResourceLoader(configDir));
|
||||
return builder(new FilesystemResourceLoader(configDir, CustomAnalyzer.class.getClassLoader()));
|
||||
}
|
||||
|
||||
/** Returns a builder for custom analyzers that loads all resources using the given {@link ResourceLoader}. */
|
||||
|
|
|
@ -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.analysis.miscellaneous;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.util.Attribute;
|
||||
import org.apache.lucene.util.AttributeSource;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
/**
|
||||
* A TokenStream that takes an array of input TokenStreams as sources, and
|
||||
* concatenates them together.
|
||||
*
|
||||
* Offsets from the second and subsequent sources are incremented to behave
|
||||
* as if all the inputs were from a single source.
|
||||
*
|
||||
* All of the input TokenStreams must have the same attribute implementations
|
||||
*/
|
||||
public final class ConcatenatingTokenStream extends TokenStream {
|
||||
|
||||
private final TokenStream[] sources;
|
||||
private final OffsetAttribute[] sourceOffsets;
|
||||
private final OffsetAttribute offsetAtt;
|
||||
|
||||
private int currentSource;
|
||||
private int offsetIncrement;
|
||||
|
||||
/**
|
||||
* Create a new ConcatenatingTokenStream from a set of inputs
|
||||
* @param sources an array of TokenStream inputs to concatenate
|
||||
*/
|
||||
public ConcatenatingTokenStream(TokenStream... sources) {
|
||||
super(combineSources(sources));
|
||||
this.sources = sources;
|
||||
this.offsetAtt = addAttribute(OffsetAttribute.class);
|
||||
this.sourceOffsets = new OffsetAttribute[sources.length];
|
||||
for (int i = 0; i < sources.length; i++) {
|
||||
this.sourceOffsets[i] = sources[i].addAttribute(OffsetAttribute.class);
|
||||
}
|
||||
}
|
||||
|
||||
private static AttributeSource combineSources(TokenStream... sources) {
|
||||
AttributeSource base = sources[0].cloneAttributes();
|
||||
try {
|
||||
for (int i = 1; i < sources.length; i++) {
|
||||
Iterator<Class<? extends Attribute>> it = sources[i].getAttributeClassesIterator();
|
||||
while (it.hasNext()) {
|
||||
base.addAttribute(it.next());
|
||||
}
|
||||
// check attributes can be captured
|
||||
sources[i].copyTo(base);
|
||||
}
|
||||
return base;
|
||||
}
|
||||
catch (IllegalArgumentException e) {
|
||||
throw new IllegalArgumentException("Attempted to concatenate TokenStreams with different attribute types", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean incrementToken() throws IOException {
|
||||
while (sources[currentSource].incrementToken() == false) {
|
||||
if (currentSource >= sources.length - 1)
|
||||
return false;
|
||||
sources[currentSource].end();
|
||||
OffsetAttribute att = sourceOffsets[currentSource];
|
||||
if (att != null)
|
||||
offsetIncrement += att.endOffset();
|
||||
currentSource++;
|
||||
}
|
||||
|
||||
clearAttributes();
|
||||
sources[currentSource].copyTo(this);
|
||||
offsetAtt.setOffset(offsetAtt.startOffset() + offsetIncrement, offsetAtt.endOffset() + offsetIncrement);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void end() throws IOException {
|
||||
sources[currentSource].end();
|
||||
super.end();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void reset() throws IOException {
|
||||
for (TokenStream source : sources) {
|
||||
source.reset();
|
||||
}
|
||||
super.reset();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
IOUtils.close(sources);
|
||||
}
|
||||
finally {
|
||||
super.close();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,75 @@
|
|||
/*
|
||||
* 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.analysis.miscellaneous;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.analysis.TokenFilter;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.tokenattributes.TermFrequencyAttribute;
|
||||
import org.apache.lucene.index.IndexOptions;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
|
||||
|
||||
/**
|
||||
* Characters before the delimiter are the "token", the textual integer after is the term frequency.
|
||||
* To use this {@code TokenFilter} the field must be indexed with
|
||||
* {@link IndexOptions#DOCS_AND_FREQS} but no positions or offsets.
|
||||
* <p>
|
||||
* For example, if the delimiter is '|', then for the string "foo|5", "foo" is the token
|
||||
* and "5" is a term frequency. If there is no delimiter, the TokenFilter does not modify
|
||||
* the term frequency.
|
||||
* <p>
|
||||
* Note make sure your Tokenizer doesn't split on the delimiter, or this won't work
|
||||
*/
|
||||
public final class DelimitedTermFrequencyTokenFilter extends TokenFilter {
|
||||
public static final char DEFAULT_DELIMITER = '|';
|
||||
|
||||
private final char delimiter;
|
||||
private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
|
||||
private final TermFrequencyAttribute tfAtt = addAttribute(TermFrequencyAttribute.class);
|
||||
|
||||
|
||||
public DelimitedTermFrequencyTokenFilter(TokenStream input) {
|
||||
this(input, DEFAULT_DELIMITER);
|
||||
}
|
||||
|
||||
public DelimitedTermFrequencyTokenFilter(TokenStream input, char delimiter) {
|
||||
super(input);
|
||||
this.delimiter = delimiter;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean incrementToken() throws IOException {
|
||||
if (input.incrementToken()) {
|
||||
final char[] buffer = termAtt.buffer();
|
||||
final int length = termAtt.length();
|
||||
for (int i = 0; i < length; i++) {
|
||||
if (buffer[i] == delimiter) {
|
||||
termAtt.setLength(i); // simply set a new length
|
||||
i++;
|
||||
tfAtt.setTermFrequency(ArrayUtil.parseInt(buffer, i, length - i));
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
}
|
|
@ -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.analysis.miscellaneous;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.util.TokenFilterFactory;
|
||||
|
||||
/**
|
||||
* Factory for {@link DelimitedTermFrequencyTokenFilter}. The field must have {@code omitPositions=true}.
|
||||
* <pre class="prettyprint">
|
||||
* <fieldType name="text_tfdl" class="solr.TextField" omitPositions="true">
|
||||
* <analyzer>
|
||||
* <tokenizer class="solr.WhitespaceTokenizerFactory"/>
|
||||
* <filter class="solr.DelimitedTermFrequencyTokenFilterFactory" delimiter="|"/>
|
||||
* </analyzer>
|
||||
* </fieldType></pre>
|
||||
*/
|
||||
public class DelimitedTermFrequencyTokenFilterFactory extends TokenFilterFactory {
|
||||
public static final String DELIMITER_ATTR = "delimiter";
|
||||
|
||||
private final char delimiter;
|
||||
|
||||
/** Creates a new DelimitedPayloadTokenFilterFactory */
|
||||
public DelimitedTermFrequencyTokenFilterFactory(Map<String, String> args) {
|
||||
super(args);
|
||||
delimiter = getChar(args, DELIMITER_ATTR, DelimitedTermFrequencyTokenFilter.DEFAULT_DELIMITER);
|
||||
if (!args.isEmpty()) {
|
||||
throw new IllegalArgumentException("Unknown parameters: " + args);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public DelimitedTermFrequencyTokenFilter create(TokenStream input) {
|
||||
return new DelimitedTermFrequencyTokenFilter(input, delimiter);
|
||||
}
|
||||
}
|
|
@ -1,84 +0,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.
|
||||
*/
|
||||
package org.apache.lucene.analysis.miscellaneous;
|
||||
|
||||
|
||||
import org.apache.lucene.analysis.Token;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Links two {@link PrefixAwareTokenFilter}.
|
||||
* <p>
|
||||
* <b>NOTE:</b> This filter might not behave correctly if used with custom Attributes, i.e. Attributes other than
|
||||
* the ones located in org.apache.lucene.analysis.tokenattributes.
|
||||
*/
|
||||
public class PrefixAndSuffixAwareTokenFilter extends TokenStream {
|
||||
|
||||
private PrefixAwareTokenFilter suffix;
|
||||
|
||||
public PrefixAndSuffixAwareTokenFilter(TokenStream prefix, TokenStream input, TokenStream suffix) {
|
||||
super(suffix);
|
||||
prefix = new PrefixAwareTokenFilter(prefix, input) {
|
||||
@Override
|
||||
public Token updateSuffixToken(Token suffixToken, Token lastInputToken) {
|
||||
return PrefixAndSuffixAwareTokenFilter.this.updateInputToken(suffixToken, lastInputToken);
|
||||
}
|
||||
};
|
||||
this.suffix = new PrefixAwareTokenFilter(prefix, suffix) {
|
||||
@Override
|
||||
public Token updateSuffixToken(Token suffixToken, Token lastInputToken) {
|
||||
return PrefixAndSuffixAwareTokenFilter.this.updateSuffixToken(suffixToken, lastInputToken);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
public Token updateInputToken(Token inputToken, Token lastPrefixToken) {
|
||||
inputToken.setOffset(lastPrefixToken.endOffset() + inputToken.startOffset(),
|
||||
lastPrefixToken.endOffset() + inputToken.endOffset());
|
||||
return inputToken;
|
||||
}
|
||||
|
||||
public Token updateSuffixToken(Token suffixToken, Token lastInputToken) {
|
||||
suffixToken.setOffset(lastInputToken.endOffset() + suffixToken.startOffset(),
|
||||
lastInputToken.endOffset() + suffixToken.endOffset());
|
||||
return suffixToken;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public final boolean incrementToken() throws IOException {
|
||||
return suffix.incrementToken();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void reset() throws IOException {
|
||||
suffix.reset();
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
suffix.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void end() throws IOException {
|
||||
suffix.end();
|
||||
}
|
||||
}
|
|
@ -1,202 +0,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.
|
||||
*/
|
||||
package org.apache.lucene.analysis.miscellaneous;
|
||||
|
||||
|
||||
import org.apache.lucene.analysis.Token;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.tokenattributes.FlagsAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
|
||||
/**
|
||||
* Joins two token streams and leaves the last token of the first stream available
|
||||
* to be used when updating the token values in the second stream based on that token.
|
||||
*
|
||||
* The default implementation adds last prefix token end offset to the suffix token start and end offsets.
|
||||
* <p>
|
||||
* <b>NOTE:</b> This filter might not behave correctly if used with custom Attributes, i.e. Attributes other than
|
||||
* the ones located in org.apache.lucene.analysis.tokenattributes.
|
||||
*/
|
||||
public class PrefixAwareTokenFilter extends TokenStream {
|
||||
|
||||
private TokenStream prefix;
|
||||
private TokenStream suffix;
|
||||
|
||||
private CharTermAttribute termAtt;
|
||||
private PositionIncrementAttribute posIncrAtt;
|
||||
private PayloadAttribute payloadAtt;
|
||||
private OffsetAttribute offsetAtt;
|
||||
private TypeAttribute typeAtt;
|
||||
private FlagsAttribute flagsAtt;
|
||||
|
||||
private CharTermAttribute p_termAtt;
|
||||
private PositionIncrementAttribute p_posIncrAtt;
|
||||
private PayloadAttribute p_payloadAtt;
|
||||
private OffsetAttribute p_offsetAtt;
|
||||
private TypeAttribute p_typeAtt;
|
||||
private FlagsAttribute p_flagsAtt;
|
||||
|
||||
public PrefixAwareTokenFilter(TokenStream prefix, TokenStream suffix) {
|
||||
super(suffix);
|
||||
this.suffix = suffix;
|
||||
this.prefix = prefix;
|
||||
prefixExhausted = false;
|
||||
|
||||
termAtt = addAttribute(CharTermAttribute.class);
|
||||
posIncrAtt = addAttribute(PositionIncrementAttribute.class);
|
||||
payloadAtt = addAttribute(PayloadAttribute.class);
|
||||
offsetAtt = addAttribute(OffsetAttribute.class);
|
||||
typeAtt = addAttribute(TypeAttribute.class);
|
||||
flagsAtt = addAttribute(FlagsAttribute.class);
|
||||
|
||||
p_termAtt = prefix.addAttribute(CharTermAttribute.class);
|
||||
p_posIncrAtt = prefix.addAttribute(PositionIncrementAttribute.class);
|
||||
p_payloadAtt = prefix.addAttribute(PayloadAttribute.class);
|
||||
p_offsetAtt = prefix.addAttribute(OffsetAttribute.class);
|
||||
p_typeAtt = prefix.addAttribute(TypeAttribute.class);
|
||||
p_flagsAtt = prefix.addAttribute(FlagsAttribute.class);
|
||||
}
|
||||
|
||||
private Token previousPrefixToken = new Token();
|
||||
private Token reusableToken = new Token();
|
||||
|
||||
private boolean prefixExhausted;
|
||||
|
||||
@Override
|
||||
public final boolean incrementToken() throws IOException {
|
||||
if (!prefixExhausted) {
|
||||
Token nextToken = getNextPrefixInputToken(reusableToken);
|
||||
if (nextToken == null) {
|
||||
prefixExhausted = true;
|
||||
} else {
|
||||
previousPrefixToken.reinit(nextToken);
|
||||
// Make it a deep copy
|
||||
BytesRef p = previousPrefixToken.getPayload();
|
||||
if (p != null) {
|
||||
previousPrefixToken.setPayload(p.clone());
|
||||
}
|
||||
setCurrentToken(nextToken);
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
Token nextToken = getNextSuffixInputToken(reusableToken);
|
||||
if (nextToken == null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
nextToken = updateSuffixToken(nextToken, previousPrefixToken);
|
||||
setCurrentToken(nextToken);
|
||||
return true;
|
||||
}
|
||||
|
||||
private void setCurrentToken(Token token) {
|
||||
if (token == null) return;
|
||||
clearAttributes();
|
||||
termAtt.copyBuffer(token.buffer(), 0, token.length());
|
||||
posIncrAtt.setPositionIncrement(token.getPositionIncrement());
|
||||
flagsAtt.setFlags(token.getFlags());
|
||||
offsetAtt.setOffset(token.startOffset(), token.endOffset());
|
||||
typeAtt.setType(token.type());
|
||||
payloadAtt.setPayload(token.getPayload());
|
||||
}
|
||||
|
||||
private Token getNextPrefixInputToken(Token token) throws IOException {
|
||||
if (!prefix.incrementToken()) return null;
|
||||
token.copyBuffer(p_termAtt.buffer(), 0, p_termAtt.length());
|
||||
token.setPositionIncrement(p_posIncrAtt.getPositionIncrement());
|
||||
token.setFlags(p_flagsAtt.getFlags());
|
||||
token.setOffset(p_offsetAtt.startOffset(), p_offsetAtt.endOffset());
|
||||
token.setType(p_typeAtt.type());
|
||||
token.setPayload(p_payloadAtt.getPayload());
|
||||
return token;
|
||||
}
|
||||
|
||||
private Token getNextSuffixInputToken(Token token) throws IOException {
|
||||
if (!suffix.incrementToken()) return null;
|
||||
token.copyBuffer(termAtt.buffer(), 0, termAtt.length());
|
||||
token.setPositionIncrement(posIncrAtt.getPositionIncrement());
|
||||
token.setFlags(flagsAtt.getFlags());
|
||||
token.setOffset(offsetAtt.startOffset(), offsetAtt.endOffset());
|
||||
token.setType(typeAtt.type());
|
||||
token.setPayload(payloadAtt.getPayload());
|
||||
return token;
|
||||
}
|
||||
|
||||
/**
|
||||
* The default implementation adds last prefix token end offset to the suffix token start and end offsets.
|
||||
*
|
||||
* @param suffixToken a token from the suffix stream
|
||||
* @param lastPrefixToken the last token from the prefix stream
|
||||
* @return consumer token
|
||||
*/
|
||||
public Token updateSuffixToken(Token suffixToken, Token lastPrefixToken) {
|
||||
suffixToken.setOffset(lastPrefixToken.endOffset() + suffixToken.startOffset(),
|
||||
lastPrefixToken.endOffset() + suffixToken.endOffset());
|
||||
return suffixToken;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void end() throws IOException {
|
||||
prefix.end();
|
||||
suffix.end();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
prefix.close();
|
||||
suffix.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void reset() throws IOException {
|
||||
super.reset();
|
||||
if (prefix != null) {
|
||||
prefixExhausted = false;
|
||||
prefix.reset();
|
||||
}
|
||||
if (suffix != null) {
|
||||
suffix.reset();
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
public TokenStream getPrefix() {
|
||||
return prefix;
|
||||
}
|
||||
|
||||
public void setPrefix(TokenStream prefix) {
|
||||
this.prefix = prefix;
|
||||
}
|
||||
|
||||
public TokenStream getSuffix() {
|
||||
return suffix;
|
||||
}
|
||||
|
||||
public void setSuffix(TokenStream suffix) {
|
||||
this.suffix = suffix;
|
||||
}
|
||||
}
|
|
@ -17,18 +17,17 @@
|
|||
package org.apache.lucene.analysis.payloads;
|
||||
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.analysis.TokenFilter;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
|
||||
/**
|
||||
* Assigns a payload to a token based on the {@link org.apache.lucene.analysis.Token#type()}
|
||||
*
|
||||
* Assigns a payload to a token based on the {@link org.apache.lucene.analysis.tokenattributes.TypeAttribute}
|
||||
**/
|
||||
public class NumericPayloadTokenFilter extends TokenFilter {
|
||||
|
||||
|
|
|
@ -17,17 +17,17 @@
|
|||
package org.apache.lucene.analysis.payloads;
|
||||
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.analysis.TokenFilter;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
|
||||
/**
|
||||
* Makes the {@link org.apache.lucene.analysis.Token#type()} a payload.
|
||||
* Makes the {@link TypeAttribute} a payload.
|
||||
*
|
||||
* Encodes the type using {@link String#getBytes(String)} with "UTF-8" as the encoding
|
||||
*
|
||||
|
|
|
@ -48,7 +48,7 @@ public final class AnalysisSPILoader<S extends AbstractAnalysisFactory> {
|
|||
}
|
||||
|
||||
public AnalysisSPILoader(Class<S> clazz, String[] suffixes) {
|
||||
this(clazz, suffixes, Thread.currentThread().getContextClassLoader());
|
||||
this(clazz, suffixes, null);
|
||||
}
|
||||
|
||||
public AnalysisSPILoader(Class<S> clazz, String[] suffixes, ClassLoader classloader) {
|
||||
|
|
|
@ -20,6 +20,8 @@ package org.apache.lucene.analysis.util;
|
|||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
|
||||
import org.apache.lucene.util.SuppressForbidden;
|
||||
|
||||
/**
|
||||
* Simple {@link ResourceLoader} that uses {@link ClassLoader#getResourceAsStream(String)}
|
||||
* and {@link Class#forName(String,boolean,ClassLoader)} to open resources and
|
||||
|
@ -30,9 +32,17 @@ public final class ClasspathResourceLoader implements ResourceLoader {
|
|||
private final ClassLoader loader;
|
||||
|
||||
/**
|
||||
* Creates an instance using the context classloader to load Resources and classes.
|
||||
* Creates an instance using the context classloader to load resources and classes.
|
||||
* Resource paths must be absolute.
|
||||
*
|
||||
* @deprecated You should not use this ctor, because it uses the thread's context
|
||||
* class loader, which is bad programming style. Please specify a reference class or
|
||||
* a {@link ClassLoader} instead.
|
||||
* @see #ClasspathResourceLoader(ClassLoader)
|
||||
* @see #ClasspathResourceLoader(Class)
|
||||
*/
|
||||
@Deprecated
|
||||
@SuppressForbidden(reason = "Deprecated method uses thread's context classloader, but there for backwards compatibility")
|
||||
public ClasspathResourceLoader() {
|
||||
this(Thread.currentThread().getContextClassLoader());
|
||||
}
|
||||
|
|
|
@ -46,11 +46,27 @@ public final class FilesystemResourceLoader implements ResourceLoader {
|
|||
* base directory (may be {@code null} to refer to CWD).
|
||||
* Files not found in file system and class lookups are delegated to context
|
||||
* classloader.
|
||||
*
|
||||
* @deprecated You should not use this ctor, because it uses the thread's context
|
||||
* class loader as fallback for resource lookups, which is bad programming style.
|
||||
* Please specify a {@link ClassLoader} instead.
|
||||
* @see #FilesystemResourceLoader(Path, ClassLoader)
|
||||
*/
|
||||
@Deprecated
|
||||
public FilesystemResourceLoader(Path baseDirectory) {
|
||||
this(baseDirectory, new ClasspathResourceLoader());
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a resource loader that resolves resources against the given
|
||||
* base directory (may be {@code null} to refer to CWD).
|
||||
* Files not found in file system and class lookups are delegated to context
|
||||
* classloader.
|
||||
*/
|
||||
public FilesystemResourceLoader(Path baseDirectory, ClassLoader delegate) {
|
||||
this(baseDirectory, new ClasspathResourceLoader(delegate));
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a resource loader that resolves resources against the given
|
||||
* base directory (may be {@code null} to refer to CWD).
|
||||
|
|
|
@ -16,9 +16,9 @@
|
|||
*/
|
||||
package org.apache.lucene.analysis.wikipedia;
|
||||
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.analysis.util.TokenizerFactory;
|
||||
import org.apache.lucene.util.AttributeFactory;
|
||||
|
@ -33,19 +33,28 @@ import org.apache.lucene.util.AttributeFactory;
|
|||
* </fieldType></pre>
|
||||
*/
|
||||
public class WikipediaTokenizerFactory extends TokenizerFactory {
|
||||
|
||||
public static final String TOKEN_OUTPUT = "tokenOutput";
|
||||
public static final String UNTOKENIZED_TYPES = "untokenizedTypes";
|
||||
|
||||
protected final int tokenOutput;
|
||||
protected Set<String> untokenizedTypes;
|
||||
|
||||
/** Creates a new WikipediaTokenizerFactory */
|
||||
public WikipediaTokenizerFactory(Map<String,String> args) {
|
||||
super(args);
|
||||
tokenOutput = getInt(args, TOKEN_OUTPUT, WikipediaTokenizer.TOKENS_ONLY);
|
||||
untokenizedTypes = getSet(args, UNTOKENIZED_TYPES);
|
||||
|
||||
if (untokenizedTypes == null) {
|
||||
untokenizedTypes = Collections.emptySet();
|
||||
}
|
||||
if (!args.isEmpty()) {
|
||||
throw new IllegalArgumentException("Unknown parameters: " + args);
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: add support for WikipediaTokenizer's advanced options.
|
||||
|
||||
@Override
|
||||
public WikipediaTokenizer create(AttributeFactory factory) {
|
||||
return new WikipediaTokenizer(factory, WikipediaTokenizer.TOKENS_ONLY,
|
||||
Collections.<String>emptySet());
|
||||
return new WikipediaTokenizer(factory, tokenOutput, untokenizedTypes);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -63,6 +63,7 @@ org.apache.lucene.analysis.miscellaneous.ASCIIFoldingFilterFactory
|
|||
org.apache.lucene.analysis.miscellaneous.CapitalizationFilterFactory
|
||||
org.apache.lucene.analysis.miscellaneous.CodepointCountFilterFactory
|
||||
org.apache.lucene.analysis.miscellaneous.DateRecognizerFilterFactory
|
||||
org.apache.lucene.analysis.miscellaneous.DelimitedTermFrequencyTokenFilterFactory
|
||||
org.apache.lucene.analysis.miscellaneous.FingerprintFilterFactory
|
||||
org.apache.lucene.analysis.miscellaneous.FixBrokenOffsetsFilterFactory
|
||||
org.apache.lucene.analysis.miscellaneous.HyphenatedWordsFilterFactory
|
||||
|
|
|
@ -21,13 +21,17 @@ import java.io.IOException;
|
|||
import java.io.Reader;
|
||||
import java.lang.reflect.Constructor;
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
|
||||
import org.apache.lucene.analysis.MockTokenizer;
|
||||
import org.apache.lucene.analysis.Tokenizer;
|
||||
import org.apache.lucene.analysis.miscellaneous.DelimitedTermFrequencyTokenFilterFactory;
|
||||
import org.apache.lucene.analysis.util.AbstractAnalysisFactory;
|
||||
import org.apache.lucene.analysis.util.CharFilterFactory;
|
||||
import org.apache.lucene.analysis.util.MultiTermAwareComponent;
|
||||
|
@ -49,6 +53,12 @@ import org.apache.lucene.util.Version;
|
|||
|
||||
// TODO: fix this to use CustomAnalyzer instead of its own FactoryAnalyzer
|
||||
public class TestFactories extends BaseTokenStreamTestCase {
|
||||
|
||||
/** Factories that are excluded from testing it with random data */
|
||||
private static final Set<Class<? extends AbstractAnalysisFactory>> EXCLUDE_FACTORIES_RANDOM_DATA = new HashSet<>(Arrays.asList(
|
||||
DelimitedTermFrequencyTokenFilterFactory.class
|
||||
));
|
||||
|
||||
public void test() throws IOException {
|
||||
for (String tokenizer : TokenizerFactory.availableTokenizers()) {
|
||||
doTestTokenizer(tokenizer);
|
||||
|
@ -77,11 +87,13 @@ public class TestFactories extends BaseTokenStreamTestCase {
|
|||
assertFalse(mtc instanceof CharFilterFactory);
|
||||
}
|
||||
|
||||
// beast it just a little, it shouldnt throw exceptions:
|
||||
// (it should have thrown them in initialize)
|
||||
Analyzer a = new FactoryAnalyzer(factory, null, null);
|
||||
checkRandomData(random(), a, 20, 20, false, false);
|
||||
a.close();
|
||||
if (!EXCLUDE_FACTORIES_RANDOM_DATA.contains(factory.getClass())) {
|
||||
// beast it just a little, it shouldnt throw exceptions:
|
||||
// (it should have thrown them in initialize)
|
||||
Analyzer a = new FactoryAnalyzer(factory, null, null);
|
||||
checkRandomData(random(), a, 20, 20, false, false);
|
||||
a.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -99,11 +111,13 @@ public class TestFactories extends BaseTokenStreamTestCase {
|
|||
assertTrue(mtc instanceof TokenFilterFactory);
|
||||
}
|
||||
|
||||
// beast it just a little, it shouldnt throw exceptions:
|
||||
// (it should have thrown them in initialize)
|
||||
Analyzer a = new FactoryAnalyzer(assertingTokenizer, factory, null);
|
||||
checkRandomData(random(), a, 20, 20, false, false);
|
||||
a.close();
|
||||
if (!EXCLUDE_FACTORIES_RANDOM_DATA.contains(factory.getClass())) {
|
||||
// beast it just a little, it shouldnt throw exceptions:
|
||||
// (it should have thrown them in initialize)
|
||||
Analyzer a = new FactoryAnalyzer(assertingTokenizer, factory, null);
|
||||
checkRandomData(random(), a, 20, 20, false, false);
|
||||
a.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -121,11 +135,13 @@ public class TestFactories extends BaseTokenStreamTestCase {
|
|||
assertTrue(mtc instanceof CharFilterFactory);
|
||||
}
|
||||
|
||||
// beast it just a little, it shouldnt throw exceptions:
|
||||
// (it should have thrown them in initialize)
|
||||
Analyzer a = new FactoryAnalyzer(assertingTokenizer, null, factory);
|
||||
checkRandomData(random(), a, 20, 20, false, false);
|
||||
a.close();
|
||||
if (!EXCLUDE_FACTORIES_RANDOM_DATA.contains(factory.getClass())) {
|
||||
// beast it just a little, it shouldnt throw exceptions:
|
||||
// (it should have thrown them in initialize)
|
||||
Analyzer a = new FactoryAnalyzer(assertingTokenizer, null, factory);
|
||||
checkRandomData(random(), a, 20, 20, false, false);
|
||||
a.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -73,6 +73,7 @@ import org.apache.lucene.analysis.compound.TestCompoundWordTokenFilter;
|
|||
import org.apache.lucene.analysis.compound.hyphenation.HyphenationTree;
|
||||
import org.apache.lucene.analysis.hunspell.Dictionary;
|
||||
import org.apache.lucene.analysis.hunspell.TestHunspellStemFilter;
|
||||
import org.apache.lucene.analysis.miscellaneous.DelimitedTermFrequencyTokenFilter;
|
||||
import org.apache.lucene.analysis.miscellaneous.HyphenatedWordsFilter;
|
||||
import org.apache.lucene.analysis.miscellaneous.LimitTokenCountFilter;
|
||||
import org.apache.lucene.analysis.miscellaneous.LimitTokenOffsetFilter;
|
||||
|
@ -159,6 +160,8 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
|
|||
WordDelimiterFilter.class,
|
||||
// Cannot correct offsets when a char filter had changed them:
|
||||
WordDelimiterGraphFilter.class,
|
||||
// requires a special encoded token value, so it may fail with random data:
|
||||
DelimitedTermFrequencyTokenFilter.class,
|
||||
// clones of core's filters:
|
||||
org.apache.lucene.analysis.core.StopFilter.class,
|
||||
org.apache.lucene.analysis.core.LowerCaseFilter.class)) {
|
||||
|
|
|
@ -0,0 +1,77 @@
|
|||
/*
|
||||
* 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.analysis.miscellaneous;
|
||||
|
||||
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.TermFrequencyAttribute;
|
||||
|
||||
public class DelimitedTermFrequencyTokenFilterTest extends BaseTokenStreamTestCase {
|
||||
|
||||
public void testTermFrequency() throws Exception {
|
||||
String test = "The quick|40 red|4 fox|06 jumped|1 over the lazy|2 brown|123 dogs|1024";
|
||||
DelimitedTermFrequencyTokenFilter filter =
|
||||
new DelimitedTermFrequencyTokenFilter(whitespaceMockTokenizer(test));
|
||||
CharTermAttribute termAtt = filter.getAttribute(CharTermAttribute.class);
|
||||
TermFrequencyAttribute tfAtt = filter.getAttribute(TermFrequencyAttribute.class);
|
||||
filter.reset();
|
||||
assertTermEquals("The", filter, termAtt, tfAtt, 1);
|
||||
assertTermEquals("quick", filter, termAtt, tfAtt, 40);
|
||||
assertTermEquals("red", filter, termAtt, tfAtt, 4);
|
||||
assertTermEquals("fox", filter, termAtt, tfAtt, 6);
|
||||
assertTermEquals("jumped", filter, termAtt, tfAtt, 1);
|
||||
assertTermEquals("over", filter, termAtt, tfAtt, 1);
|
||||
assertTermEquals("the", filter, termAtt, tfAtt, 1);
|
||||
assertTermEquals("lazy", filter, termAtt, tfAtt, 2);
|
||||
assertTermEquals("brown", filter, termAtt, tfAtt, 123);
|
||||
assertTermEquals("dogs", filter, termAtt, tfAtt, 1024);
|
||||
assertFalse(filter.incrementToken());
|
||||
filter.end();
|
||||
filter.close();
|
||||
}
|
||||
|
||||
public void testInvalidNegativeTf() throws Exception {
|
||||
String test = "foo bar|-20";
|
||||
DelimitedTermFrequencyTokenFilter filter =
|
||||
new DelimitedTermFrequencyTokenFilter(whitespaceMockTokenizer(test));
|
||||
CharTermAttribute termAtt = filter.getAttribute(CharTermAttribute.class);
|
||||
TermFrequencyAttribute tfAtt = filter.getAttribute(TermFrequencyAttribute.class);
|
||||
filter.reset();
|
||||
assertTermEquals("foo", filter, termAtt, tfAtt, 1);
|
||||
IllegalArgumentException iae = expectThrows(IllegalArgumentException.class, filter::incrementToken);
|
||||
assertEquals("Term frequency must be 1 or greater; got -20", iae.getMessage());
|
||||
}
|
||||
|
||||
public void testInvalidFloatTf() throws Exception {
|
||||
String test = "foo bar|1.2";
|
||||
DelimitedTermFrequencyTokenFilter filter =
|
||||
new DelimitedTermFrequencyTokenFilter(whitespaceMockTokenizer(test));
|
||||
CharTermAttribute termAtt = filter.getAttribute(CharTermAttribute.class);
|
||||
TermFrequencyAttribute tfAtt = filter.getAttribute(TermFrequencyAttribute.class);
|
||||
filter.reset();
|
||||
assertTermEquals("foo", filter, termAtt, tfAtt, 1);
|
||||
expectThrows(NumberFormatException.class, filter::incrementToken);
|
||||
}
|
||||
|
||||
void assertTermEquals(String expected, TokenStream stream, CharTermAttribute termAtt, TermFrequencyAttribute tfAtt, int expectedTf) throws Exception {
|
||||
assertTrue(stream.incrementToken());
|
||||
assertEquals(expected, termAtt.toString());
|
||||
assertEquals(expectedTf, tfAtt.getTermFrequency());
|
||||
}
|
||||
}
|
|
@ -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.analysis.miscellaneous;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.StringReader;
|
||||
|
||||
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
|
||||
import org.apache.lucene.analysis.MockTokenizer;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.tokenattributes.FlagsAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
|
||||
import org.apache.lucene.util.AttributeFactory;
|
||||
|
||||
public class TestConcatenatingTokenStream extends BaseTokenStreamTestCase {
|
||||
|
||||
public void testBasic() throws IOException {
|
||||
|
||||
AttributeFactory factory = newAttributeFactory();
|
||||
|
||||
final MockTokenizer first = new MockTokenizer(factory, MockTokenizer.WHITESPACE, false);
|
||||
first.setReader(new StringReader("first words "));
|
||||
final MockTokenizer second = new MockTokenizer(factory, MockTokenizer.WHITESPACE, false);
|
||||
second.setReader(new StringReader("second words"));
|
||||
final MockTokenizer third = new MockTokenizer(factory, MockTokenizer.WHITESPACE, false);
|
||||
third.setReader(new StringReader(" third words"));
|
||||
|
||||
TokenStream ts = new ConcatenatingTokenStream(first, second, new EmptyTokenStream(), third);
|
||||
assertTokenStreamContents(ts,
|
||||
new String[] { "first", "words", "second", "words", "third", "words" },
|
||||
new int[]{ 0, 6, 12, 19, 25, 31 },
|
||||
new int[]{ 5, 11, 18, 24, 30, 36 });
|
||||
|
||||
}
|
||||
|
||||
public void testInconsistentAttributes() throws IOException {
|
||||
|
||||
AttributeFactory factory = newAttributeFactory();
|
||||
|
||||
final MockTokenizer first = new MockTokenizer(factory, MockTokenizer.WHITESPACE, false);
|
||||
first.setReader(new StringReader("first words "));
|
||||
first.addAttribute(PayloadAttribute.class);
|
||||
final MockTokenizer second = new MockTokenizer(factory, MockTokenizer.WHITESPACE, false);
|
||||
second.setReader(new StringReader("second words"));
|
||||
second.addAttribute(FlagsAttribute.class);
|
||||
|
||||
TokenStream ts = new ConcatenatingTokenStream(first, second);
|
||||
assertTrue(ts.hasAttribute(FlagsAttribute.class));
|
||||
assertTrue(ts.hasAttribute(PayloadAttribute.class));
|
||||
|
||||
assertTokenStreamContents(ts,
|
||||
new String[] { "first", "words", "second", "words" },
|
||||
new int[]{ 0, 6, 12, 19, },
|
||||
new int[]{ 5, 11, 18, 24, });
|
||||
|
||||
}
|
||||
|
||||
public void testInconsistentAttributeFactories() throws IOException {
|
||||
|
||||
final MockTokenizer first = new MockTokenizer(AttributeFactory.DEFAULT_ATTRIBUTE_FACTORY, MockTokenizer.WHITESPACE, true);
|
||||
final MockTokenizer second = new MockTokenizer(TokenStream.DEFAULT_TOKEN_ATTRIBUTE_FACTORY, MockTokenizer.WHITESPACE, true);
|
||||
|
||||
expectThrows(IllegalArgumentException.class, () -> new ConcatenatingTokenStream(first, second));
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -1,49 +0,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.
|
||||
*/
|
||||
package org.apache.lucene.analysis.miscellaneous;
|
||||
|
||||
|
||||
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
|
||||
import org.apache.lucene.analysis.CannedTokenStream;
|
||||
import org.apache.lucene.analysis.MockTokenizer;
|
||||
import org.apache.lucene.analysis.Token;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.StringReader;
|
||||
|
||||
public class TestPrefixAndSuffixAwareTokenFilter extends BaseTokenStreamTestCase {
|
||||
|
||||
public void test() throws IOException {
|
||||
|
||||
final MockTokenizer input = new MockTokenizer(MockTokenizer.WHITESPACE, false);
|
||||
input.setReader(new StringReader("hello world"));
|
||||
PrefixAndSuffixAwareTokenFilter ts = new PrefixAndSuffixAwareTokenFilter(
|
||||
new CannedTokenStream(createToken("^", 0, 0)),
|
||||
input,
|
||||
new CannedTokenStream(createToken("$", 0, 0)));
|
||||
|
||||
assertTokenStreamContents(ts,
|
||||
new String[] { "^", "hello", "world", "$" },
|
||||
new int[] { 0, 0, 6, 11 },
|
||||
new int[] { 0, 5, 11, 11 });
|
||||
}
|
||||
|
||||
private static Token createToken(String term, int start, int offset)
|
||||
{
|
||||
return new Token(term, start, offset);
|
||||
}
|
||||
}
|
|
@ -1,60 +0,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.
|
||||
*/
|
||||
package org.apache.lucene.analysis.miscellaneous;
|
||||
|
||||
|
||||
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
|
||||
import org.apache.lucene.analysis.CannedTokenStream;
|
||||
import org.apache.lucene.analysis.MockTokenizer;
|
||||
import org.apache.lucene.analysis.Token;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.StringReader;
|
||||
|
||||
public class TestPrefixAwareTokenFilter extends BaseTokenStreamTestCase {
|
||||
|
||||
public void test() throws IOException {
|
||||
|
||||
PrefixAwareTokenFilter ts;
|
||||
|
||||
ts = new PrefixAwareTokenFilter(
|
||||
new CannedTokenStream(createToken("a", 0, 1)),
|
||||
new CannedTokenStream(createToken("b", 0, 1)));
|
||||
assertTokenStreamContents(ts,
|
||||
new String[] { "a", "b" },
|
||||
new int[] { 0, 1 },
|
||||
new int[] { 1, 2 });
|
||||
|
||||
// prefix and suffix using 2x prefix
|
||||
|
||||
final MockTokenizer suffix = new MockTokenizer(MockTokenizer.WHITESPACE, false);
|
||||
suffix.setReader(new StringReader("hello world"));
|
||||
ts = new PrefixAwareTokenFilter(new CannedTokenStream(createToken("^", 0, 0)),
|
||||
suffix);
|
||||
ts = new PrefixAwareTokenFilter(ts, new CannedTokenStream(createToken("$", 0, 0)));
|
||||
|
||||
assertTokenStreamContents(ts,
|
||||
new String[] { "^", "hello", "world", "$" },
|
||||
new int[] { 0, 0, 6, 11 },
|
||||
new int[] { 0, 5, 11, 11 });
|
||||
}
|
||||
|
||||
private static Token createToken(String term, int start, int offset)
|
||||
{
|
||||
return new Token(term, start, offset);
|
||||
}
|
||||
}
|
|
@ -17,34 +17,90 @@
|
|||
package org.apache.lucene.analysis.wikipedia;
|
||||
|
||||
|
||||
import java.io.Reader;
|
||||
import java.io.StringReader;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.analysis.Tokenizer;
|
||||
import org.apache.lucene.analysis.util.BaseTokenStreamFactoryTestCase;
|
||||
import org.apache.lucene.analysis.wikipedia.WikipediaTokenizer;
|
||||
|
||||
/**
|
||||
* Simple tests to ensure the wikipedia tokenizer is working.
|
||||
*/
|
||||
public class TestWikipediaTokenizerFactory extends BaseTokenStreamFactoryTestCase {
|
||||
|
||||
private final String WIKIPEDIA = "Wikipedia";
|
||||
private final String TOKEN_OUTPUT = "tokenOutput";
|
||||
private final String UNTOKENIZED_TYPES = "untokenizedTypes";
|
||||
|
||||
public void testTokenizer() throws Exception {
|
||||
Reader reader = new StringReader("This is a [[Category:foo]]");
|
||||
Tokenizer tokenizer = tokenizerFactory("Wikipedia").create(newAttributeFactory());
|
||||
tokenizer.setReader(reader);
|
||||
assertTokenStreamContents(tokenizer,
|
||||
new String[] { "This", "is", "a", "foo" },
|
||||
new int[] { 0, 5, 8, 21 },
|
||||
new int[] { 4, 7, 9, 24 },
|
||||
new String[] { "<ALPHANUM>", "<ALPHANUM>", "<ALPHANUM>", WikipediaTokenizer.CATEGORY },
|
||||
new int[] { 1, 1, 1, 1, });
|
||||
String text = "This is a [[Category:foo]]";
|
||||
Tokenizer tf = tokenizerFactory(WIKIPEDIA).create(newAttributeFactory());
|
||||
tf.setReader(new StringReader(text));
|
||||
assertTokenStreamContents(tf,
|
||||
new String[] { "This", "is", "a", "foo" },
|
||||
new int[] { 0, 5, 8, 21 },
|
||||
new int[] { 4, 7, 9, 24 },
|
||||
new String[] { "<ALPHANUM>", "<ALPHANUM>", "<ALPHANUM>", WikipediaTokenizer.CATEGORY },
|
||||
new int[] { 1, 1, 1, 1, },
|
||||
text.length());
|
||||
}
|
||||
|
||||
|
||||
public void testTokenizerTokensOnly() throws Exception {
|
||||
String text = "This is a [[Category:foo]]";
|
||||
Tokenizer tf = tokenizerFactory(WIKIPEDIA, TOKEN_OUTPUT, new Integer( WikipediaTokenizer.TOKENS_ONLY).toString()).create(newAttributeFactory());
|
||||
tf.setReader(new StringReader(text));
|
||||
assertTokenStreamContents(tf,
|
||||
new String[] { "This", "is", "a", "foo" },
|
||||
new int[] { 0, 5, 8, 21 },
|
||||
new int[] { 4, 7, 9, 24 },
|
||||
new String[] { "<ALPHANUM>", "<ALPHANUM>", "<ALPHANUM>", WikipediaTokenizer.CATEGORY },
|
||||
new int[] { 1, 1, 1, 1, },
|
||||
text.length());
|
||||
}
|
||||
|
||||
public void testTokenizerUntokenizedOnly() throws Exception {
|
||||
String test = "[[Category:a b c d]] [[Category:e f g]] [[link here]] [[link there]] ''italics here'' something ''more italics'' [[Category:h i j]]";
|
||||
Set<String> untoks = new HashSet<>();
|
||||
untoks.add(WikipediaTokenizer.CATEGORY);
|
||||
untoks.add(WikipediaTokenizer.ITALICS);
|
||||
Tokenizer tf = tokenizerFactory(WIKIPEDIA, TOKEN_OUTPUT, new Integer(WikipediaTokenizer.UNTOKENIZED_ONLY).toString(), UNTOKENIZED_TYPES, WikipediaTokenizer.CATEGORY + ", " + WikipediaTokenizer.ITALICS).create(newAttributeFactory());
|
||||
tf.setReader(new StringReader(test));
|
||||
assertTokenStreamContents(tf,
|
||||
new String[] { "a b c d", "e f g", "link", "here", "link",
|
||||
"there", "italics here", "something", "more italics", "h i j" },
|
||||
new int[] { 11, 32, 42, 47, 56, 61, 71, 86, 98, 124 },
|
||||
new int[] { 18, 37, 46, 51, 60, 66, 83, 95, 110, 133 },
|
||||
new int[] { 1, 1, 1, 1, 1, 1, 1, 1, 1, 1 }
|
||||
);
|
||||
}
|
||||
|
||||
public void testTokenizerBoth() throws Exception {
|
||||
String test = "[[Category:a b c d]] [[Category:e f g]] [[link here]] [[link there]] ''italics here'' something ''more italics'' [[Category:h i j]]";
|
||||
Tokenizer tf = tokenizerFactory(WIKIPEDIA, TOKEN_OUTPUT, new Integer(WikipediaTokenizer.BOTH).toString(), UNTOKENIZED_TYPES, WikipediaTokenizer.CATEGORY + ", " + WikipediaTokenizer.ITALICS).create(newAttributeFactory());
|
||||
tf.setReader(new StringReader(test));
|
||||
assertTokenStreamContents(tf,
|
||||
new String[] { "a b c d", "a", "b", "c", "d", "e f g", "e", "f", "g",
|
||||
"link", "here", "link", "there", "italics here", "italics", "here",
|
||||
"something", "more italics", "more", "italics", "h i j", "h", "i", "j" },
|
||||
new int[] { 11, 11, 13, 15, 17, 32, 32, 34, 36, 42, 47, 56, 61, 71, 71, 79, 86, 98, 98, 103, 124, 124, 128, 132 },
|
||||
new int[] { 18, 12, 14, 16, 18, 37, 33, 35, 37, 46, 51, 60, 66, 83, 78, 83, 95, 110, 102, 110, 133, 125, 129, 133 },
|
||||
new int[] { 1, 0, 1, 1, 1, 1, 0, 1, 1, 1, 1, 1, 1, 1, 0, 1, 1, 1, 0, 1, 1, 0, 1, 1 }
|
||||
);
|
||||
}
|
||||
|
||||
/** Test that bogus arguments result in exception */
|
||||
public void testBogusArguments() throws Exception {
|
||||
IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
|
||||
tokenizerFactory("Wikipedia", "bogusArg", "bogusValue");
|
||||
tokenizerFactory(WIKIPEDIA, "bogusArg", "bogusValue").create(newAttributeFactory());
|
||||
});
|
||||
assertTrue(expected.getMessage().contains("Unknown parameters"));
|
||||
}
|
||||
}
|
||||
|
||||
public void testIllegalArguments() throws Exception {
|
||||
IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
|
||||
Tokenizer tf = tokenizerFactory(WIKIPEDIA, TOKEN_OUTPUT, "-1").create(newAttributeFactory());
|
||||
});
|
||||
assertTrue(expected.getMessage().contains("tokenOutput must be TOKENS_ONLY, UNTOKENIZED_ONLY or BOTH"));
|
||||
}
|
||||
}
|
|
@ -21,13 +21,17 @@ import java.io.IOException;
|
|||
import java.io.Reader;
|
||||
import java.lang.reflect.Constructor;
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
|
||||
import org.apache.lucene.analysis.MockTokenizer;
|
||||
import org.apache.lucene.analysis.Tokenizer;
|
||||
import org.apache.lucene.analysis.miscellaneous.DelimitedTermFrequencyTokenFilterFactory;
|
||||
import org.apache.lucene.analysis.util.AbstractAnalysisFactory;
|
||||
import org.apache.lucene.analysis.util.CharFilterFactory;
|
||||
import org.apache.lucene.analysis.util.MultiTermAwareComponent;
|
||||
|
@ -46,6 +50,12 @@ import org.apache.lucene.util.Version;
|
|||
|
||||
// TODO: fix this to use CustomAnalyzer instead of its own FactoryAnalyzer
|
||||
public class TestFactories extends BaseTokenStreamTestCase {
|
||||
|
||||
/** Factories that are excluded from testing it with random data */
|
||||
private static final Set<Class<? extends AbstractAnalysisFactory>> EXCLUDE_FACTORIES_RANDOM_DATA = new HashSet<>(Arrays.asList(
|
||||
DelimitedTermFrequencyTokenFilterFactory.class
|
||||
));
|
||||
|
||||
public void test() throws IOException {
|
||||
for (String tokenizer : TokenizerFactory.availableTokenizers()) {
|
||||
doTestTokenizer(tokenizer);
|
||||
|
@ -74,11 +84,13 @@ public class TestFactories extends BaseTokenStreamTestCase {
|
|||
assertFalse(mtc instanceof CharFilterFactory);
|
||||
}
|
||||
|
||||
// beast it just a little, it shouldnt throw exceptions:
|
||||
// (it should have thrown them in initialize)
|
||||
Analyzer a = new FactoryAnalyzer(factory, null, null);
|
||||
checkRandomData(random(), a, 20, 20, false, false);
|
||||
a.close();
|
||||
if (!EXCLUDE_FACTORIES_RANDOM_DATA.contains(factory.getClass())) {
|
||||
// beast it just a little, it shouldnt throw exceptions:
|
||||
// (it should have thrown them in initialize)
|
||||
Analyzer a = new FactoryAnalyzer(factory, null, null);
|
||||
checkRandomData(random(), a, 20, 20, false, false);
|
||||
a.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -96,11 +108,13 @@ public class TestFactories extends BaseTokenStreamTestCase {
|
|||
assertTrue(mtc instanceof TokenFilterFactory);
|
||||
}
|
||||
|
||||
// beast it just a little, it shouldnt throw exceptions:
|
||||
// (it should have thrown them in initialize)
|
||||
Analyzer a = new FactoryAnalyzer(assertingTokenizer, factory, null);
|
||||
checkRandomData(random(), a, 20, 20, false, false);
|
||||
a.close();
|
||||
if (!EXCLUDE_FACTORIES_RANDOM_DATA.contains(factory.getClass())) {
|
||||
// beast it just a little, it shouldnt throw exceptions:
|
||||
// (it should have thrown them in initialize)
|
||||
Analyzer a = new FactoryAnalyzer(assertingTokenizer, factory, null);
|
||||
checkRandomData(random(), a, 20, 20, false, false);
|
||||
a.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -118,11 +132,13 @@ public class TestFactories extends BaseTokenStreamTestCase {
|
|||
assertTrue(mtc instanceof CharFilterFactory);
|
||||
}
|
||||
|
||||
// beast it just a little, it shouldnt throw exceptions:
|
||||
// (it should have thrown them in initialize)
|
||||
Analyzer a = new FactoryAnalyzer(assertingTokenizer, null, factory);
|
||||
checkRandomData(random(), a, 20, 20, false, false);
|
||||
a.close();
|
||||
if (!EXCLUDE_FACTORIES_RANDOM_DATA.contains(factory.getClass())) {
|
||||
// beast it just a little, it shouldnt throw exceptions:
|
||||
// (it should have thrown them in initialize)
|
||||
Analyzer a = new FactoryAnalyzer(assertingTokenizer, null, factory);
|
||||
checkRandomData(random(), a, 20, 20, false, false);
|
||||
a.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -297,7 +297,9 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
|
|||
"6.5.0-cfs",
|
||||
"6.5.0-nocfs",
|
||||
"6.5.1-cfs",
|
||||
"6.5.1-nocfs"
|
||||
"6.5.1-nocfs",
|
||||
"6.6.0-cfs",
|
||||
"6.6.0-nocfs"
|
||||
};
|
||||
|
||||
final String[] unsupportedNames = {
|
||||
|
@ -1190,7 +1192,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
|
|||
for (String name : oldNames) {
|
||||
Directory dir = oldIndexDirs.get(name);
|
||||
IndexReader r = DirectoryReader.open(dir);
|
||||
TermsEnum terms = MultiFields.getFields(r).terms("content").iterator();
|
||||
TermsEnum terms = MultiFields.getTerms(r, "content").iterator();
|
||||
BytesRef t = terms.next();
|
||||
assertNotNull(t);
|
||||
|
||||
|
|
Binary file not shown.
Binary file not shown.
|
@ -21,16 +21,14 @@ import java.util.ArrayList;
|
|||
import java.util.List;
|
||||
import java.util.Properties;
|
||||
|
||||
import org.locationtech.spatial4j.shape.Shape;
|
||||
import org.apache.lucene.benchmark.byTask.utils.Config;
|
||||
import org.apache.lucene.queries.function.FunctionQuery;
|
||||
import org.apache.lucene.queries.function.ValueSource;
|
||||
import org.apache.lucene.search.BooleanClause;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
import org.apache.lucene.queries.function.FunctionScoreQuery;
|
||||
import org.apache.lucene.search.DoubleValuesSource;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.spatial.SpatialStrategy;
|
||||
import org.apache.lucene.spatial.query.SpatialArgs;
|
||||
import org.apache.lucene.spatial.query.SpatialOperation;
|
||||
import org.locationtech.spatial4j.shape.Shape;
|
||||
|
||||
/**
|
||||
* Reads spatial data from the body field docs from an internally created {@link LineDocSource}.
|
||||
|
@ -102,11 +100,8 @@ public class SpatialFileQueryMaker extends AbstractQueryMaker {
|
|||
Query filterQuery = strategy.makeQuery(args);
|
||||
if (score) {
|
||||
//wrap with distance computing query
|
||||
ValueSource valueSource = strategy.makeDistanceValueSource(shape.getCenter());
|
||||
return new BooleanQuery.Builder()
|
||||
.add(new FunctionQuery(valueSource), BooleanClause.Occur.MUST)//matches everything and provides score
|
||||
.add(filterQuery, BooleanClause.Occur.FILTER)//filters (score isn't used)
|
||||
.build();
|
||||
DoubleValuesSource valueSource = strategy.makeDistanceValueSource(shape.getCenter());
|
||||
return new FunctionScoreQuery(filterQuery, valueSource);
|
||||
} else {
|
||||
return filterQuery; // assume constant scoring
|
||||
}
|
||||
|
|
|
@ -28,7 +28,6 @@
|
|||
<path refid="base.classpath"/>
|
||||
<pathelement path="${queries.jar}"/>
|
||||
<pathelement path="${grouping.jar}"/>
|
||||
<pathelement path="${sandbox.jar}"/>
|
||||
<pathelement path="${analyzers-common.jar}"/>
|
||||
</path>
|
||||
|
||||
|
@ -38,18 +37,17 @@
|
|||
<path refid="test.base.classpath"/>
|
||||
</path>
|
||||
|
||||
<target name="compile-core" depends="jar-sandbox,jar-grouping,jar-queries,jar-analyzers-common,common.compile-core" />
|
||||
<target name="compile-core" depends="jar-grouping,jar-queries,jar-analyzers-common,common.compile-core" />
|
||||
|
||||
<target name="jar-core" depends="common.jar-core" />
|
||||
|
||||
<target name="javadocs" depends="javadocs-sandbox,javadocs-grouping,compile-core,check-javadocs-uptodate"
|
||||
<target name="javadocs" depends="javadocs-grouping,compile-core,check-javadocs-uptodate"
|
||||
unless="javadocs-uptodate-${name}">
|
||||
<invoke-module-javadoc>
|
||||
<links>
|
||||
<link href="../queries"/>
|
||||
<link href="../analyzers-common"/>
|
||||
<link href="../grouping"/>
|
||||
<link href="../sandbox"/>
|
||||
</links>
|
||||
</invoke-module-javadoc>
|
||||
</target>
|
||||
|
|
|
@ -25,11 +25,11 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.classification.utils.NearestFuzzyQuery;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexableField;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.sandbox.queries.FuzzyLikeThisQuery;
|
||||
import org.apache.lucene.search.BooleanClause;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
|
@ -42,7 +42,7 @@ import org.apache.lucene.search.similarities.Similarity;
|
|||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/**
|
||||
* A k-Nearest Neighbor classifier based on {@link FuzzyLikeThisQuery}.
|
||||
* A k-Nearest Neighbor classifier based on {@link NearestFuzzyQuery}.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
|
@ -51,27 +51,27 @@ public class KNearestFuzzyClassifier implements Classifier<BytesRef> {
|
|||
/**
|
||||
* the name of the fields used as the input text
|
||||
*/
|
||||
protected final String[] textFieldNames;
|
||||
private final String[] textFieldNames;
|
||||
|
||||
/**
|
||||
* the name of the field used as the output text
|
||||
*/
|
||||
protected final String classFieldName;
|
||||
private final String classFieldName;
|
||||
|
||||
/**
|
||||
* an {@link IndexSearcher} used to perform queries
|
||||
*/
|
||||
protected final IndexSearcher indexSearcher;
|
||||
private final IndexSearcher indexSearcher;
|
||||
|
||||
/**
|
||||
* the no. of docs to compare in order to find the nearest neighbor to the input text
|
||||
*/
|
||||
protected final int k;
|
||||
private final int k;
|
||||
|
||||
/**
|
||||
* a {@link Query} used to filter the documents that should be used from this classifier's underlying {@link LeafReader}
|
||||
*/
|
||||
protected final Query query;
|
||||
private final Query query;
|
||||
private final Analyzer analyzer;
|
||||
|
||||
/**
|
||||
|
@ -145,11 +145,11 @@ public class KNearestFuzzyClassifier implements Classifier<BytesRef> {
|
|||
|
||||
private TopDocs knnSearch(String text) throws IOException {
|
||||
BooleanQuery.Builder bq = new BooleanQuery.Builder();
|
||||
FuzzyLikeThisQuery fuzzyLikeThisQuery = new FuzzyLikeThisQuery(300, analyzer);
|
||||
NearestFuzzyQuery nearestFuzzyQuery = new NearestFuzzyQuery(analyzer);
|
||||
for (String fieldName : textFieldNames) {
|
||||
fuzzyLikeThisQuery.addTerms(text, fieldName, 1f, 2); // TODO: make this parameters configurable
|
||||
nearestFuzzyQuery.addTerms(text, fieldName);
|
||||
}
|
||||
bq.add(fuzzyLikeThisQuery, BooleanClause.Occur.MUST);
|
||||
bq.add(nearestFuzzyQuery, BooleanClause.Occur.MUST);
|
||||
Query classFieldQuery = new WildcardQuery(new Term(classFieldName, "*"));
|
||||
bq.add(new BooleanClause(classFieldQuery, BooleanClause.Occur.MUST));
|
||||
if (query != null) {
|
||||
|
@ -165,7 +165,7 @@ public class KNearestFuzzyClassifier implements Classifier<BytesRef> {
|
|||
* @return a {@link List} of {@link ClassificationResult}, one for each existing class
|
||||
* @throws IOException if it's not possible to get the stored value of class field
|
||||
*/
|
||||
protected List<ClassificationResult<BytesRef>> buildListFromTopDocs(TopDocs topDocs) throws IOException {
|
||||
private List<ClassificationResult<BytesRef>> buildListFromTopDocs(TopDocs topDocs) throws IOException {
|
||||
Map<BytesRef, Integer> classCounts = new HashMap<>();
|
||||
Map<BytesRef, Double> classBoosts = new HashMap<>(); // this is a boost based on class ranking positions in topDocs
|
||||
float maxScore = topDocs.getMaxScore();
|
||||
|
@ -174,12 +174,7 @@ public class KNearestFuzzyClassifier implements Classifier<BytesRef> {
|
|||
if (storableField != null) {
|
||||
BytesRef cl = new BytesRef(storableField.stringValue());
|
||||
//update count
|
||||
Integer count = classCounts.get(cl);
|
||||
if (count != null) {
|
||||
classCounts.put(cl, count + 1);
|
||||
} else {
|
||||
classCounts.put(cl, 1);
|
||||
}
|
||||
classCounts.merge(cl, 1, (a, b) -> a + b);
|
||||
//update boost, the boost is based on the best score
|
||||
Double totalBoost = classBoosts.get(cl);
|
||||
double singleBoost = scoreDoc.score / maxScore;
|
||||
|
|
|
@ -122,7 +122,7 @@ public class DatasetSplitter {
|
|||
|
||||
// iterate over existing documents
|
||||
for (GroupDocs<Object> group : topGroups.groups) {
|
||||
int totalHits = group.totalHits;
|
||||
long totalHits = group.totalHits;
|
||||
double testSize = totalHits * testRatio;
|
||||
int tc = 0;
|
||||
double cvSize = totalHits * crossValidationRatio;
|
||||
|
|
|
@ -0,0 +1,333 @@
|
|||
/*
|
||||
* 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.classification.utils;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Objects;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.MultiFields;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.TermContext;
|
||||
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.BoostAttribute;
|
||||
import org.apache.lucene.search.BoostQuery;
|
||||
import org.apache.lucene.search.FuzzyTermsEnum;
|
||||
import org.apache.lucene.search.MaxNonCompetitiveBoostAttribute;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.util.AttributeSource;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.PriorityQueue;
|
||||
import org.apache.lucene.util.automaton.LevenshteinAutomata;
|
||||
|
||||
/**
|
||||
* Simplification of FuzzyLikeThisQuery, to be used in the context of KNN classification.
|
||||
*/
|
||||
public class NearestFuzzyQuery extends Query {
|
||||
|
||||
private final ArrayList<FieldVals> fieldVals = new ArrayList<>();
|
||||
private final Analyzer analyzer;
|
||||
|
||||
// fixed parameters
|
||||
private static final int MAX_VARIANTS_PER_TERM = 50;
|
||||
private static final float MIN_SIMILARITY = 1f;
|
||||
private static final int PREFIX_LENGTH = 2;
|
||||
private static final int MAX_NUM_TERMS = 300;
|
||||
|
||||
/**
|
||||
* Default constructor
|
||||
*
|
||||
* @param analyzer the analyzer used to proecss the query text
|
||||
*/
|
||||
public NearestFuzzyQuery(Analyzer analyzer) {
|
||||
this.analyzer = analyzer;
|
||||
}
|
||||
|
||||
static class FieldVals {
|
||||
final String queryString;
|
||||
final String fieldName;
|
||||
final int maxEdits;
|
||||
final int prefixLength;
|
||||
|
||||
FieldVals(String name, int maxEdits, String queryString) {
|
||||
this.fieldName = name;
|
||||
this.maxEdits = maxEdits;
|
||||
this.queryString = queryString;
|
||||
this.prefixLength = NearestFuzzyQuery.PREFIX_LENGTH;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
final int prime = 31;
|
||||
int result = 1;
|
||||
result = prime * result
|
||||
+ ((fieldName == null) ? 0 : fieldName.hashCode());
|
||||
result = prime * result + maxEdits;
|
||||
result = prime * result + prefixLength;
|
||||
result = prime * result
|
||||
+ ((queryString == null) ? 0 : queryString.hashCode());
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (this == obj)
|
||||
return true;
|
||||
if (obj == null)
|
||||
return false;
|
||||
if (getClass() != obj.getClass())
|
||||
return false;
|
||||
FieldVals other = (FieldVals) obj;
|
||||
if (fieldName == null) {
|
||||
if (other.fieldName != null)
|
||||
return false;
|
||||
} else if (!fieldName.equals(other.fieldName))
|
||||
return false;
|
||||
if (maxEdits != other.maxEdits) {
|
||||
return false;
|
||||
}
|
||||
if (prefixLength != other.prefixLength)
|
||||
return false;
|
||||
if (queryString == null) {
|
||||
if (other.queryString != null)
|
||||
return false;
|
||||
} else if (!queryString.equals(other.queryString))
|
||||
return false;
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds user input for "fuzzification"
|
||||
*
|
||||
* @param queryString The string which will be parsed by the analyzer and for which fuzzy variants will be parsed
|
||||
*/
|
||||
public void addTerms(String queryString, String fieldName) {
|
||||
int maxEdits = (int) MIN_SIMILARITY;
|
||||
if (maxEdits != MIN_SIMILARITY) {
|
||||
throw new IllegalArgumentException("MIN_SIMILARITY must integer value between 0 and " + LevenshteinAutomata.MAXIMUM_SUPPORTED_DISTANCE + ", inclusive; got " + MIN_SIMILARITY);
|
||||
}
|
||||
fieldVals.add(new FieldVals(fieldName, maxEdits, queryString));
|
||||
}
|
||||
|
||||
|
||||
private void addTerms(IndexReader reader, FieldVals f, ScoreTermQueue q) throws IOException {
|
||||
if (f.queryString == null) return;
|
||||
final Terms terms = MultiFields.getTerms(reader, f.fieldName);
|
||||
if (terms == null) {
|
||||
return;
|
||||
}
|
||||
try (TokenStream ts = analyzer.tokenStream(f.fieldName, f.queryString)) {
|
||||
CharTermAttribute termAtt = ts.addAttribute(CharTermAttribute.class);
|
||||
|
||||
int corpusNumDocs = reader.numDocs();
|
||||
HashSet<String> processedTerms = new HashSet<>();
|
||||
ts.reset();
|
||||
while (ts.incrementToken()) {
|
||||
String term = termAtt.toString();
|
||||
if (!processedTerms.contains(term)) {
|
||||
processedTerms.add(term);
|
||||
ScoreTermQueue variantsQ = new ScoreTermQueue(MAX_VARIANTS_PER_TERM); //maxNum variants considered for any one term
|
||||
float minScore = 0;
|
||||
Term startTerm = new Term(f.fieldName, term);
|
||||
AttributeSource atts = new AttributeSource();
|
||||
MaxNonCompetitiveBoostAttribute maxBoostAtt =
|
||||
atts.addAttribute(MaxNonCompetitiveBoostAttribute.class);
|
||||
FuzzyTermsEnum fe = new FuzzyTermsEnum(terms, atts, startTerm, f.maxEdits, f.prefixLength, true);
|
||||
//store the df so all variants use same idf
|
||||
int df = reader.docFreq(startTerm);
|
||||
int numVariants = 0;
|
||||
int totalVariantDocFreqs = 0;
|
||||
BytesRef possibleMatch;
|
||||
BoostAttribute boostAtt =
|
||||
fe.attributes().addAttribute(BoostAttribute.class);
|
||||
while ((possibleMatch = fe.next()) != null) {
|
||||
numVariants++;
|
||||
totalVariantDocFreqs += fe.docFreq();
|
||||
float score = boostAtt.getBoost();
|
||||
if (variantsQ.size() < MAX_VARIANTS_PER_TERM || score > minScore) {
|
||||
ScoreTerm st = new ScoreTerm(new Term(startTerm.field(), BytesRef.deepCopyOf(possibleMatch)), score, startTerm);
|
||||
variantsQ.insertWithOverflow(st);
|
||||
minScore = variantsQ.top().score; // maintain minScore
|
||||
}
|
||||
maxBoostAtt.setMaxNonCompetitiveBoost(variantsQ.size() >= MAX_VARIANTS_PER_TERM ? minScore : Float.NEGATIVE_INFINITY);
|
||||
}
|
||||
|
||||
if (numVariants > 0) {
|
||||
int avgDf = totalVariantDocFreqs / numVariants;
|
||||
if (df == 0)//no direct match we can use as df for all variants
|
||||
{
|
||||
df = avgDf; //use avg df of all variants
|
||||
}
|
||||
|
||||
// take the top variants (scored by edit distance) and reset the score
|
||||
// to include an IDF factor then add to the global queue for ranking
|
||||
// overall top query terms
|
||||
int size = variantsQ.size();
|
||||
for (int i = 0; i < size; i++) {
|
||||
ScoreTerm st = variantsQ.pop();
|
||||
if (st != null) {
|
||||
st.score = (st.score * st.score) * idf(df, corpusNumDocs);
|
||||
q.insertWithOverflow(st);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
ts.end();
|
||||
}
|
||||
}
|
||||
|
||||
private float idf(int docFreq, int docCount) {
|
||||
return (float)(Math.log((docCount+1)/(double)(docFreq+1)) + 1.0);
|
||||
}
|
||||
|
||||
private Query newTermQuery(IndexReader reader, Term term) throws IOException {
|
||||
// we build an artificial TermContext that will give an overall df and ttf
|
||||
// equal to 1
|
||||
TermContext context = new TermContext(reader.getContext());
|
||||
for (LeafReaderContext leafContext : reader.leaves()) {
|
||||
Terms terms = leafContext.reader().terms(term.field());
|
||||
if (terms != null) {
|
||||
TermsEnum termsEnum = terms.iterator();
|
||||
if (termsEnum.seekExact(term.bytes())) {
|
||||
int freq = 1 - context.docFreq(); // we want the total df and ttf to be 1
|
||||
context.register(termsEnum.termState(), leafContext.ord, freq, freq);
|
||||
}
|
||||
}
|
||||
}
|
||||
return new TermQuery(term, context);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Query rewrite(IndexReader reader) throws IOException {
|
||||
ScoreTermQueue q = new ScoreTermQueue(MAX_NUM_TERMS);
|
||||
//load up the list of possible terms
|
||||
for (FieldVals f : fieldVals) {
|
||||
addTerms(reader, f, q);
|
||||
}
|
||||
|
||||
BooleanQuery.Builder bq = new BooleanQuery.Builder();
|
||||
|
||||
//create BooleanQueries to hold the variants for each token/field pair and ensure it
|
||||
// has no coord factor
|
||||
//Step 1: sort the termqueries by term/field
|
||||
HashMap<Term, ArrayList<ScoreTerm>> variantQueries = new HashMap<>();
|
||||
int size = q.size();
|
||||
for (int i = 0; i < size; i++) {
|
||||
ScoreTerm st = q.pop();
|
||||
if (st != null) {
|
||||
ArrayList<ScoreTerm> l = variantQueries.computeIfAbsent(st.fuzziedSourceTerm, k -> new ArrayList<>());
|
||||
l.add(st);
|
||||
}
|
||||
}
|
||||
//Step 2: Organize the sorted termqueries into zero-coord scoring boolean queries
|
||||
for (ArrayList<ScoreTerm> variants : variantQueries.values()) {
|
||||
if (variants.size() == 1) {
|
||||
//optimize where only one selected variant
|
||||
ScoreTerm st = variants.get(0);
|
||||
Query tq = newTermQuery(reader, st.term);
|
||||
// set the boost to a mix of IDF and score
|
||||
bq.add(new BoostQuery(tq, st.score), BooleanClause.Occur.SHOULD);
|
||||
} else {
|
||||
BooleanQuery.Builder termVariants = new BooleanQuery.Builder();
|
||||
for (ScoreTerm st : variants) {
|
||||
// found a match
|
||||
Query tq = newTermQuery(reader, st.term);
|
||||
// set the boost using the ScoreTerm's score
|
||||
termVariants.add(new BoostQuery(tq, st.score), BooleanClause.Occur.SHOULD); // add to query
|
||||
}
|
||||
bq.add(termVariants.build(), BooleanClause.Occur.SHOULD); // add to query
|
||||
}
|
||||
}
|
||||
//TODO possible alternative step 3 - organize above booleans into a new layer of field-based
|
||||
// booleans with a minimum-should-match of NumFields-1?
|
||||
return bq.build();
|
||||
}
|
||||
|
||||
//Holds info for a fuzzy term variant - initially score is set to edit distance (for ranking best
|
||||
// term variants) then is reset with IDF for use in ranking against all other
|
||||
// terms/fields
|
||||
private static class ScoreTerm {
|
||||
public final Term term;
|
||||
public float score;
|
||||
final Term fuzziedSourceTerm;
|
||||
|
||||
ScoreTerm(Term term, float score, Term fuzziedSourceTerm) {
|
||||
this.term = term;
|
||||
this.score = score;
|
||||
this.fuzziedSourceTerm = fuzziedSourceTerm;
|
||||
}
|
||||
}
|
||||
|
||||
private static class ScoreTermQueue extends PriorityQueue<ScoreTerm> {
|
||||
ScoreTermQueue(int size) {
|
||||
super(size);
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.apache.lucene.util.PriorityQueue#lessThan(java.lang.Object, java.lang.Object)
|
||||
*/
|
||||
@Override
|
||||
protected boolean lessThan(ScoreTerm termA, ScoreTerm termB) {
|
||||
if (termA.score == termB.score)
|
||||
return termA.term.compareTo(termB.term) > 0;
|
||||
else
|
||||
return termA.score < termB.score;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString(String field) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
int prime = 31;
|
||||
int result = classHash();
|
||||
result = prime * result + Objects.hashCode(analyzer);
|
||||
result = prime * result + Objects.hashCode(fieldVals);
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object other) {
|
||||
return sameClassAs(other) &&
|
||||
equalsTo(getClass().cast(other));
|
||||
}
|
||||
|
||||
private boolean equalsTo(NearestFuzzyQuery other) {
|
||||
return Objects.equals(analyzer, other.analyzer) &&
|
||||
Objects.equals(fieldVals, other.fieldVals);
|
||||
}
|
||||
|
||||
}
|
|
@ -28,7 +28,7 @@ import org.apache.lucene.util.BytesRef;
|
|||
import org.junit.Test;
|
||||
|
||||
/**
|
||||
* Testcase for {@link KNearestFuzzyClassifier}
|
||||
* Tests for {@link KNearestFuzzyClassifier}
|
||||
*/
|
||||
public class KNearestFuzzyClassifierTest extends ClassificationTestBase<BytesRef> {
|
||||
|
||||
|
|
|
@ -1076,7 +1076,7 @@
|
|||
<propertyref prefix="tests.leaveTemporary" />
|
||||
<propertyref prefix="tests.leavetemporary" />
|
||||
<propertyref prefix="solr.test.leavetmpdir" />
|
||||
<propertyref prefix="solr.tests.preferPointFields"/>
|
||||
<propertyref prefix="solr.tests.use.numeric.points" />
|
||||
</syspropertyset>
|
||||
|
||||
<!-- Pass randomized settings to the forked JVM. -->
|
||||
|
|
|
@ -235,7 +235,7 @@ public abstract class Analyzer implements Closeable {
|
|||
}
|
||||
filteredText = builder.toString();
|
||||
} catch (IOException e) {
|
||||
throw new IllegalStateException("Normalization threw an unexpected exeption", e);
|
||||
throw new IllegalStateException("Normalization threw an unexpected exception", e);
|
||||
}
|
||||
|
||||
final AttributeFactory attributeFactory = attributeFactory(fieldName);
|
||||
|
@ -258,7 +258,7 @@ public abstract class Analyzer implements Closeable {
|
|||
return term;
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new IllegalStateException("Normalization threw an unexpected exeption", e);
|
||||
throw new IllegalStateException("Normalization threw an unexpected exception", e);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -40,12 +40,7 @@ import org.apache.lucene.util.AttributeSource;
|
|||
* <li>{@link TokenFilter}, a <code>TokenStream</code> whose input is another
|
||||
* <code>TokenStream</code>.
|
||||
* </ul>
|
||||
* A new <code>TokenStream</code> API has been introduced with Lucene 2.9. This API
|
||||
* has moved from being {@link Token}-based to {@link Attribute}-based. While
|
||||
* {@link Token} still exists in 2.9 as a convenience class, the preferred way
|
||||
* to store the information of a {@link Token} is to use {@link AttributeImpl}s.
|
||||
* <p>
|
||||
* <code>TokenStream</code> now extends {@link AttributeSource}, which provides
|
||||
* <code>TokenStream</code> extends {@link AttributeSource}, which provides
|
||||
* access to all of the token {@link Attribute}s for the <code>TokenStream</code>.
|
||||
* Note that only one instance per {@link AttributeImpl} is created and reused
|
||||
* for every token. This approach reduces object creation and allows local
|
||||
|
|
|
@ -253,11 +253,6 @@ public class CharTermAttributeImpl extends AttributeImpl implements CharTermAttr
|
|||
/**
|
||||
* Returns solely the term text as specified by the
|
||||
* {@link CharSequence} interface.
|
||||
* <p>This method changed the behavior with Lucene 3.1,
|
||||
* before it returned a String representation of the whole
|
||||
* term with all attributes.
|
||||
* This affects especially the
|
||||
* {@link org.apache.lucene.analysis.Token} subclass.
|
||||
*/
|
||||
@Override
|
||||
public String toString() {
|
||||
|
|
|
@ -26,15 +26,18 @@ import org.apache.lucene.util.AttributeReflector;
|
|||
* <li>{@link PositionIncrementAttribute}
|
||||
* <li>{@link PositionLengthAttribute}
|
||||
* <li>{@link OffsetAttribute}
|
||||
* <li>{@link TermFrequencyAttribute}
|
||||
* </ul>*/
|
||||
public class PackedTokenAttributeImpl extends CharTermAttributeImpl
|
||||
implements TypeAttribute, PositionIncrementAttribute,
|
||||
PositionLengthAttribute, OffsetAttribute {
|
||||
PositionLengthAttribute, OffsetAttribute,
|
||||
TermFrequencyAttribute {
|
||||
|
||||
private int startOffset,endOffset;
|
||||
private String type = DEFAULT_TYPE;
|
||||
private int positionIncrement = 1;
|
||||
private int positionLength = 1;
|
||||
private int termFrequency = 1;
|
||||
|
||||
/** Constructs the attribute implementation. */
|
||||
public PackedTokenAttributeImpl() {
|
||||
|
@ -132,12 +135,26 @@ public class PackedTokenAttributeImpl extends CharTermAttributeImpl
|
|||
this.type = type;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final void setTermFrequency(int termFrequency) {
|
||||
if (termFrequency < 1) {
|
||||
throw new IllegalArgumentException("Term frequency must be 1 or greater; got " + termFrequency);
|
||||
}
|
||||
this.termFrequency = termFrequency;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final int getTermFrequency() {
|
||||
return termFrequency;
|
||||
}
|
||||
|
||||
/** Resets the attributes
|
||||
*/
|
||||
@Override
|
||||
public void clear() {
|
||||
super.clear();
|
||||
positionIncrement = positionLength = 1;
|
||||
termFrequency = 1;
|
||||
startOffset = endOffset = 0;
|
||||
type = DEFAULT_TYPE;
|
||||
}
|
||||
|
@ -147,10 +164,8 @@ public class PackedTokenAttributeImpl extends CharTermAttributeImpl
|
|||
@Override
|
||||
public void end() {
|
||||
super.end();
|
||||
// super.end already calls this.clear, so we only set values that are different from clear:
|
||||
positionIncrement = 0;
|
||||
positionLength = 1;
|
||||
startOffset = endOffset = 0;
|
||||
type = DEFAULT_TYPE;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -170,6 +185,7 @@ public class PackedTokenAttributeImpl extends CharTermAttributeImpl
|
|||
positionIncrement == other.positionIncrement &&
|
||||
positionLength == other.positionLength &&
|
||||
(type == null ? other.type == null : type.equals(other.type)) &&
|
||||
termFrequency == other.termFrequency &&
|
||||
super.equals(obj)
|
||||
);
|
||||
} else
|
||||
|
@ -185,6 +201,7 @@ public class PackedTokenAttributeImpl extends CharTermAttributeImpl
|
|||
code = code * 31 + positionLength;
|
||||
if (type != null)
|
||||
code = code * 31 + type.hashCode();
|
||||
code = code * 31 + termFrequency;;
|
||||
return code;
|
||||
}
|
||||
|
||||
|
@ -198,12 +215,14 @@ public class PackedTokenAttributeImpl extends CharTermAttributeImpl
|
|||
to.startOffset = startOffset;
|
||||
to.endOffset = endOffset;
|
||||
to.type = type;
|
||||
to.termFrequency = termFrequency;
|
||||
} else {
|
||||
super.copyTo(target);
|
||||
((OffsetAttribute) target).setOffset(startOffset, endOffset);
|
||||
((PositionIncrementAttribute) target).setPositionIncrement(positionIncrement);
|
||||
((PositionLengthAttribute) target).setPositionLength(positionLength);
|
||||
((TypeAttribute) target).setType(type);
|
||||
((TermFrequencyAttribute) target).setTermFrequency(termFrequency);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -215,6 +234,6 @@ public class PackedTokenAttributeImpl extends CharTermAttributeImpl
|
|||
reflector.reflect(PositionIncrementAttribute.class, "positionIncrement", positionIncrement);
|
||||
reflector.reflect(PositionLengthAttribute.class, "positionLength", positionLength);
|
||||
reflector.reflect(TypeAttribute.class, "type", type);
|
||||
reflector.reflect(TermFrequencyAttribute.class, "termFrequency", termFrequency);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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.analysis.tokenattributes;
|
||||
|
||||
import org.apache.lucene.index.IndexOptions;
|
||||
import org.apache.lucene.util.Attribute;
|
||||
|
||||
/** Sets the custom term frequency of a term within one document. If this attribute
|
||||
* is present in your analysis chain for a given field, that field must be indexed with
|
||||
* {@link IndexOptions#DOCS_AND_FREQS}. */
|
||||
public interface TermFrequencyAttribute extends Attribute {
|
||||
|
||||
/** Set the custom term frequency of the current term within one document. */
|
||||
public void setTermFrequency(int termFrequency);
|
||||
|
||||
/** Returns the custom term frequencey. */
|
||||
public int getTermFrequency();
|
||||
}
|
|
@ -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.analysis.tokenattributes;
|
||||
|
||||
|
||||
import org.apache.lucene.util.AttributeImpl;
|
||||
import org.apache.lucene.util.AttributeReflector;
|
||||
|
||||
/** Default implementation of {@link TermFrequencyAttribute}. */
|
||||
public class TermFrequencyAttributeImpl extends AttributeImpl implements TermFrequencyAttribute, Cloneable {
|
||||
private int termFrequency = 1;
|
||||
|
||||
/** Initialize this attribute with term frequencey of 1 */
|
||||
public TermFrequencyAttributeImpl() {}
|
||||
|
||||
@Override
|
||||
public void setTermFrequency(int termFrequency) {
|
||||
if (termFrequency < 1) {
|
||||
throw new IllegalArgumentException("Term frequency must be 1 or greater; got " + termFrequency);
|
||||
}
|
||||
this.termFrequency = termFrequency;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getTermFrequency() {
|
||||
return termFrequency;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clear() {
|
||||
this.termFrequency = 1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void end() {
|
||||
this.termFrequency = 1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object other) {
|
||||
if (other == this) {
|
||||
return true;
|
||||
}
|
||||
|
||||
if (other instanceof TermFrequencyAttributeImpl) {
|
||||
TermFrequencyAttributeImpl _other = (TermFrequencyAttributeImpl) other;
|
||||
return termFrequency == _other.termFrequency;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Integer.hashCode(termFrequency);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void copyTo(AttributeImpl target) {
|
||||
TermFrequencyAttribute t = (TermFrequencyAttribute) target;
|
||||
t.setTermFrequency(termFrequency);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void reflectWith(AttributeReflector reflector) {
|
||||
reflector.reflect(TermFrequencyAttribute.class, "termFrequency", termFrequency);
|
||||
}
|
||||
}
|
|
@ -121,12 +121,6 @@ public final class BlockTreeTermsReader extends FieldsProducer {
|
|||
|
||||
private final TreeMap<String,FieldReader> fields = new TreeMap<>();
|
||||
|
||||
/** File offset where the directory starts in the terms file. */
|
||||
private long dirOffset;
|
||||
|
||||
/** File offset where the directory starts in the index file. */
|
||||
private long indexDirOffset;
|
||||
|
||||
final String segment;
|
||||
|
||||
final int version;
|
||||
|
@ -167,8 +161,8 @@ public final class BlockTreeTermsReader extends FieldsProducer {
|
|||
CodecUtil.retrieveChecksum(termsIn);
|
||||
|
||||
// Read per-field details
|
||||
seekDir(termsIn, dirOffset);
|
||||
seekDir(indexIn, indexDirOffset);
|
||||
seekDir(termsIn);
|
||||
seekDir(indexIn);
|
||||
|
||||
final int numFields = termsIn.readVInt();
|
||||
if (numFields < 0) {
|
||||
|
@ -181,13 +175,7 @@ public final class BlockTreeTermsReader extends FieldsProducer {
|
|||
if (numTerms <= 0) {
|
||||
throw new CorruptIndexException("Illegal numTerms for field number: " + field, termsIn);
|
||||
}
|
||||
final int numBytes = termsIn.readVInt();
|
||||
if (numBytes < 0) {
|
||||
throw new CorruptIndexException("invalid rootCode for field number: " + field + ", numBytes=" + numBytes, termsIn);
|
||||
}
|
||||
final BytesRef rootCode = new BytesRef(new byte[numBytes]);
|
||||
termsIn.readBytes(rootCode.bytes, 0, numBytes);
|
||||
rootCode.length = numBytes;
|
||||
final BytesRef rootCode = readBytesRef(termsIn);
|
||||
final FieldInfo fieldInfo = state.fieldInfos.fieldInfo(field);
|
||||
if (fieldInfo == null) {
|
||||
throw new CorruptIndexException("invalid field number: " + field, termsIn);
|
||||
|
@ -230,19 +218,24 @@ public final class BlockTreeTermsReader extends FieldsProducer {
|
|||
}
|
||||
|
||||
private static BytesRef readBytesRef(IndexInput in) throws IOException {
|
||||
int numBytes = in.readVInt();
|
||||
if (numBytes < 0) {
|
||||
throw new CorruptIndexException("invalid bytes length: " + numBytes, in);
|
||||
}
|
||||
|
||||
BytesRef bytes = new BytesRef();
|
||||
bytes.length = in.readVInt();
|
||||
bytes.bytes = new byte[bytes.length];
|
||||
in.readBytes(bytes.bytes, 0, bytes.length);
|
||||
bytes.length = numBytes;
|
||||
bytes.bytes = new byte[numBytes];
|
||||
in.readBytes(bytes.bytes, 0, numBytes);
|
||||
|
||||
return bytes;
|
||||
}
|
||||
|
||||
/** Seek {@code input} to the directory offset. */
|
||||
private void seekDir(IndexInput input, long dirOffset)
|
||||
throws IOException {
|
||||
private static void seekDir(IndexInput input) throws IOException {
|
||||
input.seek(input.length() - CodecUtil.footerLength() - 8);
|
||||
dirOffset = input.readLong();
|
||||
input.seek(dirOffset);
|
||||
long offset = input.readLong();
|
||||
input.seek(offset);
|
||||
}
|
||||
|
||||
// for debugging
|
||||
|
|
|
@ -19,22 +19,20 @@ package org.apache.lucene.document;
|
|||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Objects;
|
||||
import java.util.function.IntPredicate;
|
||||
import java.util.function.Predicate;
|
||||
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.PointValues;
|
||||
import org.apache.lucene.index.PointValues.Relation;
|
||||
import org.apache.lucene.index.PointValues.IntersectVisitor;
|
||||
import org.apache.lucene.index.PointValues.Relation;
|
||||
import org.apache.lucene.search.ConstantScoreScorer;
|
||||
import org.apache.lucene.search.ConstantScoreWeight;
|
||||
import org.apache.lucene.search.DocIdSet;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
import org.apache.lucene.search.ScorerSupplier;
|
||||
import org.apache.lucene.search.Weight;
|
||||
import org.apache.lucene.util.DocIdSetBuilder;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
|
@ -60,13 +58,167 @@ abstract class RangeFieldQuery extends Query {
|
|||
/** Used by {@code RangeFieldQuery} to check how each internal or leaf node relates to the query. */
|
||||
enum QueryType {
|
||||
/** Use this for intersects queries. */
|
||||
INTERSECTS,
|
||||
INTERSECTS {
|
||||
|
||||
@Override
|
||||
Relation compare(byte[] queryPackedValue, byte[] minPackedValue, byte[] maxPackedValue,
|
||||
int numDims, int bytesPerDim, int dim) {
|
||||
int minOffset = dim * bytesPerDim;
|
||||
int maxOffset = minOffset + bytesPerDim * numDims;
|
||||
|
||||
if (StringHelper.compare(bytesPerDim, queryPackedValue, maxOffset, minPackedValue, minOffset) < 0
|
||||
|| StringHelper.compare(bytesPerDim, queryPackedValue, minOffset, maxPackedValue, maxOffset) > 0) {
|
||||
// disjoint
|
||||
return Relation.CELL_OUTSIDE_QUERY;
|
||||
}
|
||||
|
||||
if (StringHelper.compare(bytesPerDim, queryPackedValue, maxOffset, maxPackedValue, minOffset) >= 0
|
||||
&& StringHelper.compare(bytesPerDim, queryPackedValue, minOffset, minPackedValue, maxOffset) <= 0) {
|
||||
return Relation.CELL_INSIDE_QUERY;
|
||||
}
|
||||
|
||||
return Relation.CELL_CROSSES_QUERY;
|
||||
}
|
||||
|
||||
@Override
|
||||
boolean matches(byte[] queryPackedValue, byte[] packedValue, int numDims, int bytesPerDim, int dim) {
|
||||
int minOffset = dim * bytesPerDim;
|
||||
int maxOffset = minOffset + bytesPerDim * numDims;
|
||||
return StringHelper.compare(bytesPerDim, queryPackedValue, maxOffset, packedValue, minOffset) >= 0
|
||||
&& StringHelper.compare(bytesPerDim, queryPackedValue, minOffset, packedValue, maxOffset) <= 0;
|
||||
}
|
||||
|
||||
},
|
||||
/** Use this for within queries. */
|
||||
WITHIN,
|
||||
WITHIN {
|
||||
|
||||
@Override
|
||||
Relation compare(byte[] queryPackedValue, byte[] minPackedValue, byte[] maxPackedValue,
|
||||
int numDims, int bytesPerDim, int dim) {
|
||||
int minOffset = dim * bytesPerDim;
|
||||
int maxOffset = minOffset + bytesPerDim * numDims;
|
||||
|
||||
if (StringHelper.compare(bytesPerDim, queryPackedValue, maxOffset, minPackedValue, maxOffset) < 0
|
||||
|| StringHelper.compare(bytesPerDim, queryPackedValue, minOffset, maxPackedValue, minOffset) > 0) {
|
||||
// all ranges have at least one point outside of the query
|
||||
return Relation.CELL_OUTSIDE_QUERY;
|
||||
}
|
||||
|
||||
if (StringHelper.compare(bytesPerDim, queryPackedValue, maxOffset, maxPackedValue, maxOffset) >= 0
|
||||
&& StringHelper.compare(bytesPerDim, queryPackedValue, minOffset, minPackedValue, minOffset) <= 0) {
|
||||
return Relation.CELL_INSIDE_QUERY;
|
||||
}
|
||||
|
||||
return Relation.CELL_CROSSES_QUERY;
|
||||
}
|
||||
|
||||
@Override
|
||||
boolean matches(byte[] queryPackedValue, byte[] packedValue, int numDims, int bytesPerDim, int dim) {
|
||||
int minOffset = dim * bytesPerDim;
|
||||
int maxOffset = minOffset + bytesPerDim * numDims;
|
||||
return StringHelper.compare(bytesPerDim, queryPackedValue, minOffset, packedValue, minOffset) <= 0
|
||||
&& StringHelper.compare(bytesPerDim, queryPackedValue, maxOffset, packedValue, maxOffset) >= 0;
|
||||
}
|
||||
|
||||
},
|
||||
/** Use this for contains */
|
||||
CONTAINS,
|
||||
CONTAINS {
|
||||
|
||||
@Override
|
||||
Relation compare(byte[] queryPackedValue, byte[] minPackedValue, byte[] maxPackedValue,
|
||||
int numDims, int bytesPerDim, int dim) {
|
||||
int minOffset = dim * bytesPerDim;
|
||||
int maxOffset = minOffset + bytesPerDim * numDims;
|
||||
|
||||
if (StringHelper.compare(bytesPerDim, queryPackedValue, maxOffset, maxPackedValue, maxOffset) > 0
|
||||
|| StringHelper.compare(bytesPerDim, queryPackedValue, minOffset, minPackedValue, minOffset) < 0) {
|
||||
// all ranges are either less than the query max or greater than the query min
|
||||
return Relation.CELL_OUTSIDE_QUERY;
|
||||
}
|
||||
|
||||
if (StringHelper.compare(bytesPerDim, queryPackedValue, maxOffset, minPackedValue, maxOffset) <= 0
|
||||
&& StringHelper.compare(bytesPerDim, queryPackedValue, minOffset, maxPackedValue, minOffset) >= 0) {
|
||||
return Relation.CELL_INSIDE_QUERY;
|
||||
}
|
||||
|
||||
return Relation.CELL_CROSSES_QUERY;
|
||||
}
|
||||
|
||||
@Override
|
||||
boolean matches(byte[] queryPackedValue, byte[] packedValue, int numDims, int bytesPerDim, int dim) {
|
||||
int minOffset = dim * bytesPerDim;
|
||||
int maxOffset = minOffset + bytesPerDim * numDims;
|
||||
return StringHelper.compare(bytesPerDim, queryPackedValue, minOffset, packedValue, minOffset) >= 0
|
||||
&& StringHelper.compare(bytesPerDim, queryPackedValue, maxOffset, packedValue, maxOffset) <= 0;
|
||||
}
|
||||
|
||||
},
|
||||
/** Use this for crosses queries */
|
||||
CROSSES
|
||||
CROSSES {
|
||||
|
||||
@Override
|
||||
Relation compare(byte[] queryPackedValue, byte[] minPackedValue, byte[] maxPackedValue,
|
||||
int numDims, int bytesPerDim, int dim) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
boolean matches(byte[] queryPackedValue, byte[] packedValue, int numDims, int bytesPerDim, int dim) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
Relation compare(byte[] queryPackedValue, byte[] minPackedValue, byte[] maxPackedValue,
|
||||
int numDims, int bytesPerDim) {
|
||||
Relation intersectRelation = QueryType.INTERSECTS.compare(queryPackedValue, minPackedValue, maxPackedValue, numDims, bytesPerDim);
|
||||
if (intersectRelation == Relation.CELL_OUTSIDE_QUERY) {
|
||||
return Relation.CELL_OUTSIDE_QUERY;
|
||||
}
|
||||
|
||||
Relation withinRelation = QueryType.WITHIN.compare(queryPackedValue, minPackedValue, maxPackedValue, numDims, bytesPerDim);
|
||||
if (withinRelation == Relation.CELL_INSIDE_QUERY) {
|
||||
return Relation.CELL_OUTSIDE_QUERY;
|
||||
}
|
||||
|
||||
if (intersectRelation == Relation.CELL_INSIDE_QUERY && withinRelation == Relation.CELL_OUTSIDE_QUERY) {
|
||||
return Relation.CELL_INSIDE_QUERY;
|
||||
}
|
||||
|
||||
return Relation.CELL_CROSSES_QUERY;
|
||||
}
|
||||
|
||||
boolean matches(byte[] queryPackedValue, byte[] packedValue, int numDims, int bytesPerDim) {
|
||||
return INTERSECTS.matches(queryPackedValue, packedValue, numDims, bytesPerDim)
|
||||
&& WITHIN.matches(queryPackedValue, packedValue, numDims, bytesPerDim) == false;
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
abstract Relation compare(byte[] queryPackedValue, byte[] minPackedValue, byte[] maxPackedValue, int numDims, int bytesPerDim, int dim);
|
||||
|
||||
Relation compare(byte[] queryPackedValue, byte[] minPackedValue, byte[] maxPackedValue, int numDims, int bytesPerDim) {
|
||||
boolean inside = true;
|
||||
for (int dim = 0; dim < numDims; ++dim) {
|
||||
Relation relation = compare(queryPackedValue, minPackedValue, maxPackedValue, numDims, bytesPerDim, dim);
|
||||
if (relation == Relation.CELL_OUTSIDE_QUERY) {
|
||||
return Relation.CELL_OUTSIDE_QUERY;
|
||||
} else if (relation != Relation.CELL_INSIDE_QUERY) {
|
||||
inside = false;
|
||||
}
|
||||
}
|
||||
return inside ? Relation.CELL_INSIDE_QUERY : Relation.CELL_CROSSES_QUERY;
|
||||
}
|
||||
|
||||
abstract boolean matches(byte[] queryPackedValue, byte[] packedValue, int numDims, int bytesPerDim, int dim);
|
||||
|
||||
boolean matches(byte[] queryPackedValue, byte[] packedValue, int numDims, int bytesPerDim) {
|
||||
for (int dim = 0; dim < numDims; ++dim) {
|
||||
if (matches(queryPackedValue, packedValue, numDims, bytesPerDim, dim) == false) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -111,54 +263,33 @@ abstract class RangeFieldQuery extends Query {
|
|||
@Override
|
||||
public final Weight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException {
|
||||
return new ConstantScoreWeight(this, boost) {
|
||||
final RangeFieldComparator target = new RangeFieldComparator();
|
||||
|
||||
private DocIdSet buildMatchingDocIdSet(LeafReader reader, PointValues values) throws IOException {
|
||||
DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc(), values, field);
|
||||
values.intersect(
|
||||
new IntersectVisitor() {
|
||||
DocIdSetBuilder.BulkAdder adder;
|
||||
@Override
|
||||
public void grow(int count) {
|
||||
adder = result.grow(count);
|
||||
}
|
||||
@Override
|
||||
public void visit(int docID) throws IOException {
|
||||
adder.add(docID);
|
||||
}
|
||||
@Override
|
||||
public void visit(int docID, byte[] leaf) throws IOException {
|
||||
if (target.matches(leaf)) {
|
||||
adder.add(docID);
|
||||
}
|
||||
}
|
||||
@Override
|
||||
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
return compareRange(minPackedValue, maxPackedValue);
|
||||
}
|
||||
});
|
||||
return result.build();
|
||||
}
|
||||
|
||||
private Relation compareRange(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
byte[] node = getInternalRange(minPackedValue, maxPackedValue);
|
||||
// compute range relation for BKD traversal
|
||||
if (target.intersects(node) == false) {
|
||||
return Relation.CELL_OUTSIDE_QUERY;
|
||||
} else if (target.within(node)) {
|
||||
// target within cell; continue traversing:
|
||||
return Relation.CELL_CROSSES_QUERY;
|
||||
} else if (target.contains(node)) {
|
||||
// target contains cell; add iff queryType is not a CONTAINS or CROSSES query:
|
||||
return (queryType == QueryType.CONTAINS || queryType == QueryType.CROSSES) ?
|
||||
Relation.CELL_OUTSIDE_QUERY : Relation.CELL_INSIDE_QUERY;
|
||||
}
|
||||
// target intersects cell; continue traversing:
|
||||
return Relation.CELL_CROSSES_QUERY;
|
||||
private IntersectVisitor getIntersectVisitor(DocIdSetBuilder result) {
|
||||
return new IntersectVisitor() {
|
||||
DocIdSetBuilder.BulkAdder adder;
|
||||
@Override
|
||||
public void grow(int count) {
|
||||
adder = result.grow(count);
|
||||
}
|
||||
@Override
|
||||
public void visit(int docID) throws IOException {
|
||||
adder.add(docID);
|
||||
}
|
||||
@Override
|
||||
public void visit(int docID, byte[] leaf) throws IOException {
|
||||
if (queryType.matches(ranges, leaf, numDims, bytesPerDim)) {
|
||||
adder.add(docID);
|
||||
}
|
||||
}
|
||||
@Override
|
||||
public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
|
||||
return queryType.compare(ranges, minPackedValue, maxPackedValue, numDims, bytesPerDim);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(LeafReaderContext context) throws IOException {
|
||||
public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException {
|
||||
LeafReader reader = context.reader();
|
||||
PointValues values = reader.getPointValues(field);
|
||||
if (values == null) {
|
||||
|
@ -173,115 +304,59 @@ abstract class RangeFieldQuery extends Query {
|
|||
checkFieldInfo(fieldInfo);
|
||||
boolean allDocsMatch = false;
|
||||
if (values.getDocCount() == reader.maxDoc()
|
||||
&& compareRange(values.getMinPackedValue(), values.getMaxPackedValue()) == Relation.CELL_INSIDE_QUERY) {
|
||||
&& queryType.compare(ranges, values.getMinPackedValue(), values.getMaxPackedValue(), numDims, bytesPerDim) == Relation.CELL_INSIDE_QUERY) {
|
||||
allDocsMatch = true;
|
||||
}
|
||||
|
||||
DocIdSetIterator iterator = allDocsMatch == true ?
|
||||
DocIdSetIterator.all(reader.maxDoc()) : buildMatchingDocIdSet(reader, values).iterator();
|
||||
return new ConstantScoreScorer(this, score(), iterator);
|
||||
}
|
||||
final Weight weight = this;
|
||||
if (allDocsMatch) {
|
||||
return new ScorerSupplier() {
|
||||
@Override
|
||||
public Scorer get(boolean randomAccess) {
|
||||
return new ConstantScoreScorer(weight, score(), DocIdSetIterator.all(reader.maxDoc()));
|
||||
}
|
||||
|
||||
/** get an encoded byte representation of the internal node; this is
|
||||
* the lower half of the min array and the upper half of the max array */
|
||||
private byte[] getInternalRange(byte[] min, byte[] max) {
|
||||
byte[] range = new byte[min.length];
|
||||
final int dimSize = numDims * bytesPerDim;
|
||||
System.arraycopy(min, 0, range, 0, dimSize);
|
||||
System.arraycopy(max, dimSize, range, dimSize, dimSize);
|
||||
return range;
|
||||
}
|
||||
};
|
||||
}
|
||||
@Override
|
||||
public long cost() {
|
||||
return reader.maxDoc();
|
||||
}
|
||||
};
|
||||
} else {
|
||||
return new ScorerSupplier() {
|
||||
|
||||
/**
|
||||
* RangeFieldComparator class provides the core comparison logic for accepting or rejecting indexed
|
||||
* {@code RangeField} types based on the defined query range and relation.
|
||||
*/
|
||||
class RangeFieldComparator {
|
||||
final Predicate<byte[]> predicate;
|
||||
final DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc(), values, field);
|
||||
final IntersectVisitor visitor = getIntersectVisitor(result);
|
||||
long cost = -1;
|
||||
|
||||
/** constructs the comparator based on the query type */
|
||||
RangeFieldComparator() {
|
||||
switch (queryType) {
|
||||
case INTERSECTS:
|
||||
predicate = this::intersects;
|
||||
break;
|
||||
case WITHIN:
|
||||
predicate = this::contains;
|
||||
break;
|
||||
case CONTAINS:
|
||||
predicate = this::within;
|
||||
break;
|
||||
case CROSSES:
|
||||
// crosses first checks intersection (disjoint automatic fails),
|
||||
// then ensures the query doesn't wholly contain the leaf:
|
||||
predicate = (byte[] leaf) -> this.intersects(leaf)
|
||||
&& this.contains(leaf) == false;
|
||||
break;
|
||||
default:
|
||||
throw new IllegalArgumentException("invalid queryType [" + queryType + "] found.");
|
||||
}
|
||||
}
|
||||
@Override
|
||||
public Scorer get(boolean randomAccess) throws IOException {
|
||||
values.intersect(visitor);
|
||||
DocIdSetIterator iterator = result.build().iterator();
|
||||
return new ConstantScoreScorer(weight, score(), iterator);
|
||||
}
|
||||
|
||||
/** determines if the candidate range matches the query request */
|
||||
private boolean matches(final byte[] candidate) {
|
||||
return (Arrays.equals(ranges, candidate) && queryType != QueryType.CROSSES)
|
||||
|| predicate.test(candidate);
|
||||
}
|
||||
|
||||
/** check if query intersects candidate range */
|
||||
private boolean intersects(final byte[] candidate) {
|
||||
return relate((int d) -> compareMinMax(candidate, d) > 0 || compareMaxMin(candidate, d) < 0);
|
||||
}
|
||||
|
||||
/** check if query is within candidate range */
|
||||
private boolean within(final byte[] candidate) {
|
||||
return relate((int d) -> compareMinMin(candidate, d) < 0 || compareMaxMax(candidate, d) > 0);
|
||||
}
|
||||
|
||||
/** check if query contains candidate range */
|
||||
private boolean contains(final byte[] candidate) {
|
||||
return relate((int d) -> compareMinMin(candidate, d) > 0 || compareMaxMax(candidate, d) < 0);
|
||||
}
|
||||
|
||||
/** internal method used by each relation method to test range relation logic */
|
||||
private boolean relate(IntPredicate predicate) {
|
||||
for (int d=0; d<numDims; ++d) {
|
||||
if (predicate.test(d)) {
|
||||
return false;
|
||||
@Override
|
||||
public long cost() {
|
||||
if (cost == -1) {
|
||||
// Computing the cost may be expensive, so only do it if necessary
|
||||
cost = values.estimatePointCount(visitor);
|
||||
assert cost >= 0;
|
||||
}
|
||||
return cost;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
/** compare the encoded min value (for the defined query dimension) with the encoded min value in the byte array */
|
||||
private int compareMinMin(byte[] b, int dimension) {
|
||||
// convert dimension to offset:
|
||||
dimension *= bytesPerDim;
|
||||
return StringHelper.compare(bytesPerDim, ranges, dimension, b, dimension);
|
||||
}
|
||||
|
||||
/** compare the encoded min value (for the defined query dimension) with the encoded max value in the byte array */
|
||||
private int compareMinMax(byte[] b, int dimension) {
|
||||
// convert dimension to offset:
|
||||
dimension *= bytesPerDim;
|
||||
return StringHelper.compare(bytesPerDim, ranges, dimension, b, numDims * bytesPerDim + dimension);
|
||||
}
|
||||
|
||||
/** compare the encoded max value (for the defined query dimension) with the encoded min value in the byte array */
|
||||
private int compareMaxMin(byte[] b, int dimension) {
|
||||
// convert dimension to offset:
|
||||
dimension *= bytesPerDim;
|
||||
return StringHelper.compare(bytesPerDim, ranges, numDims * bytesPerDim + dimension, b, dimension);
|
||||
}
|
||||
|
||||
/** compare the encoded max value (for the defined query dimension) with the encoded max value in the byte array */
|
||||
private int compareMaxMax(byte[] b, int dimension) {
|
||||
// convert dimension to max offset:
|
||||
dimension = numDims * bytesPerDim + dimension * bytesPerDim;
|
||||
return StringHelper.compare(bytesPerDim, ranges, dimension, b, dimension);
|
||||
}
|
||||
@Override
|
||||
public Scorer scorer(LeafReaderContext context) throws IOException {
|
||||
ScorerSupplier scorerSupplier = scorerSupplier(context);
|
||||
if (scorerSupplier == null) {
|
||||
return null;
|
||||
}
|
||||
return scorerSupplier.get(false);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -22,6 +22,7 @@ import org.apache.lucene.search.DocIdSetIterator;
|
|||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
import org.apache.lucene.util.InPlaceMergeSorter;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
import org.apache.lucene.util.packed.PagedGrowableWriter;
|
||||
import org.apache.lucene.util.packed.PagedMutable;
|
||||
|
@ -35,22 +36,24 @@ import org.apache.lucene.util.packed.PagedMutable;
|
|||
class BinaryDocValuesFieldUpdates extends DocValuesFieldUpdates {
|
||||
|
||||
final static class Iterator extends DocValuesFieldUpdates.Iterator {
|
||||
private final PagedGrowableWriter offsets;
|
||||
private final int size;
|
||||
private final PagedGrowableWriter offsets;
|
||||
private final PagedGrowableWriter lengths;
|
||||
private final PagedMutable docs;
|
||||
private long idx = 0; // long so we don't overflow if size == Integer.MAX_VALUE
|
||||
private int doc = -1;
|
||||
private final BytesRef value;
|
||||
private int offset, length;
|
||||
private final long delGen;
|
||||
|
||||
Iterator(int size, PagedGrowableWriter offsets, PagedGrowableWriter lengths,
|
||||
PagedMutable docs, BytesRef values) {
|
||||
PagedMutable docs, BytesRef values, long delGen) {
|
||||
this.offsets = offsets;
|
||||
this.size = size;
|
||||
this.lengths = lengths;
|
||||
this.docs = docs;
|
||||
value = values.clone();
|
||||
this.delGen = delGen;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -69,6 +72,7 @@ class BinaryDocValuesFieldUpdates extends DocValuesFieldUpdates {
|
|||
doc = (int) docs.get(idx);
|
||||
++idx;
|
||||
while (idx < size && docs.get(idx) == doc) {
|
||||
// scan forward to last update to this doc
|
||||
++idx;
|
||||
}
|
||||
// idx points to the "next" element
|
||||
|
@ -87,10 +91,8 @@ class BinaryDocValuesFieldUpdates extends DocValuesFieldUpdates {
|
|||
}
|
||||
|
||||
@Override
|
||||
void reset() {
|
||||
doc = -1;
|
||||
offset = -1;
|
||||
idx = 0;
|
||||
long delGen() {
|
||||
return delGen;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -100,18 +102,29 @@ class BinaryDocValuesFieldUpdates extends DocValuesFieldUpdates {
|
|||
private int size;
|
||||
private final int bitsPerValue;
|
||||
|
||||
public BinaryDocValuesFieldUpdates(String field, int maxDoc) {
|
||||
super(field, DocValuesType.BINARY);
|
||||
public BinaryDocValuesFieldUpdates(long delGen, String field, int maxDoc) {
|
||||
super(maxDoc, delGen, field, DocValuesType.BINARY);
|
||||
bitsPerValue = PackedInts.bitsRequired(maxDoc - 1);
|
||||
docs = new PagedMutable(1, PAGE_SIZE, bitsPerValue, PackedInts.COMPACT);
|
||||
offsets = new PagedGrowableWriter(1, PAGE_SIZE, 1, PackedInts.FAST);
|
||||
lengths = new PagedGrowableWriter(1, PAGE_SIZE, 1, PackedInts.FAST);
|
||||
values = new BytesRefBuilder();
|
||||
size = 0;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void add(int doc, Object value) {
|
||||
public int size() {
|
||||
return size;
|
||||
}
|
||||
|
||||
// NOTE: we fully consume the incoming BytesRef so caller is free to reuse it after we return:
|
||||
@Override
|
||||
synchronized public void add(int doc, Object value) {
|
||||
if (finished) {
|
||||
throw new IllegalStateException("already finished");
|
||||
}
|
||||
|
||||
assert doc < maxDoc: "doc=" + doc + " maxDoc=" + maxDoc;
|
||||
|
||||
// TODO: if the Sorter interface changes to take long indexes, we can remove that limitation
|
||||
if (size == Integer.MAX_VALUE) {
|
||||
throw new IllegalStateException("cannot support more than Integer.MAX_VALUE doc/value entries");
|
||||
|
@ -134,11 +147,19 @@ class BinaryDocValuesFieldUpdates extends DocValuesFieldUpdates {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Iterator iterator() {
|
||||
final PagedMutable docs = this.docs;
|
||||
final PagedGrowableWriter offsets = this.offsets;
|
||||
final PagedGrowableWriter lengths = this.lengths;
|
||||
final BytesRef values = this.values.get();
|
||||
public void finish() {
|
||||
if (finished) {
|
||||
throw new IllegalStateException("already finished");
|
||||
}
|
||||
finished = true;
|
||||
|
||||
// shrink wrap
|
||||
if (size < docs.size()) {
|
||||
docs = docs.resize(size);
|
||||
offsets = offsets.resize(size);
|
||||
lengths = lengths.resize(size);
|
||||
}
|
||||
|
||||
new InPlaceMergeSorter() {
|
||||
@Override
|
||||
protected void swap(int i, int j) {
|
||||
|
@ -157,36 +178,20 @@ class BinaryDocValuesFieldUpdates extends DocValuesFieldUpdates {
|
|||
|
||||
@Override
|
||||
protected int compare(int i, int j) {
|
||||
int x = (int) docs.get(i);
|
||||
int y = (int) docs.get(j);
|
||||
return (x < y) ? -1 : ((x == y) ? 0 : 1);
|
||||
// increasing docID order:
|
||||
// NOTE: we can have ties here, when the same docID was updated in the same segment, in which case we rely on sort being
|
||||
// stable and preserving original order so the last update to that docID wins
|
||||
return Integer.compare((int) docs.get(i), (int) docs.get(j));
|
||||
}
|
||||
}.sort(0, size);
|
||||
|
||||
return new Iterator(size, offsets, lengths, docs, values);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void merge(DocValuesFieldUpdates other) {
|
||||
BinaryDocValuesFieldUpdates otherUpdates = (BinaryDocValuesFieldUpdates) other;
|
||||
if (otherUpdates.size > Integer.MAX_VALUE - size) {
|
||||
throw new IllegalStateException(
|
||||
"cannot support more than Integer.MAX_VALUE doc/value entries; size="
|
||||
+ size + " other.size=" + otherUpdates.size);
|
||||
public Iterator iterator() {
|
||||
if (finished == false) {
|
||||
throw new IllegalStateException("call finish first");
|
||||
}
|
||||
final int newSize = size + otherUpdates.size;
|
||||
docs = docs.grow(newSize);
|
||||
offsets = offsets.grow(newSize);
|
||||
lengths = lengths.grow(newSize);
|
||||
for (int i = 0; i < otherUpdates.size; i++) {
|
||||
int doc = (int) otherUpdates.docs.get(i);
|
||||
docs.set(size, doc);
|
||||
offsets.set(size, values.length() + otherUpdates.offsets.get(i)); // correct relative offset
|
||||
lengths.set(size, otherUpdates.lengths.get(i));
|
||||
++size;
|
||||
}
|
||||
|
||||
values.append(otherUpdates.values);
|
||||
return new Iterator(size, offsets, lengths, docs, values.get(), delGen);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -195,13 +200,13 @@ class BinaryDocValuesFieldUpdates extends DocValuesFieldUpdates {
|
|||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesPerDoc() {
|
||||
long bytesPerDoc = (long) Math.ceil((double) (bitsPerValue) / 8); // docs
|
||||
final int capacity = estimateCapacity(size);
|
||||
bytesPerDoc += (long) Math.ceil((double) offsets.ramBytesUsed() / capacity); // offsets
|
||||
bytesPerDoc += (long) Math.ceil((double) lengths.ramBytesUsed() / capacity); // lengths
|
||||
bytesPerDoc += (long) Math.ceil((double) values.length() / size); // values
|
||||
return bytesPerDoc;
|
||||
public long ramBytesUsed() {
|
||||
return offsets.ramBytesUsed()
|
||||
+ lengths.ramBytesUsed()
|
||||
+ docs.ramBytesUsed()
|
||||
+ RamUsageEstimator.NUM_BYTES_OBJECT_HEADER
|
||||
+ 4 * RamUsageEstimator.NUM_BYTES_INT
|
||||
+ 5 * RamUsageEstimator.NUM_BYTES_OBJECT_REF
|
||||
+ values.bytes().length;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -30,11 +30,12 @@ import org.apache.lucene.index.DocValuesUpdate.NumericDocValuesUpdate;
|
|||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
/* Holds buffered deletes and updates, by docID, term or query for a
|
||||
* single segment. This is used to hold buffered pending
|
||||
* deletes and updates against the to-be-flushed segment. Once the
|
||||
* deletes and updates are pushed (on flush in DocumentsWriter), they
|
||||
* are converted to a FrozenBufferedUpdates instance. */
|
||||
/** Holds buffered deletes and updates, by docID, term or query for a
|
||||
* single segment. This is used to hold buffered pending
|
||||
* deletes and updates against the to-be-flushed segment. Once the
|
||||
* deletes and updates are pushed (on flush in DocumentsWriter), they
|
||||
* are converted to a {@link FrozenBufferedUpdates} instance and
|
||||
* pushed to the {@link BufferedUpdatesStream}. */
|
||||
|
||||
// NOTE: instances of this class are accessed either via a private
|
||||
// instance on DocumentWriterPerThread, or via sync'd code by
|
||||
|
@ -128,10 +129,9 @@ class BufferedUpdates {
|
|||
final AtomicInteger numNumericUpdates = new AtomicInteger();
|
||||
final AtomicInteger numBinaryUpdates = new AtomicInteger();
|
||||
|
||||
// TODO: rename thes three: put "deleted" prefix in front:
|
||||
final Map<Term,Integer> terms = new HashMap<>();
|
||||
final Map<Query,Integer> queries = new HashMap<>();
|
||||
final List<Integer> docIDs = new ArrayList<>();
|
||||
final Map<Term,Integer> deleteTerms = new HashMap<>();
|
||||
final Map<Query,Integer> deleteQueries = new HashMap<>();
|
||||
final List<Integer> deleteDocIDs = new ArrayList<>();
|
||||
|
||||
// Map<dvField,Map<updateTerm,NumericUpdate>>
|
||||
// For each field we keep an ordered list of NumericUpdates, key'd by the
|
||||
|
@ -169,19 +169,19 @@ class BufferedUpdates {
|
|||
@Override
|
||||
public String toString() {
|
||||
if (VERBOSE_DELETES) {
|
||||
return "gen=" + gen + " numTerms=" + numTermDeletes + ", terms=" + terms
|
||||
+ ", queries=" + queries + ", docIDs=" + docIDs + ", numericUpdates=" + numericUpdates
|
||||
return "gen=" + gen + " numTerms=" + numTermDeletes + ", deleteTerms=" + deleteTerms
|
||||
+ ", deleteQueries=" + deleteQueries + ", deleteDocIDs=" + deleteDocIDs + ", numericUpdates=" + numericUpdates
|
||||
+ ", binaryUpdates=" + binaryUpdates + ", bytesUsed=" + bytesUsed;
|
||||
} else {
|
||||
String s = "gen=" + gen;
|
||||
if (numTermDeletes.get() != 0) {
|
||||
s += " " + numTermDeletes.get() + " deleted terms (unique count=" + terms.size() + ")";
|
||||
s += " " + numTermDeletes.get() + " deleted terms (unique count=" + deleteTerms.size() + ")";
|
||||
}
|
||||
if (queries.size() != 0) {
|
||||
s += " " + queries.size() + " deleted queries";
|
||||
if (deleteQueries.size() != 0) {
|
||||
s += " " + deleteQueries.size() + " deleted queries";
|
||||
}
|
||||
if (docIDs.size() != 0) {
|
||||
s += " " + docIDs.size() + " deleted docIDs";
|
||||
if (deleteDocIDs.size() != 0) {
|
||||
s += " " + deleteDocIDs.size() + " deleted docIDs";
|
||||
}
|
||||
if (numNumericUpdates.get() != 0) {
|
||||
s += " " + numNumericUpdates.get() + " numeric updates (unique count=" + numericUpdates.size() + ")";
|
||||
|
@ -198,7 +198,7 @@ class BufferedUpdates {
|
|||
}
|
||||
|
||||
public void addQuery(Query query, int docIDUpto) {
|
||||
Integer current = queries.put(query, docIDUpto);
|
||||
Integer current = deleteQueries.put(query, docIDUpto);
|
||||
// increment bytes used only if the query wasn't added so far.
|
||||
if (current == null) {
|
||||
bytesUsed.addAndGet(BYTES_PER_DEL_QUERY);
|
||||
|
@ -206,12 +206,12 @@ class BufferedUpdates {
|
|||
}
|
||||
|
||||
public void addDocID(int docID) {
|
||||
docIDs.add(Integer.valueOf(docID));
|
||||
deleteDocIDs.add(Integer.valueOf(docID));
|
||||
bytesUsed.addAndGet(BYTES_PER_DEL_DOCID);
|
||||
}
|
||||
|
||||
public void addTerm(Term term, int docIDUpto) {
|
||||
Integer current = terms.get(term);
|
||||
Integer current = deleteTerms.get(term);
|
||||
if (current != null && docIDUpto < current) {
|
||||
// Only record the new number if it's greater than the
|
||||
// current one. This is important because if multiple
|
||||
|
@ -223,7 +223,7 @@ class BufferedUpdates {
|
|||
return;
|
||||
}
|
||||
|
||||
terms.put(term, Integer.valueOf(docIDUpto));
|
||||
deleteTerms.put(term, Integer.valueOf(docIDUpto));
|
||||
// note that if current != null then it means there's already a buffered
|
||||
// delete on that term, therefore we seem to over-count. this over-counting
|
||||
// is done to respect IndexWriterConfig.setMaxBufferedDeleteTerms.
|
||||
|
@ -290,11 +290,16 @@ class BufferedUpdates {
|
|||
bytesUsed.addAndGet(BYTES_PER_BINARY_UPDATE_ENTRY + update.sizeInBytes());
|
||||
}
|
||||
}
|
||||
|
||||
void clearDeleteTerms() {
|
||||
deleteTerms.clear();
|
||||
numTermDeletes.set(0);
|
||||
}
|
||||
|
||||
void clear() {
|
||||
terms.clear();
|
||||
queries.clear();
|
||||
docIDs.clear();
|
||||
deleteTerms.clear();
|
||||
deleteQueries.clear();
|
||||
deleteDocIDs.clear();
|
||||
numericUpdates.clear();
|
||||
binaryUpdates.clear();
|
||||
numTermDeletes.set(0);
|
||||
|
@ -304,6 +309,6 @@ class BufferedUpdates {
|
|||
}
|
||||
|
||||
boolean any() {
|
||||
return terms.size() > 0 || docIDs.size() > 0 || queries.size() > 0 || numericUpdates.size() > 0 || binaryUpdates.size() > 0;
|
||||
return deleteTerms.size() > 0 || deleteDocIDs.size() > 0 || deleteQueries.size() > 0 || numericUpdates.size() > 0 || binaryUpdates.size() > 0;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -14,40 +14,33 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.lucene.index;
|
||||
|
||||
package org.apache.lucene.index;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
import org.apache.lucene.search.Weight;
|
||||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.Accountable;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
import org.apache.lucene.util.PriorityQueue;
|
||||
|
||||
/* Tracks the stream of {@link BufferedDeletes}.
|
||||
/** Tracks the stream of {@link FrozenBufferedUpdates}.
|
||||
* When DocumentsWriterPerThread flushes, its buffered
|
||||
* deletes and updates are appended to this stream. We later
|
||||
* apply them (resolve them to the actual
|
||||
* docIDs, per segment) when a merge is started
|
||||
* (only to the to-be-merged segments). We
|
||||
* also apply to all segments when NRT reader is pulled,
|
||||
* commit/close is called, or when too many deletes or updates are
|
||||
* deletes and updates are appended to this stream and immediately
|
||||
* resolved (to actual docIDs, per segment) using the indexing
|
||||
* thread that triggered the flush for concurrency. When a
|
||||
* merge kicks off, we sync to ensure all resolving packets
|
||||
* complete. We also apply to all segments when NRT reader is pulled,
|
||||
* commit/close is called, or when too many deletes or updates are
|
||||
* buffered and must be flushed (by RAM usage or by count).
|
||||
*
|
||||
* Each packet is assigned a generation, and each flushed or
|
||||
|
@ -57,23 +50,24 @@ import org.apache.lucene.util.PriorityQueue;
|
|||
|
||||
class BufferedUpdatesStream implements Accountable {
|
||||
|
||||
// TODO: maybe linked list?
|
||||
private final List<FrozenBufferedUpdates> updates = new ArrayList<>();
|
||||
private final Set<FrozenBufferedUpdates> updates = new HashSet<>();
|
||||
|
||||
// Starts at 1 so that SegmentInfos that have never had
|
||||
// deletes applied (whose bufferedDelGen defaults to 0)
|
||||
// will be correct:
|
||||
private long nextGen = 1;
|
||||
|
||||
// used only by assert
|
||||
private BytesRef lastDeleteTerm;
|
||||
|
||||
private final FinishedSegments finishedSegments;
|
||||
private final InfoStream infoStream;
|
||||
private final AtomicLong bytesUsed = new AtomicLong();
|
||||
private final AtomicInteger numTerms = new AtomicInteger();
|
||||
private final IndexWriter writer;
|
||||
private boolean closed;
|
||||
|
||||
public BufferedUpdatesStream(InfoStream infoStream) {
|
||||
this.infoStream = infoStream;
|
||||
public BufferedUpdatesStream(IndexWriter writer) {
|
||||
this.writer = writer;
|
||||
this.infoStream = writer.infoStream;
|
||||
this.finishedSegments = new FinishedSegments(infoStream);
|
||||
}
|
||||
|
||||
// Appends a new packet of buffered deletes to the stream,
|
||||
|
@ -89,21 +83,27 @@ class BufferedUpdatesStream implements Accountable {
|
|||
packet.setDelGen(nextGen++);
|
||||
assert packet.any();
|
||||
assert checkDeleteStats();
|
||||
assert packet.delGen() < nextGen;
|
||||
assert updates.isEmpty() || updates.get(updates.size()-1).delGen() < packet.delGen() : "Delete packets must be in order";
|
||||
|
||||
updates.add(packet);
|
||||
numTerms.addAndGet(packet.numTermDeletes);
|
||||
bytesUsed.addAndGet(packet.bytesUsed);
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", "push deletes " + packet + " segmentPrivate?=" + packet.isSegmentPrivate + " delGen=" + packet.delGen() + " packetCount=" + updates.size() + " totBytesUsed=" + bytesUsed.get());
|
||||
infoStream.message("BD", String.format(Locale.ROOT, "push new packet (%s), packetCount=%d, bytesUsed=%.3f MB", packet, updates.size(), bytesUsed.get()/1024./1024.));
|
||||
}
|
||||
assert checkDeleteStats();
|
||||
|
||||
return packet.delGen();
|
||||
}
|
||||
|
||||
public synchronized int getPendingUpdatesCount() {
|
||||
return updates.size();
|
||||
}
|
||||
|
||||
/** Only used by IW.rollback */
|
||||
public synchronized void clear() {
|
||||
updates.clear();
|
||||
nextGen = 1;
|
||||
finishedSegments.clear();
|
||||
numTerms.set(0);
|
||||
bytesUsed.set(0);
|
||||
}
|
||||
|
@ -121,253 +121,148 @@ class BufferedUpdatesStream implements Accountable {
|
|||
return bytesUsed.get();
|
||||
}
|
||||
|
||||
private synchronized void ensureOpen() {
|
||||
if (closed) {
|
||||
throw new AlreadyClosedException("already closed");
|
||||
}
|
||||
}
|
||||
|
||||
public static class ApplyDeletesResult {
|
||||
|
||||
// True if any actual deletes took place:
|
||||
public final boolean anyDeletes;
|
||||
|
||||
// Current gen, for the merged segment:
|
||||
public final long gen;
|
||||
|
||||
// If non-null, contains segments that are 100% deleted
|
||||
public final List<SegmentCommitInfo> allDeleted;
|
||||
|
||||
ApplyDeletesResult(boolean anyDeletes, long gen, List<SegmentCommitInfo> allDeleted) {
|
||||
ApplyDeletesResult(boolean anyDeletes, List<SegmentCommitInfo> allDeleted) {
|
||||
this.anyDeletes = anyDeletes;
|
||||
this.gen = gen;
|
||||
this.allDeleted = allDeleted;
|
||||
}
|
||||
}
|
||||
|
||||
// Sorts SegmentInfos from smallest to biggest bufferedDelGen:
|
||||
private static final Comparator<SegmentCommitInfo> sortSegInfoByDelGen = new Comparator<SegmentCommitInfo>() {
|
||||
@Override
|
||||
public int compare(SegmentCommitInfo si1, SegmentCommitInfo si2) {
|
||||
return Long.compare(si1.getBufferedDeletesGen(), si2.getBufferedDeletesGen());
|
||||
/** Waits for all in-flight packets, which are already being resolved concurrently
|
||||
* by indexing threads, to finish. Returns true if there were any
|
||||
* new deletes or updates. This is called for refresh, commit. */
|
||||
public void waitApplyAll() throws IOException {
|
||||
|
||||
assert Thread.holdsLock(writer) == false;
|
||||
|
||||
final long t0 = System.nanoTime();
|
||||
|
||||
Set<FrozenBufferedUpdates> waitFor;
|
||||
synchronized (this) {
|
||||
waitFor = new HashSet<>(updates);
|
||||
}
|
||||
};
|
||||
|
||||
waitApply(waitFor);
|
||||
}
|
||||
|
||||
/** Returns true if this delGen is still running. */
|
||||
public boolean stillRunning(long delGen) {
|
||||
return finishedSegments.stillRunning(delGen);
|
||||
}
|
||||
|
||||
public void finishedSegment(long delGen) {
|
||||
finishedSegments.finishedSegment(delGen);
|
||||
}
|
||||
|
||||
/** Resolves the buffered deleted Term/Query/docIDs, into
|
||||
* actual deleted docIDs in the liveDocs MutableBits for
|
||||
* each SegmentReader. */
|
||||
public synchronized ApplyDeletesResult applyDeletesAndUpdates(IndexWriter.ReaderPool pool, List<SegmentCommitInfo> infos) throws IOException {
|
||||
final long t0 = System.currentTimeMillis();
|
||||
/** Called by indexing threads once they are fully done resolving all deletes for the provided
|
||||
* delGen. We track the completed delGens and record the maximum delGen for which all prior
|
||||
* delGens, inclusive, are completed, so that it's safe for doc values updates to apply and write. */
|
||||
|
||||
final long gen = nextGen++;
|
||||
public synchronized void finished(FrozenBufferedUpdates packet) {
|
||||
// TODO: would be a bit more memory efficient to track this per-segment, so when each segment writes it writes all packets finished for
|
||||
// it, rather than only recording here, across all segments. But, more complex code, and more CPU, and maybe not so much impact in
|
||||
// practice?
|
||||
|
||||
if (infos.size() == 0) {
|
||||
return new ApplyDeletesResult(false, gen, null);
|
||||
packet.applied.countDown();
|
||||
|
||||
updates.remove(packet);
|
||||
numTerms.addAndGet(-packet.numTermDeletes);
|
||||
bytesUsed.addAndGet(-packet.bytesUsed);
|
||||
|
||||
finishedSegment(packet.delGen());
|
||||
}
|
||||
|
||||
/** All frozen packets up to and including this del gen are guaranteed to be finished. */
|
||||
public long getCompletedDelGen() {
|
||||
return finishedSegments.getCompletedDelGen();
|
||||
}
|
||||
|
||||
/** Waits only for those in-flight packets that apply to these merge segments. This is
|
||||
* called when a merge needs to finish and must ensure all deletes to the merging
|
||||
* segments are resolved. */
|
||||
public void waitApplyForMerge(List<SegmentCommitInfo> mergeInfos) throws IOException {
|
||||
assert Thread.holdsLock(writer) == false;
|
||||
|
||||
final long t0 = System.nanoTime();
|
||||
|
||||
long maxDelGen = Long.MIN_VALUE;
|
||||
for (SegmentCommitInfo info : mergeInfos) {
|
||||
maxDelGen = Math.max(maxDelGen, info.getBufferedDeletesGen());
|
||||
}
|
||||
|
||||
// We only init these on demand, when we find our first deletes that need to be applied:
|
||||
SegmentState[] segStates = null;
|
||||
|
||||
long totDelCount = 0;
|
||||
long totTermVisitedCount = 0;
|
||||
|
||||
boolean success = false;
|
||||
|
||||
ApplyDeletesResult result = null;
|
||||
|
||||
try {
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", String.format(Locale.ROOT, "applyDeletes: open segment readers took %d msec", System.currentTimeMillis()-t0));
|
||||
}
|
||||
|
||||
assert checkDeleteStats();
|
||||
|
||||
if (!any()) {
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", "applyDeletes: no segments; skipping");
|
||||
Set<FrozenBufferedUpdates> waitFor = new HashSet<>();
|
||||
synchronized (this) {
|
||||
for (FrozenBufferedUpdates packet : updates) {
|
||||
if (packet.delGen() <= maxDelGen) {
|
||||
// We must wait for this packet before finishing the merge because its
|
||||
// deletes apply to a subset of the segments being merged:
|
||||
waitFor.add(packet);
|
||||
}
|
||||
return new ApplyDeletesResult(false, gen, null);
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", "applyDeletes: infos=" + infos + " packetCount=" + updates.size());
|
||||
}
|
||||
|
||||
infos = sortByDelGen(infos);
|
||||
|
||||
CoalescedUpdates coalescedUpdates = null;
|
||||
int infosIDX = infos.size()-1;
|
||||
int delIDX = updates.size()-1;
|
||||
|
||||
// Backwards merge sort the segment delGens with the packet delGens in the buffered stream:
|
||||
while (infosIDX >= 0) {
|
||||
final FrozenBufferedUpdates packet = delIDX >= 0 ? updates.get(delIDX) : null;
|
||||
final SegmentCommitInfo info = infos.get(infosIDX);
|
||||
final long segGen = info.getBufferedDeletesGen();
|
||||
|
||||
if (packet != null && segGen < packet.delGen()) {
|
||||
if (!packet.isSegmentPrivate && packet.any()) {
|
||||
/*
|
||||
* Only coalesce if we are NOT on a segment private del packet: the segment private del packet
|
||||
* must only apply to segments with the same delGen. Yet, if a segment is already deleted
|
||||
* from the SI since it had no more documents remaining after some del packets younger than
|
||||
* its segPrivate packet (higher delGen) have been applied, the segPrivate packet has not been
|
||||
* removed.
|
||||
*/
|
||||
if (coalescedUpdates == null) {
|
||||
coalescedUpdates = new CoalescedUpdates();
|
||||
}
|
||||
coalescedUpdates.update(packet);
|
||||
}
|
||||
|
||||
delIDX--;
|
||||
} else if (packet != null && segGen == packet.delGen()) {
|
||||
assert packet.isSegmentPrivate : "Packet and Segments deletegen can only match on a segment private del packet gen=" + segGen;
|
||||
|
||||
if (segStates == null) {
|
||||
segStates = openSegmentStates(pool, infos);
|
||||
}
|
||||
|
||||
SegmentState segState = segStates[infosIDX];
|
||||
|
||||
// Lock order: IW -> BD -> RP
|
||||
assert pool.infoIsLive(info);
|
||||
int delCount = 0;
|
||||
final DocValuesFieldUpdates.Container dvUpdates = new DocValuesFieldUpdates.Container();
|
||||
|
||||
// first apply segment-private deletes/updates
|
||||
delCount += applyQueryDeletes(packet.queriesIterable(), segState);
|
||||
applyDocValuesUpdates(Arrays.asList(packet.numericDVUpdates), segState, dvUpdates);
|
||||
applyDocValuesUpdates(Arrays.asList(packet.binaryDVUpdates), segState, dvUpdates);
|
||||
|
||||
// ... then coalesced deletes/updates, so that if there is an update that appears in both, the coalesced updates (carried from
|
||||
// updates ahead of the segment-privates ones) win:
|
||||
if (coalescedUpdates != null) {
|
||||
delCount += applyQueryDeletes(coalescedUpdates.queriesIterable(), segState);
|
||||
applyDocValuesUpdatesList(coalescedUpdates.numericDVUpdates, segState, dvUpdates);
|
||||
applyDocValuesUpdatesList(coalescedUpdates.binaryDVUpdates, segState, dvUpdates);
|
||||
}
|
||||
if (dvUpdates.any()) {
|
||||
segState.rld.writeFieldUpdates(info.info.dir, dvUpdates);
|
||||
}
|
||||
|
||||
totDelCount += delCount;
|
||||
|
||||
/*
|
||||
* Since we are on a segment private del packet we must not
|
||||
* update the coalescedUpdates here! We can simply advance to the
|
||||
* next packet and seginfo.
|
||||
*/
|
||||
delIDX--;
|
||||
infosIDX--;
|
||||
|
||||
} else {
|
||||
if (coalescedUpdates != null) {
|
||||
if (segStates == null) {
|
||||
segStates = openSegmentStates(pool, infos);
|
||||
}
|
||||
SegmentState segState = segStates[infosIDX];
|
||||
// Lock order: IW -> BD -> RP
|
||||
assert pool.infoIsLive(info);
|
||||
int delCount = 0;
|
||||
delCount += applyQueryDeletes(coalescedUpdates.queriesIterable(), segState);
|
||||
DocValuesFieldUpdates.Container dvUpdates = new DocValuesFieldUpdates.Container();
|
||||
applyDocValuesUpdatesList(coalescedUpdates.numericDVUpdates, segState, dvUpdates);
|
||||
applyDocValuesUpdatesList(coalescedUpdates.binaryDVUpdates, segState, dvUpdates);
|
||||
if (dvUpdates.any()) {
|
||||
segState.rld.writeFieldUpdates(info.info.dir, dvUpdates);
|
||||
}
|
||||
|
||||
totDelCount += delCount;
|
||||
}
|
||||
|
||||
infosIDX--;
|
||||
}
|
||||
}
|
||||
|
||||
// Now apply all term deletes:
|
||||
if (coalescedUpdates != null && coalescedUpdates.totalTermCount != 0) {
|
||||
if (segStates == null) {
|
||||
segStates = openSegmentStates(pool, infos);
|
||||
}
|
||||
totTermVisitedCount += applyTermDeletes(coalescedUpdates, segStates);
|
||||
}
|
||||
|
||||
assert checkDeleteStats();
|
||||
|
||||
success = true;
|
||||
|
||||
} finally {
|
||||
if (segStates != null) {
|
||||
result = closeSegmentStates(pool, segStates, success, gen);
|
||||
}
|
||||
}
|
||||
|
||||
if (result == null) {
|
||||
result = new ApplyDeletesResult(false, gen, null);
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", "waitApplyForMerge: " + waitFor.size() + " packets, " + mergeInfos.size() + " merging segments");
|
||||
}
|
||||
|
||||
waitApply(waitFor);
|
||||
}
|
||||
|
||||
private void waitApply(Set<FrozenBufferedUpdates> waitFor) throws IOException {
|
||||
|
||||
long startNS = System.nanoTime();
|
||||
|
||||
int packetCount = waitFor.size();
|
||||
|
||||
if (waitFor.isEmpty()) {
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", "waitApply: no deletes to apply");
|
||||
}
|
||||
return;
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", "waitApply: " + waitFor.size() + " packets: " + waitFor);
|
||||
}
|
||||
|
||||
long totalDelCount = 0;
|
||||
for (FrozenBufferedUpdates packet : waitFor) {
|
||||
// Frozen packets are now resolved, concurrently, by the indexing threads that
|
||||
// create them, by adding a DocumentsWriter.ResolveUpdatesEvent to the events queue,
|
||||
// but if we get here and the packet is not yet resolved, we resolve it now ourselves:
|
||||
packet.apply(writer);
|
||||
totalDelCount += packet.totalDelCount;
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD",
|
||||
String.format(Locale.ROOT,
|
||||
"applyDeletes took %d msec for %d segments, %d newly deleted docs (query deletes), %d visited terms, allDeleted=%s",
|
||||
System.currentTimeMillis()-t0, infos.size(), totDelCount, totTermVisitedCount, result.allDeleted));
|
||||
String.format(Locale.ROOT, "waitApply: done %d packets; totalDelCount=%d; totBytesUsed=%d; took %.2f msec",
|
||||
packetCount,
|
||||
totalDelCount,
|
||||
bytesUsed.get(),
|
||||
(System.nanoTime() - startNS) / 1000000.));
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
private List<SegmentCommitInfo> sortByDelGen(List<SegmentCommitInfo> infos) {
|
||||
infos = new ArrayList<>(infos);
|
||||
// Smaller delGens come first:
|
||||
Collections.sort(infos, sortSegInfoByDelGen);
|
||||
return infos;
|
||||
}
|
||||
|
||||
synchronized long getNextGen() {
|
||||
return nextGen++;
|
||||
}
|
||||
|
||||
// Lock order IW -> BD
|
||||
/* Removes any BufferedDeletes that we no longer need to
|
||||
* store because all segments in the index have had the
|
||||
* deletes applied. */
|
||||
public synchronized void prune(SegmentInfos segmentInfos) {
|
||||
assert checkDeleteStats();
|
||||
long minGen = Long.MAX_VALUE;
|
||||
for(SegmentCommitInfo info : segmentInfos) {
|
||||
minGen = Math.min(info.getBufferedDeletesGen(), minGen);
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", "prune sis=" + segmentInfos + " minGen=" + minGen + " packetCount=" + updates.size());
|
||||
}
|
||||
final int limit = updates.size();
|
||||
for(int delIDX=0;delIDX<limit;delIDX++) {
|
||||
if (updates.get(delIDX).delGen() >= minGen) {
|
||||
prune(delIDX);
|
||||
assert checkDeleteStats();
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
// All deletes pruned
|
||||
prune(limit);
|
||||
assert !any();
|
||||
assert checkDeleteStats();
|
||||
}
|
||||
|
||||
private synchronized void prune(int count) {
|
||||
if (count > 0) {
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", "pruneDeletes: prune " + count + " packets; " + (updates.size() - count) + " packets remain");
|
||||
}
|
||||
for(int delIDX=0;delIDX<count;delIDX++) {
|
||||
final FrozenBufferedUpdates packet = updates.get(delIDX);
|
||||
numTerms.addAndGet(-packet.numTermDeletes);
|
||||
assert numTerms.get() >= 0;
|
||||
bytesUsed.addAndGet(-packet.bytesUsed);
|
||||
assert bytesUsed.get() >= 0;
|
||||
}
|
||||
updates.subList(0, count).clear();
|
||||
}
|
||||
}
|
||||
|
||||
static class SegmentState {
|
||||
/** Holds all per-segment internal state used while resolving deletions. */
|
||||
public static final class SegmentState {
|
||||
final long delGen;
|
||||
final ReadersAndUpdates rld;
|
||||
final SegmentReader reader;
|
||||
|
@ -376,7 +271,6 @@ class BufferedUpdatesStream implements Accountable {
|
|||
TermsEnum termsEnum;
|
||||
PostingsEnum postingsEnum;
|
||||
BytesRef term;
|
||||
boolean any;
|
||||
|
||||
public SegmentState(IndexWriter.ReaderPool pool, SegmentCommitInfo info) throws IOException {
|
||||
rld = pool.get(info, true);
|
||||
|
@ -392,58 +286,54 @@ class BufferedUpdatesStream implements Accountable {
|
|||
pool.release(rld);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Does a merge sort by current term across all segments. */
|
||||
static class SegmentQueue extends PriorityQueue<SegmentState> {
|
||||
public SegmentQueue(int size) {
|
||||
super(size);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean lessThan(SegmentState a, SegmentState b) {
|
||||
return a.term.compareTo(b.term) < 0;
|
||||
public String toString() {
|
||||
return "SegmentState(" + rld.info + ")";
|
||||
}
|
||||
}
|
||||
|
||||
/** Opens SegmentReader and inits SegmentState for each segment. */
|
||||
private SegmentState[] openSegmentStates(IndexWriter.ReaderPool pool, List<SegmentCommitInfo> infos) throws IOException {
|
||||
int numReaders = infos.size();
|
||||
SegmentState[] segStates = new SegmentState[numReaders];
|
||||
public SegmentState[] openSegmentStates(IndexWriter.ReaderPool pool, List<SegmentCommitInfo> infos,
|
||||
Set<SegmentCommitInfo> alreadySeenSegments, long delGen) throws IOException {
|
||||
ensureOpen();
|
||||
|
||||
List<SegmentState> segStates = new ArrayList<>();
|
||||
boolean success = false;
|
||||
try {
|
||||
for(int i=0;i<numReaders;i++) {
|
||||
segStates[i] = new SegmentState(pool, infos.get(i));
|
||||
for (SegmentCommitInfo info : infos) {
|
||||
if (info.getBufferedDeletesGen() <= delGen && alreadySeenSegments.contains(info) == false) {
|
||||
segStates.add(new SegmentState(pool, info));
|
||||
alreadySeenSegments.add(info);
|
||||
}
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (success == false) {
|
||||
for(int j=0;j<numReaders;j++) {
|
||||
if (segStates[j] != null) {
|
||||
for(SegmentState segState : segStates) {
|
||||
try {
|
||||
segStates[j].finish(pool);
|
||||
segState.finish(pool);
|
||||
} catch (Throwable th) {
|
||||
// suppress so we keep throwing original exc
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return segStates;
|
||||
|
||||
return segStates.toArray(new SegmentState[0]);
|
||||
}
|
||||
|
||||
/** Close segment states previously opened with openSegmentStates. */
|
||||
private ApplyDeletesResult closeSegmentStates(IndexWriter.ReaderPool pool, SegmentState[] segStates, boolean success, long gen) throws IOException {
|
||||
int numReaders = segStates.length;
|
||||
public ApplyDeletesResult closeSegmentStates(IndexWriter.ReaderPool pool, SegmentState[] segStates, boolean success) throws IOException {
|
||||
int count = segStates.length;
|
||||
Throwable firstExc = null;
|
||||
List<SegmentCommitInfo> allDeleted = null;
|
||||
long totDelCount = 0;
|
||||
for (int j=0;j<numReaders;j++) {
|
||||
|
||||
for (int j=0;j<count;j++) {
|
||||
SegmentState segState = segStates[j];
|
||||
if (success) {
|
||||
totDelCount += segState.rld.getPendingDeleteCount() - segState.startDelCount;
|
||||
segState.reader.getSegmentInfo().setBufferedDeletesGen(gen);
|
||||
int fullDelCount = segState.rld.info.getDelCount() + segState.rld.getPendingDeleteCount();
|
||||
assert fullDelCount <= segState.rld.info.info.maxDoc();
|
||||
if (fullDelCount == segState.rld.info.info.maxDoc()) {
|
||||
|
@ -469,278 +359,10 @@ class BufferedUpdatesStream implements Accountable {
|
|||
}
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", "applyDeletes: " + totDelCount + " new deleted documents");
|
||||
infoStream.message("BD", "closeSegmentStates: " + totDelCount + " new deleted documents; pool " + updates.size() + " packets; bytesUsed=" + pool.ramBytesUsed());
|
||||
}
|
||||
|
||||
return new ApplyDeletesResult(totDelCount > 0, gen, allDeleted);
|
||||
}
|
||||
|
||||
/** Merge sorts the deleted terms and all segments to resolve terms to docIDs for deletion. */
|
||||
private synchronized long applyTermDeletes(CoalescedUpdates updates, SegmentState[] segStates) throws IOException {
|
||||
|
||||
long startNS = System.nanoTime();
|
||||
|
||||
int numReaders = segStates.length;
|
||||
|
||||
long delTermVisitedCount = 0;
|
||||
long segTermVisitedCount = 0;
|
||||
|
||||
FieldTermIterator iter = updates.termIterator();
|
||||
|
||||
String field = null;
|
||||
SegmentQueue queue = null;
|
||||
|
||||
BytesRef term;
|
||||
|
||||
while ((term = iter.next()) != null) {
|
||||
|
||||
if (iter.field() != field) {
|
||||
// field changed
|
||||
field = iter.field();
|
||||
|
||||
queue = new SegmentQueue(numReaders);
|
||||
|
||||
long segTermCount = 0;
|
||||
for(int i=0;i<numReaders;i++) {
|
||||
SegmentState state = segStates[i];
|
||||
Terms terms = state.reader.fields().terms(field);
|
||||
if (terms != null) {
|
||||
segTermCount += terms.size();
|
||||
state.termsEnum = terms.iterator();
|
||||
state.term = state.termsEnum.next();
|
||||
if (state.term != null) {
|
||||
queue.add(state);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
assert checkDeleteTerm(null);
|
||||
}
|
||||
|
||||
assert checkDeleteTerm(term);
|
||||
|
||||
delTermVisitedCount++;
|
||||
|
||||
long delGen = iter.delGen();
|
||||
|
||||
while (queue.size() != 0) {
|
||||
|
||||
// Get next term merged across all segments
|
||||
SegmentState state = queue.top();
|
||||
segTermVisitedCount++;
|
||||
|
||||
int cmp = term.compareTo(state.term);
|
||||
|
||||
if (cmp < 0) {
|
||||
break;
|
||||
} else if (cmp == 0) {
|
||||
// fall through
|
||||
} else {
|
||||
TermsEnum.SeekStatus status = state.termsEnum.seekCeil(term);
|
||||
if (status == TermsEnum.SeekStatus.FOUND) {
|
||||
// fallthrough
|
||||
} else {
|
||||
if (status == TermsEnum.SeekStatus.NOT_FOUND) {
|
||||
state.term = state.termsEnum.term();
|
||||
queue.updateTop();
|
||||
} else {
|
||||
// No more terms in this segment
|
||||
queue.pop();
|
||||
}
|
||||
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
||||
assert state.delGen != delGen;
|
||||
|
||||
if (state.delGen < delGen) {
|
||||
|
||||
// we don't need term frequencies for this
|
||||
final Bits acceptDocs = state.rld.getLiveDocs();
|
||||
state.postingsEnum = state.termsEnum.postings(state.postingsEnum, PostingsEnum.NONE);
|
||||
|
||||
assert state.postingsEnum != null;
|
||||
|
||||
while (true) {
|
||||
final int docID = state.postingsEnum.nextDoc();
|
||||
if (docID == DocIdSetIterator.NO_MORE_DOCS) {
|
||||
break;
|
||||
}
|
||||
if (acceptDocs != null && acceptDocs.get(docID) == false) {
|
||||
continue;
|
||||
}
|
||||
if (!state.any) {
|
||||
state.rld.initWritableLiveDocs();
|
||||
state.any = true;
|
||||
}
|
||||
|
||||
// NOTE: there is no limit check on the docID
|
||||
// when deleting by Term (unlike by Query)
|
||||
// because on flush we apply all Term deletes to
|
||||
// each segment. So all Term deleting here is
|
||||
// against prior segments:
|
||||
state.rld.delete(docID);
|
||||
}
|
||||
}
|
||||
|
||||
state.term = state.termsEnum.next();
|
||||
if (state.term == null) {
|
||||
queue.pop();
|
||||
} else {
|
||||
queue.updateTop();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD",
|
||||
String.format(Locale.ROOT, "applyTermDeletes took %.1f msec for %d segments and %d packets; %d del terms visited; %d seg terms visited",
|
||||
(System.nanoTime()-startNS)/1000000.,
|
||||
numReaders,
|
||||
updates.terms.size(),
|
||||
delTermVisitedCount, segTermVisitedCount));
|
||||
}
|
||||
|
||||
return delTermVisitedCount;
|
||||
}
|
||||
|
||||
private synchronized void applyDocValuesUpdatesList(List<List<DocValuesUpdate>> updates,
|
||||
SegmentState segState, DocValuesFieldUpdates.Container dvUpdatesContainer) throws IOException {
|
||||
// we walk backwards through the segments, appending deletion packets to the coalesced updates, so we must apply the packets in reverse
|
||||
// so that newer packets override older ones:
|
||||
for(int idx=updates.size()-1;idx>=0;idx--) {
|
||||
applyDocValuesUpdates(updates.get(idx), segState, dvUpdatesContainer);
|
||||
}
|
||||
}
|
||||
|
||||
// DocValues updates
|
||||
private synchronized void applyDocValuesUpdates(List<DocValuesUpdate> updates,
|
||||
SegmentState segState, DocValuesFieldUpdates.Container dvUpdatesContainer) throws IOException {
|
||||
Fields fields = segState.reader.fields();
|
||||
|
||||
// TODO: we can process the updates per DV field, from last to first so that
|
||||
// if multiple terms affect same document for the same field, we add an update
|
||||
// only once (that of the last term). To do that, we can keep a bitset which
|
||||
// marks which documents have already been updated. So e.g. if term T1
|
||||
// updates doc 7, and then we process term T2 and it updates doc 7 as well,
|
||||
// we don't apply the update since we know T1 came last and therefore wins
|
||||
// the update.
|
||||
// We can also use that bitset as 'liveDocs' to pass to TermEnum.docs(), so
|
||||
// that these documents aren't even returned.
|
||||
|
||||
String currentField = null;
|
||||
TermsEnum termsEnum = null;
|
||||
PostingsEnum postingsEnum = null;
|
||||
|
||||
for (DocValuesUpdate update : updates) {
|
||||
Term term = update.term;
|
||||
int limit = update.docIDUpto;
|
||||
|
||||
// TODO: we traverse the terms in update order (not term order) so that we
|
||||
// apply the updates in the correct order, i.e. if two terms udpate the
|
||||
// same document, the last one that came in wins, irrespective of the
|
||||
// terms lexical order.
|
||||
// we can apply the updates in terms order if we keep an updatesGen (and
|
||||
// increment it with every update) and attach it to each NumericUpdate. Note
|
||||
// that we cannot rely only on docIDUpto because an app may send two updates
|
||||
// which will get same docIDUpto, yet will still need to respect the order
|
||||
// those updates arrived.
|
||||
|
||||
if (!term.field().equals(currentField)) {
|
||||
// if we change the code to process updates in terms order, enable this assert
|
||||
// assert currentField == null || currentField.compareTo(term.field()) < 0;
|
||||
currentField = term.field();
|
||||
Terms terms = fields.terms(currentField);
|
||||
if (terms != null) {
|
||||
termsEnum = terms.iterator();
|
||||
} else {
|
||||
termsEnum = null;
|
||||
}
|
||||
}
|
||||
|
||||
if (termsEnum == null) {
|
||||
// no terms in this field
|
||||
continue;
|
||||
}
|
||||
|
||||
if (termsEnum.seekExact(term.bytes())) {
|
||||
// we don't need term frequencies for this
|
||||
final Bits acceptDocs = segState.rld.getLiveDocs();
|
||||
postingsEnum = termsEnum.postings(postingsEnum, PostingsEnum.NONE);
|
||||
|
||||
DocValuesFieldUpdates dvUpdates = dvUpdatesContainer.getUpdates(update.field, update.type);
|
||||
if (dvUpdates == null) {
|
||||
dvUpdates = dvUpdatesContainer.newUpdates(update.field, update.type, segState.reader.maxDoc());
|
||||
}
|
||||
int doc;
|
||||
while ((doc = postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
|
||||
if (doc >= limit) {
|
||||
break; // no more docs that can be updated for this term
|
||||
}
|
||||
if (acceptDocs != null && acceptDocs.get(doc) == false) {
|
||||
continue;
|
||||
}
|
||||
dvUpdates.add(doc, update.value);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static class QueryAndLimit {
|
||||
public final Query query;
|
||||
public final int limit;
|
||||
public QueryAndLimit(Query query, int limit) {
|
||||
this.query = query;
|
||||
this.limit = limit;
|
||||
}
|
||||
}
|
||||
|
||||
// Delete by query
|
||||
private static long applyQueryDeletes(Iterable<QueryAndLimit> queriesIter, SegmentState segState) throws IOException {
|
||||
long delCount = 0;
|
||||
final LeafReaderContext readerContext = segState.reader.getContext();
|
||||
for (QueryAndLimit ent : queriesIter) {
|
||||
Query query = ent.query;
|
||||
int limit = ent.limit;
|
||||
final IndexSearcher searcher = new IndexSearcher(readerContext.reader());
|
||||
searcher.setQueryCache(null);
|
||||
final Weight weight = searcher.createNormalizedWeight(query, false);
|
||||
final Scorer scorer = weight.scorer(readerContext);
|
||||
if (scorer != null) {
|
||||
final DocIdSetIterator it = scorer.iterator();
|
||||
final Bits liveDocs = readerContext.reader().getLiveDocs();
|
||||
while (true) {
|
||||
int doc = it.nextDoc();
|
||||
if (doc >= limit) {
|
||||
break;
|
||||
}
|
||||
if (liveDocs != null && liveDocs.get(doc) == false) {
|
||||
continue;
|
||||
}
|
||||
|
||||
if (!segState.any) {
|
||||
segState.rld.initWritableLiveDocs();
|
||||
segState.any = true;
|
||||
}
|
||||
if (segState.rld.delete(doc)) {
|
||||
delCount++;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return delCount;
|
||||
}
|
||||
|
||||
// used only by assert
|
||||
private boolean checkDeleteTerm(BytesRef term) {
|
||||
if (term != null) {
|
||||
assert lastDeleteTerm == null || term.compareTo(lastDeleteTerm) >= 0: "lastTerm=" + lastDeleteTerm + " vs term=" + term;
|
||||
}
|
||||
// TODO: we re-use term now in our merged iterable, but we shouldn't clone, instead copy for this assert
|
||||
lastDeleteTerm = term == null ? null : BytesRef.deepCopyOf(term);
|
||||
return true;
|
||||
return new ApplyDeletesResult(totDelCount > 0, allDeleted);
|
||||
}
|
||||
|
||||
// only for assert
|
||||
|
@ -755,4 +377,52 @@ class BufferedUpdatesStream implements Accountable {
|
|||
assert bytesUsed2 == bytesUsed.get(): "bytesUsed2=" + bytesUsed2 + " vs " + bytesUsed;
|
||||
return true;
|
||||
}
|
||||
|
||||
/** Tracks the contiguous range of packets that have finished resolving. We need this because the packets
|
||||
* are concurrently resolved, and we can only write to disk the contiguous completed
|
||||
* packets. */
|
||||
private static class FinishedSegments {
|
||||
|
||||
/** Largest del gen, inclusive, for which all prior packets have finished applying. */
|
||||
private long completedDelGen;
|
||||
|
||||
/** This lets us track the "holes" in the current frontier of applying del
|
||||
* gens; once the holes are filled in we can advance completedDelGen. */
|
||||
private final Set<Long> finishedDelGens = new HashSet<>();
|
||||
|
||||
private final InfoStream infoStream;
|
||||
|
||||
public FinishedSegments(InfoStream infoStream) {
|
||||
this.infoStream = infoStream;
|
||||
}
|
||||
|
||||
public synchronized void clear() {
|
||||
finishedDelGens.clear();
|
||||
completedDelGen = 0;
|
||||
}
|
||||
|
||||
public synchronized boolean stillRunning(long delGen) {
|
||||
return delGen > completedDelGen && finishedDelGens.contains(delGen) == false;
|
||||
}
|
||||
|
||||
public synchronized long getCompletedDelGen() {
|
||||
return completedDelGen;
|
||||
}
|
||||
|
||||
public synchronized void finishedSegment(long delGen) {
|
||||
finishedDelGens.add(delGen);
|
||||
while (true) {
|
||||
if (finishedDelGens.contains(completedDelGen + 1)) {
|
||||
finishedDelGens.remove(completedDelGen + 1);
|
||||
completedDelGen++;
|
||||
} else {
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", "finished packet delGen=" + delGen + " now completedDelGen=" + completedDelGen);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,109 +0,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.
|
||||
*/
|
||||
package org.apache.lucene.index;
|
||||
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.index.BufferedUpdatesStream.QueryAndLimit;
|
||||
import org.apache.lucene.index.DocValuesUpdate.BinaryDocValuesUpdate;
|
||||
import org.apache.lucene.index.DocValuesUpdate.NumericDocValuesUpdate;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
class CoalescedUpdates {
|
||||
final Map<Query,Integer> queries = new HashMap<>();
|
||||
final List<PrefixCodedTerms> terms = new ArrayList<>();
|
||||
final List<List<DocValuesUpdate>> numericDVUpdates = new ArrayList<>();
|
||||
final List<List<DocValuesUpdate>> binaryDVUpdates = new ArrayList<>();
|
||||
long totalTermCount;
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
// note: we could add/collect more debugging information
|
||||
return "CoalescedUpdates(termSets=" + terms.size()
|
||||
+ ",totalTermCount=" + totalTermCount
|
||||
+ ",queries=" + queries.size() + ",numericDVUpdates=" + numericDVUpdates.size()
|
||||
+ ",binaryDVUpdates=" + binaryDVUpdates.size() + ")";
|
||||
}
|
||||
|
||||
void update(FrozenBufferedUpdates in) {
|
||||
totalTermCount += in.terms.size();
|
||||
terms.add(in.terms);
|
||||
|
||||
for (int queryIdx = 0; queryIdx < in.queries.length; queryIdx++) {
|
||||
final Query query = in.queries[queryIdx];
|
||||
queries.put(query, BufferedUpdates.MAX_INT);
|
||||
}
|
||||
|
||||
List<DocValuesUpdate> numericPacket = new ArrayList<>();
|
||||
numericDVUpdates.add(numericPacket);
|
||||
for (NumericDocValuesUpdate nu : in.numericDVUpdates) {
|
||||
NumericDocValuesUpdate clone = new NumericDocValuesUpdate(nu.term, nu.field, (Long) nu.value);
|
||||
clone.docIDUpto = Integer.MAX_VALUE;
|
||||
numericPacket.add(clone);
|
||||
}
|
||||
|
||||
List<DocValuesUpdate> binaryPacket = new ArrayList<>();
|
||||
binaryDVUpdates.add(binaryPacket);
|
||||
for (BinaryDocValuesUpdate bu : in.binaryDVUpdates) {
|
||||
BinaryDocValuesUpdate clone = new BinaryDocValuesUpdate(bu.term, bu.field, (BytesRef) bu.value);
|
||||
clone.docIDUpto = Integer.MAX_VALUE;
|
||||
binaryPacket.add(clone);
|
||||
}
|
||||
}
|
||||
|
||||
public FieldTermIterator termIterator() {
|
||||
if (terms.size() == 1) {
|
||||
return terms.get(0).iterator();
|
||||
} else {
|
||||
return new MergedPrefixCodedTermsIterator(terms);
|
||||
}
|
||||
}
|
||||
|
||||
public Iterable<QueryAndLimit> queriesIterable() {
|
||||
return new Iterable<QueryAndLimit>() {
|
||||
|
||||
@Override
|
||||
public Iterator<QueryAndLimit> iterator() {
|
||||
return new Iterator<QueryAndLimit>() {
|
||||
private final Iterator<Map.Entry<Query,Integer>> iter = queries.entrySet().iterator();
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
return iter.hasNext();
|
||||
}
|
||||
|
||||
@Override
|
||||
public QueryAndLimit next() {
|
||||
final Map.Entry<Query,Integer> ent = iter.next();
|
||||
return new QueryAndLimit(ent.getKey(), ent.getValue());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void remove() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
|
@ -98,12 +98,15 @@ public abstract class CodecReader extends LeafReader implements Accountable {
|
|||
throw new IndexOutOfBoundsException("docID must be >= 0 and < maxDoc=" + maxDoc() + " (got docID=" + docID + ")");
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public final Fields fields() {
|
||||
return getPostingsReader();
|
||||
public final Terms terms(String field) throws IOException {
|
||||
//ensureOpen(); no; getPostingsReader calls this
|
||||
// We could check the FieldInfo IndexOptions but there's no point since
|
||||
// PostingsReader will simply return null for fields that don't exist or that have no terms index.
|
||||
return getPostingsReader().terms(field);
|
||||
}
|
||||
|
||||
|
||||
// returns the FieldInfo that corresponds to the given field and type, or
|
||||
// null if the field does not exist, or not indexed as the requested
|
||||
// DovDocValuesType.
|
||||
|
|
|
@ -770,10 +770,12 @@ final class DefaultIndexingChain extends DocConsumer {
|
|||
}
|
||||
invertState.lastStartOffset = startOffset;
|
||||
|
||||
invertState.length++;
|
||||
if (invertState.length < 0) {
|
||||
throw new IllegalArgumentException("too many tokens in field '" + field.name() + "'");
|
||||
try {
|
||||
invertState.length = Math.addExact(invertState.length, invertState.termFreqAttribute.getTermFrequency());
|
||||
} catch (ArithmeticException ae) {
|
||||
throw new IllegalArgumentException("too many tokens for field \"" + field.name() + "\"");
|
||||
}
|
||||
|
||||
//System.out.println(" term=" + invertState.termAttribute);
|
||||
|
||||
// If we hit an exception in here, we abort
|
||||
|
|
|
@ -16,15 +16,13 @@
|
|||
*/
|
||||
package org.apache.lucene.index;
|
||||
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.util.packed.PagedGrowableWriter;
|
||||
import org.apache.lucene.util.PriorityQueue;
|
||||
|
||||
import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
|
||||
|
||||
/**
|
||||
* Holds updates of a single DocValues field, for a set of documents.
|
||||
* Holds updates of a single DocValues field, for a set of documents within one segment.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
|
@ -54,100 +52,114 @@ abstract class DocValuesFieldUpdates {
|
|||
* {@code null} value means that it was unset for this document.
|
||||
*/
|
||||
abstract Object value();
|
||||
|
||||
/**
|
||||
* Reset the iterator's state. Should be called before {@link #nextDoc()}
|
||||
* and {@link #value()}.
|
||||
*/
|
||||
abstract void reset();
|
||||
|
||||
|
||||
/** Returns delGen for this packet. */
|
||||
abstract long delGen();
|
||||
}
|
||||
|
||||
static class Container {
|
||||
|
||||
final Map<String,NumericDocValuesFieldUpdates> numericDVUpdates = new HashMap<>();
|
||||
final Map<String,BinaryDocValuesFieldUpdates> binaryDVUpdates = new HashMap<>();
|
||||
|
||||
boolean any() {
|
||||
for (NumericDocValuesFieldUpdates updates : numericDVUpdates.values()) {
|
||||
if (updates.any()) {
|
||||
return true;
|
||||
/** Merge-sorts multiple iterators, one per delGen, favoring the largest delGen that has updates for a given docID. */
|
||||
public static Iterator mergedIterator(Iterator[] subs) {
|
||||
|
||||
if (subs.length == 1) {
|
||||
return subs[0];
|
||||
}
|
||||
|
||||
PriorityQueue<Iterator> queue = new PriorityQueue<Iterator>(subs.length) {
|
||||
@Override
|
||||
protected boolean lessThan(Iterator a, Iterator b) {
|
||||
// sort by smaller docID
|
||||
int cmp = Integer.compare(a.doc(), b.doc());
|
||||
if (cmp == 0) {
|
||||
// then by larger delGen
|
||||
cmp = Long.compare(b.delGen(), a.delGen());
|
||||
|
||||
// delGens are unique across our subs:
|
||||
assert cmp != 0;
|
||||
}
|
||||
|
||||
return cmp < 0;
|
||||
}
|
||||
};
|
||||
|
||||
for (Iterator sub : subs) {
|
||||
if (sub.nextDoc() != NO_MORE_DOCS) {
|
||||
queue.add(sub);
|
||||
}
|
||||
for (BinaryDocValuesFieldUpdates updates : binaryDVUpdates.values()) {
|
||||
if (updates.any()) {
|
||||
return true;
|
||||
}
|
||||
|
||||
if (queue.size() == 0) {
|
||||
return null;
|
||||
}
|
||||
|
||||
return new Iterator() {
|
||||
private int doc;
|
||||
|
||||
private boolean first = true;
|
||||
|
||||
@Override
|
||||
public int nextDoc() {
|
||||
// TODO: can we do away with this first boolean?
|
||||
if (first == false) {
|
||||
// Advance all sub iterators past current doc
|
||||
while (true) {
|
||||
if (queue.size() == 0) {
|
||||
doc = NO_MORE_DOCS;
|
||||
break;
|
||||
}
|
||||
int newDoc = queue.top().doc();
|
||||
if (newDoc != doc) {
|
||||
assert newDoc > doc: "doc=" + doc + " newDoc=" + newDoc;
|
||||
doc = newDoc;
|
||||
break;
|
||||
}
|
||||
if (queue.top().nextDoc() == NO_MORE_DOCS) {
|
||||
queue.pop();
|
||||
} else {
|
||||
queue.updateTop();
|
||||
}
|
||||
}
|
||||
} else {
|
||||
doc = queue.top().doc();
|
||||
first = false;
|
||||
}
|
||||
return doc;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
int size() {
|
||||
return numericDVUpdates.size() + binaryDVUpdates.size();
|
||||
}
|
||||
|
||||
long ramBytesPerDoc() {
|
||||
long ramBytesPerDoc = 0;
|
||||
for (NumericDocValuesFieldUpdates updates : numericDVUpdates.values()) {
|
||||
ramBytesPerDoc += updates.ramBytesPerDoc();
|
||||
|
||||
@Override
|
||||
public int doc() {
|
||||
return doc;
|
||||
}
|
||||
for (BinaryDocValuesFieldUpdates updates : binaryDVUpdates.values()) {
|
||||
ramBytesPerDoc += updates.ramBytesPerDoc();
|
||||
|
||||
@Override
|
||||
public Object value() {
|
||||
return queue.top().value();
|
||||
}
|
||||
return ramBytesPerDoc;
|
||||
}
|
||||
|
||||
DocValuesFieldUpdates getUpdates(String field, DocValuesType type) {
|
||||
switch (type) {
|
||||
case NUMERIC:
|
||||
return numericDVUpdates.get(field);
|
||||
case BINARY:
|
||||
return binaryDVUpdates.get(field);
|
||||
default:
|
||||
throw new IllegalArgumentException("unsupported type: " + type);
|
||||
|
||||
@Override
|
||||
public long delGen() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
||||
|
||||
DocValuesFieldUpdates newUpdates(String field, DocValuesType type, int maxDoc) {
|
||||
switch (type) {
|
||||
case NUMERIC:
|
||||
assert numericDVUpdates.get(field) == null;
|
||||
NumericDocValuesFieldUpdates numericUpdates = new NumericDocValuesFieldUpdates(field, maxDoc);
|
||||
numericDVUpdates.put(field, numericUpdates);
|
||||
return numericUpdates;
|
||||
case BINARY:
|
||||
assert binaryDVUpdates.get(field) == null;
|
||||
BinaryDocValuesFieldUpdates binaryUpdates = new BinaryDocValuesFieldUpdates(field, maxDoc);
|
||||
binaryDVUpdates.put(field, binaryUpdates);
|
||||
return binaryUpdates;
|
||||
default:
|
||||
throw new IllegalArgumentException("unsupported type: " + type);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "numericDVUpdates=" + numericDVUpdates + " binaryDVUpdates=" + binaryDVUpdates;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
final String field;
|
||||
final DocValuesType type;
|
||||
|
||||
protected DocValuesFieldUpdates(String field, DocValuesType type) {
|
||||
final long delGen;
|
||||
protected boolean finished;
|
||||
protected final int maxDoc;
|
||||
|
||||
protected DocValuesFieldUpdates(int maxDoc, long delGen, String field, DocValuesType type) {
|
||||
this.maxDoc = maxDoc;
|
||||
this.delGen = delGen;
|
||||
this.field = field;
|
||||
if (type == null) {
|
||||
throw new NullPointerException("DocValuesType must not be null");
|
||||
}
|
||||
this.type = type;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the estimated capacity of a {@link PagedGrowableWriter} given the
|
||||
* actual number of stored elements.
|
||||
*/
|
||||
protected static int estimateCapacity(int size) {
|
||||
return (int) Math.ceil((double) size / PAGE_SIZE) * PAGE_SIZE;
|
||||
|
||||
public boolean getFinished() {
|
||||
return finished;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -160,19 +172,17 @@ abstract class DocValuesFieldUpdates {
|
|||
* Returns an {@link Iterator} over the updated documents and their
|
||||
* values.
|
||||
*/
|
||||
// TODO: also use this for merging, instead of having to write through to disk first
|
||||
public abstract Iterator iterator();
|
||||
|
||||
/**
|
||||
* Merge with another {@link DocValuesFieldUpdates}. This is called for a
|
||||
* segment which received updates while it was being merged. The given updates
|
||||
* should override whatever updates are in that instance.
|
||||
*/
|
||||
public abstract void merge(DocValuesFieldUpdates other);
|
||||
|
||||
/** Freezes internal data structures and sorts updates by docID for efficient iteration. */
|
||||
public abstract void finish();
|
||||
|
||||
/** Returns true if this instance contains any updates. */
|
||||
public abstract boolean any();
|
||||
|
||||
/** Returns approximate RAM bytes used per document. */
|
||||
public abstract long ramBytesPerDoc();
|
||||
/** Returns approximate RAM bytes used. */
|
||||
public abstract long ramBytesUsed();
|
||||
|
||||
public abstract int size();
|
||||
}
|
||||
|
|
|
@ -85,7 +85,6 @@ abstract class DocValuesUpdate {
|
|||
long valueSizeInBytes() {
|
||||
return RAW_VALUE_SIZE_IN_BYTES + ((BytesRef) value).bytes.length;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/** An in-place update to a numeric DocValues field */
|
||||
|
@ -99,7 +98,5 @@ abstract class DocValuesUpdate {
|
|||
long valueSizeInBytes() {
|
||||
return Long.BYTES;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -107,7 +107,7 @@ final class DocumentsWriter implements Closeable, Accountable {
|
|||
private final AtomicInteger numDocsInRAM = new AtomicInteger(0);
|
||||
|
||||
// TODO: cut over to BytesRefHash in BufferedDeletes
|
||||
volatile DocumentsWriterDeleteQueue deleteQueue = new DocumentsWriterDeleteQueue();
|
||||
volatile DocumentsWriterDeleteQueue deleteQueue;
|
||||
private final DocumentsWriterFlushQueue ticketQueue = new DocumentsWriterFlushQueue();
|
||||
/*
|
||||
* we preserve changes during a full flush since IW might not checkout before
|
||||
|
@ -129,6 +129,7 @@ final class DocumentsWriter implements Closeable, Accountable {
|
|||
this.directory = directory;
|
||||
this.config = config;
|
||||
this.infoStream = config.getInfoStream();
|
||||
this.deleteQueue = new DocumentsWriterDeleteQueue(infoStream);
|
||||
this.perThreadPool = config.getIndexerThreadPool();
|
||||
flushPolicy = config.getFlushPolicy();
|
||||
this.writer = writer;
|
||||
|
@ -141,10 +142,10 @@ final class DocumentsWriter implements Closeable, Accountable {
|
|||
final DocumentsWriterDeleteQueue deleteQueue = this.deleteQueue;
|
||||
long seqNo = deleteQueue.addDelete(queries);
|
||||
flushControl.doOnDelete();
|
||||
lastSeqNo = Math.max(lastSeqNo, seqNo);
|
||||
if (applyAllDeletes(deleteQueue)) {
|
||||
seqNo = -seqNo;
|
||||
}
|
||||
lastSeqNo = Math.max(lastSeqNo, seqNo);
|
||||
return seqNo;
|
||||
}
|
||||
|
||||
|
@ -160,10 +161,10 @@ final class DocumentsWriter implements Closeable, Accountable {
|
|||
final DocumentsWriterDeleteQueue deleteQueue = this.deleteQueue;
|
||||
long seqNo = deleteQueue.addDelete(terms);
|
||||
flushControl.doOnDelete();
|
||||
lastSeqNo = Math.max(lastSeqNo, seqNo);
|
||||
if (applyAllDeletes(deleteQueue)) {
|
||||
seqNo = -seqNo;
|
||||
}
|
||||
lastSeqNo = Math.max(lastSeqNo, seqNo);
|
||||
return seqNo;
|
||||
}
|
||||
|
||||
|
@ -171,20 +172,21 @@ final class DocumentsWriter implements Closeable, Accountable {
|
|||
final DocumentsWriterDeleteQueue deleteQueue = this.deleteQueue;
|
||||
long seqNo = deleteQueue.addDocValuesUpdates(updates);
|
||||
flushControl.doOnDelete();
|
||||
lastSeqNo = Math.max(lastSeqNo, seqNo);
|
||||
if (applyAllDeletes(deleteQueue)) {
|
||||
seqNo = -seqNo;
|
||||
}
|
||||
lastSeqNo = Math.max(lastSeqNo, seqNo);
|
||||
return seqNo;
|
||||
}
|
||||
|
||||
DocumentsWriterDeleteQueue currentDeleteSession() {
|
||||
return deleteQueue;
|
||||
}
|
||||
|
||||
|
||||
/** If buffered deletes are using too much heap, resolve them and write disk and return true. */
|
||||
private boolean applyAllDeletes(DocumentsWriterDeleteQueue deleteQueue) throws IOException {
|
||||
if (flushControl.getAndResetApplyAllDeletes()) {
|
||||
if (deleteQueue != null && !flushControl.isFullFlush()) {
|
||||
if (deleteQueue != null) {
|
||||
ticketQueue.addDeletes(deleteQueue);
|
||||
}
|
||||
putEvent(ApplyDeletesEvent.INSTANCE); // apply deletes event forces a purge
|
||||
|
@ -200,7 +202,6 @@ final class DocumentsWriter implements Closeable, Accountable {
|
|||
return ticketQueue.tryPurge(writer);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/** Returns how many docs are currently buffered in RAM. */
|
||||
int getNumDocs() {
|
||||
|
@ -246,11 +247,13 @@ final class DocumentsWriter implements Closeable, Accountable {
|
|||
}
|
||||
|
||||
/** Returns how many documents were aborted. */
|
||||
synchronized long lockAndAbortAll(IndexWriter indexWriter) {
|
||||
synchronized long lockAndAbortAll(IndexWriter indexWriter) throws IOException {
|
||||
assert indexWriter.holdsFullFlushLock();
|
||||
if (infoStream.isEnabled("DW")) {
|
||||
infoStream.message("DW", "lockAndAbortAll");
|
||||
}
|
||||
// Make sure we move all pending tickets into the flush queue:
|
||||
ticketQueue.forcePurge(indexWriter);
|
||||
long abortedDocCount = 0;
|
||||
boolean success = false;
|
||||
try {
|
||||
|
@ -578,9 +581,7 @@ final class DocumentsWriter implements Closeable, Accountable {
|
|||
|
||||
flushingDWPT = flushControl.nextPendingFlush();
|
||||
}
|
||||
if (hasEvents) {
|
||||
putEvent(MergePendingEvent.INSTANCE);
|
||||
}
|
||||
|
||||
// If deletes alone are consuming > 1/2 our RAM
|
||||
// buffer, force them all to apply now. This is to
|
||||
// prevent too-frequent flushing of a long tail of
|
||||
|
@ -589,9 +590,9 @@ final class DocumentsWriter implements Closeable, Accountable {
|
|||
if (ramBufferSizeMB != IndexWriterConfig.DISABLE_AUTO_FLUSH &&
|
||||
flushControl.getDeleteBytesUsed() > (1024*1024*ramBufferSizeMB/2)) {
|
||||
hasEvents = true;
|
||||
if (!this.applyAllDeletes(deleteQueue)) {
|
||||
if (applyAllDeletes(deleteQueue) == false) {
|
||||
if (infoStream.isEnabled("DW")) {
|
||||
infoStream.message("DW", String.format(Locale.ROOT, "force apply deletes bytesUsed=%.1f MB vs ramBuffer=%.1f MB",
|
||||
infoStream.message("DW", String.format(Locale.ROOT, "force apply deletes after flush bytesUsed=%.1f MB vs ramBuffer=%.1f MB",
|
||||
flushControl.getDeleteBytesUsed()/(1024.*1024.),
|
||||
ramBufferSizeMB));
|
||||
}
|
||||
|
@ -654,7 +655,7 @@ final class DocumentsWriter implements Closeable, Accountable {
|
|||
}
|
||||
// If a concurrent flush is still in flight wait for it
|
||||
flushControl.waitForFlush();
|
||||
if (!anythingFlushed && flushingDeleteQueue.anyChanges()) { // apply deletes if we did not flush any document
|
||||
if (anythingFlushed == false && flushingDeleteQueue.anyChanges()) { // apply deletes if we did not flush any document
|
||||
if (infoStream.isEnabled("DW")) {
|
||||
infoStream.message("DW", Thread.currentThread().getName() + ": flush naked frozen global deletes");
|
||||
}
|
||||
|
@ -695,7 +696,7 @@ final class DocumentsWriter implements Closeable, Accountable {
|
|||
return config;
|
||||
}
|
||||
|
||||
private void putEvent(Event event) {
|
||||
void putEvent(Event event) {
|
||||
events.add(event);
|
||||
}
|
||||
|
||||
|
@ -704,6 +705,25 @@ final class DocumentsWriter implements Closeable, Accountable {
|
|||
return flushControl.ramBytesUsed();
|
||||
}
|
||||
|
||||
static final class ResolveUpdatesEvent implements Event {
|
||||
|
||||
private final FrozenBufferedUpdates packet;
|
||||
|
||||
ResolveUpdatesEvent(FrozenBufferedUpdates packet) {
|
||||
this.packet = packet;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void process(IndexWriter writer, boolean triggerMerge, boolean forcePurge) throws IOException {
|
||||
try {
|
||||
packet.apply(writer);
|
||||
} catch (Throwable t) {
|
||||
writer.tragicEvent(t, "applyUpdatesPacket");
|
||||
}
|
||||
writer.flushDeletesCount.incrementAndGet();
|
||||
}
|
||||
}
|
||||
|
||||
static final class ApplyDeletesEvent implements Event {
|
||||
static final Event INSTANCE = new ApplyDeletesEvent();
|
||||
private int instCount = 0;
|
||||
|
@ -717,21 +737,7 @@ final class DocumentsWriter implements Closeable, Accountable {
|
|||
writer.applyDeletesAndPurge(true); // we always purge!
|
||||
}
|
||||
}
|
||||
|
||||
static final class MergePendingEvent implements Event {
|
||||
static final Event INSTANCE = new MergePendingEvent();
|
||||
private int instCount = 0;
|
||||
private MergePendingEvent() {
|
||||
assert instCount == 0;
|
||||
instCount++;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void process(IndexWriter writer, boolean triggerMerge, boolean forcePurge) throws IOException {
|
||||
writer.doAfterSegmentFlushed(triggerMerge, forcePurge);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
static final class ForcedPurgeEvent implements Event {
|
||||
static final Event INSTANCE = new ForcedPurgeEvent();
|
||||
private int instCount = 0;
|
||||
|
|
|
@ -16,6 +16,7 @@
|
|||
*/
|
||||
package org.apache.lucene.index;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
@ -25,6 +26,7 @@ import org.apache.lucene.index.DocValuesUpdate.NumericDocValuesUpdate;
|
|||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.util.Accountable;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
|
||||
/**
|
||||
* {@link DocumentsWriterDeleteQueue} is a non-blocking linked pending deletes
|
||||
|
@ -85,19 +87,22 @@ final class DocumentsWriterDeleteQueue implements Accountable {
|
|||
/** Generates the sequence number that IW returns to callers changing the index, showing the effective serialization of all operations. */
|
||||
private final AtomicLong nextSeqNo;
|
||||
|
||||
private final InfoStream infoStream;
|
||||
|
||||
// for asserts
|
||||
long maxSeqNo = Long.MAX_VALUE;
|
||||
|
||||
DocumentsWriterDeleteQueue() {
|
||||
DocumentsWriterDeleteQueue(InfoStream infoStream) {
|
||||
// seqNo must start at 1 because some APIs negate this to also return a boolean
|
||||
this(0, 1);
|
||||
this(infoStream, 0, 1);
|
||||
}
|
||||
|
||||
DocumentsWriterDeleteQueue(long generation, long startSeqNo) {
|
||||
this(new BufferedUpdates("global"), generation, startSeqNo);
|
||||
DocumentsWriterDeleteQueue(InfoStream infoStream, long generation, long startSeqNo) {
|
||||
this(infoStream, new BufferedUpdates("global"), generation, startSeqNo);
|
||||
}
|
||||
|
||||
DocumentsWriterDeleteQueue(BufferedUpdates globalBufferedUpdates, long generation, long startSeqNo) {
|
||||
DocumentsWriterDeleteQueue(InfoStream infoStream, BufferedUpdates globalBufferedUpdates, long generation, long startSeqNo) {
|
||||
this.infoStream = infoStream;
|
||||
this.globalBufferedUpdates = globalBufferedUpdates;
|
||||
this.generation = generation;
|
||||
this.nextSeqNo = new AtomicLong(startSeqNo);
|
||||
|
@ -189,7 +194,7 @@ final class DocumentsWriterDeleteQueue implements Accountable {
|
|||
}
|
||||
}
|
||||
|
||||
FrozenBufferedUpdates freezeGlobalBuffer(DeleteSlice callerSlice) {
|
||||
FrozenBufferedUpdates freezeGlobalBuffer(DeleteSlice callerSlice) throws IOException {
|
||||
globalBufferLock.lock();
|
||||
/*
|
||||
* Here we freeze the global buffer so we need to lock it, apply all
|
||||
|
@ -209,9 +214,13 @@ final class DocumentsWriterDeleteQueue implements Accountable {
|
|||
globalSlice.apply(globalBufferedUpdates, BufferedUpdates.MAX_INT);
|
||||
}
|
||||
|
||||
final FrozenBufferedUpdates packet = new FrozenBufferedUpdates(globalBufferedUpdates, false);
|
||||
globalBufferedUpdates.clear();
|
||||
return packet;
|
||||
if (globalBufferedUpdates.any()) {
|
||||
final FrozenBufferedUpdates packet = new FrozenBufferedUpdates(infoStream, globalBufferedUpdates, null);
|
||||
globalBufferedUpdates.clear();
|
||||
return packet;
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
} finally {
|
||||
globalBufferLock.unlock();
|
||||
}
|
||||
|
@ -426,7 +435,7 @@ final class DocumentsWriterDeleteQueue implements Accountable {
|
|||
globalBufferLock.lock();
|
||||
try {
|
||||
forceApplyGlobalSlice();
|
||||
return globalBufferedUpdates.terms.size();
|
||||
return globalBufferedUpdates.deleteTerms.size();
|
||||
} finally {
|
||||
globalBufferLock.unlock();
|
||||
}
|
||||
|
|
|
@ -424,22 +424,16 @@ final class DocumentsWriterFlushControl implements Accountable {
|
|||
};
|
||||
}
|
||||
|
||||
|
||||
|
||||
synchronized void doOnDelete() {
|
||||
// pass null this is a global delete no update
|
||||
flushPolicy.onDelete(this, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the number of delete terms in the global pool
|
||||
*/
|
||||
public int getNumGlobalTermDeletes() {
|
||||
return documentsWriter.deleteQueue.numGlobalTermDeletes() + bufferedUpdatesStream.numTerms();
|
||||
}
|
||||
|
||||
/** Returns heap bytes currently consumed by buffered deletes/updates that would be
|
||||
* freed if we pushed all deletes. This does not include bytes consumed by
|
||||
* already pushed delete/update packets. */
|
||||
public long getDeleteBytesUsed() {
|
||||
return documentsWriter.deleteQueue.ramBytesUsed() + bufferedUpdatesStream.ramBytesUsed();
|
||||
return documentsWriter.deleteQueue.ramBytesUsed();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -501,7 +495,7 @@ final class DocumentsWriterFlushControl implements Accountable {
|
|||
seqNo = documentsWriter.deleteQueue.getLastSequenceNumber() + perThreadPool.getActiveThreadStateCount() + 2;
|
||||
flushingQueue.maxSeqNo = seqNo+1;
|
||||
|
||||
DocumentsWriterDeleteQueue newQueue = new DocumentsWriterDeleteQueue(flushingQueue.generation+1, seqNo+1);
|
||||
DocumentsWriterDeleteQueue newQueue = new DocumentsWriterDeleteQueue(infoStream, flushingQueue.generation+1, seqNo+1);
|
||||
|
||||
documentsWriter.deleteQueue = newQueue;
|
||||
}
|
||||
|
@ -648,8 +642,7 @@ final class DocumentsWriterFlushControl implements Accountable {
|
|||
}
|
||||
for (BlockedFlush blockedFlush : blockedFlushes) {
|
||||
try {
|
||||
flushingWriters
|
||||
.put(blockedFlush.dwpt, Long.valueOf(blockedFlush.bytes));
|
||||
flushingWriters.put(blockedFlush.dwpt, Long.valueOf(blockedFlush.bytes));
|
||||
documentsWriter.subtractFlushedNumDocs(blockedFlush.dwpt.getNumDocsInRAM());
|
||||
blockedFlush.dwpt.abort();
|
||||
} catch (Throwable ex) {
|
||||
|
@ -720,6 +713,4 @@ final class DocumentsWriterFlushControl implements Accountable {
|
|||
public InfoStream getInfoStream() {
|
||||
return infoStream;
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
|
|
@ -60,7 +60,7 @@ class DocumentsWriterFlushQueue {
|
|||
assert numTickets >= 0;
|
||||
}
|
||||
|
||||
synchronized SegmentFlushTicket addFlushTicket(DocumentsWriterPerThread dwpt) {
|
||||
synchronized SegmentFlushTicket addFlushTicket(DocumentsWriterPerThread dwpt) throws IOException {
|
||||
// Each flush is assigned a ticket in the order they acquire the ticketQueue
|
||||
// lock
|
||||
incTickets();
|
||||
|
@ -168,11 +168,11 @@ class DocumentsWriterFlushQueue {
|
|||
protected boolean published = false;
|
||||
|
||||
protected FlushTicket(FrozenBufferedUpdates frozenUpdates) {
|
||||
assert frozenUpdates != null;
|
||||
this.frozenUpdates = frozenUpdates;
|
||||
}
|
||||
|
||||
protected abstract void publish(IndexWriter writer) throws IOException;
|
||||
|
||||
protected abstract boolean canPublish();
|
||||
|
||||
/**
|
||||
|
@ -186,33 +186,31 @@ class DocumentsWriterFlushQueue {
|
|||
assert newSegment != null;
|
||||
assert newSegment.segmentInfo != null;
|
||||
final FrozenBufferedUpdates segmentUpdates = newSegment.segmentUpdates;
|
||||
//System.out.println("FLUSH: " + newSegment.segmentInfo.info.name);
|
||||
if (indexWriter.infoStream.isEnabled("DW")) {
|
||||
indexWriter.infoStream.message("DW", "publishFlushedSegment seg-private updates=" + segmentUpdates);
|
||||
indexWriter.infoStream.message("DW", "publishFlushedSegment seg-private updates=" + segmentUpdates);
|
||||
}
|
||||
|
||||
if (segmentUpdates != null && indexWriter.infoStream.isEnabled("DW")) {
|
||||
indexWriter.infoStream.message("DW", "flush: push buffered seg private updates: " + segmentUpdates);
|
||||
indexWriter.infoStream.message("DW", "flush: push buffered seg private updates: " + segmentUpdates);
|
||||
}
|
||||
// now publish!
|
||||
indexWriter.publishFlushedSegment(newSegment.segmentInfo, segmentUpdates, globalPacket);
|
||||
indexWriter.publishFlushedSegment(newSegment.segmentInfo, segmentUpdates, globalPacket, newSegment.sortMap);
|
||||
}
|
||||
|
||||
protected final void finishFlush(IndexWriter indexWriter, FlushedSegment newSegment, FrozenBufferedUpdates bufferedUpdates)
|
||||
throws IOException {
|
||||
// Finish the flushed segment and publish it to IndexWriter
|
||||
if (newSegment == null) {
|
||||
assert bufferedUpdates != null;
|
||||
if (bufferedUpdates != null && bufferedUpdates.any()) {
|
||||
indexWriter.publishFrozenUpdates(bufferedUpdates);
|
||||
if (indexWriter.infoStream.isEnabled("DW")) {
|
||||
indexWriter.infoStream.message("DW", "flush: push buffered updates: " + bufferedUpdates);
|
||||
}
|
||||
// Finish the flushed segment and publish it to IndexWriter
|
||||
if (newSegment == null) {
|
||||
if (bufferedUpdates != null && bufferedUpdates.any()) {
|
||||
indexWriter.publishFrozenUpdates(bufferedUpdates);
|
||||
if (indexWriter.infoStream.isEnabled("DW")) {
|
||||
indexWriter.infoStream.message("DW", "flush: push buffered updates: " + bufferedUpdates);
|
||||
}
|
||||
} else {
|
||||
publishFlushedSegment(indexWriter, newSegment, bufferedUpdates);
|
||||
}
|
||||
} else {
|
||||
publishFlushedSegment(indexWriter, newSegment, bufferedUpdates);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
static final class GlobalDeletesTicket extends FlushTicket {
|
||||
|
@ -220,6 +218,7 @@ class DocumentsWriterFlushQueue {
|
|||
protected GlobalDeletesTicket(FrozenBufferedUpdates frozenUpdates) {
|
||||
super(frozenUpdates);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void publish(IndexWriter writer) throws IOException {
|
||||
assert !published : "ticket was already publised - can not publish twice";
|
||||
|
@ -264,4 +263,4 @@ class DocumentsWriterFlushQueue {
|
|||
return segment != null || failed;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -96,15 +96,18 @@ class DocumentsWriterPerThread {
|
|||
final FieldInfos fieldInfos;
|
||||
final FrozenBufferedUpdates segmentUpdates;
|
||||
final MutableBits liveDocs;
|
||||
final Sorter.DocMap sortMap;
|
||||
final int delCount;
|
||||
|
||||
private FlushedSegment(SegmentCommitInfo segmentInfo, FieldInfos fieldInfos,
|
||||
BufferedUpdates segmentUpdates, MutableBits liveDocs, int delCount) {
|
||||
private FlushedSegment(InfoStream infoStream, SegmentCommitInfo segmentInfo, FieldInfos fieldInfos,
|
||||
BufferedUpdates segmentUpdates, MutableBits liveDocs, int delCount, Sorter.DocMap sortMap)
|
||||
throws IOException {
|
||||
this.segmentInfo = segmentInfo;
|
||||
this.fieldInfos = fieldInfos;
|
||||
this.segmentUpdates = segmentUpdates != null && segmentUpdates.any() ? new FrozenBufferedUpdates(segmentUpdates, true) : null;
|
||||
this.segmentUpdates = segmentUpdates != null && segmentUpdates.any() ? new FrozenBufferedUpdates(infoStream, segmentUpdates, segmentInfo) : null;
|
||||
this.liveDocs = liveDocs;
|
||||
this.delCount = delCount;
|
||||
this.sortMap = sortMap;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -142,7 +145,7 @@ class DocumentsWriterPerThread {
|
|||
SegmentWriteState flushState;
|
||||
// Updates for our still-in-RAM (to be flushed next) segment
|
||||
final BufferedUpdates pendingUpdates;
|
||||
private final SegmentInfo segmentInfo; // Current segment we are working on
|
||||
final SegmentInfo segmentInfo; // Current segment we are working on
|
||||
boolean aborted = false; // True if we aborted
|
||||
|
||||
private final FieldInfos.Builder fieldInfos;
|
||||
|
@ -395,7 +398,7 @@ class DocumentsWriterPerThread {
|
|||
* {@link DocumentsWriterDeleteQueue}s global buffer and apply all pending
|
||||
* deletes to this DWPT.
|
||||
*/
|
||||
FrozenBufferedUpdates prepareFlush() {
|
||||
FrozenBufferedUpdates prepareFlush() throws IOException {
|
||||
assert numDocsInRAM > 0;
|
||||
final FrozenBufferedUpdates globalUpdates = deleteQueue.freezeGlobalBuffer(deleteSlice);
|
||||
/* deleteSlice can possibly be null if we have hit non-aborting exceptions during indexing and never succeeded
|
||||
|
@ -421,14 +424,14 @@ class DocumentsWriterPerThread {
|
|||
// Apply delete-by-docID now (delete-byDocID only
|
||||
// happens when an exception is hit processing that
|
||||
// doc, eg if analyzer has some problem w/ the text):
|
||||
if (pendingUpdates.docIDs.size() > 0) {
|
||||
if (pendingUpdates.deleteDocIDs.size() > 0) {
|
||||
flushState.liveDocs = codec.liveDocsFormat().newLiveDocs(numDocsInRAM);
|
||||
for(int delDocID : pendingUpdates.docIDs) {
|
||||
for(int delDocID : pendingUpdates.deleteDocIDs) {
|
||||
flushState.liveDocs.clear(delDocID);
|
||||
}
|
||||
flushState.delCountOnFlush = pendingUpdates.docIDs.size();
|
||||
pendingUpdates.bytesUsed.addAndGet(-pendingUpdates.docIDs.size() * BufferedUpdates.BYTES_PER_DEL_DOCID);
|
||||
pendingUpdates.docIDs.clear();
|
||||
flushState.delCountOnFlush = pendingUpdates.deleteDocIDs.size();
|
||||
pendingUpdates.bytesUsed.addAndGet(-pendingUpdates.deleteDocIDs.size() * BufferedUpdates.BYTES_PER_DEL_DOCID);
|
||||
pendingUpdates.deleteDocIDs.clear();
|
||||
}
|
||||
|
||||
if (aborted) {
|
||||
|
@ -446,7 +449,8 @@ class DocumentsWriterPerThread {
|
|||
final Sorter.DocMap sortMap;
|
||||
try {
|
||||
sortMap = consumer.flush(flushState);
|
||||
pendingUpdates.terms.clear();
|
||||
// We clear this here because we already resolved them (private to this segment) when writing postings:
|
||||
pendingUpdates.clearDeleteTerms();
|
||||
segmentInfo.setFiles(new HashSet<>(directory.getCreatedFiles()));
|
||||
|
||||
final SegmentCommitInfo segmentInfoPerCommit = new SegmentCommitInfo(segmentInfo, 0, -1L, -1L, -1L);
|
||||
|
@ -463,7 +467,7 @@ class DocumentsWriterPerThread {
|
|||
}
|
||||
|
||||
final BufferedUpdates segmentDeletes;
|
||||
if (pendingUpdates.queries.isEmpty() && pendingUpdates.numericUpdates.isEmpty() && pendingUpdates.binaryUpdates.isEmpty()) {
|
||||
if (pendingUpdates.deleteQueries.isEmpty() && pendingUpdates.numericUpdates.isEmpty() && pendingUpdates.binaryUpdates.isEmpty()) {
|
||||
pendingUpdates.clear();
|
||||
segmentDeletes = null;
|
||||
} else {
|
||||
|
@ -480,13 +484,14 @@ class DocumentsWriterPerThread {
|
|||
|
||||
assert segmentInfo != null;
|
||||
|
||||
FlushedSegment fs = new FlushedSegment(segmentInfoPerCommit, flushState.fieldInfos,
|
||||
segmentDeletes, flushState.liveDocs, flushState.delCountOnFlush);
|
||||
FlushedSegment fs = new FlushedSegment(infoStream, segmentInfoPerCommit, flushState.fieldInfos,
|
||||
segmentDeletes, flushState.liveDocs, flushState.delCountOnFlush,
|
||||
sortMap);
|
||||
sealFlushedSegment(fs, sortMap);
|
||||
if (infoStream.isEnabled("DWPT")) {
|
||||
infoStream.message("DWPT", "flush time " + ((System.nanoTime() - t0)/1000000.0) + " msec");
|
||||
}
|
||||
|
||||
|
||||
return fs;
|
||||
} catch (Throwable th) {
|
||||
abort();
|
||||
|
|
|
@ -16,11 +16,12 @@
|
|||
*/
|
||||
package org.apache.lucene.index;
|
||||
|
||||
import org.apache.lucene.util.ThreadInterruptedException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
import org.apache.lucene.util.ThreadInterruptedException;
|
||||
|
||||
/**
|
||||
* {@link DocumentsWriterPerThreadPool} controls {@link ThreadState} instances
|
||||
* and their thread assignments during indexing. Each {@link ThreadState} holds
|
||||
|
@ -163,6 +164,9 @@ final class DocumentsWriterPerThreadPool {
|
|||
// don't recycle DWPT by default
|
||||
}
|
||||
|
||||
// TODO: maybe we should try to do load leveling here: we want roughly even numbers
|
||||
// of items (docs, deletes, DV updates) to most take advantage of concurrency while flushing
|
||||
|
||||
/** This method is used by DocumentsWriter/FlushControl to obtain a ThreadState to do an indexing operation (add/updateDocument). */
|
||||
ThreadState getAndLock(Thread requestingThread, DocumentsWriter documentsWriter) {
|
||||
ThreadState threadState = null;
|
||||
|
@ -208,9 +212,8 @@ final class DocumentsWriterPerThreadPool {
|
|||
state.unlock();
|
||||
synchronized (this) {
|
||||
freeList.add(state);
|
||||
// In case any thread is waiting, wake one of them up since we just released a thread state; notify() should be sufficient but we do
|
||||
// notifyAll defensively:
|
||||
notifyAll();
|
||||
// In case any thread is waiting, wake one of them up since we just released a thread state:
|
||||
notify();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -17,14 +17,13 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
|
||||
import org.apache.lucene.index.FilterLeafReader.FilterFields;
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.FilterLeafReader.FilterTerms;
|
||||
import org.apache.lucene.index.FilterLeafReader.FilterTermsEnum;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.automaton.CompiledAutomaton;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
|
||||
/**
|
||||
* The {@link ExitableDirectoryReader} wraps a real index {@link DirectoryReader} and
|
||||
|
@ -79,14 +78,12 @@ public class ExitableDirectoryReader extends FilterDirectoryReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Fields fields() throws IOException {
|
||||
Fields fields = super.fields();
|
||||
if (queryTimeout.isTimeoutEnabled()) {
|
||||
return new ExitableFields(fields, queryTimeout);
|
||||
}
|
||||
else {
|
||||
return fields; // break out of wrapper as soon as possible
|
||||
public Terms terms(String field) throws IOException {
|
||||
Terms terms = in.terms(field);
|
||||
if (terms == null) {
|
||||
return null;
|
||||
}
|
||||
return (queryTimeout.isTimeoutEnabled()) ? new ExitableTerms(terms, queryTimeout) : terms;
|
||||
}
|
||||
|
||||
// this impl does not change deletes or data so we can delegate the
|
||||
|
@ -103,29 +100,6 @@ public class ExitableDirectoryReader extends FilterDirectoryReader {
|
|||
|
||||
}
|
||||
|
||||
/**
|
||||
* Wrapper class for another Fields implementation that is used by the ExitableFilterAtomicReader.
|
||||
*/
|
||||
public static class ExitableFields extends FilterFields {
|
||||
|
||||
private QueryTimeout queryTimeout;
|
||||
|
||||
/** Constructor **/
|
||||
public ExitableFields(Fields fields, QueryTimeout queryTimeout) {
|
||||
super(fields);
|
||||
this.queryTimeout = queryTimeout;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Terms terms(String field) throws IOException {
|
||||
Terms terms = in.terms(field);
|
||||
if (terms == null) {
|
||||
return null;
|
||||
}
|
||||
return new ExitableTerms(terms, queryTimeout);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Wrapper class for another Terms implementation that is used by ExitableFields.
|
||||
*/
|
||||
|
|
|
@ -20,6 +20,7 @@ import org.apache.lucene.analysis.TokenStream; // javadocs
|
|||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.TermFrequencyAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
|
||||
import org.apache.lucene.util.AttributeSource;
|
||||
|
||||
|
@ -48,6 +49,7 @@ public final class FieldInvertState {
|
|||
PositionIncrementAttribute posIncrAttribute;
|
||||
PayloadAttribute payloadAttribute;
|
||||
TermToBytesRefAttribute termAttribute;
|
||||
TermFrequencyAttribute termFreqAttribute;
|
||||
|
||||
/** Creates {code FieldInvertState} for the specified
|
||||
* field name. */
|
||||
|
@ -88,6 +90,7 @@ public final class FieldInvertState {
|
|||
if (this.attributeSource != attributeSource) {
|
||||
this.attributeSource = attributeSource;
|
||||
termAttribute = attributeSource.getAttribute(TermToBytesRefAttribute.class);
|
||||
termFreqAttribute = attributeSource.addAttribute(TermFrequencyAttribute.class);
|
||||
posIncrAttribute = attributeSource.addAttribute(PositionIncrementAttribute.class);
|
||||
offsetAttribute = attributeSource.addAttribute(OffsetAttribute.class);
|
||||
payloadAttribute = attributeSource.getAttribute(PayloadAttribute.class);
|
||||
|
|
|
@ -20,9 +20,15 @@ package org.apache.lucene.index;
|
|||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
|
||||
/** Flex API for access to fields and terms
|
||||
* @lucene.experimental */
|
||||
import org.apache.lucene.codecs.FieldsProducer;
|
||||
|
||||
/**
|
||||
* Provides a {@link Terms} index for fields that have it, and lists which fields do.
|
||||
* This is primarily an internal/experimental API (see {@link FieldsProducer}),
|
||||
* although it is also used to expose the set of term vectors per document.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public abstract class Fields implements Iterable<String> {
|
||||
|
||||
/** Sole constructor. (For invocation by subclass
|
||||
|
|
|
@ -345,11 +345,11 @@ public abstract class FilterLeafReader extends LeafReader {
|
|||
protected void doClose() throws IOException {
|
||||
in.close();
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Fields fields() throws IOException {
|
||||
public Terms terms(String field) throws IOException {
|
||||
ensureOpen();
|
||||
return in.fields();
|
||||
return in.terms(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -29,8 +29,7 @@ import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState;
|
|||
* <li>
|
||||
* {@link #onDelete(DocumentsWriterFlushControl, DocumentsWriterPerThreadPool.ThreadState)}
|
||||
* - applies pending delete operations based on the global number of buffered
|
||||
* delete terms iff {@link IndexWriterConfig#getMaxBufferedDeleteTerms()} is
|
||||
* enabled</li>
|
||||
* delete terms if the consumed memory is greater than {@link IndexWriterConfig#getRAMBufferSizeMB()}</li>.
|
||||
* <li>
|
||||
* {@link #onInsert(DocumentsWriterFlushControl, DocumentsWriterPerThreadPool.ThreadState)}
|
||||
* - flushes either on the number of documents per
|
||||
|
@ -60,21 +59,12 @@ class FlushByRamOrCountsPolicy extends FlushPolicy {
|
|||
|
||||
@Override
|
||||
public void onDelete(DocumentsWriterFlushControl control, ThreadState state) {
|
||||
if (flushOnDeleteTerms()) {
|
||||
// Flush this state by num del terms
|
||||
final int maxBufferedDeleteTerms = indexWriterConfig
|
||||
.getMaxBufferedDeleteTerms();
|
||||
if (control.getNumGlobalTermDeletes() >= maxBufferedDeleteTerms) {
|
||||
control.setApplyAllDeletes();
|
||||
if ((flushOnRAM() && control.getDeleteBytesUsed() > 1024*1024*indexWriterConfig.getRAMBufferSizeMB())) {
|
||||
control.setApplyAllDeletes();
|
||||
if (infoStream.isEnabled("FP")) {
|
||||
infoStream.message("FP", "force apply deletes bytesUsed=" + control.getDeleteBytesUsed() + " vs ramBufferMB=" + indexWriterConfig.getRAMBufferSizeMB());
|
||||
}
|
||||
}
|
||||
if ((flushOnRAM() &&
|
||||
control.getDeleteBytesUsed() > (1024*1024*indexWriterConfig.getRAMBufferSizeMB()))) {
|
||||
control.setApplyAllDeletes();
|
||||
if (infoStream.isEnabled("FP")) {
|
||||
infoStream.message("FP", "force apply deletes bytesUsed=" + control.getDeleteBytesUsed() + " vs ramBufferMB=" + indexWriterConfig.getRAMBufferSizeMB());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -114,15 +104,6 @@ class FlushByRamOrCountsPolicy extends FlushPolicy {
|
|||
return indexWriterConfig.getMaxBufferedDocs() != IndexWriterConfig.DISABLE_AUTO_FLUSH;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns <code>true</code> if this {@link FlushPolicy} flushes on
|
||||
* {@link IndexWriterConfig#getMaxBufferedDeleteTerms()}, otherwise
|
||||
* <code>false</code>.
|
||||
*/
|
||||
protected boolean flushOnDeleteTerms() {
|
||||
return indexWriterConfig.getMaxBufferedDeleteTerms() != IndexWriterConfig.DISABLE_AUTO_FLUSH;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns <code>true</code> if this {@link FlushPolicy} flushes on
|
||||
* {@link IndexWriterConfig#getRAMBufferSizeMB()}, otherwise
|
||||
|
|
|
@ -33,9 +33,6 @@ import org.apache.lucene.util.InfoStream;
|
|||
* <li>Number of RAM resident documents - configured via
|
||||
* {@link IndexWriterConfig#setMaxBufferedDocs(int)}</li>
|
||||
* </ul>
|
||||
* The policy also applies pending delete operations (by term and/or query),
|
||||
* given the threshold set in
|
||||
* {@link IndexWriterConfig#setMaxBufferedDeleteTerms(int)}.
|
||||
* <p>
|
||||
* {@link IndexWriter} consults the provided {@link FlushPolicy} to control the
|
||||
* flushing process. The policy is informed for each added or updated document
|
||||
|
|
|
@ -34,11 +34,10 @@ final class FreqProxTermsWriter extends TermsHash {
|
|||
}
|
||||
|
||||
private void applyDeletes(SegmentWriteState state, Fields fields) throws IOException {
|
||||
|
||||
// Process any pending Term deletes for this newly
|
||||
// flushed segment:
|
||||
if (state.segUpdates != null && state.segUpdates.terms.size() > 0) {
|
||||
Map<Term,Integer> segDeletes = state.segUpdates.terms;
|
||||
if (state.segUpdates != null && state.segUpdates.deleteTerms.size() > 0) {
|
||||
Map<Term,Integer> segDeletes = state.segUpdates.deleteTerms;
|
||||
List<Term> deleteTerms = new ArrayList<>(segDeletes.keySet());
|
||||
Collections.sort(deleteTerms);
|
||||
String lastField = null;
|
||||
|
|
|
@ -113,9 +113,10 @@ final class FreqProxTermsWriterPerField extends TermsHashPerField {
|
|||
if (!hasFreq) {
|
||||
assert postings.termFreqs == null;
|
||||
postings.lastDocCodes[termID] = docState.docID;
|
||||
fieldState.maxTermFrequency = Math.max(1, fieldState.maxTermFrequency);
|
||||
} else {
|
||||
postings.lastDocCodes[termID] = docState.docID << 1;
|
||||
postings.termFreqs[termID] = 1;
|
||||
postings.termFreqs[termID] = getTermFreq();
|
||||
if (hasProx) {
|
||||
writeProx(termID, fieldState.position);
|
||||
if (hasOffsets) {
|
||||
|
@ -124,19 +125,21 @@ final class FreqProxTermsWriterPerField extends TermsHashPerField {
|
|||
} else {
|
||||
assert !hasOffsets;
|
||||
}
|
||||
fieldState.maxTermFrequency = Math.max(postings.termFreqs[termID], fieldState.maxTermFrequency);
|
||||
}
|
||||
fieldState.maxTermFrequency = Math.max(1, fieldState.maxTermFrequency);
|
||||
fieldState.uniqueTermCount++;
|
||||
}
|
||||
|
||||
@Override
|
||||
void addTerm(final int termID) {
|
||||
final FreqProxPostingsArray postings = freqProxPostingsArray;
|
||||
|
||||
assert !hasFreq || postings.termFreqs[termID] > 0;
|
||||
|
||||
if (!hasFreq) {
|
||||
assert postings.termFreqs == null;
|
||||
if (termFreqAtt.getTermFrequency() != 1) {
|
||||
throw new IllegalStateException("field \"" + fieldInfo.name + "\": must index term freq while using custom TermFrequencyAttribute");
|
||||
}
|
||||
if (docState.docID != postings.lastDocIDs[termID]) {
|
||||
// New document; now encode docCode for previous doc:
|
||||
assert docState.docID > postings.lastDocIDs[termID];
|
||||
|
@ -160,8 +163,8 @@ final class FreqProxTermsWriterPerField extends TermsHashPerField {
|
|||
}
|
||||
|
||||
// Init freq for the current document
|
||||
postings.termFreqs[termID] = 1;
|
||||
fieldState.maxTermFrequency = Math.max(1, fieldState.maxTermFrequency);
|
||||
postings.termFreqs[termID] = getTermFreq();
|
||||
fieldState.maxTermFrequency = Math.max(postings.termFreqs[termID], fieldState.maxTermFrequency);
|
||||
postings.lastDocCodes[termID] = (docState.docID - postings.lastDocIDs[termID]) << 1;
|
||||
postings.lastDocIDs[termID] = docState.docID;
|
||||
if (hasProx) {
|
||||
|
@ -175,7 +178,8 @@ final class FreqProxTermsWriterPerField extends TermsHashPerField {
|
|||
}
|
||||
fieldState.uniqueTermCount++;
|
||||
} else {
|
||||
fieldState.maxTermFrequency = Math.max(fieldState.maxTermFrequency, ++postings.termFreqs[termID]);
|
||||
postings.termFreqs[termID] = Math.addExact(postings.termFreqs[termID], getTermFreq());
|
||||
fieldState.maxTermFrequency = Math.max(fieldState.maxTermFrequency, postings.termFreqs[termID]);
|
||||
if (hasProx) {
|
||||
writeProx(termID, fieldState.position-postings.lastPositions[termID]);
|
||||
if (hasOffsets) {
|
||||
|
@ -185,6 +189,17 @@ final class FreqProxTermsWriterPerField extends TermsHashPerField {
|
|||
}
|
||||
}
|
||||
|
||||
private int getTermFreq() {
|
||||
int freq = termFreqAtt.getTermFrequency();
|
||||
if (freq != 1) {
|
||||
if (hasProx) {
|
||||
throw new IllegalStateException("field \"" + fieldInfo.name + "\": cannot index positions while using custom TermFrequencyAttribute");
|
||||
}
|
||||
}
|
||||
|
||||
return freq;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void newPostingsArray() {
|
||||
freqProxPostingsArray = (FreqProxPostingsArray) postingsArray;
|
||||
|
|
|
@ -16,167 +16,846 @@
|
|||
*/
|
||||
package org.apache.lucene.index;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
|
||||
import org.apache.lucene.index.BufferedUpdatesStream.QueryAndLimit;
|
||||
import org.apache.lucene.index.DocValuesUpdate.BinaryDocValuesUpdate;
|
||||
import org.apache.lucene.index.DocValuesUpdate.NumericDocValuesUpdate;
|
||||
import org.apache.lucene.index.PrefixCodedTerms.TermIterator;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
import org.apache.lucene.search.Weight;
|
||||
import org.apache.lucene.store.ByteArrayDataInput;
|
||||
import org.apache.lucene.store.RAMOutputStream;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
/**
|
||||
* Holds buffered deletes and updates by term or query, once pushed. Pushed
|
||||
* deletes/updates are write-once, so we shift to more memory efficient data
|
||||
* structure to hold them. We don't hold docIDs because these are applied on
|
||||
* structure to hold them. We don't hold docIDs because these are applied on
|
||||
* flush.
|
||||
*/
|
||||
class FrozenBufferedUpdates {
|
||||
|
||||
/* NOTE: we now apply this frozen packet immediately on creation, yet this process is heavy, and runs
|
||||
* in multiple threads, and this compression is sizable (~8.3% of the original size), so it's important
|
||||
* we run this before applying the deletes/updates. */
|
||||
|
||||
/* Query we often undercount (say 24 bytes), plus int. */
|
||||
final static int BYTES_PER_DEL_QUERY = RamUsageEstimator.NUM_BYTES_OBJECT_REF + Integer.BYTES + 24;
|
||||
|
||||
// Terms, in sorted order:
|
||||
final PrefixCodedTerms terms;
|
||||
final PrefixCodedTerms deleteTerms;
|
||||
|
||||
// Parallel array of deleted query, and the docIDUpto for each
|
||||
final Query[] queries;
|
||||
final int[] queryLimits;
|
||||
final Query[] deleteQueries;
|
||||
final int[] deleteQueryLimits;
|
||||
|
||||
// numeric DV update term and their updates
|
||||
final NumericDocValuesUpdate[] numericDVUpdates;
|
||||
final byte[] numericDVUpdates;
|
||||
|
||||
// binary DV update term and their updates
|
||||
final BinaryDocValuesUpdate[] binaryDVUpdates;
|
||||
final byte[] binaryDVUpdates;
|
||||
|
||||
private int numericDVUpdateCount;
|
||||
private int binaryDVUpdateCount;
|
||||
|
||||
/** Counts down once all deletes/updates have been applied */
|
||||
public final CountDownLatch applied = new CountDownLatch(1);
|
||||
|
||||
/** How many total documents were deleted/updated. */
|
||||
public long totalDelCount;
|
||||
|
||||
final int bytesUsed;
|
||||
final int numTermDeletes;
|
||||
private long gen = -1; // assigned by BufferedUpdatesStream once pushed
|
||||
final boolean isSegmentPrivate; // set to true iff this frozen packet represents
|
||||
|
||||
private long delGen = -1; // assigned by BufferedUpdatesStream once pushed
|
||||
|
||||
final SegmentCommitInfo privateSegment; // non-null iff this frozen packet represents
|
||||
// a segment private deletes. in that case is should
|
||||
// only have Queries
|
||||
// only have Queries and doc values updates
|
||||
private final InfoStream infoStream;
|
||||
|
||||
|
||||
public FrozenBufferedUpdates(BufferedUpdates deletes, boolean isSegmentPrivate) {
|
||||
this.isSegmentPrivate = isSegmentPrivate;
|
||||
assert !isSegmentPrivate || deletes.terms.size() == 0 : "segment private package should only have del queries";
|
||||
Term termsArray[] = deletes.terms.keySet().toArray(new Term[deletes.terms.size()]);
|
||||
public FrozenBufferedUpdates(InfoStream infoStream, BufferedUpdates updates, SegmentCommitInfo privateSegment) throws IOException {
|
||||
this.infoStream = infoStream;
|
||||
this.privateSegment = privateSegment;
|
||||
assert updates.deleteDocIDs.isEmpty();
|
||||
assert privateSegment == null || updates.deleteTerms.isEmpty() : "segment private packet should only have del queries";
|
||||
Term termsArray[] = updates.deleteTerms.keySet().toArray(new Term[updates.deleteTerms.size()]);
|
||||
ArrayUtil.timSort(termsArray);
|
||||
PrefixCodedTerms.Builder builder = new PrefixCodedTerms.Builder();
|
||||
for (Term term : termsArray) {
|
||||
builder.add(term);
|
||||
}
|
||||
terms = builder.finish();
|
||||
deleteTerms = builder.finish();
|
||||
|
||||
queries = new Query[deletes.queries.size()];
|
||||
queryLimits = new int[deletes.queries.size()];
|
||||
deleteQueries = new Query[updates.deleteQueries.size()];
|
||||
deleteQueryLimits = new int[updates.deleteQueries.size()];
|
||||
int upto = 0;
|
||||
for(Map.Entry<Query,Integer> ent : deletes.queries.entrySet()) {
|
||||
queries[upto] = ent.getKey();
|
||||
queryLimits[upto] = ent.getValue();
|
||||
for(Map.Entry<Query,Integer> ent : updates.deleteQueries.entrySet()) {
|
||||
deleteQueries[upto] = ent.getKey();
|
||||
deleteQueryLimits[upto] = ent.getValue();
|
||||
upto++;
|
||||
}
|
||||
|
||||
// TODO if a Term affects multiple fields, we could keep the updates key'd by Term
|
||||
// so that it maps to all fields it affects, sorted by their docUpto, and traverse
|
||||
// that Term only once, applying the update to all fields that still need to be
|
||||
// updated.
|
||||
List<NumericDocValuesUpdate> allNumericUpdates = new ArrayList<>();
|
||||
int numericUpdatesSize = 0;
|
||||
for (LinkedHashMap<Term,NumericDocValuesUpdate> numericUpdates : deletes.numericUpdates.values()) {
|
||||
for (NumericDocValuesUpdate update : numericUpdates.values()) {
|
||||
allNumericUpdates.add(update);
|
||||
numericUpdatesSize += update.sizeInBytes();
|
||||
}
|
||||
}
|
||||
numericDVUpdates = allNumericUpdates.toArray(new NumericDocValuesUpdate[allNumericUpdates.size()]);
|
||||
// updated.
|
||||
numericDVUpdates = freezeNumericDVUpdates(updates.numericUpdates);
|
||||
|
||||
// TODO if a Term affects multiple fields, we could keep the updates key'd by Term
|
||||
// so that it maps to all fields it affects, sorted by their docUpto, and traverse
|
||||
// that Term only once, applying the update to all fields that still need to be
|
||||
// updated.
|
||||
List<BinaryDocValuesUpdate> allBinaryUpdates = new ArrayList<>();
|
||||
int binaryUpdatesSize = 0;
|
||||
for (LinkedHashMap<Term,BinaryDocValuesUpdate> binaryUpdates : deletes.binaryUpdates.values()) {
|
||||
for (BinaryDocValuesUpdate update : binaryUpdates.values()) {
|
||||
allBinaryUpdates.add(update);
|
||||
binaryUpdatesSize += update.sizeInBytes();
|
||||
binaryDVUpdates = freezeBinaryDVUpdates(updates.binaryUpdates);
|
||||
|
||||
bytesUsed = (int) (deleteTerms.ramBytesUsed() + deleteQueries.length * BYTES_PER_DEL_QUERY
|
||||
+ numericDVUpdates.length + binaryDVUpdates.length);
|
||||
|
||||
numTermDeletes = updates.numTermDeletes.get();
|
||||
if (infoStream != null && infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", String.format(Locale.ROOT,
|
||||
"compressed %d to %d bytes (%.2f%%) for deletes/updates; private segment %s",
|
||||
updates.bytesUsed.get(), bytesUsed, 100.*bytesUsed/updates.bytesUsed.get(),
|
||||
privateSegment));
|
||||
}
|
||||
}
|
||||
|
||||
private byte[] freezeNumericDVUpdates(Map<String,LinkedHashMap<Term,NumericDocValuesUpdate>> numericDVUpdates)
|
||||
throws IOException {
|
||||
// TODO: we could do better here, e.g. collate the updates by field
|
||||
// so if you are updating 2 fields interleaved we don't keep writing the field strings
|
||||
|
||||
RAMOutputStream out = new RAMOutputStream();
|
||||
String lastTermField = null;
|
||||
String lastUpdateField = null;
|
||||
for (LinkedHashMap<Term,NumericDocValuesUpdate> numericUpdates : numericDVUpdates.values()) {
|
||||
numericDVUpdateCount += numericUpdates.size();
|
||||
for (NumericDocValuesUpdate update : numericUpdates.values()) {
|
||||
|
||||
int code = update.term.bytes().length << 2;
|
||||
|
||||
String termField = update.term.field();
|
||||
if (termField.equals(lastTermField) == false) {
|
||||
code |= 1;
|
||||
}
|
||||
String updateField = update.field;
|
||||
if (updateField.equals(lastUpdateField) == false) {
|
||||
code |= 2;
|
||||
}
|
||||
out.writeVInt(code);
|
||||
out.writeVInt(update.docIDUpto);
|
||||
if ((code & 1) != 0) {
|
||||
out.writeString(termField);
|
||||
lastTermField = termField;
|
||||
}
|
||||
if ((code & 2) != 0) {
|
||||
out.writeString(updateField);
|
||||
lastUpdateField = updateField;
|
||||
}
|
||||
|
||||
out.writeBytes(update.term.bytes().bytes, update.term.bytes().offset, update.term.bytes().length);
|
||||
out.writeZLong(((Long) update.value).longValue());
|
||||
}
|
||||
}
|
||||
binaryDVUpdates = allBinaryUpdates.toArray(new BinaryDocValuesUpdate[allBinaryUpdates.size()]);
|
||||
byte[] bytes = new byte[(int) out.getFilePointer()];
|
||||
out.writeTo(bytes, 0);
|
||||
return bytes;
|
||||
}
|
||||
|
||||
private byte[] freezeBinaryDVUpdates(Map<String,LinkedHashMap<Term,BinaryDocValuesUpdate>> binaryDVUpdates)
|
||||
throws IOException {
|
||||
// TODO: we could do better here, e.g. collate the updates by field
|
||||
// so if you are updating 2 fields interleaved we don't keep writing the field strings
|
||||
|
||||
RAMOutputStream out = new RAMOutputStream();
|
||||
String lastTermField = null;
|
||||
String lastUpdateField = null;
|
||||
for (LinkedHashMap<Term,BinaryDocValuesUpdate> binaryUpdates : binaryDVUpdates.values()) {
|
||||
binaryDVUpdateCount += binaryUpdates.size();
|
||||
for (BinaryDocValuesUpdate update : binaryUpdates.values()) {
|
||||
|
||||
int code = update.term.bytes().length << 2;
|
||||
|
||||
String termField = update.term.field();
|
||||
if (termField.equals(lastTermField) == false) {
|
||||
code |= 1;
|
||||
}
|
||||
String updateField = update.field;
|
||||
if (updateField.equals(lastUpdateField) == false) {
|
||||
code |= 2;
|
||||
}
|
||||
out.writeVInt(code);
|
||||
out.writeVInt(update.docIDUpto);
|
||||
if (termField.equals(lastTermField) == false) {
|
||||
out.writeString(termField);
|
||||
lastTermField = termField;
|
||||
}
|
||||
if (updateField.equals(lastUpdateField) == false) {
|
||||
out.writeString(updateField);
|
||||
lastUpdateField = updateField;
|
||||
}
|
||||
out.writeBytes(update.term.bytes().bytes, update.term.bytes().offset, update.term.bytes().length);
|
||||
|
||||
BytesRef value = (BytesRef) update.value;
|
||||
out.writeVInt(value.length);
|
||||
out.writeBytes(value.bytes, value.offset, value.length);
|
||||
}
|
||||
}
|
||||
byte[] bytes = new byte[(int) out.getFilePointer()];
|
||||
out.writeTo(bytes, 0);
|
||||
return bytes;
|
||||
}
|
||||
|
||||
/** Returns the {@link SegmentCommitInfo} that this packet is supposed to apply its deletes to, or null
|
||||
* if the private segment was already merged away. */
|
||||
private List<SegmentCommitInfo> getInfosToApply(IndexWriter writer) {
|
||||
assert Thread.holdsLock(writer);
|
||||
List<SegmentCommitInfo> infos;
|
||||
if (privateSegment != null) {
|
||||
if (writer.segmentInfos.indexOf(privateSegment) == -1) {
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", "private segment already gone; skip processing updates");
|
||||
}
|
||||
return null;
|
||||
} else {
|
||||
infos = Collections.singletonList(privateSegment);
|
||||
}
|
||||
} else {
|
||||
infos = writer.segmentInfos.asList();
|
||||
}
|
||||
return infos;
|
||||
}
|
||||
|
||||
/** Translates a frozen packet of delete term/query, or doc values
|
||||
* updates, into their actual docIDs in the index, and applies the change. This is a heavy
|
||||
* operation and is done concurrently by incoming indexing threads. */
|
||||
|
||||
public synchronized void apply(IndexWriter writer) throws IOException {
|
||||
if (applied.getCount() == 0) {
|
||||
// already done
|
||||
return;
|
||||
}
|
||||
|
||||
long startNS = System.nanoTime();
|
||||
|
||||
assert any();
|
||||
|
||||
Set<SegmentCommitInfo> seenSegments = new HashSet<>();
|
||||
|
||||
int iter = 0;
|
||||
int totalSegmentCount = 0;
|
||||
long totalDelCount = 0;
|
||||
|
||||
// Optimistic concurrency: assume we are free to resolve the deletes against all current segments in the index, despite that
|
||||
// concurrent merges are running. Once we are done, we check to see if a merge completed while we were running. If so, we must retry
|
||||
// resolving against the newly merged segment(s). Eventually no merge finishes while we were running and we are done.
|
||||
while (true) {
|
||||
String messagePrefix;
|
||||
if (iter == 0) {
|
||||
messagePrefix = "";
|
||||
} else {
|
||||
messagePrefix = "iter " + iter;
|
||||
}
|
||||
|
||||
long iterStartNS = System.nanoTime();
|
||||
|
||||
long mergeGenStart = writer.mergeFinishedGen.get();
|
||||
|
||||
Set<String> delFiles = new HashSet<>();
|
||||
BufferedUpdatesStream.SegmentState[] segStates;
|
||||
|
||||
synchronized (writer) {
|
||||
List<SegmentCommitInfo> infos = getInfosToApply(writer);
|
||||
if (infos == null) {
|
||||
break;
|
||||
}
|
||||
|
||||
for (SegmentCommitInfo info : infos) {
|
||||
delFiles.addAll(info.files());
|
||||
}
|
||||
|
||||
// Must open while holding IW lock so that e.g. segments are not merged
|
||||
// away, dropped from 100% deletions, etc., before we can open the readers
|
||||
segStates = writer.bufferedUpdatesStream.openSegmentStates(writer.readerPool, infos, seenSegments, delGen());
|
||||
|
||||
if (segStates.length == 0) {
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", "packet matches no segments");
|
||||
}
|
||||
|
||||
break;
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", String.format(Locale.ROOT,
|
||||
messagePrefix + "now apply del packet (%s) to %d segments, mergeGen %d",
|
||||
this, segStates.length, mergeGenStart));
|
||||
}
|
||||
|
||||
totalSegmentCount += segStates.length;
|
||||
|
||||
// Important, else IFD may try to delete our files while we are still using them,
|
||||
// if e.g. a merge finishes on some of the segments we are resolving on:
|
||||
writer.deleter.incRef(delFiles);
|
||||
}
|
||||
|
||||
boolean success = false;
|
||||
long delCount;
|
||||
try {
|
||||
// don't hold IW monitor lock here so threads are free concurrently resolve deletes/updates:
|
||||
delCount = apply(segStates);
|
||||
success = true;
|
||||
} finally {
|
||||
finishApply(writer, segStates, success, delFiles);
|
||||
}
|
||||
|
||||
// Since we jus resolved some more deletes/updates, now is a good time to write them:
|
||||
writer.readerPool.writeSomeDocValuesUpdates();
|
||||
|
||||
// It's OK to add this here, even if the while loop retries, because delCount only includes newly
|
||||
// deleted documents, on the segments we didn't already do in previous iterations:
|
||||
totalDelCount += delCount;
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", String.format(Locale.ROOT,
|
||||
messagePrefix + "done apply del packet (%s) to %d segments; %d new deletes/updates; took %.3f sec",
|
||||
this, segStates.length, delCount, (System.nanoTime() - iterStartNS) / 1000000000.));
|
||||
}
|
||||
|
||||
if (privateSegment != null) {
|
||||
// No need to retry for a segment-private packet: the merge that folds in our private segment already waits for all deletes to
|
||||
// be applied before it kicks off, so this private segment must already not be in the set of merging segments
|
||||
|
||||
break;
|
||||
}
|
||||
|
||||
// Must sync on writer here so that IW.mergeCommit is not running concurrently, so that if we exit, we know mergeCommit will succeed
|
||||
// in pulling all our delGens into a merge:
|
||||
synchronized (writer) {
|
||||
long mergeGenCur = writer.mergeFinishedGen.get();
|
||||
|
||||
if (mergeGenCur == mergeGenStart) {
|
||||
|
||||
// No merge finished while we were applying, so we are done!
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", messagePrefix + "concurrent merges finished; move to next iter");
|
||||
}
|
||||
|
||||
// A merge completed while we were running. In this case, that merge may have picked up some of the updates we did, but not
|
||||
// necessarily all of them, so we cycle again, re-applying all our updates to the newly merged segment.
|
||||
|
||||
iter++;
|
||||
}
|
||||
|
||||
// Record that this packet is finished:
|
||||
writer.bufferedUpdatesStream.finished(this);
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
String message = String.format(Locale.ROOT,
|
||||
"done apply del packet (%s) to %d segments; %d new deletes/updates; took %.3f sec",
|
||||
this, totalSegmentCount, totalDelCount, (System.nanoTime() - startNS) / 1000000000.);
|
||||
if (iter > 0) {
|
||||
message += "; " + (iter+1) + " iters due to concurrent merges";
|
||||
}
|
||||
message += "; " + writer.bufferedUpdatesStream.getPendingUpdatesCount() + " packets remain";
|
||||
infoStream.message("BD", message);
|
||||
}
|
||||
}
|
||||
|
||||
private void finishApply(IndexWriter writer, BufferedUpdatesStream.SegmentState[] segStates,
|
||||
boolean success, Set<String> delFiles) throws IOException {
|
||||
synchronized (writer) {
|
||||
|
||||
BufferedUpdatesStream.ApplyDeletesResult result;
|
||||
try {
|
||||
result = writer.bufferedUpdatesStream.closeSegmentStates(writer.readerPool, segStates, success);
|
||||
} finally {
|
||||
// Matches the incRef we did above, but we must do the decRef after closing segment states else
|
||||
// IFD can't delete still-open files
|
||||
writer.deleter.decRef(delFiles);
|
||||
}
|
||||
|
||||
if (result.anyDeletes) {
|
||||
writer.maybeMerge.set(true);
|
||||
writer.checkpoint();
|
||||
}
|
||||
|
||||
if (writer.keepFullyDeletedSegments == false && result.allDeleted != null) {
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "drop 100% deleted segments: " + writer.segString(result.allDeleted));
|
||||
}
|
||||
for (SegmentCommitInfo info : result.allDeleted) {
|
||||
writer.dropDeletedSegment(info);
|
||||
}
|
||||
writer.checkpoint();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Applies pending delete-by-term, delete-by-query and doc values updates to all segments in the index, returning
|
||||
* the number of new deleted or updated documents. */
|
||||
private synchronized long apply(BufferedUpdatesStream.SegmentState[] segStates) throws IOException {
|
||||
|
||||
if (delGen == -1) {
|
||||
// we were not yet pushed
|
||||
throw new IllegalArgumentException("gen is not yet set; call BufferedUpdatesStream.push first");
|
||||
}
|
||||
|
||||
if (applied.getCount() == 0) {
|
||||
// already done
|
||||
return totalDelCount;
|
||||
}
|
||||
|
||||
if (privateSegment != null) {
|
||||
assert segStates.length == 1;
|
||||
assert privateSegment == segStates[0].reader.getSegmentInfo();
|
||||
}
|
||||
|
||||
totalDelCount += applyTermDeletes(segStates);
|
||||
totalDelCount += applyQueryDeletes(segStates);
|
||||
totalDelCount += applyDocValuesUpdates(segStates);
|
||||
|
||||
return totalDelCount;
|
||||
}
|
||||
|
||||
private long applyDocValuesUpdates(BufferedUpdatesStream.SegmentState[] segStates) throws IOException {
|
||||
|
||||
if (numericDVUpdates.length == 0 && binaryDVUpdates.length == 0) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
long startNS = System.nanoTime();
|
||||
|
||||
long updateCount = 0;
|
||||
|
||||
for (BufferedUpdatesStream.SegmentState segState : segStates) {
|
||||
|
||||
if (delGen < segState.delGen) {
|
||||
// segment is newer than this deletes packet
|
||||
continue;
|
||||
}
|
||||
|
||||
if (segState.rld.refCount() == 1) {
|
||||
// This means we are the only remaining reference to this segment, meaning
|
||||
// it was merged away while we were running, so we can safely skip running
|
||||
// because we will run on the newly merged segment next:
|
||||
continue;
|
||||
}
|
||||
|
||||
if (numericDVUpdates.length > 0) {
|
||||
updateCount += applyDocValuesUpdates(segState, numericDVUpdates, true);
|
||||
}
|
||||
|
||||
if (binaryDVUpdates.length > 0) {
|
||||
updateCount += applyDocValuesUpdates(segState, binaryDVUpdates, false);
|
||||
}
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD",
|
||||
String.format(Locale.ROOT, "applyDocValuesUpdates %.1f msec for %d segments, %d numeric updates and %d binary updates; %d new updates",
|
||||
(System.nanoTime()-startNS)/1000000.,
|
||||
segStates.length,
|
||||
numericDVUpdateCount,
|
||||
binaryDVUpdateCount,
|
||||
updateCount));
|
||||
}
|
||||
|
||||
return updateCount;
|
||||
}
|
||||
|
||||
private long applyDocValuesUpdates(BufferedUpdatesStream.SegmentState segState,
|
||||
byte[] updates, boolean isNumeric) throws IOException {
|
||||
|
||||
TermsEnum termsEnum = null;
|
||||
PostingsEnum postingsEnum = null;
|
||||
|
||||
// TODO: we can process the updates per DV field, from last to first so that
|
||||
// if multiple terms affect same document for the same field, we add an update
|
||||
// only once (that of the last term). To do that, we can keep a bitset which
|
||||
// marks which documents have already been updated. So e.g. if term T1
|
||||
// updates doc 7, and then we process term T2 and it updates doc 7 as well,
|
||||
// we don't apply the update since we know T1 came last and therefore wins
|
||||
// the update.
|
||||
// We can also use that bitset as 'liveDocs' to pass to TermEnum.docs(), so
|
||||
// that these documents aren't even returned.
|
||||
|
||||
long updateCount = 0;
|
||||
|
||||
// We first write all our updates private, and only in the end publish to the ReadersAndUpdates */
|
||||
Map<String, DocValuesFieldUpdates> holder = new HashMap<>();
|
||||
|
||||
ByteArrayDataInput in = new ByteArrayDataInput(updates);
|
||||
|
||||
String termField = null;
|
||||
String updateField = null;
|
||||
BytesRef term = new BytesRef();
|
||||
term.bytes = new byte[16];
|
||||
|
||||
bytesUsed = (int) (terms.ramBytesUsed() + queries.length * BYTES_PER_DEL_QUERY
|
||||
+ numericUpdatesSize + RamUsageEstimator.shallowSizeOf(numericDVUpdates)
|
||||
+ binaryUpdatesSize + RamUsageEstimator.shallowSizeOf(binaryDVUpdates));
|
||||
BytesRef scratch = new BytesRef();
|
||||
scratch.bytes = new byte[16];
|
||||
|
||||
numTermDeletes = deletes.numTermDeletes.get();
|
||||
while (in.getPosition() != updates.length) {
|
||||
int code = in.readVInt();
|
||||
int docIDUpto = in.readVInt();
|
||||
term.length = code >> 2;
|
||||
|
||||
if ((code & 1) != 0) {
|
||||
termField = in.readString();
|
||||
}
|
||||
if ((code & 2) != 0) {
|
||||
updateField = in.readString();
|
||||
}
|
||||
|
||||
if (term.bytes.length < term.length) {
|
||||
term.bytes = ArrayUtil.grow(term.bytes, term.length);
|
||||
}
|
||||
in.readBytes(term.bytes, 0, term.length);
|
||||
|
||||
int limit;
|
||||
if (delGen == segState.delGen) {
|
||||
assert privateSegment != null;
|
||||
limit = docIDUpto;
|
||||
} else {
|
||||
limit = Integer.MAX_VALUE;
|
||||
}
|
||||
|
||||
// TODO: we traverse the terms in update order (not term order) so that we
|
||||
// apply the updates in the correct order, i.e. if two terms udpate the
|
||||
// same document, the last one that came in wins, irrespective of the
|
||||
// terms lexical order.
|
||||
// we can apply the updates in terms order if we keep an updatesGen (and
|
||||
// increment it with every update) and attach it to each NumericUpdate. Note
|
||||
// that we cannot rely only on docIDUpto because an app may send two updates
|
||||
// which will get same docIDUpto, yet will still need to respect the order
|
||||
// those updates arrived.
|
||||
|
||||
// TODO: we could at least *collate* by field?
|
||||
|
||||
// This is the field used to resolve to docIDs, e.g. an "id" field, not the doc values field we are updating!
|
||||
if ((code & 1) != 0) {
|
||||
Terms terms = segState.reader.terms(termField);
|
||||
if (terms != null) {
|
||||
termsEnum = terms.iterator();
|
||||
} else {
|
||||
termsEnum = null;
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: can we avoid boxing here w/o fully forking this method?
|
||||
Object value;
|
||||
if (isNumeric) {
|
||||
value = Long.valueOf(in.readZLong());
|
||||
} else {
|
||||
value = scratch;
|
||||
scratch.length = in.readVInt();
|
||||
if (scratch.bytes.length < scratch.length) {
|
||||
scratch.bytes = ArrayUtil.grow(scratch.bytes, scratch.length);
|
||||
}
|
||||
in.readBytes(scratch.bytes, 0, scratch.length);
|
||||
}
|
||||
|
||||
if (termsEnum == null) {
|
||||
// no terms in this segment for this field
|
||||
continue;
|
||||
}
|
||||
|
||||
if (termsEnum.seekExact(term)) {
|
||||
|
||||
// we don't need term frequencies for this
|
||||
postingsEnum = termsEnum.postings(postingsEnum, PostingsEnum.NONE);
|
||||
|
||||
DocValuesFieldUpdates dvUpdates = holder.get(updateField);
|
||||
if (dvUpdates == null) {
|
||||
if (isNumeric) {
|
||||
dvUpdates = new NumericDocValuesFieldUpdates(delGen, updateField, segState.reader.maxDoc());
|
||||
} else {
|
||||
dvUpdates = new BinaryDocValuesFieldUpdates(delGen, updateField, segState.reader.maxDoc());
|
||||
}
|
||||
|
||||
holder.put(updateField, dvUpdates);
|
||||
}
|
||||
|
||||
if (segState.rld.sortMap != null && privateSegment != null) {
|
||||
// This segment was sorted on flush; we must apply seg-private deletes carefully in this case:
|
||||
int doc;
|
||||
final Bits acceptDocs = segState.rld.getLiveDocs();
|
||||
while ((doc = postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
|
||||
|
||||
if (acceptDocs != null && acceptDocs.get(doc) == false) {
|
||||
continue;
|
||||
}
|
||||
|
||||
// The limit is in the pre-sorted doc space:
|
||||
if (segState.rld.sortMap.newToOld(doc) < limit) {
|
||||
dvUpdates.add(doc, value);
|
||||
updateCount++;
|
||||
}
|
||||
}
|
||||
} else {
|
||||
int doc;
|
||||
final Bits acceptDocs = segState.rld.getLiveDocs();
|
||||
while ((doc = postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
|
||||
if (doc >= limit) {
|
||||
break; // no more docs that can be updated for this term
|
||||
}
|
||||
if (acceptDocs != null && acceptDocs.get(doc) == false) {
|
||||
continue;
|
||||
}
|
||||
dvUpdates.add(doc, value);
|
||||
updateCount++;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// now freeze & publish:
|
||||
for (DocValuesFieldUpdates update : holder.values()) {
|
||||
if (update.any()) {
|
||||
update.finish();
|
||||
segState.rld.addDVUpdate(update);
|
||||
}
|
||||
}
|
||||
|
||||
return updateCount;
|
||||
}
|
||||
|
||||
// Delete by query
|
||||
private long applyQueryDeletes(BufferedUpdatesStream.SegmentState[] segStates) throws IOException {
|
||||
|
||||
if (deleteQueries.length == 0) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
long startNS = System.nanoTime();
|
||||
|
||||
long delCount = 0;
|
||||
for (BufferedUpdatesStream.SegmentState segState : segStates) {
|
||||
|
||||
if (delGen < segState.delGen) {
|
||||
// segment is newer than this deletes packet
|
||||
continue;
|
||||
}
|
||||
|
||||
if (segState.rld.refCount() == 1) {
|
||||
// This means we are the only remaining reference to this segment, meaning
|
||||
// it was merged away while we were running, so we can safely skip running
|
||||
// because we will run on the newly merged segment next:
|
||||
continue;
|
||||
}
|
||||
|
||||
final LeafReaderContext readerContext = segState.reader.getContext();
|
||||
for (int i = 0; i < deleteQueries.length; i++) {
|
||||
Query query = deleteQueries[i];
|
||||
int limit;
|
||||
if (delGen == segState.delGen) {
|
||||
assert privateSegment != null;
|
||||
limit = deleteQueryLimits[i];
|
||||
} else {
|
||||
limit = Integer.MAX_VALUE;
|
||||
}
|
||||
final IndexSearcher searcher = new IndexSearcher(readerContext.reader());
|
||||
searcher.setQueryCache(null);
|
||||
final Weight weight = searcher.createNormalizedWeight(query, false);
|
||||
final Scorer scorer = weight.scorer(readerContext);
|
||||
if (scorer != null) {
|
||||
final DocIdSetIterator it = scorer.iterator();
|
||||
|
||||
int docID;
|
||||
while ((docID = it.nextDoc()) < limit) {
|
||||
if (segState.rld.delete(docID)) {
|
||||
delCount++;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD",
|
||||
String.format(Locale.ROOT, "applyQueryDeletes took %.2f msec for %d segments and %d queries; %d new deletions",
|
||||
(System.nanoTime()-startNS)/1000000.,
|
||||
segStates.length,
|
||||
deleteQueries.length,
|
||||
delCount));
|
||||
}
|
||||
|
||||
return delCount;
|
||||
}
|
||||
|
||||
public void setDelGen(long gen) {
|
||||
assert this.gen == -1;
|
||||
this.gen = gen;
|
||||
terms.setDelGen(gen);
|
||||
private long applyTermDeletes(BufferedUpdatesStream.SegmentState[] segStates) throws IOException {
|
||||
|
||||
if (deleteTerms.size() == 0) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
// We apply segment-private deletes on flush:
|
||||
assert privateSegment == null;
|
||||
|
||||
try {
|
||||
long startNS = System.nanoTime();
|
||||
|
||||
long delCount = 0;
|
||||
|
||||
for (BufferedUpdatesStream.SegmentState segState : segStates) {
|
||||
assert segState.delGen != delGen: "segState.delGen=" + segState.delGen + " vs this.gen=" + delGen;
|
||||
if (segState.delGen > delGen) {
|
||||
// our deletes don't apply to this segment
|
||||
continue;
|
||||
}
|
||||
if (segState.rld.refCount() == 1) {
|
||||
// This means we are the only remaining reference to this segment, meaning
|
||||
// it was merged away while we were running, so we can safely skip running
|
||||
// because we will run on the newly merged segment next:
|
||||
continue;
|
||||
}
|
||||
|
||||
FieldTermIterator iter = deleteTerms.iterator();
|
||||
|
||||
BytesRef delTerm;
|
||||
String field = null;
|
||||
TermsEnum termsEnum = null;
|
||||
BytesRef readerTerm = null;
|
||||
PostingsEnum postingsEnum = null;
|
||||
while ((delTerm = iter.next()) != null) {
|
||||
|
||||
if (iter.field() != field) {
|
||||
// field changed
|
||||
field = iter.field();
|
||||
Terms terms = segState.reader.terms(field);
|
||||
if (terms != null) {
|
||||
termsEnum = terms.iterator();
|
||||
readerTerm = termsEnum.next();
|
||||
} else {
|
||||
termsEnum = null;
|
||||
}
|
||||
}
|
||||
|
||||
if (termsEnum != null) {
|
||||
int cmp = delTerm.compareTo(readerTerm);
|
||||
if (cmp < 0) {
|
||||
// TODO: can we advance across del terms here?
|
||||
// move to next del term
|
||||
continue;
|
||||
} else if (cmp == 0) {
|
||||
// fall through
|
||||
} else if (cmp > 0) {
|
||||
TermsEnum.SeekStatus status = termsEnum.seekCeil(delTerm);
|
||||
if (status == TermsEnum.SeekStatus.FOUND) {
|
||||
// fall through
|
||||
} else if (status == TermsEnum.SeekStatus.NOT_FOUND) {
|
||||
readerTerm = termsEnum.term();
|
||||
continue;
|
||||
} else {
|
||||
// TODO: can we advance to next field in deleted terms?
|
||||
// no more terms in this segment
|
||||
termsEnum = null;
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
||||
// we don't need term frequencies for this
|
||||
postingsEnum = termsEnum.postings(postingsEnum, PostingsEnum.NONE);
|
||||
|
||||
assert postingsEnum != null;
|
||||
|
||||
int docID;
|
||||
while ((docID = postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
|
||||
|
||||
// NOTE: there is no limit check on the docID
|
||||
// when deleting by Term (unlike by Query)
|
||||
// because on flush we apply all Term deletes to
|
||||
// each segment. So all Term deleting here is
|
||||
// against prior segments:
|
||||
if (segState.rld.delete(docID)) {
|
||||
delCount++;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD",
|
||||
String.format(Locale.ROOT, "applyTermDeletes took %.2f msec for %d segments and %d del terms; %d new deletions",
|
||||
(System.nanoTime()-startNS)/1000000.,
|
||||
segStates.length,
|
||||
deleteTerms.size(),
|
||||
delCount));
|
||||
}
|
||||
|
||||
return delCount;
|
||||
|
||||
} catch (Throwable t) {
|
||||
throw IOUtils.rethrowAlways(t);
|
||||
}
|
||||
}
|
||||
|
||||
public void setDelGen(long delGen) {
|
||||
assert this.delGen == -1: "delGen was already previously set to " + this.delGen;
|
||||
this.delGen = delGen;
|
||||
deleteTerms.setDelGen(delGen);
|
||||
}
|
||||
|
||||
public long delGen() {
|
||||
assert gen != -1;
|
||||
return gen;
|
||||
}
|
||||
|
||||
public TermIterator termIterator() {
|
||||
return terms.iterator();
|
||||
}
|
||||
|
||||
public Iterable<QueryAndLimit> queriesIterable() {
|
||||
return new Iterable<QueryAndLimit>() {
|
||||
@Override
|
||||
public Iterator<QueryAndLimit> iterator() {
|
||||
return new Iterator<QueryAndLimit>() {
|
||||
private int upto;
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
return upto < queries.length;
|
||||
}
|
||||
|
||||
@Override
|
||||
public QueryAndLimit next() {
|
||||
QueryAndLimit ret = new QueryAndLimit(queries[upto], queryLimits[upto]);
|
||||
upto++;
|
||||
return ret;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void remove() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
assert delGen != -1;
|
||||
return delGen;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
String s = "";
|
||||
String s = "delGen=" + delGen;
|
||||
if (numTermDeletes != 0) {
|
||||
s += " " + numTermDeletes + " deleted terms (unique count=" + terms.size() + ")";
|
||||
s += " numDeleteTerms=" + numTermDeletes;
|
||||
if (numTermDeletes != deleteTerms.size()) {
|
||||
s += " (" + deleteTerms.size() + " unique)";
|
||||
}
|
||||
}
|
||||
if (queries.length != 0) {
|
||||
s += " " + queries.length + " deleted queries";
|
||||
if (deleteQueries.length != 0) {
|
||||
s += " numDeleteQuerys=" + deleteQueries.length;
|
||||
}
|
||||
if (numericDVUpdates.length > 0) {
|
||||
s += " numNumericDVUpdates=" + numericDVUpdateCount;
|
||||
}
|
||||
if (binaryDVUpdates.length > 0) {
|
||||
s += " numBinaryDVUpdates=" + binaryDVUpdateCount;
|
||||
}
|
||||
if (bytesUsed != 0) {
|
||||
s += " bytesUsed=" + bytesUsed;
|
||||
}
|
||||
if (privateSegment != null) {
|
||||
s += " privateSegment=" + privateSegment;
|
||||
}
|
||||
|
||||
return s;
|
||||
}
|
||||
|
||||
boolean any() {
|
||||
return terms.size() > 0 || queries.length > 0 || numericDVUpdates.length > 0 || binaryDVUpdates.length > 0;
|
||||
return deleteTerms.size() > 0 || deleteQueries.length > 0 || numericDVUpdates.length > 0 || binaryDVUpdates.length > 0;
|
||||
}
|
||||
|
||||
boolean anyDeleteTerms() {
|
||||
return deleteTerms.size() > 0;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -510,9 +510,8 @@ final class IndexFileDeleter implements Closeable {
|
|||
assert locked();
|
||||
|
||||
assert Thread.holdsLock(writer);
|
||||
long t0 = 0;
|
||||
long t0 = System.nanoTime();
|
||||
if (infoStream.isEnabled("IFD")) {
|
||||
t0 = System.nanoTime();
|
||||
infoStream.message("IFD", "now checkpoint \"" + writer.segString(writer.toLiveInfos(segmentInfos)) + "\" [" + segmentInfos.size() + " segments " + "; isCommit = " + isCommit + "]");
|
||||
}
|
||||
|
||||
|
@ -698,7 +697,9 @@ final class IndexFileDeleter implements Closeable {
|
|||
ensureOpen();
|
||||
|
||||
if (infoStream.isEnabled("IFD")) {
|
||||
infoStream.message("IFD", "delete " + names + "");
|
||||
if (names.size() > 0) {
|
||||
infoStream.message("IFD", "delete " + names + "");
|
||||
}
|
||||
}
|
||||
|
||||
// We make two passes, first deleting any segments_N files, second deleting the rest. We do this so that if we throw exc or JVM
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -91,8 +91,12 @@ public final class IndexWriterConfig extends LiveIndexWriterConfig {
|
|||
*/
|
||||
public final static double DEFAULT_RAM_BUFFER_SIZE_MB = 16.0;
|
||||
|
||||
/** Default setting for {@link #setReaderPooling}. */
|
||||
public final static boolean DEFAULT_READER_POOLING = false;
|
||||
/** Default setting (true) for {@link #setReaderPooling}. */
|
||||
// We changed this default to true with concurrent deletes/updates (LUCENE-7868),
|
||||
// because we will otherwise need to open and close segment readers more frequently.
|
||||
// False is still supported, but will have worse performance since readers will
|
||||
// be forced to aggressively move all state to disk.
|
||||
public final static boolean DEFAULT_READER_POOLING = true;
|
||||
|
||||
/** Default value is 1945. Change using {@link #setRAMPerThreadHardLimitMB(int)} */
|
||||
public static final int DEFAULT_RAM_PER_THREAD_HARD_LIMIT_MB = 1945;
|
||||
|
@ -323,7 +327,6 @@ public final class IndexWriterConfig extends LiveIndexWriterConfig {
|
|||
* Expert: Controls when segments are flushed to disk during indexing.
|
||||
* The {@link FlushPolicy} initialized during {@link IndexWriter} instantiation and once initialized
|
||||
* the given instance is bound to this {@link IndexWriter} and should not be used with another writer.
|
||||
* @see #setMaxBufferedDeleteTerms(int)
|
||||
* @see #setMaxBufferedDocs(int)
|
||||
* @see #setRAMBufferSizeMB(double)
|
||||
*/
|
||||
|
@ -374,11 +377,6 @@ public final class IndexWriterConfig extends LiveIndexWriterConfig {
|
|||
return super.getAnalyzer();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxBufferedDeleteTerms() {
|
||||
return super.getMaxBufferedDeleteTerms();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxBufferedDocs() {
|
||||
return super.getMaxBufferedDocs();
|
||||
|
@ -424,11 +422,6 @@ public final class IndexWriterConfig extends LiveIndexWriterConfig {
|
|||
return (IndexWriterConfig) super.setMergePolicy(mergePolicy);
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexWriterConfig setMaxBufferedDeleteTerms(int maxBufferedDeleteTerms) {
|
||||
return (IndexWriterConfig) super.setMaxBufferedDeleteTerms(maxBufferedDeleteTerms);
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexWriterConfig setMaxBufferedDocs(int maxBufferedDocs) {
|
||||
return (IndexWriterConfig) super.setMaxBufferedDocs(maxBufferedDocs);
|
||||
|
|
|
@ -18,7 +18,6 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.IndexReader.CacheHelper;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
||||
/** {@code LeafReader} is an abstract class, providing an interface for accessing an
|
||||
|
@ -60,7 +59,7 @@ public abstract class LeafReader extends IndexReader {
|
|||
}
|
||||
|
||||
/**
|
||||
* Optional method: Return a {@link CacheHelper} that can be used to cache
|
||||
* Optional method: Return a {@link IndexReader.CacheHelper} that can be used to cache
|
||||
* based on the content of this leaf regardless of deletions. Two readers
|
||||
* that have the same data but different sets of deleted documents or doc
|
||||
* values updates may be considered equal. Consider using
|
||||
|
@ -73,12 +72,6 @@ public abstract class LeafReader extends IndexReader {
|
|||
*/
|
||||
public abstract CacheHelper getCoreCacheHelper();
|
||||
|
||||
/**
|
||||
* Returns {@link Fields} for this reader.
|
||||
* This method will not return null.
|
||||
*/
|
||||
public abstract Fields fields() throws IOException;
|
||||
|
||||
@Override
|
||||
public final int docFreq(Term term) throws IOException {
|
||||
final Terms terms = terms(term.field());
|
||||
|
@ -139,10 +132,8 @@ public abstract class LeafReader extends IndexReader {
|
|||
return terms.getSumTotalTermFreq();
|
||||
}
|
||||
|
||||
/** This may return null if the field does not exist.*/
|
||||
public final Terms terms(String field) throws IOException {
|
||||
return fields().terms(field);
|
||||
}
|
||||
/** Returns the {@link Terms} index for this field, or null if it has none. */
|
||||
public abstract Terms terms(String field) throws IOException;
|
||||
|
||||
/** Returns {@link PostingsEnum} for the specified term.
|
||||
* This will return null if either the field or
|
||||
|
|
|
@ -42,7 +42,6 @@ public class LiveIndexWriterConfig {
|
|||
|
||||
private volatile int maxBufferedDocs;
|
||||
private volatile double ramBufferSizeMB;
|
||||
private volatile int maxBufferedDeleteTerms;
|
||||
private volatile IndexReaderWarmer mergedSegmentWarmer;
|
||||
|
||||
// modified by IndexWriterConfig
|
||||
|
@ -109,7 +108,6 @@ public class LiveIndexWriterConfig {
|
|||
this.analyzer = analyzer;
|
||||
ramBufferSizeMB = IndexWriterConfig.DEFAULT_RAM_BUFFER_SIZE_MB;
|
||||
maxBufferedDocs = IndexWriterConfig.DEFAULT_MAX_BUFFERED_DOCS;
|
||||
maxBufferedDeleteTerms = IndexWriterConfig.DEFAULT_MAX_BUFFERED_DELETE_TERMS;
|
||||
mergedSegmentWarmer = null;
|
||||
delPolicy = new KeepOnlyLastCommitDeletionPolicy();
|
||||
commit = null;
|
||||
|
@ -135,43 +133,6 @@ public class LiveIndexWriterConfig {
|
|||
return analyzer;
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines the maximum number of delete-by-term operations that will be
|
||||
* buffered before both the buffered in-memory delete terms and queries are
|
||||
* applied and flushed.
|
||||
* <p>
|
||||
* Disabled by default (writer flushes by RAM usage).
|
||||
* <p>
|
||||
* NOTE: This setting won't trigger a segment flush.
|
||||
*
|
||||
* <p>
|
||||
* Takes effect immediately, but only the next time a document is added,
|
||||
* updated or deleted. Also, if you only delete-by-query, this setting has no
|
||||
* effect, i.e. delete queries are buffered until the next segment is flushed.
|
||||
*
|
||||
* @throws IllegalArgumentException
|
||||
* if maxBufferedDeleteTerms is enabled but smaller than 1
|
||||
*
|
||||
* @see #setRAMBufferSizeMB
|
||||
*/
|
||||
public LiveIndexWriterConfig setMaxBufferedDeleteTerms(int maxBufferedDeleteTerms) {
|
||||
if (maxBufferedDeleteTerms != IndexWriterConfig.DISABLE_AUTO_FLUSH && maxBufferedDeleteTerms < 1) {
|
||||
throw new IllegalArgumentException("maxBufferedDeleteTerms must at least be 1 when enabled");
|
||||
}
|
||||
this.maxBufferedDeleteTerms = maxBufferedDeleteTerms;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the number of buffered deleted terms that will trigger a flush of all
|
||||
* buffered deletes if enabled.
|
||||
*
|
||||
* @see #setMaxBufferedDeleteTerms(int)
|
||||
*/
|
||||
public int getMaxBufferedDeleteTerms() {
|
||||
return maxBufferedDeleteTerms;
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines the amount of RAM that may be used for buffering added documents
|
||||
* and deletions before they are flushed to the Directory. Generally for
|
||||
|
@ -195,12 +156,8 @@ public class LiveIndexWriterConfig {
|
|||
* <b>NOTE</b>: the account of RAM usage for pending deletions is only
|
||||
* approximate. Specifically, if you delete by Query, Lucene currently has no
|
||||
* way to measure the RAM usage of individual Queries so the accounting will
|
||||
* under-estimate and you should compensate by either calling commit()
|
||||
* periodically yourself, or by using {@link #setMaxBufferedDeleteTerms(int)}
|
||||
* to flush and apply buffered deletes by count instead of RAM usage (for each
|
||||
* buffered delete Query a constant number of bytes is used to estimate RAM
|
||||
* usage). Note that enabling {@link #setMaxBufferedDeleteTerms(int)} will not
|
||||
* trigger any segment flushes.
|
||||
* under-estimate and you should compensate by either calling commit() or refresh()
|
||||
* periodically yourself.
|
||||
* <p>
|
||||
* <b>NOTE</b>: It's not guaranteed that all memory resident documents are
|
||||
* flushed once this limit is exceeded. Depending on the configured
|
||||
|
@ -476,7 +433,6 @@ public class LiveIndexWriterConfig {
|
|||
sb.append("analyzer=").append(analyzer == null ? "null" : analyzer.getClass().getName()).append("\n");
|
||||
sb.append("ramBufferSizeMB=").append(getRAMBufferSizeMB()).append("\n");
|
||||
sb.append("maxBufferedDocs=").append(getMaxBufferedDocs()).append("\n");
|
||||
sb.append("maxBufferedDeleteTerms=").append(getMaxBufferedDeleteTerms()).append("\n");
|
||||
sb.append("mergedSegmentWarmer=").append(getMergedSegmentWarmer()).append("\n");
|
||||
sb.append("delPolicy=").append(getIndexDeletionPolicy().getClass().getName()).append("\n");
|
||||
IndexCommit commit = getIndexCommit();
|
||||
|
|
|
@ -70,8 +70,11 @@ class MergeReaderWrapper extends LeafReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Fields fields() throws IOException {
|
||||
return fields;
|
||||
public Terms terms(String field) throws IOException {
|
||||
ensureOpen();
|
||||
// We could check the FieldInfo IndexOptions but there's no point since
|
||||
// PostingsReader will simply return null for fields that don't exist or that have no terms index.
|
||||
return fields.terms(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -1,132 +0,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.
|
||||
*/
|
||||
package org.apache.lucene.index;
|
||||
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.index.PrefixCodedTerms.TermIterator;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.PriorityQueue;
|
||||
|
||||
/** Merges multiple {@link FieldTermIterator}s */
|
||||
class MergedPrefixCodedTermsIterator extends FieldTermIterator {
|
||||
|
||||
private static class TermMergeQueue extends PriorityQueue<TermIterator> {
|
||||
TermMergeQueue(int size) {
|
||||
super(size);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean lessThan(TermIterator a, TermIterator b) {
|
||||
int cmp = a.bytes.compareTo(b.bytes);
|
||||
if (cmp < 0) {
|
||||
return true;
|
||||
} else if (cmp > 0) {
|
||||
return false;
|
||||
} else {
|
||||
return a.delGen() > b.delGen();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static class FieldMergeQueue extends PriorityQueue<TermIterator> {
|
||||
FieldMergeQueue(int size) {
|
||||
super(size);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean lessThan(TermIterator a, TermIterator b) {
|
||||
return a.field.compareTo(b.field) < 0;
|
||||
}
|
||||
}
|
||||
|
||||
final TermMergeQueue termQueue;
|
||||
final FieldMergeQueue fieldQueue;
|
||||
|
||||
public MergedPrefixCodedTermsIterator(List<PrefixCodedTerms> termsList) {
|
||||
fieldQueue = new FieldMergeQueue(termsList.size());
|
||||
for (PrefixCodedTerms terms : termsList) {
|
||||
TermIterator iter = terms.iterator();
|
||||
iter.next();
|
||||
if (iter.field != null) {
|
||||
fieldQueue.add(iter);
|
||||
}
|
||||
}
|
||||
|
||||
termQueue = new TermMergeQueue(termsList.size());
|
||||
}
|
||||
|
||||
String field;
|
||||
|
||||
@Override
|
||||
public BytesRef next() {
|
||||
if (termQueue.size() == 0) {
|
||||
// No more terms in current field:
|
||||
if (fieldQueue.size() == 0) {
|
||||
// No more fields:
|
||||
field = null;
|
||||
return null;
|
||||
}
|
||||
|
||||
// Transfer all iterators on the next field into the term queue:
|
||||
TermIterator top = fieldQueue.pop();
|
||||
termQueue.add(top);
|
||||
field = top.field;
|
||||
assert field != null;
|
||||
|
||||
while (fieldQueue.size() != 0 && fieldQueue.top().field.equals(top.field)) {
|
||||
TermIterator iter = fieldQueue.pop();
|
||||
assert iter.field.equals(field);
|
||||
// TODO: a little bit evil; we do this so we can == on field down below:
|
||||
iter.field = field;
|
||||
termQueue.add(iter);
|
||||
}
|
||||
|
||||
return termQueue.top().bytes;
|
||||
} else {
|
||||
TermIterator top = termQueue.top();
|
||||
if (top.next() == null) {
|
||||
termQueue.pop();
|
||||
} else if (top.field() != field) {
|
||||
// Field changed
|
||||
termQueue.pop();
|
||||
fieldQueue.add(top);
|
||||
} else {
|
||||
termQueue.updateTop();
|
||||
}
|
||||
if (termQueue.size() == 0) {
|
||||
// Recurse (just once) to go to next field:
|
||||
return next();
|
||||
} else {
|
||||
// Still terms left in this field
|
||||
return termQueue.top().bytes;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String field() {
|
||||
return field;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long delGen() {
|
||||
return termQueue.top().delGen();
|
||||
}
|
||||
}
|
||||
|
|
@ -20,6 +20,7 @@ package org.apache.lucene.index;
|
|||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
@ -31,11 +32,12 @@ import org.apache.lucene.util.BytesRef;
|
|||
import org.apache.lucene.util.MergedIterator;
|
||||
|
||||
/**
|
||||
* Exposes flex API, merged from flex API of sub-segments.
|
||||
* Provides a single {@link Fields} term index view over an
|
||||
* {@link IndexReader}.
|
||||
* This is useful when you're interacting with an {@link
|
||||
* IndexReader} implementation that consists of sequential
|
||||
* sub-readers (eg {@link DirectoryReader} or {@link
|
||||
* MultiReader}).
|
||||
* MultiReader}) and you must treat it as a {@link LeafReader}.
|
||||
*
|
||||
* <p><b>NOTE</b>: for composite readers, you'll get better
|
||||
* performance by gathering the sub readers using
|
||||
|
@ -45,7 +47,6 @@ import org.apache.lucene.util.MergedIterator;
|
|||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
|
||||
public final class MultiFields extends Fields {
|
||||
private final Fields[] subs;
|
||||
private final ReaderSlice[] subSlices;
|
||||
|
@ -64,13 +65,13 @@ public final class MultiFields extends Fields {
|
|||
switch (leaves.size()) {
|
||||
case 1:
|
||||
// already an atomic reader / reader with one leave
|
||||
return leaves.get(0).reader().fields();
|
||||
return new LeafReaderFields(leaves.get(0).reader());
|
||||
default:
|
||||
final List<Fields> fields = new ArrayList<>(leaves.size());
|
||||
final List<ReaderSlice> slices = new ArrayList<>(leaves.size());
|
||||
for (final LeafReaderContext ctx : leaves) {
|
||||
final LeafReader r = ctx.reader();
|
||||
final Fields f = r.fields();
|
||||
final Fields f = new LeafReaderFields(r);
|
||||
fields.add(f);
|
||||
slices.add(new ReaderSlice(ctx.docBase, r.maxDoc(), fields.size()-1));
|
||||
}
|
||||
|
@ -115,9 +116,31 @@ public final class MultiFields extends Fields {
|
|||
}
|
||||
}
|
||||
|
||||
/** This method may return null if the field does not exist.*/
|
||||
/** This method may return null if the field does not exist or if it has no terms. */
|
||||
public static Terms getTerms(IndexReader r, String field) throws IOException {
|
||||
return getFields(r).terms(field);
|
||||
final List<LeafReaderContext> leaves = r.leaves();
|
||||
if (leaves.size() == 1) {
|
||||
return leaves.get(0).reader().terms(field);
|
||||
}
|
||||
|
||||
final List<Terms> termsPerLeaf = new ArrayList<>(leaves.size());
|
||||
final List<ReaderSlice> slicePerLeaf = new ArrayList<>(leaves.size());
|
||||
|
||||
for (int leafIdx = 0; leafIdx < leaves.size(); leafIdx++) {
|
||||
LeafReaderContext ctx = leaves.get(leafIdx);
|
||||
Terms subTerms = ctx.reader().terms(field);
|
||||
if (subTerms != null) {
|
||||
termsPerLeaf.add(subTerms);
|
||||
slicePerLeaf.add(new ReaderSlice(ctx.docBase, r.maxDoc(), leafIdx - 1));
|
||||
}
|
||||
}
|
||||
|
||||
if (termsPerLeaf.size() == 0) {
|
||||
return null;
|
||||
} else {
|
||||
return new MultiTerms(termsPerLeaf.toArray(Terms.EMPTY_ARRAY),
|
||||
slicePerLeaf.toArray(ReaderSlice.EMPTY_ARRAY));
|
||||
}
|
||||
}
|
||||
|
||||
/** Returns {@link PostingsEnum} for the specified field and
|
||||
|
@ -264,5 +287,37 @@ public final class MultiFields extends Fields {
|
|||
}
|
||||
return fields;
|
||||
}
|
||||
|
||||
private static class LeafReaderFields extends Fields {
|
||||
|
||||
private final LeafReader leafReader;
|
||||
private final List<String> indexedFields;
|
||||
|
||||
LeafReaderFields(LeafReader leafReader) {
|
||||
this.leafReader = leafReader;
|
||||
this.indexedFields = new ArrayList<>();
|
||||
for (FieldInfo fieldInfo : leafReader.getFieldInfos()) {
|
||||
if (fieldInfo.getIndexOptions() != IndexOptions.NONE) {
|
||||
indexedFields.add(fieldInfo.name);
|
||||
}
|
||||
}
|
||||
Collections.sort(indexedFields);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<String> iterator() {
|
||||
return Collections.unmodifiableList(indexedFields).iterator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int size() {
|
||||
return indexedFields.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Terms terms(String field) throws IOException {
|
||||
return leafReader.terms(field);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.lucene.index;
|
|||
import org.apache.lucene.document.NumericDocValuesField;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.util.InPlaceMergeSorter;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
import org.apache.lucene.util.packed.PagedGrowableWriter;
|
||||
import org.apache.lucene.util.packed.PagedMutable;
|
||||
|
@ -40,11 +41,13 @@ class NumericDocValuesFieldUpdates extends DocValuesFieldUpdates {
|
|||
private long idx = 0; // long so we don't overflow if size == Integer.MAX_VALUE
|
||||
private int doc = -1;
|
||||
private Long value = null;
|
||||
private final long delGen;
|
||||
|
||||
Iterator(int size, PagedGrowableWriter values, PagedMutable docs) {
|
||||
Iterator(int size, PagedGrowableWriter values, PagedMutable docs, long delGen) {
|
||||
this.size = size;
|
||||
this.values = values;
|
||||
this.docs = docs;
|
||||
this.delGen = delGen;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -61,6 +64,7 @@ class NumericDocValuesFieldUpdates extends DocValuesFieldUpdates {
|
|||
doc = (int) docs.get(idx);
|
||||
++idx;
|
||||
while (idx < size && docs.get(idx) == doc) {
|
||||
// scan forward to last update to this doc
|
||||
++idx;
|
||||
}
|
||||
// idx points to the "next" element
|
||||
|
@ -72,12 +76,10 @@ class NumericDocValuesFieldUpdates extends DocValuesFieldUpdates {
|
|||
int doc() {
|
||||
return doc;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
void reset() {
|
||||
doc = -1;
|
||||
value = null;
|
||||
idx = 0;
|
||||
long delGen() {
|
||||
return delGen;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -86,16 +88,26 @@ class NumericDocValuesFieldUpdates extends DocValuesFieldUpdates {
|
|||
private PagedGrowableWriter values;
|
||||
private int size;
|
||||
|
||||
public NumericDocValuesFieldUpdates(String field, int maxDoc) {
|
||||
super(field, DocValuesType.NUMERIC);
|
||||
public NumericDocValuesFieldUpdates(long delGen, String field, int maxDoc) {
|
||||
super(maxDoc, delGen, field, DocValuesType.NUMERIC);
|
||||
bitsPerValue = PackedInts.bitsRequired(maxDoc - 1);
|
||||
docs = new PagedMutable(1, PAGE_SIZE, bitsPerValue, PackedInts.COMPACT);
|
||||
values = new PagedGrowableWriter(1, PAGE_SIZE, 1, PackedInts.FAST);
|
||||
size = 0;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void add(int doc, Object value) {
|
||||
public int size() {
|
||||
return size;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void add(int doc, Object value) {
|
||||
if (finished) {
|
||||
throw new IllegalStateException("already finished");
|
||||
}
|
||||
|
||||
assert doc < maxDoc;
|
||||
|
||||
// TODO: if the Sorter interface changes to take long indexes, we can remove that limitation
|
||||
if (size == Integer.MAX_VALUE) {
|
||||
throw new IllegalStateException("cannot support more than Integer.MAX_VALUE doc/value entries");
|
||||
|
@ -113,11 +125,20 @@ class NumericDocValuesFieldUpdates extends DocValuesFieldUpdates {
|
|||
values.set(size, val.longValue());
|
||||
++size;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Iterator iterator() {
|
||||
final PagedMutable docs = this.docs;
|
||||
final PagedGrowableWriter values = this.values;
|
||||
public void finish() {
|
||||
if (finished) {
|
||||
throw new IllegalStateException("already finished");
|
||||
}
|
||||
finished = true;
|
||||
|
||||
// shrink wrap
|
||||
if (size < docs.size()) {
|
||||
docs = docs.resize(size);
|
||||
values = values.resize(size);
|
||||
}
|
||||
|
||||
new InPlaceMergeSorter() {
|
||||
@Override
|
||||
protected void swap(int i, int j) {
|
||||
|
@ -129,48 +150,36 @@ class NumericDocValuesFieldUpdates extends DocValuesFieldUpdates {
|
|||
values.set(j, values.get(i));
|
||||
values.set(i, tmpVal);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
protected int compare(int i, int j) {
|
||||
int x = (int) docs.get(i);
|
||||
int y = (int) docs.get(j);
|
||||
return (x < y) ? -1 : ((x == y) ? 0 : 1);
|
||||
// increasing docID order:
|
||||
// NOTE: we can have ties here, when the same docID was updated in the same segment, in which case we rely on sort being
|
||||
// stable and preserving original order so the last update to that docID wins
|
||||
return Integer.compare((int) docs.get(i), (int) docs.get(j));
|
||||
}
|
||||
}.sort(0, size);
|
||||
|
||||
return new Iterator(size, values, docs);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void merge(DocValuesFieldUpdates other) {
|
||||
assert other instanceof NumericDocValuesFieldUpdates;
|
||||
NumericDocValuesFieldUpdates otherUpdates = (NumericDocValuesFieldUpdates) other;
|
||||
if (otherUpdates.size > Integer.MAX_VALUE - size) {
|
||||
throw new IllegalStateException(
|
||||
"cannot support more than Integer.MAX_VALUE doc/value entries; size="
|
||||
+ size + " other.size=" + otherUpdates.size);
|
||||
}
|
||||
docs = docs.grow(size + otherUpdates.size);
|
||||
values = values.grow(size + otherUpdates.size);
|
||||
for (int i = 0; i < otherUpdates.size; i++) {
|
||||
int doc = (int) otherUpdates.docs.get(i);
|
||||
docs.set(size, doc);
|
||||
values.set(size, otherUpdates.values.get(i));
|
||||
++size;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator iterator() {
|
||||
if (finished == false) {
|
||||
throw new IllegalStateException("call finish first");
|
||||
}
|
||||
return new Iterator(size, values, docs, delGen);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean any() {
|
||||
return size > 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ramBytesPerDoc() {
|
||||
long bytesPerDoc = (long) Math.ceil((double) (bitsPerValue) / 8);
|
||||
final int capacity = estimateCapacity(size);
|
||||
bytesPerDoc += (long) Math.ceil((double) values.ramBytesUsed() / capacity); // values
|
||||
return bytesPerDoc;
|
||||
public long ramBytesUsed() {
|
||||
return values.ramBytesUsed()
|
||||
+ docs.ramBytesUsed()
|
||||
+ RamUsageEstimator.NUM_BYTES_OBJECT_HEADER
|
||||
+ 2 * RamUsageEstimator.NUM_BYTES_INT
|
||||
+ 2 * RamUsageEstimator.NUM_BYTES_OBJECT_REF;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.IdentityHashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
|
@ -50,7 +51,6 @@ import org.apache.lucene.util.Version;
|
|||
*/
|
||||
public class ParallelLeafReader extends LeafReader {
|
||||
private final FieldInfos fieldInfos;
|
||||
private final ParallelFields fields = new ParallelFields();
|
||||
private final LeafReader[] parallelReaders, storedFieldsReaders;
|
||||
private final Set<LeafReader> completeReaderSet =
|
||||
Collections.newSetFromMap(new IdentityHashMap<LeafReader,Boolean>());
|
||||
|
@ -58,9 +58,10 @@ public class ParallelLeafReader extends LeafReader {
|
|||
private final int maxDoc, numDocs;
|
||||
private final boolean hasDeletions;
|
||||
private final LeafMetaData metaData;
|
||||
private final SortedMap<String,LeafReader> fieldToReader = new TreeMap<>();
|
||||
private final SortedMap<String,LeafReader> tvFieldToReader = new TreeMap<>();
|
||||
|
||||
private final SortedMap<String,LeafReader> fieldToReader = new TreeMap<>();//TODO needn't sort?
|
||||
private final Map<String,LeafReader> termsFieldToReader = new HashMap<>();
|
||||
|
||||
/** Create a ParallelLeafReader based on the provided
|
||||
* readers; auto-closes the given readers on {@link #close()}. */
|
||||
public ParallelLeafReader(LeafReader... readers) throws IOException {
|
||||
|
@ -130,9 +131,15 @@ public class ParallelLeafReader extends LeafReader {
|
|||
if (!fieldToReader.containsKey(fieldInfo.name)) {
|
||||
builder.add(fieldInfo);
|
||||
fieldToReader.put(fieldInfo.name, reader);
|
||||
// only add these if the reader responsible for that field name is the current:
|
||||
// TODO consider populating 1st leaf with vectors even if the field name has been seen on a previous leaf
|
||||
if (fieldInfo.hasVectors()) {
|
||||
tvFieldToReader.put(fieldInfo.name, reader);
|
||||
}
|
||||
// TODO consider populating 1st leaf with terms even if the field name has been seen on a previous leaf
|
||||
if (fieldInfo.getIndexOptions() != IndexOptions.NONE) {
|
||||
termsFieldToReader.put(fieldInfo.name, reader);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -154,17 +161,6 @@ public class ParallelLeafReader extends LeafReader {
|
|||
|
||||
fieldInfos = builder.finish();
|
||||
this.metaData = new LeafMetaData(createdVersionMajor, minVersion, indexSort);
|
||||
|
||||
// build Fields instance
|
||||
for (final LeafReader reader : this.parallelReaders) {
|
||||
final Fields readerFields = reader.fields();
|
||||
for (String field : readerFields) {
|
||||
// only add if the reader responsible for that field name is the current:
|
||||
if (fieldToReader.get(field) == reader) {
|
||||
this.fields.addField(field, readerFields.terms(field));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// do this finally so any Exceptions occurred before don't affect refcounts:
|
||||
for (LeafReader reader : completeReaderSet) {
|
||||
|
@ -230,13 +226,14 @@ public class ParallelLeafReader extends LeafReader {
|
|||
ensureOpen();
|
||||
return hasDeletions ? parallelReaders[0].getLiveDocs() : null;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Fields fields() {
|
||||
public Terms terms(String field) throws IOException {
|
||||
ensureOpen();
|
||||
return fields;
|
||||
LeafReader leafReader = termsFieldToReader.get(field);
|
||||
return leafReader == null ? null : leafReader.terms(field);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public int numDocs() {
|
||||
// Don't call ensureOpen() here (it could affect performance)
|
||||
|
|
|
@ -129,7 +129,7 @@ public class PrefixCodedTerms implements Accountable {
|
|||
|
||||
private TermIterator(long delGen, RAMFile buffer) {
|
||||
try {
|
||||
input = new RAMInputStream("MergedPrefixCodedTermsIterator", buffer);
|
||||
input = new RAMInputStream("PrefixCodedTermsIterator", buffer);
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
|
|
|
@ -18,11 +18,17 @@ package org.apache.lucene.index;
|
|||
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
|
@ -36,6 +42,7 @@ import org.apache.lucene.store.TrackingDirectoryWrapper;
|
|||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
import org.apache.lucene.util.MutableBits;
|
||||
|
||||
// Used by IndexWriter to hold open SegmentReaders (for
|
||||
|
@ -76,8 +83,20 @@ class ReadersAndUpdates {
|
|||
// That way, when the segment is done merging, IndexWriter can apply the
|
||||
// updates on the merged segment too.
|
||||
private boolean isMerging = false;
|
||||
|
||||
private final Map<String,DocValuesFieldUpdates> mergingDVUpdates = new HashMap<>();
|
||||
|
||||
// Holds resolved (to docIDs) doc values updates that have not yet been
|
||||
// written to the index
|
||||
private final Map<String,List<DocValuesFieldUpdates>> pendingDVUpdates = new HashMap<>();
|
||||
|
||||
// Holds resolved (to docIDs) doc values updates that were resolved while
|
||||
// this segment was being merged; at the end of the merge we carry over
|
||||
// these updates (remapping their docIDs) to the newly merged segment
|
||||
private final Map<String,List<DocValuesFieldUpdates>> mergingDVUpdates = new HashMap<>();
|
||||
|
||||
// Only set if there are doc values updates against this segment, and the index is sorted:
|
||||
Sorter.DocMap sortMap;
|
||||
|
||||
public final AtomicLong ramBytesUsed = new AtomicLong();
|
||||
|
||||
public ReadersAndUpdates(IndexWriter writer, SegmentCommitInfo info) {
|
||||
this.writer = writer;
|
||||
|
@ -100,12 +119,12 @@ class ReadersAndUpdates {
|
|||
|
||||
public void incRef() {
|
||||
final int rc = refCount.incrementAndGet();
|
||||
assert rc > 1;
|
||||
assert rc > 1: "seg=" + info;
|
||||
}
|
||||
|
||||
public void decRef() {
|
||||
final int rc = refCount.decrementAndGet();
|
||||
assert rc >= 0;
|
||||
assert rc >= 0: "seg=" + info;
|
||||
}
|
||||
|
||||
public int refCount() {
|
||||
|
@ -117,6 +136,52 @@ class ReadersAndUpdates {
|
|||
public synchronized int getPendingDeleteCount() {
|
||||
return pendingDeleteCount;
|
||||
}
|
||||
|
||||
private synchronized boolean assertNoDupGen(List<DocValuesFieldUpdates> fieldUpdates, DocValuesFieldUpdates update) {
|
||||
for (int i=0;i<fieldUpdates.size();i++) {
|
||||
DocValuesFieldUpdates oldUpdate = fieldUpdates.get(i);
|
||||
if (oldUpdate.delGen == update.delGen) {
|
||||
throw new AssertionError("duplicate delGen=" + update.delGen + " for seg=" + info);
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
/** Adds a new resolved (meaning it maps docIDs to new values) doc values packet. We buffer these in RAM and write to disk when too much
|
||||
* RAM is used or when a merge needs to kick off, or a commit/refresh. */
|
||||
public synchronized void addDVUpdate(DocValuesFieldUpdates update) {
|
||||
if (update.getFinished() == false) {
|
||||
throw new IllegalArgumentException("call finish first");
|
||||
}
|
||||
List<DocValuesFieldUpdates> fieldUpdates = pendingDVUpdates.get(update.field);
|
||||
if (fieldUpdates == null) {
|
||||
fieldUpdates = new ArrayList<>();
|
||||
pendingDVUpdates.put(update.field, fieldUpdates);
|
||||
}
|
||||
|
||||
assert assertNoDupGen(fieldUpdates, update);
|
||||
|
||||
ramBytesUsed.addAndGet(update.ramBytesUsed());
|
||||
|
||||
fieldUpdates.add(update);
|
||||
|
||||
if (isMerging) {
|
||||
fieldUpdates = mergingDVUpdates.get(update.field);
|
||||
if (fieldUpdates == null) {
|
||||
fieldUpdates = new ArrayList<>();
|
||||
mergingDVUpdates.put(update.field, fieldUpdates);
|
||||
}
|
||||
fieldUpdates.add(update);
|
||||
}
|
||||
}
|
||||
|
||||
public synchronized long getNumDVUpdates() {
|
||||
long count = 0;
|
||||
for (List<DocValuesFieldUpdates> updates : pendingDVUpdates.values()) {
|
||||
count += updates.size();
|
||||
}
|
||||
return count;
|
||||
}
|
||||
|
||||
// Call only from assert!
|
||||
public synchronized boolean verifyDocCounts() {
|
||||
|
@ -137,7 +202,7 @@ class ReadersAndUpdates {
|
|||
}
|
||||
|
||||
/** Returns a {@link SegmentReader}. */
|
||||
public SegmentReader getReader(IOContext context) throws IOException {
|
||||
public synchronized SegmentReader getReader(IOContext context) throws IOException {
|
||||
if (reader == null) {
|
||||
// We steal returned ref:
|
||||
reader = new SegmentReader(info, writer.segmentInfos.getIndexCreatedVersionMajor(), context);
|
||||
|
@ -156,16 +221,15 @@ class ReadersAndUpdates {
|
|||
sr.decRef();
|
||||
}
|
||||
|
||||
public synchronized boolean delete(int docID) {
|
||||
public synchronized boolean delete(int docID) throws IOException {
|
||||
initWritableLiveDocs();
|
||||
assert liveDocs != null;
|
||||
assert Thread.holdsLock(writer);
|
||||
assert docID >= 0 && docID < liveDocs.length() : "out of bounds: docid=" + docID + " liveDocsLength=" + liveDocs.length() + " seg=" + info.info.name + " maxDoc=" + info.info.maxDoc();
|
||||
assert !liveDocsShared;
|
||||
final boolean didDelete = liveDocs.get(docID);
|
||||
if (didDelete) {
|
||||
((MutableBits) liveDocs).clear(docID);
|
||||
pendingDeleteCount++;
|
||||
//System.out.println(" new del seg=" + info + " docID=" + docID + " pendingDelCount=" + pendingDeleteCount + " totDelCount=" + (info.info.maxDoc()-liveDocs.count()));
|
||||
}
|
||||
return didDelete;
|
||||
}
|
||||
|
@ -175,7 +239,6 @@ class ReadersAndUpdates {
|
|||
// TODO: can we somehow use IOUtils here...? problem is
|
||||
// we are calling .decRef not .close)...
|
||||
if (reader != null) {
|
||||
//System.out.println(" pool.drop info=" + info + " rc=" + reader.getRefCount());
|
||||
try {
|
||||
reader.decRef();
|
||||
} finally {
|
||||
|
@ -207,10 +270,8 @@ class ReadersAndUpdates {
|
|||
}
|
||||
}
|
||||
|
||||
public synchronized void initWritableLiveDocs() throws IOException {
|
||||
assert Thread.holdsLock(writer);
|
||||
private synchronized void initWritableLiveDocs() throws IOException {
|
||||
assert info.info.maxDoc() > 0;
|
||||
//System.out.println("initWritableLivedocs seg=" + info + " liveDocs=" + liveDocs + " shared=" + shared);
|
||||
if (liveDocsShared) {
|
||||
// Copy on write: this means we've cloned a
|
||||
// SegmentReader sharing the current liveDocs
|
||||
|
@ -218,7 +279,6 @@ class ReadersAndUpdates {
|
|||
// change it:
|
||||
LiveDocsFormat liveDocsFormat = info.info.getCodec().liveDocsFormat();
|
||||
if (liveDocs == null) {
|
||||
//System.out.println("create BV seg=" + info);
|
||||
liveDocs = liveDocsFormat.newLiveDocs(info.info.maxDoc());
|
||||
} else {
|
||||
liveDocs = liveDocsFormat.newLiveDocs(liveDocs);
|
||||
|
@ -228,21 +288,16 @@ class ReadersAndUpdates {
|
|||
}
|
||||
|
||||
public synchronized Bits getLiveDocs() {
|
||||
assert Thread.holdsLock(writer);
|
||||
return liveDocs;
|
||||
}
|
||||
|
||||
public synchronized Bits getReadOnlyLiveDocs() {
|
||||
//System.out.println("getROLiveDocs seg=" + info);
|
||||
assert Thread.holdsLock(writer);
|
||||
liveDocsShared = true;
|
||||
//if (liveDocs != null) {
|
||||
//System.out.println(" liveCount=" + liveDocs.count());
|
||||
//}
|
||||
return liveDocs;
|
||||
}
|
||||
|
||||
public synchronized void dropChanges() {
|
||||
assert Thread.holdsLock(writer);
|
||||
// Discard (don't save) changes when we are dropping
|
||||
// the reader; this is used only on the sub-readers
|
||||
// after a successful merge. If deletes had
|
||||
|
@ -258,8 +313,6 @@ class ReadersAndUpdates {
|
|||
// _X_N updates files) to the directory; returns true if it wrote any file
|
||||
// and false if there were no new deletes or updates to write:
|
||||
public synchronized boolean writeLiveDocs(Directory dir) throws IOException {
|
||||
assert Thread.holdsLock(writer);
|
||||
//System.out.println("rld.writeLiveDocs seg=" + info + " pendingDelCount=" + pendingDeleteCount + " numericUpdates=" + numericUpdates);
|
||||
if (pendingDeleteCount == 0) {
|
||||
return false;
|
||||
}
|
||||
|
@ -304,16 +357,43 @@ class ReadersAndUpdates {
|
|||
}
|
||||
|
||||
@SuppressWarnings("synthetic-access")
|
||||
private void handleNumericDVUpdates(FieldInfos infos, Map<String,NumericDocValuesFieldUpdates> updates,
|
||||
Directory dir, DocValuesFormat dvFormat, final SegmentReader reader, Map<Integer,Set<String>> fieldFiles) throws IOException {
|
||||
for (Entry<String,NumericDocValuesFieldUpdates> e : updates.entrySet()) {
|
||||
final String field = e.getKey();
|
||||
final NumericDocValuesFieldUpdates fieldUpdates = e.getValue();
|
||||
private synchronized void handleNumericDVUpdates(FieldInfos infos,
|
||||
Directory dir, DocValuesFormat dvFormat, final SegmentReader reader,
|
||||
Map<Integer,Set<String>> fieldFiles, long maxDelGen, InfoStream infoStream) throws IOException {
|
||||
|
||||
for (Entry<String,List<DocValuesFieldUpdates>> ent : pendingDVUpdates.entrySet()) {
|
||||
final String field = ent.getKey();
|
||||
final List<DocValuesFieldUpdates> updates = ent.getValue();
|
||||
if (updates.get(0).type != DocValuesType.NUMERIC) {
|
||||
continue;
|
||||
}
|
||||
|
||||
final List<DocValuesFieldUpdates> updatesToApply = new ArrayList<>();
|
||||
long bytes = 0;
|
||||
for(DocValuesFieldUpdates update : updates) {
|
||||
if (update.delGen <= maxDelGen) {
|
||||
// safe to apply this one
|
||||
bytes += update.ramBytesUsed();
|
||||
updatesToApply.add(update);
|
||||
}
|
||||
}
|
||||
if (updatesToApply.isEmpty()) {
|
||||
// nothing to apply yet
|
||||
continue;
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", String.format(Locale.ROOT,
|
||||
"now write %d pending numeric DV updates for field=%s, seg=%s, bytes=%.3f MB",
|
||||
updatesToApply.size(),
|
||||
field,
|
||||
info,
|
||||
bytes/1024./1024.));
|
||||
}
|
||||
|
||||
final long nextDocValuesGen = info.getNextDocValuesGen();
|
||||
final String segmentSuffix = Long.toString(nextDocValuesGen, Character.MAX_RADIX);
|
||||
final long estUpdatesSize = fieldUpdates.ramBytesPerDoc() * info.info.maxDoc();
|
||||
final IOContext updatesContext = new IOContext(new FlushInfo(info.info.maxDoc(), estUpdatesSize));
|
||||
final IOContext updatesContext = new IOContext(new FlushInfo(info.info.maxDoc(), bytes));
|
||||
final FieldInfo fieldInfo = infos.fieldInfo(field);
|
||||
assert fieldInfo != null;
|
||||
fieldInfo.setDocValuesGen(nextDocValuesGen);
|
||||
|
@ -330,11 +410,14 @@ class ReadersAndUpdates {
|
|||
throw new IllegalArgumentException("wrong fieldInfo");
|
||||
}
|
||||
final int maxDoc = reader.maxDoc();
|
||||
DocValuesFieldUpdates.Iterator[] subs = new DocValuesFieldUpdates.Iterator[updatesToApply.size()];
|
||||
for(int i=0;i<subs.length;i++) {
|
||||
subs[i] = updatesToApply.get(i).iterator();
|
||||
}
|
||||
|
||||
final NumericDocValuesFieldUpdates.Iterator updatesIter = fieldUpdates.iterator();
|
||||
final DocValuesFieldUpdates.Iterator updatesIter = DocValuesFieldUpdates.mergedIterator(subs);
|
||||
|
||||
final NumericDocValues currentValues = reader.getNumericDocValues(field);
|
||||
updatesIter.reset();
|
||||
|
||||
// Merge sort of the original doc values with updated doc values:
|
||||
return new NumericDocValues() {
|
||||
|
@ -394,7 +477,7 @@ class ReadersAndUpdates {
|
|||
} else {
|
||||
docIDOut = updateDocID;
|
||||
if (docIDOut != NO_MORE_DOCS) {
|
||||
value = updatesIter.value();
|
||||
value = (Long) updatesIter.value();
|
||||
}
|
||||
}
|
||||
return docIDOut;
|
||||
|
@ -410,16 +493,42 @@ class ReadersAndUpdates {
|
|||
}
|
||||
|
||||
@SuppressWarnings("synthetic-access")
|
||||
private void handleBinaryDVUpdates(FieldInfos infos, Map<String,BinaryDocValuesFieldUpdates> updates,
|
||||
TrackingDirectoryWrapper dir, DocValuesFormat dvFormat, final SegmentReader reader, Map<Integer,Set<String>> fieldFiles) throws IOException {
|
||||
for (Entry<String,BinaryDocValuesFieldUpdates> e : updates.entrySet()) {
|
||||
final String field = e.getKey();
|
||||
final BinaryDocValuesFieldUpdates fieldUpdates = e.getValue();
|
||||
private synchronized void handleBinaryDVUpdates(FieldInfos infos,
|
||||
TrackingDirectoryWrapper dir, DocValuesFormat dvFormat, final SegmentReader reader,
|
||||
Map<Integer,Set<String>> fieldFiles, long maxDelGen, InfoStream infoStream) throws IOException {
|
||||
for (Entry<String,List<DocValuesFieldUpdates>> ent : pendingDVUpdates.entrySet()) {
|
||||
final String field = ent.getKey();
|
||||
final List<DocValuesFieldUpdates> updates = ent.getValue();
|
||||
if (updates.get(0).type != DocValuesType.BINARY) {
|
||||
continue;
|
||||
}
|
||||
|
||||
final List<DocValuesFieldUpdates> updatesToApply = new ArrayList<>();
|
||||
long bytes = 0;
|
||||
for(DocValuesFieldUpdates update : updates) {
|
||||
if (update.delGen <= maxDelGen) {
|
||||
// safe to apply this one
|
||||
bytes += update.ramBytesUsed();
|
||||
updatesToApply.add(update);
|
||||
}
|
||||
}
|
||||
if (updatesToApply.isEmpty()) {
|
||||
// nothing to apply yet
|
||||
continue;
|
||||
}
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", String.format(Locale.ROOT,
|
||||
"now write %d pending binary DV updates for field=%s, seg=%s, bytes=%.3fMB",
|
||||
updatesToApply.size(),
|
||||
field,
|
||||
info,
|
||||
bytes/1024./1024.));
|
||||
}
|
||||
|
||||
final long nextDocValuesGen = info.getNextDocValuesGen();
|
||||
final String segmentSuffix = Long.toString(nextDocValuesGen, Character.MAX_RADIX);
|
||||
final long estUpdatesSize = fieldUpdates.ramBytesPerDoc() * info.info.maxDoc();
|
||||
final IOContext updatesContext = new IOContext(new FlushInfo(info.info.maxDoc(), estUpdatesSize));
|
||||
final IOContext updatesContext = new IOContext(new FlushInfo(info.info.maxDoc(), bytes));
|
||||
final FieldInfo fieldInfo = infos.fieldInfo(field);
|
||||
assert fieldInfo != null;
|
||||
fieldInfo.setDocValuesGen(nextDocValuesGen);
|
||||
|
@ -438,8 +547,12 @@ class ReadersAndUpdates {
|
|||
}
|
||||
final int maxDoc = reader.maxDoc();
|
||||
|
||||
final BinaryDocValuesFieldUpdates.Iterator updatesIter = fieldUpdates.iterator();
|
||||
updatesIter.reset();
|
||||
DocValuesFieldUpdates.Iterator[] subs = new DocValuesFieldUpdates.Iterator[updatesToApply.size()];
|
||||
for(int i=0;i<subs.length;i++) {
|
||||
subs[i] = updatesToApply.get(i).iterator();
|
||||
}
|
||||
|
||||
final DocValuesFieldUpdates.Iterator updatesIter = DocValuesFieldUpdates.mergedIterator(subs);
|
||||
|
||||
final BinaryDocValues currentValues = reader.getBinaryDocValues(field);
|
||||
|
||||
|
@ -500,7 +613,7 @@ class ReadersAndUpdates {
|
|||
} else {
|
||||
docIDOut = updateDocID;
|
||||
if (docIDOut != NO_MORE_DOCS) {
|
||||
value = updatesIter.value();
|
||||
value = (BytesRef) updatesIter.value();
|
||||
}
|
||||
}
|
||||
return docIDOut;
|
||||
|
@ -515,7 +628,7 @@ class ReadersAndUpdates {
|
|||
}
|
||||
}
|
||||
|
||||
private Set<String> writeFieldInfosGen(FieldInfos fieldInfos, Directory dir, DocValuesFormat dvFormat,
|
||||
private synchronized Set<String> writeFieldInfosGen(FieldInfos fieldInfos, Directory dir, DocValuesFormat dvFormat,
|
||||
FieldInfosFormat infosFormat) throws IOException {
|
||||
final long nextFieldInfosGen = info.getNextFieldInfosGen();
|
||||
final String segmentSuffix = Long.toString(nextFieldInfosGen, Character.MAX_RADIX);
|
||||
|
@ -531,28 +644,52 @@ class ReadersAndUpdates {
|
|||
return trackingDir.getCreatedFiles();
|
||||
}
|
||||
|
||||
// Writes field updates (new _X_N updates files) to the directory
|
||||
public synchronized void writeFieldUpdates(Directory dir, DocValuesFieldUpdates.Container dvUpdates) throws IOException {
|
||||
public synchronized boolean writeFieldUpdates(Directory dir, long maxDelGen, InfoStream infoStream) throws IOException {
|
||||
|
||||
long startTimeNS = System.nanoTime();
|
||||
|
||||
assert Thread.holdsLock(writer);
|
||||
//System.out.println("rld.writeFieldUpdates: seg=" + info + " numericFieldUpdates=" + numericFieldUpdates);
|
||||
|
||||
assert dvUpdates.any();
|
||||
|
||||
// Do this so we can delete any created files on
|
||||
// exception; this saves all codecs from having to do
|
||||
// it:
|
||||
TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(dir);
|
||||
|
||||
|
||||
final Map<Integer,Set<String>> newDVFiles = new HashMap<>();
|
||||
Set<String> fieldInfosFiles = null;
|
||||
FieldInfos fieldInfos = null;
|
||||
|
||||
boolean any = false;
|
||||
int count = 0;
|
||||
for (List<DocValuesFieldUpdates> updates : pendingDVUpdates.values()) {
|
||||
// Sort by increasing delGen:
|
||||
Collections.sort(updates, (a, b) -> Long.compare(a.delGen, b.delGen));
|
||||
count += updates.size();
|
||||
for (DocValuesFieldUpdates update : updates) {
|
||||
if (update.delGen <= maxDelGen && update.any()) {
|
||||
any = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (any == false) {
|
||||
// no updates
|
||||
return false;
|
||||
}
|
||||
|
||||
// Do this so we can delete any created files on
|
||||
// exception; this saves all codecs from having to do it:
|
||||
TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(dir);
|
||||
|
||||
boolean success = false;
|
||||
try {
|
||||
final Codec codec = info.info.getCodec();
|
||||
|
||||
// reader could be null e.g. for a just merged segment (from
|
||||
// IndexWriter.commitMergedDeletes).
|
||||
final SegmentReader reader = this.reader == null ? new SegmentReader(info, writer.segmentInfos.getIndexCreatedVersionMajor(), IOContext.READONCE) : this.reader;
|
||||
final SegmentReader reader;
|
||||
if (this.reader == null) {
|
||||
reader = new SegmentReader(info, writer.segmentInfos.getIndexCreatedVersionMajor(), IOContext.READONCE);
|
||||
} else {
|
||||
reader = this.reader;
|
||||
}
|
||||
|
||||
try {
|
||||
// clone FieldInfos so that we can update their dvGen separately from
|
||||
// the reader's infos and write them to a new fieldInfos_gen file
|
||||
|
@ -567,38 +704,30 @@ class ReadersAndUpdates {
|
|||
}
|
||||
clone.setDocValuesGen(fi.getDocValuesGen());
|
||||
}
|
||||
// create new fields or update existing ones to have NumericDV type
|
||||
for (String f : dvUpdates.numericDVUpdates.keySet()) {
|
||||
FieldInfo fieldInfo = builder.getOrAdd(f);
|
||||
fieldInfo.setDocValuesType(DocValuesType.NUMERIC);
|
||||
}
|
||||
// create new fields or update existing ones to have BinaryDV type
|
||||
for (String f : dvUpdates.binaryDVUpdates.keySet()) {
|
||||
FieldInfo fieldInfo = builder.getOrAdd(f);
|
||||
fieldInfo.setDocValuesType(DocValuesType.BINARY);
|
||||
|
||||
// create new fields with the right DV type
|
||||
for (List<DocValuesFieldUpdates> updates : pendingDVUpdates.values()) {
|
||||
DocValuesFieldUpdates update = updates.get(0);
|
||||
FieldInfo fieldInfo = builder.getOrAdd(update.field);
|
||||
fieldInfo.setDocValuesType(update.type);
|
||||
}
|
||||
|
||||
fieldInfos = builder.finish();
|
||||
final DocValuesFormat docValuesFormat = codec.docValuesFormat();
|
||||
|
||||
// System.out.println("[" + Thread.currentThread().getName() + "] RLD.writeFieldUpdates: applying numeric updates; seg=" + info + " updates=" + numericFieldUpdates);
|
||||
handleNumericDVUpdates(fieldInfos, dvUpdates.numericDVUpdates, trackingDir, docValuesFormat, reader, newDVFiles);
|
||||
|
||||
// System.out.println("[" + Thread.currentThread().getName() + "] RAU.writeFieldUpdates: applying binary updates; seg=" + info + " updates=" + dvUpdates.binaryDVUpdates);
|
||||
handleBinaryDVUpdates(fieldInfos, dvUpdates.binaryDVUpdates, trackingDir, docValuesFormat, reader, newDVFiles);
|
||||
handleNumericDVUpdates(fieldInfos, trackingDir, docValuesFormat, reader, newDVFiles, maxDelGen, infoStream);
|
||||
handleBinaryDVUpdates(fieldInfos, trackingDir, docValuesFormat, reader, newDVFiles, maxDelGen, infoStream);
|
||||
|
||||
// System.out.println("[" + Thread.currentThread().getName() + "] RAU.writeFieldUpdates: write fieldInfos; seg=" + info);
|
||||
fieldInfosFiles = writeFieldInfosGen(fieldInfos, trackingDir, docValuesFormat, codec.fieldInfosFormat());
|
||||
} finally {
|
||||
if (reader != this.reader) {
|
||||
// System.out.println("[" + Thread.currentThread().getName() + "] RLD.writeLiveDocs: closeReader " + reader);
|
||||
reader.close();
|
||||
}
|
||||
}
|
||||
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
if (success == false) {
|
||||
// Advance only the nextWriteFieldInfosGen and nextWriteDocValuesGen, so
|
||||
// that a 2nd attempt to write will write to a new file
|
||||
info.advanceNextWriteFieldInfosGen();
|
||||
|
@ -610,27 +739,48 @@ class ReadersAndUpdates {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
// copy all the updates to mergingUpdates, so they can later be applied to the merged segment
|
||||
if (isMerging) {
|
||||
for (Entry<String,NumericDocValuesFieldUpdates> e : dvUpdates.numericDVUpdates.entrySet()) {
|
||||
DocValuesFieldUpdates updates = mergingDVUpdates.get(e.getKey());
|
||||
if (updates == null) {
|
||||
mergingDVUpdates.put(e.getKey(), e.getValue());
|
||||
|
||||
// Prune the now-written DV updates:
|
||||
long bytesFreed = 0;
|
||||
Iterator<Map.Entry<String,List<DocValuesFieldUpdates>>> it = pendingDVUpdates.entrySet().iterator();
|
||||
while (it.hasNext()) {
|
||||
Map.Entry<String,List<DocValuesFieldUpdates>> ent = it.next();
|
||||
int upto = 0;
|
||||
List<DocValuesFieldUpdates> updates = ent.getValue();
|
||||
for (DocValuesFieldUpdates update : updates) {
|
||||
if (update.delGen > maxDelGen) {
|
||||
// not yet applied
|
||||
updates.set(upto, update);
|
||||
upto++;
|
||||
} else {
|
||||
updates.merge(e.getValue());
|
||||
bytesFreed += update.ramBytesUsed();
|
||||
}
|
||||
}
|
||||
for (Entry<String,BinaryDocValuesFieldUpdates> e : dvUpdates.binaryDVUpdates.entrySet()) {
|
||||
DocValuesFieldUpdates updates = mergingDVUpdates.get(e.getKey());
|
||||
if (updates == null) {
|
||||
mergingDVUpdates.put(e.getKey(), e.getValue());
|
||||
} else {
|
||||
updates.merge(e.getValue());
|
||||
if (upto == 0) {
|
||||
it.remove();
|
||||
} else {
|
||||
updates.subList(upto, updates.size()).clear();
|
||||
}
|
||||
}
|
||||
|
||||
long bytes = ramBytesUsed.addAndGet(-bytesFreed);
|
||||
assert bytes >= 0;
|
||||
|
||||
// if there is a reader open, reopen it to reflect the updates
|
||||
if (reader != null) {
|
||||
SegmentReader newReader = new SegmentReader(info, reader, liveDocs, info.info.maxDoc() - info.getDelCount() - pendingDeleteCount);
|
||||
boolean success2 = false;
|
||||
try {
|
||||
reader.decRef();
|
||||
reader = newReader;
|
||||
success2 = true;
|
||||
} finally {
|
||||
if (success2 == false) {
|
||||
newReader.decRef();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
// writing field updates succeeded
|
||||
assert fieldInfosFiles != null;
|
||||
info.setFieldInfosFiles(fieldInfosFiles);
|
||||
|
@ -639,46 +789,75 @@ class ReadersAndUpdates {
|
|||
// of files, hence we copy from the existing map all fields w/ updates that
|
||||
// were not updated in this session, and add new mappings for fields that
|
||||
// were updated now.
|
||||
assert !newDVFiles.isEmpty();
|
||||
assert newDVFiles.isEmpty() == false;
|
||||
for (Entry<Integer,Set<String>> e : info.getDocValuesUpdatesFiles().entrySet()) {
|
||||
if (!newDVFiles.containsKey(e.getKey())) {
|
||||
if (newDVFiles.containsKey(e.getKey()) == false) {
|
||||
newDVFiles.put(e.getKey(), e.getValue());
|
||||
}
|
||||
}
|
||||
info.setDocValuesUpdatesFiles(newDVFiles);
|
||||
|
||||
// wrote new files, should checkpoint()
|
||||
writer.checkpoint();
|
||||
|
||||
// if there is a reader open, reopen it to reflect the updates
|
||||
if (reader != null) {
|
||||
SegmentReader newReader = new SegmentReader(info, reader, liveDocs, info.info.maxDoc() - info.getDelCount() - pendingDeleteCount);
|
||||
boolean reopened = false;
|
||||
// wrote new files, should checkpoint()
|
||||
writer.checkpointNoSIS();
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", String.format(Locale.ROOT, "done write field updates for seg=%s; took %.3fs; new files: %s",
|
||||
info, (System.nanoTime() - startTimeNS)/1000000000.0, newDVFiles));
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
/** Returns a reader for merge, with the latest doc values updates and deletions. */
|
||||
synchronized SegmentReader getReaderForMerge(IOContext context) throws IOException {
|
||||
|
||||
// This ensures any newly resolved doc value updates while we are merging are
|
||||
// saved for re-applying after this segment is done merging:
|
||||
isMerging = true;
|
||||
|
||||
assert mergingDVUpdates.isEmpty();
|
||||
|
||||
// We must carry over any still-pending DV updates because they were not
|
||||
// successfully written, e.g. because there was a hole in the delGens,
|
||||
// or they arrived after we wrote all DVs for merge but before we set
|
||||
// isMerging here:
|
||||
for (Map.Entry<String, List<DocValuesFieldUpdates>> ent : pendingDVUpdates.entrySet()) {
|
||||
List<DocValuesFieldUpdates> mergingUpdates = mergingDVUpdates.get(ent.getKey());
|
||||
if (mergingUpdates == null) {
|
||||
mergingUpdates = new ArrayList<>();
|
||||
mergingDVUpdates.put(ent.getKey(), mergingUpdates);
|
||||
}
|
||||
mergingUpdates.addAll(ent.getValue());
|
||||
}
|
||||
|
||||
SegmentReader reader = getReader(context);
|
||||
int delCount = pendingDeleteCount + info.getDelCount();
|
||||
if (delCount != reader.numDeletedDocs()) {
|
||||
|
||||
// beware of zombies:
|
||||
assert delCount > reader.numDeletedDocs(): "delCount=" + delCount + " reader.numDeletedDocs()=" + reader.numDeletedDocs();
|
||||
|
||||
assert liveDocs != null;
|
||||
|
||||
// Create a new reader with the latest live docs:
|
||||
SegmentReader newReader = new SegmentReader(info, reader, liveDocs, info.info.maxDoc() - delCount);
|
||||
boolean success = false;
|
||||
try {
|
||||
reader.decRef();
|
||||
reader = newReader;
|
||||
reopened = true;
|
||||
success = true;
|
||||
} finally {
|
||||
if (!reopened) {
|
||||
if (success == false) {
|
||||
newReader.decRef();
|
||||
}
|
||||
}
|
||||
reader = newReader;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a reader for merge. This method applies field updates if there are
|
||||
* any and marks that this segment is currently merging.
|
||||
*/
|
||||
synchronized SegmentReader getReaderForMerge(IOContext context) throws IOException {
|
||||
assert Thread.holdsLock(writer);
|
||||
// must execute these two statements as atomic operation, otherwise we
|
||||
// could lose updates if e.g. another thread calls writeFieldUpdates in
|
||||
// between, or the updates are applied to the obtained reader, but then
|
||||
// re-applied in IW.commitMergedDeletes (unnecessary work and potential
|
||||
// bugs).
|
||||
isMerging = true;
|
||||
return getReader(context);
|
||||
liveDocsShared = true;
|
||||
|
||||
assert verifyDocCounts();
|
||||
|
||||
return reader;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -689,12 +868,13 @@ class ReadersAndUpdates {
|
|||
mergingDVUpdates.clear();
|
||||
isMerging = false;
|
||||
}
|
||||
|
||||
/** Returns updates that came in while this segment was merging. */
|
||||
public synchronized Map<String,DocValuesFieldUpdates> getMergingFieldUpdates() {
|
||||
|
||||
public synchronized Map<String,List<DocValuesFieldUpdates>> getMergingDVUpdates() {
|
||||
// We must atomically (in single sync'd block) clear isMerging when we return the DV updates otherwise we can lose updates:
|
||||
isMerging = false;
|
||||
return mergingDVUpdates;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
|
@ -703,5 +883,4 @@ class ReadersAndUpdates {
|
|||
sb.append(" liveDocsShared=").append(liveDocsShared);
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -70,6 +70,10 @@ public class SegmentCommitInfo {
|
|||
|
||||
private volatile long sizeInBytes = -1;
|
||||
|
||||
// NOTE: only used in-RAM by IW to track buffered deletes;
|
||||
// this is never written to/read from the Directory
|
||||
private long bufferedDeletesGen = -1;
|
||||
|
||||
/**
|
||||
* Sole constructor.
|
||||
*
|
||||
|
@ -236,17 +240,17 @@ public class SegmentCommitInfo {
|
|||
return files;
|
||||
}
|
||||
|
||||
// NOTE: only used in-RAM by IW to track buffered deletes;
|
||||
// this is never written to/read from the Directory
|
||||
private long bufferedDeletesGen;
|
||||
|
||||
long getBufferedDeletesGen() {
|
||||
return bufferedDeletesGen;
|
||||
}
|
||||
|
||||
void setBufferedDeletesGen(long v) {
|
||||
bufferedDeletesGen = v;
|
||||
sizeInBytes = -1;
|
||||
if (bufferedDeletesGen == -1) {
|
||||
bufferedDeletesGen = v;
|
||||
sizeInBytes = -1;
|
||||
} else {
|
||||
throw new IllegalStateException("buffered deletes gen should only be set once");
|
||||
}
|
||||
}
|
||||
|
||||
/** Returns true if there are any deletions for the
|
||||
|
|
|
@ -92,7 +92,6 @@ final class SegmentCoreReaders {
|
|||
|
||||
final Codec codec = si.info.getCodec();
|
||||
final Directory cfsDir; // confusing name: if (cfs) it's the cfsdir, otherwise it's the segment's directory.
|
||||
|
||||
boolean success = false;
|
||||
|
||||
try {
|
||||
|
@ -164,7 +163,6 @@ final class SegmentCoreReaders {
|
|||
|
||||
void decRef() throws IOException {
|
||||
if (ref.decrementAndGet() == 0) {
|
||||
// System.err.println("--- closing core readers");
|
||||
Throwable th = null;
|
||||
try {
|
||||
IOUtils.close(termVectorsLocal, fieldsReaderLocal, fields, termVectorsReaderOrig, fieldsReaderOrig,
|
||||
|
|
|
@ -172,7 +172,7 @@ public final class SegmentInfo {
|
|||
/** Return all files referenced by this SegmentInfo. */
|
||||
public Set<String> files() {
|
||||
if (setFiles == null) {
|
||||
throw new IllegalStateException("files were not computed yet");
|
||||
throw new IllegalStateException("files were not computed yet; segment=" + name + " maxDoc=" + maxDoc);
|
||||
}
|
||||
return Collections.unmodifiableSet(setFiles);
|
||||
}
|
||||
|
|
|
@ -158,15 +158,22 @@ public final class SegmentReader extends CodecReader {
|
|||
* init most recent DocValues for the current commit
|
||||
*/
|
||||
private DocValuesProducer initDocValuesProducer() throws IOException {
|
||||
final Directory dir = core.cfsReader != null ? core.cfsReader : si.info.dir;
|
||||
|
||||
if (!fieldInfos.hasDocValues()) {
|
||||
if (fieldInfos.hasDocValues() == false) {
|
||||
return null;
|
||||
} else if (si.hasFieldUpdates()) {
|
||||
return new SegmentDocValuesProducer(si, dir, core.coreFieldInfos, fieldInfos, segDocValues);
|
||||
} else {
|
||||
// simple case, no DocValues updates
|
||||
return segDocValues.getDocValuesProducer(-1L, si, dir, fieldInfos);
|
||||
Directory dir;
|
||||
if (core.cfsReader != null) {
|
||||
dir = core.cfsReader;
|
||||
} else {
|
||||
dir = si.info.dir;
|
||||
}
|
||||
if (si.hasFieldUpdates()) {
|
||||
return new SegmentDocValuesProducer(si, dir, core.coreFieldInfos, fieldInfos, segDocValues);
|
||||
} else {
|
||||
// simple case, no DocValues updates
|
||||
return segDocValues.getDocValuesProducer(-1L, si, dir, fieldInfos);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -32,11 +32,11 @@ public class SerialMergeScheduler extends MergeScheduler {
|
|||
* multiple threads, only one merge may run at a time. */
|
||||
@Override
|
||||
synchronized public void merge(IndexWriter writer, MergeTrigger trigger, boolean newMergesFound) throws IOException {
|
||||
|
||||
while(true) {
|
||||
MergePolicy.OneMerge merge = writer.getNextMerge();
|
||||
if (merge == null)
|
||||
if (merge == null) {
|
||||
break;
|
||||
}
|
||||
writer.merge(merge);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,6 +18,8 @@ package org.apache.lucene.index;
|
|||
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesProducer;
|
||||
|
@ -284,21 +286,27 @@ public final class SlowCodecReaderWrapper {
|
|||
}
|
||||
|
||||
private static FieldsProducer readerToFieldsProducer(final LeafReader reader) throws IOException {
|
||||
final Fields fields = reader.fields();
|
||||
ArrayList<String> indexedFields = new ArrayList<>();
|
||||
for (FieldInfo fieldInfo : reader.getFieldInfos()) {
|
||||
if (fieldInfo.getIndexOptions() != IndexOptions.NONE) {
|
||||
indexedFields.add(fieldInfo.name);
|
||||
}
|
||||
}
|
||||
Collections.sort(indexedFields);
|
||||
return new FieldsProducer() {
|
||||
@Override
|
||||
public Iterator<String> iterator() {
|
||||
return fields.iterator();
|
||||
return indexedFields.iterator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Terms terms(String field) throws IOException {
|
||||
return fields.terms(field);
|
||||
return reader.terms(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int size() {
|
||||
return fields.size();
|
||||
return indexedFields.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -49,6 +49,7 @@ import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
|
|||
*/
|
||||
class SortingLeafReader extends FilterLeafReader {
|
||||
|
||||
//TODO remove from here; move to FreqProxTermsWriter or FreqProxFields?
|
||||
static class SortingFields extends FilterFields {
|
||||
|
||||
private final Sorter.DocMap docMap;
|
||||
|
@ -1042,8 +1043,9 @@ class SortingLeafReader extends FilterLeafReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Fields fields() throws IOException {
|
||||
return new SortingFields(in.fields(), in.getFieldInfos(), docMap);
|
||||
public Terms terms(String field) throws IOException {
|
||||
Terms terms = super.terms(field);
|
||||
return terms==null ? null : new SortingTerms(terms, in.getFieldInfos().fieldInfo(field).getIndexOptions(), docMap);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -109,6 +109,7 @@ final class TermVectorsConsumerPerField extends TermsHashPerField {
|
|||
|
||||
@Override
|
||||
boolean start(IndexableField field, boolean first) {
|
||||
super.start(field, first);
|
||||
assert field.fieldType().indexOptions() != IndexOptions.NONE;
|
||||
|
||||
if (first) {
|
||||
|
@ -224,7 +225,7 @@ final class TermVectorsConsumerPerField extends TermsHashPerField {
|
|||
void newTerm(final int termID) {
|
||||
TermVectorsPostingsArray postings = termVectorsPostingsArray;
|
||||
|
||||
postings.freqs[termID] = 1;
|
||||
postings.freqs[termID] = getTermFreq();
|
||||
postings.lastOffsets[termID] = 0;
|
||||
postings.lastPositions[termID] = 0;
|
||||
|
||||
|
@ -235,11 +236,25 @@ final class TermVectorsConsumerPerField extends TermsHashPerField {
|
|||
void addTerm(final int termID) {
|
||||
TermVectorsPostingsArray postings = termVectorsPostingsArray;
|
||||
|
||||
postings.freqs[termID]++;
|
||||
postings.freqs[termID] += getTermFreq();
|
||||
|
||||
writeProx(postings, termID);
|
||||
}
|
||||
|
||||
private int getTermFreq() {
|
||||
int freq = termFreqAtt.getTermFrequency();
|
||||
if (freq != 1) {
|
||||
if (doVectorPositions) {
|
||||
throw new IllegalArgumentException("field \"" + fieldInfo.name + "\": cannot index term vector positions while using custom TermFrequencyAttribute");
|
||||
}
|
||||
if (doVectorOffsets) {
|
||||
throw new IllegalArgumentException("field \"" + fieldInfo.name + "\": cannot index term vector offsets while using custom TermFrequencyAttribute");
|
||||
}
|
||||
}
|
||||
|
||||
return freq;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void newPostingsArray() {
|
||||
termVectorsPostingsArray = (TermVectorsPostingsArray) postingsArray;
|
||||
|
|
|
@ -19,12 +19,13 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.analysis.tokenattributes.TermFrequencyAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
|
||||
import org.apache.lucene.util.ByteBlockPool;
|
||||
import org.apache.lucene.util.BytesRefHash.BytesStartArray;
|
||||
import org.apache.lucene.util.BytesRefHash;
|
||||
import org.apache.lucene.util.Counter;
|
||||
import org.apache.lucene.util.IntBlockPool;
|
||||
import org.apache.lucene.util.BytesRefHash.BytesStartArray;
|
||||
|
||||
abstract class TermsHashPerField implements Comparable<TermsHashPerField> {
|
||||
private static final int HASH_INIT_SIZE = 4;
|
||||
|
@ -35,6 +36,7 @@ abstract class TermsHashPerField implements Comparable<TermsHashPerField> {
|
|||
protected final DocumentsWriterPerThread.DocState docState;
|
||||
protected final FieldInvertState fieldState;
|
||||
TermToBytesRefAttribute termAtt;
|
||||
protected TermFrequencyAttribute termFreqAtt;
|
||||
|
||||
// Copied from our perThread
|
||||
final IntBlockPool intPool;
|
||||
|
@ -287,6 +289,7 @@ abstract class TermsHashPerField implements Comparable<TermsHashPerField> {
|
|||
* document. */
|
||||
boolean start(IndexableField field, boolean first) {
|
||||
termAtt = fieldState.termAttribute;
|
||||
termFreqAtt = fieldState.termFreqAttribute;
|
||||
if (nextPerField != null) {
|
||||
doNextCall = nextPerField.start(field, first);
|
||||
}
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.util.ArrayList;
|
|||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
|
@ -237,26 +238,20 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
|
||||
private class SegmentByteSizeDescending implements Comparator<SegmentCommitInfo> {
|
||||
|
||||
private final IndexWriter writer;
|
||||
private final Map<SegmentCommitInfo, Long> sizeInBytes;
|
||||
|
||||
SegmentByteSizeDescending(IndexWriter writer) {
|
||||
this.writer = writer;
|
||||
SegmentByteSizeDescending(Map<SegmentCommitInfo, Long> sizeInBytes) {
|
||||
this.sizeInBytes = sizeInBytes;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compare(SegmentCommitInfo o1, SegmentCommitInfo o2) {
|
||||
try {
|
||||
final long sz1 = size(o1, writer);
|
||||
final long sz2 = size(o2, writer);
|
||||
if (sz1 > sz2) {
|
||||
return -1;
|
||||
} else if (sz2 > sz1) {
|
||||
return 1;
|
||||
} else {
|
||||
return o1.info.name.compareTo(o2.info.name);
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
throw new RuntimeException(ioe);
|
||||
// Sort by largest size:
|
||||
int cmp = Long.compare(sizeInBytes.get(o2), sizeInBytes.get(o1));
|
||||
if (cmp == 0) {
|
||||
cmp = o1.info.name.compareTo(o2.info.name);
|
||||
}
|
||||
return cmp;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -277,6 +272,14 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
abstract String getExplanation();
|
||||
}
|
||||
|
||||
private Map<SegmentCommitInfo,Long> getSegmentSizes(IndexWriter writer, Collection<SegmentCommitInfo> infos) throws IOException {
|
||||
Map<SegmentCommitInfo,Long> sizeInBytes = new HashMap<>();
|
||||
for (SegmentCommitInfo info : infos) {
|
||||
sizeInBytes.put(info, size(info, writer));
|
||||
}
|
||||
return sizeInBytes;
|
||||
}
|
||||
|
||||
@Override
|
||||
public MergeSpecification findMerges(MergeTrigger mergeTrigger, SegmentInfos infos, IndexWriter writer) throws IOException {
|
||||
if (verbose(writer)) {
|
||||
|
@ -289,13 +292,19 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
final Collection<SegmentCommitInfo> toBeMerged = new HashSet<>();
|
||||
|
||||
final List<SegmentCommitInfo> infosSorted = new ArrayList<>(infos.asList());
|
||||
Collections.sort(infosSorted, new SegmentByteSizeDescending(writer));
|
||||
|
||||
// The size can change concurrently while we are running here, because deletes
|
||||
// are now applied concurrently, and this can piss off TimSort! So we
|
||||
// call size() once per segment and sort by that:
|
||||
Map<SegmentCommitInfo,Long> sizeInBytes = getSegmentSizes(writer, infos.asList());
|
||||
|
||||
Collections.sort(infosSorted, new SegmentByteSizeDescending(sizeInBytes));
|
||||
|
||||
// Compute total index bytes & print details about the index
|
||||
long totIndexBytes = 0;
|
||||
long minSegmentBytes = Long.MAX_VALUE;
|
||||
for(SegmentCommitInfo info : infosSorted) {
|
||||
final long segBytes = size(info, writer);
|
||||
final long segBytes = sizeInBytes.get(info);
|
||||
if (verbose(writer)) {
|
||||
String extra = merging.contains(info) ? " [merging]" : "";
|
||||
if (segBytes >= maxMergedSegmentBytes/2.0) {
|
||||
|
@ -315,7 +324,7 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
// of the maxSegmentCount:
|
||||
int tooBigCount = 0;
|
||||
while (tooBigCount < infosSorted.size()) {
|
||||
long segBytes = size(infosSorted.get(tooBigCount), writer);
|
||||
long segBytes = sizeInBytes.get(infosSorted.get(tooBigCount));
|
||||
if (segBytes < maxMergedSegmentBytes/2.0) {
|
||||
break;
|
||||
}
|
||||
|
@ -355,7 +364,7 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
for(int idx = tooBigCount; idx<infosSorted.size(); idx++) {
|
||||
final SegmentCommitInfo info = infosSorted.get(idx);
|
||||
if (merging.contains(info)) {
|
||||
mergingBytes += size(info, writer);
|
||||
mergingBytes += sizeInBytes.get(info);
|
||||
} else if (!toBeMerged.contains(info)) {
|
||||
eligible.add(info);
|
||||
}
|
||||
|
@ -388,7 +397,7 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
boolean hitTooLarge = false;
|
||||
for(int idx = startIdx;idx<eligible.size() && candidate.size() < maxMergeAtOnce;idx++) {
|
||||
final SegmentCommitInfo info = eligible.get(idx);
|
||||
final long segBytes = size(info, writer);
|
||||
final long segBytes = sizeInBytes.get(info);
|
||||
|
||||
if (totAfterMergeBytes + segBytes > maxMergedSegmentBytes) {
|
||||
hitTooLarge = true;
|
||||
|
@ -408,7 +417,7 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
// segments, and already pre-excluded the too-large segments:
|
||||
assert candidate.size() > 0;
|
||||
|
||||
final MergeScore score = score(candidate, hitTooLarge, mergingBytes, writer);
|
||||
final MergeScore score = score(candidate, hitTooLarge, mergingBytes, writer, sizeInBytes);
|
||||
if (verbose(writer)) {
|
||||
message(" maybe=" + writer.segString(candidate) + " score=" + score.getScore() + " " + score.getExplanation() + " tooLarge=" + hitTooLarge + " size=" + String.format(Locale.ROOT, "%.3f MB", totAfterMergeBytes/1024./1024.), writer);
|
||||
}
|
||||
|
@ -447,12 +456,12 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
}
|
||||
|
||||
/** Expert: scores one merge; subclasses can override. */
|
||||
protected MergeScore score(List<SegmentCommitInfo> candidate, boolean hitTooLarge, long mergingBytes, IndexWriter writer) throws IOException {
|
||||
protected MergeScore score(List<SegmentCommitInfo> candidate, boolean hitTooLarge, long mergingBytes, IndexWriter writer, Map<SegmentCommitInfo, Long> sizeInBytes) throws IOException {
|
||||
long totBeforeMergeBytes = 0;
|
||||
long totAfterMergeBytes = 0;
|
||||
long totAfterMergeBytesFloored = 0;
|
||||
for(SegmentCommitInfo info : candidate) {
|
||||
final long segBytes = size(info, writer);
|
||||
final long segBytes = sizeInBytes.get(info);
|
||||
totAfterMergeBytes += segBytes;
|
||||
totAfterMergeBytesFloored += floorSize(segBytes);
|
||||
totBeforeMergeBytes += info.sizeInBytes();
|
||||
|
@ -472,7 +481,7 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
// over time:
|
||||
skew = 1.0/maxMergeAtOnce;
|
||||
} else {
|
||||
skew = ((double) floorSize(size(candidate.get(0), writer)))/totAfterMergeBytesFloored;
|
||||
skew = ((double) floorSize(sizeInBytes.get(candidate.get(0))))/totAfterMergeBytesFloored;
|
||||
}
|
||||
|
||||
// Strongly favor merges with less skew (smaller
|
||||
|
@ -519,7 +528,7 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
final Boolean isOriginal = segmentsToMerge.get(info);
|
||||
if (isOriginal != null) {
|
||||
segmentIsOriginal = isOriginal;
|
||||
if (!merging.contains(info)) {
|
||||
if (merging.contains(info) == false) {
|
||||
eligible.add(info);
|
||||
} else {
|
||||
forceMergeRunning = true;
|
||||
|
@ -531,6 +540,11 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
return null;
|
||||
}
|
||||
|
||||
// The size can change concurrently while we are running here, because deletes
|
||||
// are now applied concurrently, and this can piss off TimSort! So we
|
||||
// call size() once per segment and sort by that:
|
||||
Map<SegmentCommitInfo,Long> sizeInBytes = getSegmentSizes(writer, eligible);
|
||||
|
||||
if ((maxSegmentCount > 1 && eligible.size() <= maxSegmentCount) ||
|
||||
(maxSegmentCount == 1 && eligible.size() == 1 && (!segmentIsOriginal || isMerged(infos, eligible.get(0), writer)))) {
|
||||
if (verbose(writer)) {
|
||||
|
@ -539,7 +553,7 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
return null;
|
||||
}
|
||||
|
||||
Collections.sort(eligible, new SegmentByteSizeDescending(writer));
|
||||
Collections.sort(eligible, new SegmentByteSizeDescending(sizeInBytes));
|
||||
|
||||
if (verbose(writer)) {
|
||||
message("eligible=" + eligible, writer);
|
||||
|
@ -595,7 +609,12 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
return null;
|
||||
}
|
||||
|
||||
Collections.sort(eligible, new SegmentByteSizeDescending(writer));
|
||||
// The size can change concurrently while we are running here, because deletes
|
||||
// are now applied concurrently, and this can piss off TimSort! So we
|
||||
// call size() once per segment and sort by that:
|
||||
Map<SegmentCommitInfo,Long> sizeInBytes = getSegmentSizes(writer, infos.asList());
|
||||
|
||||
Collections.sort(eligible, new SegmentByteSizeDescending(sizeInBytes));
|
||||
|
||||
if (verbose(writer)) {
|
||||
message("eligible=" + eligible, writer);
|
||||
|
|
|
@ -51,6 +51,7 @@ public class AutomatonQuery extends MultiTermQuery {
|
|||
protected final CompiledAutomaton compiled;
|
||||
/** term containing the field, and possibly some pattern structure */
|
||||
protected final Term term;
|
||||
protected final boolean automatonIsBinary;
|
||||
|
||||
/**
|
||||
* Create a new AutomatonQuery from an {@link Automaton}.
|
||||
|
@ -98,6 +99,7 @@ public class AutomatonQuery extends MultiTermQuery {
|
|||
super(term.field());
|
||||
this.term = term;
|
||||
this.automaton = automaton;
|
||||
this.automatonIsBinary = isBinary;
|
||||
// TODO: we could take isFinite too, to save a bit of CPU in CompiledAutomaton ctor?:
|
||||
this.compiled = new CompiledAutomaton(automaton, null, true, maxDeterminizedStates, isBinary);
|
||||
}
|
||||
|
@ -154,4 +156,9 @@ public class AutomatonQuery extends MultiTermQuery {
|
|||
public Automaton getAutomaton() {
|
||||
return automaton;
|
||||
}
|
||||
|
||||
/** Is this a binary (byte) oriented automaton. See the constructor. */
|
||||
public boolean isAutomatonBinary() {
|
||||
return automatonIsBinary;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -178,6 +178,14 @@ public final class DisjunctionMaxQuery extends Query implements Iterable<Query>
|
|||
return disjuncts[0];
|
||||
}
|
||||
|
||||
if (tieBreakerMultiplier == 1.0f) {
|
||||
BooleanQuery.Builder builder = new BooleanQuery.Builder();
|
||||
for (Query sub : disjuncts) {
|
||||
builder.add(sub, BooleanClause.Occur.SHOULD);
|
||||
}
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
boolean actuallyRewritten = false;
|
||||
List<Query> rewrittenDisjuncts = new ArrayList<>();
|
||||
for (Query sub : disjuncts) {
|
||||
|
|
|
@ -35,4 +35,25 @@ public abstract class DoubleValues {
|
|||
*/
|
||||
public abstract boolean advanceExact(int doc) throws IOException;
|
||||
|
||||
/**
|
||||
* Wrap a DoubleValues instance, returning a default if the wrapped instance has no value
|
||||
*/
|
||||
public static DoubleValues withDefault(DoubleValues in, double missingValue) {
|
||||
return new DoubleValues() {
|
||||
|
||||
boolean hasValue = false;
|
||||
|
||||
@Override
|
||||
public double doubleValue() throws IOException {
|
||||
return hasValue ? in.doubleValue() : missingValue;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean advanceExact(int doc) throws IOException {
|
||||
hasValue = in.advanceExact(doc);
|
||||
return true;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -20,9 +20,7 @@ package org.apache.lucene.search;
|
|||
import java.io.IOException;
|
||||
import java.util.Objects;
|
||||
import java.util.function.DoubleToLongFunction;
|
||||
import java.util.function.DoubleUnaryOperator;
|
||||
import java.util.function.LongToDoubleFunction;
|
||||
import java.util.function.ToDoubleBiFunction;
|
||||
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
|
@ -66,7 +64,12 @@ public abstract class DoubleValuesSource {
|
|||
* @return an Explanation for the value
|
||||
* @throws IOException if an {@link IOException} occurs
|
||||
*/
|
||||
public abstract Explanation explain(LeafReaderContext ctx, int docId, Explanation scoreExplanation) throws IOException;
|
||||
public Explanation explain(LeafReaderContext ctx, int docId, Explanation scoreExplanation) throws IOException {
|
||||
DoubleValues dv = getValues(ctx, DoubleValuesSource.constant(scoreExplanation.getValue()).getValues(ctx, null));
|
||||
if (dv.advanceExact(docId))
|
||||
return Explanation.match((float) dv.doubleValue(), this.toString());
|
||||
return Explanation.noMatch(this.toString());
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a sort field based on the value of this producer
|
||||
|
@ -76,14 +79,33 @@ public abstract class DoubleValuesSource {
|
|||
return new DoubleValuesSortField(this, reverse);
|
||||
}
|
||||
|
||||
@Override
|
||||
public abstract int hashCode();
|
||||
|
||||
@Override
|
||||
public abstract boolean equals(Object obj);
|
||||
|
||||
@Override
|
||||
public abstract String toString();
|
||||
|
||||
/**
|
||||
* Convert to a LongValuesSource by casting the double values to longs
|
||||
*/
|
||||
public final LongValuesSource toLongValuesSource() {
|
||||
return new LongValuesSource() {
|
||||
@Override
|
||||
return new LongDoubleValuesSource(this);
|
||||
}
|
||||
|
||||
private static class LongDoubleValuesSource extends LongValuesSource {
|
||||
|
||||
private final DoubleValuesSource inner;
|
||||
|
||||
private LongDoubleValuesSource(DoubleValuesSource inner) {
|
||||
this.inner = inner;
|
||||
}
|
||||
|
||||
@Override
|
||||
public LongValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
|
||||
DoubleValues in = DoubleValuesSource.this.getValues(ctx, scores);
|
||||
DoubleValues in = inner.getValues(ctx, scores);
|
||||
return new LongValues() {
|
||||
@Override
|
||||
public long longValue() throws IOException {
|
||||
|
@ -99,9 +121,27 @@ public abstract class DoubleValuesSource {
|
|||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return DoubleValuesSource.this.needsScores();
|
||||
return inner.needsScores();
|
||||
}
|
||||
};
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
LongDoubleValuesSource that = (LongDoubleValuesSource) o;
|
||||
return Objects.equals(inner, that.inner);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(inner);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "long(" + inner.toString() + ")";
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -164,115 +204,80 @@ public abstract class DoubleValuesSource {
|
|||
public Explanation explain(LeafReaderContext ctx, int docId, Explanation scoreExplanation) {
|
||||
return scoreExplanation;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
return obj == this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "scores";
|
||||
}
|
||||
};
|
||||
|
||||
/**
|
||||
* Creates a DoubleValuesSource that always returns a constant value
|
||||
*/
|
||||
public static DoubleValuesSource constant(double value) {
|
||||
return new DoubleValuesSource() {
|
||||
@Override
|
||||
public DoubleValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
|
||||
return new DoubleValues() {
|
||||
@Override
|
||||
public double doubleValue() throws IOException {
|
||||
return value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean advanceExact(int doc) throws IOException {
|
||||
return true;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Explanation explain(LeafReaderContext ctx, int docId, Explanation scoreExplanation) {
|
||||
return Explanation.match((float) value, "constant(" + value + ")");
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "constant(" + value + ")";
|
||||
}
|
||||
};
|
||||
return new ConstantValuesSource(value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a DoubleValuesSource that is a function of another DoubleValuesSource
|
||||
*/
|
||||
public static DoubleValuesSource function(DoubleValuesSource in, String description, DoubleUnaryOperator function) {
|
||||
return new DoubleValuesSource() {
|
||||
@Override
|
||||
public DoubleValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
|
||||
DoubleValues inputs = in.getValues(ctx, scores);
|
||||
return new DoubleValues() {
|
||||
@Override
|
||||
public double doubleValue() throws IOException {
|
||||
return function.applyAsDouble(inputs.doubleValue());
|
||||
}
|
||||
private static class ConstantValuesSource extends DoubleValuesSource {
|
||||
|
||||
@Override
|
||||
public boolean advanceExact(int doc) throws IOException {
|
||||
return inputs.advanceExact(doc);
|
||||
}
|
||||
};
|
||||
}
|
||||
private final double value;
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return in.needsScores();
|
||||
}
|
||||
private ConstantValuesSource(double value) {
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Explanation explain(LeafReaderContext ctx, int docId, Explanation scoreExplanation) throws IOException {
|
||||
Explanation inner = in.explain(ctx, docId, scoreExplanation);
|
||||
return Explanation.match((float) function.applyAsDouble(inner.getValue()), description + ", computed from:", inner, scoreExplanation);
|
||||
}
|
||||
};
|
||||
}
|
||||
@Override
|
||||
public DoubleValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
|
||||
return new DoubleValues() {
|
||||
@Override
|
||||
public double doubleValue() throws IOException {
|
||||
return value;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a DoubleValuesSource that is a function of another DoubleValuesSource and a score
|
||||
* @param in the DoubleValuesSource to use as an input
|
||||
* @param description a description of the function
|
||||
* @param function a function of the form (source, score) == result
|
||||
*/
|
||||
public static DoubleValuesSource scoringFunction(DoubleValuesSource in, String description, ToDoubleBiFunction<Double, Double> function) {
|
||||
return new DoubleValuesSource() {
|
||||
@Override
|
||||
public DoubleValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
|
||||
DoubleValues inputs = in.getValues(ctx, scores);
|
||||
return new DoubleValues() {
|
||||
@Override
|
||||
public double doubleValue() throws IOException {
|
||||
return function.applyAsDouble(inputs.doubleValue(), scores.doubleValue());
|
||||
}
|
||||
@Override
|
||||
public boolean advanceExact(int doc) throws IOException {
|
||||
return true;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean advanceExact(int doc) throws IOException {
|
||||
return inputs.advanceExact(doc);
|
||||
}
|
||||
};
|
||||
}
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true;
|
||||
}
|
||||
@Override
|
||||
public Explanation explain(LeafReaderContext ctx, int docId, Explanation scoreExplanation) {
|
||||
return Explanation.match((float) value, "constant(" + value + ")");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Explanation explain(LeafReaderContext ctx, int docId, Explanation scoreExplanation) throws IOException {
|
||||
Explanation inner = in.explain(ctx, docId, scoreExplanation);
|
||||
return Explanation.match((float) function.applyAsDouble((double)inner.getValue(), (double)scoreExplanation.getValue()),
|
||||
description + ", computed from:", inner, scoreExplanation);
|
||||
}
|
||||
};
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
ConstantValuesSource that = (ConstantValuesSource) o;
|
||||
return Double.compare(that.value, value) == 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "constant(" + value + ")";
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -312,6 +317,11 @@ public abstract class DoubleValuesSource {
|
|||
Objects.equals(decoder, that.decoder);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "double(" + field + ")";
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(field, decoder);
|
||||
|
@ -342,9 +352,9 @@ public abstract class DoubleValuesSource {
|
|||
public Explanation explain(LeafReaderContext ctx, int docId, Explanation scoreExplanation) throws IOException {
|
||||
DoubleValues values = getValues(ctx, null);
|
||||
if (values.advanceExact(docId))
|
||||
return Explanation.match((float)values.doubleValue(), "double(" + field + ")");
|
||||
return Explanation.match((float)values.doubleValue(), this.toString());
|
||||
else
|
||||
return Explanation.noMatch("double(" + field + ")");
|
||||
return Explanation.noMatch(this.toString());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -52,6 +52,15 @@ public abstract class LongValuesSource {
|
|||
*/
|
||||
public abstract boolean needsScores();
|
||||
|
||||
@Override
|
||||
public abstract int hashCode();
|
||||
|
||||
@Override
|
||||
public abstract boolean equals(Object obj);
|
||||
|
||||
@Override
|
||||
public abstract String toString();
|
||||
|
||||
/**
|
||||
* Create a sort field based on the value of this producer
|
||||
* @param reverse true if the sort should be decreasing
|
||||
|
@ -78,27 +87,55 @@ public abstract class LongValuesSource {
|
|||
* Creates a LongValuesSource that always returns a constant value
|
||||
*/
|
||||
public static LongValuesSource constant(long value) {
|
||||
return new LongValuesSource() {
|
||||
@Override
|
||||
public LongValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
|
||||
return new LongValues() {
|
||||
@Override
|
||||
public long longValue() throws IOException {
|
||||
return value;
|
||||
}
|
||||
return new ConstantLongValuesSource(value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean advanceExact(int doc) throws IOException {
|
||||
return true;
|
||||
}
|
||||
};
|
||||
}
|
||||
private static class ConstantLongValuesSource extends LongValuesSource {
|
||||
|
||||
private final long value;
|
||||
|
||||
private ConstantLongValuesSource(long value) {
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public LongValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException {
|
||||
return new LongValues() {
|
||||
@Override
|
||||
public long longValue() throws IOException {
|
||||
return value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean advanceExact(int doc) throws IOException {
|
||||
return true;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
ConstantLongValuesSource that = (ConstantLongValuesSource) o;
|
||||
return value == that.value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "constant(" + value + ")";
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return false;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
private static class FieldValuesSource extends LongValuesSource {
|
||||
|
@ -117,6 +154,11 @@ public abstract class LongValuesSource {
|
|||
return Objects.equals(field, that.field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "long(" + field + ")";
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(field);
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue