mirror of https://github.com/apache/lucene.git
LUCENE-2621: move TermVectors,FieldInfos,SegmentInfos to codec
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1202842 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
0780806efe
commit
873f199924
|
@ -10,8 +10,6 @@
|
|||
<classpathentry kind="src" path="lucene/contrib/demo/src/test"/>
|
||||
<classpathentry kind="src" path="lucene/contrib/highlighter/src/java"/>
|
||||
<classpathentry kind="src" path="lucene/contrib/highlighter/src/test"/>
|
||||
<classpathentry kind="src" path="lucene/contrib/instantiated/src/java"/>
|
||||
<classpathentry kind="src" path="lucene/contrib/instantiated/src/test"/>
|
||||
<classpathentry kind="src" path="lucene/contrib/memory/src/java"/>
|
||||
<classpathentry kind="src" path="lucene/contrib/memory/src/test"/>
|
||||
<classpathentry kind="src" path="lucene/contrib/misc/src/java"/>
|
||||
|
|
|
@ -1,17 +0,0 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<module type="JAVA_MODULE" version="4">
|
||||
<component name="NewModuleRootManager" inherit-compiler-output="false">
|
||||
<output url="file://$MODULE_DIR$/../../build/contrib/instantiated/classes/java" />
|
||||
<output-test url="file://$MODULE_DIR$/../../build/contrib/instantiated/classes/test" />
|
||||
<exclude-output />
|
||||
<content url="file://$MODULE_DIR$">
|
||||
<sourceFolder url="file://$MODULE_DIR$/src/java" isTestSource="false" />
|
||||
<sourceFolder url="file://$MODULE_DIR$/src/test" isTestSource="true" />
|
||||
</content>
|
||||
<orderEntry type="inheritedJdk" />
|
||||
<orderEntry type="sourceFolder" forTests="false" />
|
||||
<orderEntry type="library" scope="TEST" name="JUnit" level="project" />
|
||||
<orderEntry type="module" module-name="misc" />
|
||||
<orderEntry type="module" module-name="lucene" />
|
||||
</component>
|
||||
</module>
|
|
@ -1,66 +0,0 @@
|
|||
<project xmlns="http://maven.apache.org/POM/4.0.0"
|
||||
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
|
||||
<!--
|
||||
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.
|
||||
-->
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<parent>
|
||||
<groupId>org.apache.lucene</groupId>
|
||||
<artifactId>lucene-parent</artifactId>
|
||||
<version>@version@</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
<groupId>org.apache.lucene</groupId>
|
||||
<artifactId>lucene-instantiated</artifactId>
|
||||
<packaging>jar</packaging>
|
||||
<name>Lucene InstantiatedIndex</name>
|
||||
<description>InstantiatedIndex, alternative RAM store for small corpora.</description>
|
||||
<properties>
|
||||
<module-directory>lucene/contrib/instantiated</module-directory>
|
||||
<build-directory>../../build/contrib/instantiated</build-directory>
|
||||
</properties>
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>${project.groupId}</groupId>
|
||||
<artifactId>lucene-core</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>${project.groupId}</groupId>
|
||||
<artifactId>lucene-test-framework</artifactId>
|
||||
<version>${project.version}</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
<build>
|
||||
<directory>${build-directory}</directory>
|
||||
<outputDirectory>${build-directory}/classes/java</outputDirectory>
|
||||
<testOutputDirectory>${build-directory}/classes/test</testOutputDirectory>
|
||||
<sourceDirectory>src/java</sourceDirectory>
|
||||
<testSourceDirectory>src/test</testSourceDirectory>
|
||||
<testResources>
|
||||
<testResource>
|
||||
<directory>${project.build.testSourceDirectory}</directory>
|
||||
<excludes>
|
||||
<exclude>**/*.java</exclude>
|
||||
</excludes>
|
||||
</testResource>
|
||||
</testResources>
|
||||
</build>
|
||||
</project>
|
|
@ -33,7 +33,6 @@
|
|||
<modules>
|
||||
<module>demo</module>
|
||||
<module>highlighter</module>
|
||||
<module>instantiated</module>
|
||||
<module>memory</module>
|
||||
<module>misc</module>
|
||||
<module>sandbox</module>
|
||||
|
|
|
@ -32,6 +32,11 @@ Changes in backwards compatibility policy
|
|||
now specify the skipDocs explicitly (typically this will be the
|
||||
deleted docs, but in general you can provide any Bits).
|
||||
|
||||
- The term vectors APIs (TermFreqVector, TermPositionVector,
|
||||
TermVectorMapper) have been removed in favor of the above
|
||||
flexible indexing APIs, presenting a single-document inverted
|
||||
index of the document from the term vectors.
|
||||
|
||||
- MultiReader ctor now throws IOException
|
||||
|
||||
- Directory.copy/Directory.copyTo now copies all files (not just
|
||||
|
|
|
@ -252,7 +252,6 @@
|
|||
|
||||
<packageset dir="contrib/demo/src/java"/>
|
||||
<packageset dir="contrib/highlighter/src/java"/>
|
||||
<packageset dir="contrib/instantiated/src/java"/>
|
||||
<packageset dir="contrib/memory/src/java"/>
|
||||
<packageset dir="contrib/misc/src/java"/>
|
||||
<packageset dir="contrib/sandbox/src/java"/>
|
||||
|
@ -268,7 +267,6 @@
|
|||
<group title="contrib: Demo" packages="org.apache.lucene.demo*"/>
|
||||
<group title="contrib: ICU" packages="org.apache.lucene.collation*"/>
|
||||
<group title="contrib: Highlighter" packages="org.apache.lucene.search.highlight*:org.apache.lucene.search.vectorhighlight*"/>
|
||||
<group title="contrib: Instantiated" packages="org.apache.lucene.store.instantiated*"/>
|
||||
<group title="contrib: Memory" packages="org.apache.lucene.index.memory*"/>
|
||||
<group title="contrib: Misc " packages="org.apache.lucene.misc*"/>
|
||||
<group title="contrib: Sandbox" packages="org.apache.lucene.sandbox*"/>
|
||||
|
|
|
@ -808,7 +808,7 @@
|
|||
<!-- <compilerarg line="-Xmaxwarns 10000000"/>
|
||||
<compilerarg line="-Xmaxerrs 10000000"/> -->
|
||||
<!-- for generics in Java 1.5: -->
|
||||
<compilerarg line="-Xlint -Xlint:-deprecation -Xlint:-serial"/>
|
||||
<compilerarg line="-Xlint -Xlint:-deprecation -Xlint:-serial"/>
|
||||
</javac>
|
||||
</sequential>
|
||||
</macrodef>
|
||||
|
|
|
@ -13,6 +13,8 @@ Changes in Runtime Behavior
|
|||
provide an analyzer that returns 1 from the offsetGap method.
|
||||
(Mike McCandless)
|
||||
|
||||
* LUCENE-2621: Removed contrib/instantiated. (Robert Muir)
|
||||
|
||||
Build
|
||||
|
||||
* LUCENE-2845: Moved contrib/benchmark to modules.
|
||||
|
|
|
@ -32,10 +32,11 @@ import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
|||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.index.DocsAndPositionsEnum;
|
||||
import org.apache.lucene.index.Fields;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.TermFreqVector;
|
||||
import org.apache.lucene.index.TermPositionVector;
|
||||
import org.apache.lucene.index.TermVectorOffsetInfo;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
|
@ -69,12 +70,14 @@ public class TokenSources {
|
|||
String field, Document doc, Analyzer analyzer) throws IOException {
|
||||
TokenStream ts = null;
|
||||
|
||||
TermFreqVector tfv = reader.getTermFreqVector(docId, field);
|
||||
if (tfv != null) {
|
||||
if (tfv instanceof TermPositionVector) {
|
||||
ts = getTokenStream((TermPositionVector) tfv);
|
||||
Fields vectors = reader.getTermVectors(docId);
|
||||
if (vectors != null) {
|
||||
Terms vector = vectors.terms(field);
|
||||
if (vector != null) {
|
||||
ts = getTokenStream(vector);
|
||||
}
|
||||
}
|
||||
|
||||
// No token info stored so fall back to analyzing raw content
|
||||
if (ts == null) {
|
||||
ts = getTokenStream(doc, field, analyzer);
|
||||
|
@ -99,12 +102,14 @@ public class TokenSources {
|
|||
String field, Analyzer analyzer) throws IOException {
|
||||
TokenStream ts = null;
|
||||
|
||||
TermFreqVector tfv = reader.getTermFreqVector(docId, field);
|
||||
if (tfv != null) {
|
||||
if (tfv instanceof TermPositionVector) {
|
||||
ts = getTokenStream((TermPositionVector) tfv);
|
||||
Fields vectors = reader.getTermVectors(docId);
|
||||
if (vectors != null) {
|
||||
Terms vector = vectors.terms(field);
|
||||
if (vector != null) {
|
||||
ts = getTokenStream(vector);
|
||||
}
|
||||
}
|
||||
|
||||
// No token info stored so fall back to analyzing raw content
|
||||
if (ts == null) {
|
||||
ts = getTokenStream(reader, docId, field, analyzer);
|
||||
|
@ -112,10 +117,25 @@ public class TokenSources {
|
|||
return ts;
|
||||
}
|
||||
|
||||
public static TokenStream getTokenStream(TermPositionVector tpv) {
|
||||
public static TokenStream getTokenStream(Terms vector) throws IOException {
|
||||
// assumes the worst and makes no assumptions about token position
|
||||
// sequences.
|
||||
return getTokenStream(tpv, false);
|
||||
return getTokenStream(vector, false);
|
||||
}
|
||||
|
||||
private static boolean hasPositions(Terms vector) throws IOException {
|
||||
final TermsEnum termsEnum = vector.iterator(null);
|
||||
if (termsEnum.next() != null) {
|
||||
DocsAndPositionsEnum dpEnum = termsEnum.docsAndPositions(null, null);
|
||||
if (dpEnum != null) {
|
||||
int pos = dpEnum.nextPosition();
|
||||
if (pos >= 0) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -144,9 +164,10 @@ public class TokenSources {
|
|||
* numbers have no overlaps or gaps. If looking to eek out the last
|
||||
* drops of performance, set to true. If in doubt, set to false.
|
||||
*/
|
||||
public static TokenStream getTokenStream(TermPositionVector tpv,
|
||||
boolean tokenPositionsGuaranteedContiguous) {
|
||||
if (!tokenPositionsGuaranteedContiguous && tpv.getTermPositions(0) != null) {
|
||||
public static TokenStream getTokenStream(Terms tpv,
|
||||
boolean tokenPositionsGuaranteedContiguous)
|
||||
throws IOException {
|
||||
if (!tokenPositionsGuaranteedContiguous && hasPositions(tpv)) {
|
||||
return new TokenStreamFromTermPositionVector(tpv);
|
||||
}
|
||||
|
||||
|
@ -186,56 +207,57 @@ public class TokenSources {
|
|||
}
|
||||
}
|
||||
// code to reconstruct the original sequence of Tokens
|
||||
BytesRef[] terms = tpv.getTerms();
|
||||
int[] freq = tpv.getTermFrequencies();
|
||||
TermsEnum termsEnum = tpv.iterator(null);
|
||||
int totalTokens = 0;
|
||||
for (int t = 0; t < freq.length; t++) {
|
||||
totalTokens += freq[t];
|
||||
while(termsEnum.next() != null) {
|
||||
totalTokens += (int) termsEnum.totalTermFreq();
|
||||
}
|
||||
Token tokensInOriginalOrder[] = new Token[totalTokens];
|
||||
ArrayList<Token> unsortedTokens = null;
|
||||
for (int t = 0; t < freq.length; t++) {
|
||||
TermVectorOffsetInfo[] offsets = tpv.getOffsets(t);
|
||||
if (offsets == null) {
|
||||
termsEnum = tpv.iterator(null);
|
||||
BytesRef text;
|
||||
DocsAndPositionsEnum dpEnum = null;
|
||||
while ((text = termsEnum.next()) != null) {
|
||||
|
||||
dpEnum = termsEnum.docsAndPositions(null, dpEnum);
|
||||
if (dpEnum == null || (!dpEnum.attributes().hasAttribute(OffsetAttribute.class))) {
|
||||
throw new IllegalArgumentException(
|
||||
"Required TermVector Offset information was not found");
|
||||
}
|
||||
|
||||
int[] pos = null;
|
||||
if (tokenPositionsGuaranteedContiguous) {
|
||||
// try get the token position info to speed up assembly of tokens into
|
||||
// sorted sequence
|
||||
pos = tpv.getTermPositions(t);
|
||||
}
|
||||
if (pos == null) {
|
||||
// tokens NOT stored with positions or not guaranteed contiguous - must
|
||||
// add to list and sort later
|
||||
if (unsortedTokens == null) {
|
||||
unsortedTokens = new ArrayList<Token>();
|
||||
}
|
||||
for (int tp = 0; tp < offsets.length; tp++) {
|
||||
Token token = new Token(terms[t].utf8ToString(),
|
||||
offsets[tp].getStartOffset(), offsets[tp].getEndOffset());
|
||||
final String term = text.utf8ToString();
|
||||
|
||||
final OffsetAttribute offsetAtt = dpEnum.attributes().getAttribute(OffsetAttribute.class);
|
||||
dpEnum.nextDoc();
|
||||
final int freq = dpEnum.freq();
|
||||
for(int posUpto=0;posUpto<freq;posUpto++) {
|
||||
final int pos = dpEnum.nextPosition();
|
||||
final Token token = new Token(term,
|
||||
offsetAtt.startOffset(),
|
||||
offsetAtt.endOffset());
|
||||
if (tokenPositionsGuaranteedContiguous && pos != -1) {
|
||||
// We have positions stored and a guarantee that the token position
|
||||
// information is contiguous
|
||||
|
||||
// This may be fast BUT wont work if Tokenizers used which create >1
|
||||
// token in same position or
|
||||
// creates jumps in position numbers - this code would fail under those
|
||||
// circumstances
|
||||
|
||||
// tokens stored with positions - can use this to index straight into
|
||||
// sorted array
|
||||
tokensInOriginalOrder[pos] = token;
|
||||
} else {
|
||||
// tokens NOT stored with positions or not guaranteed contiguous - must
|
||||
// add to list and sort later
|
||||
if (unsortedTokens == null) {
|
||||
unsortedTokens = new ArrayList<Token>();
|
||||
}
|
||||
unsortedTokens.add(token);
|
||||
}
|
||||
} else {
|
||||
// We have positions stored and a guarantee that the token position
|
||||
// information is contiguous
|
||||
|
||||
// This may be fast BUT wont work if Tokenizers used which create >1
|
||||
// token in same position or
|
||||
// creates jumps in position numbers - this code would fail under those
|
||||
// circumstances
|
||||
|
||||
// tokens stored with positions - can use this to index straight into
|
||||
// sorted array
|
||||
for (int tp = 0; tp < pos.length; tp++) {
|
||||
Token token = new Token(terms[t].utf8ToString(),
|
||||
offsets[tp].getStartOffset(), offsets[tp].getEndOffset());
|
||||
tokensInOriginalOrder[pos[tp]] = token;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// If the field has been stored without position data we must perform a sort
|
||||
if (unsortedTokens != null) {
|
||||
tokensInOriginalOrder = unsortedTokens.toArray(new Token[unsortedTokens
|
||||
|
@ -253,18 +275,25 @@ public class TokenSources {
|
|||
|
||||
public static TokenStream getTokenStream(IndexReader reader, int docId,
|
||||
String field) throws IOException {
|
||||
TermFreqVector tfv = reader.getTermFreqVector(docId, field);
|
||||
if (tfv == null) {
|
||||
|
||||
Fields vectors = reader.getTermVectors(docId);
|
||||
if (vectors == null) {
|
||||
throw new IllegalArgumentException(field + " in doc #" + docId
|
||||
+ "does not have any term position data stored");
|
||||
}
|
||||
if (tfv instanceof TermPositionVector) {
|
||||
TermPositionVector tpv = (TermPositionVector) reader.getTermFreqVector(
|
||||
docId, field);
|
||||
return getTokenStream(tpv);
|
||||
|
||||
Terms vector = vectors.terms(field);
|
||||
if (vector == null) {
|
||||
throw new IllegalArgumentException(field + " in doc #" + docId
|
||||
+ "does not have any term position data stored");
|
||||
}
|
||||
throw new IllegalArgumentException(field + " in doc #" + docId
|
||||
+ "does not have any term position data stored");
|
||||
|
||||
if (!hasPositions(vector)) {
|
||||
throw new IllegalArgumentException(field + " in doc #" + docId
|
||||
+ "does not have any term position data stored");
|
||||
}
|
||||
|
||||
return getTokenStream(vector);
|
||||
}
|
||||
|
||||
// convenience method
|
||||
|
|
|
@ -27,8 +27,9 @@ import org.apache.lucene.analysis.TokenStream;
|
|||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
import org.apache.lucene.index.TermPositionVector;
|
||||
import org.apache.lucene.index.TermVectorOffsetInfo;
|
||||
import org.apache.lucene.index.DocsAndPositionsEnum;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CollectionUtil;
|
||||
|
||||
|
@ -47,31 +48,42 @@ public final class TokenStreamFromTermPositionVector extends TokenStream {
|
|||
/**
|
||||
* Constructor.
|
||||
*
|
||||
* @param termPositionVector TermPositionVector that contains the data for
|
||||
* @param vector Terms that contains the data for
|
||||
* creating the TokenStream. Must have positions and offsets.
|
||||
*/
|
||||
public TokenStreamFromTermPositionVector(
|
||||
final TermPositionVector termPositionVector) {
|
||||
final Terms vector) throws IOException {
|
||||
termAttribute = addAttribute(CharTermAttribute.class);
|
||||
positionIncrementAttribute = addAttribute(PositionIncrementAttribute.class);
|
||||
offsetAttribute = addAttribute(OffsetAttribute.class);
|
||||
final BytesRef[] terms = termPositionVector.getTerms();
|
||||
for (int i = 0; i < terms.length; i++) {
|
||||
final TermVectorOffsetInfo[] offsets = termPositionVector.getOffsets(i);
|
||||
final int[] termPositions = termPositionVector.getTermPositions(i);
|
||||
for (int j = 0; j < termPositions.length; j++) {
|
||||
final TermsEnum termsEnum = vector.iterator(null);
|
||||
BytesRef text;
|
||||
DocsAndPositionsEnum dpEnum = null;
|
||||
while((text = termsEnum.next()) != null) {
|
||||
dpEnum = termsEnum.docsAndPositions(null, dpEnum);
|
||||
dpEnum.nextDoc();
|
||||
final int freq = dpEnum.freq();
|
||||
final OffsetAttribute offsetAtt;
|
||||
if (dpEnum.attributes().hasAttribute(OffsetAttribute.class)) {
|
||||
offsetAtt = dpEnum.attributes().getAttribute(OffsetAttribute.class);
|
||||
} else {
|
||||
offsetAtt = null;
|
||||
}
|
||||
for (int j = 0; j < freq; j++) {
|
||||
int pos = dpEnum.nextPosition();
|
||||
Token token;
|
||||
if (offsets != null) {
|
||||
token = new Token(terms[i].utf8ToString(),
|
||||
offsets[j].getStartOffset(), offsets[j].getEndOffset());
|
||||
if (offsetAtt != null) {
|
||||
token = new Token(text.utf8ToString(),
|
||||
offsetAtt.startOffset(),
|
||||
offsetAtt.endOffset());
|
||||
} else {
|
||||
token = new Token();
|
||||
token.setEmpty().append(terms[i].utf8ToString());
|
||||
token.setEmpty().append(text.utf8ToString());
|
||||
}
|
||||
// Yes - this is the position, not the increment! This is for
|
||||
// sorting. This value
|
||||
// will be corrected before use.
|
||||
token.setPositionIncrement(termPositions[j]);
|
||||
token.setPositionIncrement(pos);
|
||||
this.positionedTokens.add(token);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,10 +21,12 @@ import java.util.Collections;
|
|||
import java.util.LinkedList;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.index.DocsAndPositionsEnum;
|
||||
import org.apache.lucene.index.Fields;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.TermFreqVector;
|
||||
import org.apache.lucene.index.TermPositionVector;
|
||||
import org.apache.lucene.index.TermVectorOffsetInfo;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CharsRef;
|
||||
|
||||
|
@ -76,30 +78,55 @@ public class FieldTermStack {
|
|||
// just return to make null snippet if un-matched fieldName specified when fieldMatch == true
|
||||
if( termSet == null ) return;
|
||||
|
||||
TermFreqVector tfv = reader.getTermFreqVector( docId, fieldName );
|
||||
if( tfv == null ) return; // just return to make null snippets
|
||||
TermPositionVector tpv = null;
|
||||
try{
|
||||
tpv = (TermPositionVector)tfv;
|
||||
final Fields vectors = reader.getTermVectors(docId);
|
||||
if (vectors == null) {
|
||||
// null snippet
|
||||
return;
|
||||
}
|
||||
catch( ClassCastException e ){
|
||||
return; // just return to make null snippets
|
||||
|
||||
final Terms vector = vectors.terms(fieldName);
|
||||
if (vector == null) {
|
||||
// null snippet
|
||||
return;
|
||||
}
|
||||
|
||||
|
||||
final CharsRef spare = new CharsRef();
|
||||
for( BytesRef term : tpv.getTerms() ){
|
||||
if( !termSet.contains( term.utf8ToChars(spare).toString() ) ) continue;
|
||||
int index = tpv.indexOf( term );
|
||||
TermVectorOffsetInfo[] tvois = tpv.getOffsets( index );
|
||||
if( tvois == null ) return; // just return to make null snippets
|
||||
int[] poss = tpv.getTermPositions( index );
|
||||
if( poss == null ) return; // just return to make null snippets
|
||||
for( int i = 0; i < tvois.length; i++ )
|
||||
termList.add( new TermInfo( term.utf8ToChars(spare).toString(), tvois[i].getStartOffset(), tvois[i].getEndOffset(), poss[i] ) );
|
||||
final TermsEnum termsEnum = vector.iterator(null);
|
||||
DocsAndPositionsEnum dpEnum = null;
|
||||
BytesRef text;
|
||||
while ((text = termsEnum.next()) != null) {
|
||||
final String term = text.utf8ToChars(spare).toString();
|
||||
if (!termSet.contains(term)) {
|
||||
continue;
|
||||
}
|
||||
dpEnum = termsEnum.docsAndPositions(null, dpEnum);
|
||||
if (dpEnum == null) {
|
||||
// null snippet
|
||||
return;
|
||||
}
|
||||
|
||||
if (!dpEnum.attributes().hasAttribute(OffsetAttribute.class)) {
|
||||
// null snippet
|
||||
return;
|
||||
}
|
||||
dpEnum.nextDoc();
|
||||
|
||||
final OffsetAttribute offsetAtt = dpEnum.attributes().getAttribute(OffsetAttribute.class);
|
||||
|
||||
final int freq = dpEnum.freq();
|
||||
|
||||
for(int i = 0;i < freq;i++) {
|
||||
final int pos = dpEnum.nextPosition();
|
||||
if (pos == -1) {
|
||||
// null snippet
|
||||
return;
|
||||
}
|
||||
termList.add(new TermInfo(term, offsetAtt.startOffset(), offsetAtt.endOffset(), pos));
|
||||
}
|
||||
}
|
||||
|
||||
// sort by position
|
||||
Collections.sort( termList );
|
||||
Collections.sort(termList);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -95,7 +95,7 @@ will be generated:</p>
|
|||
</pre>
|
||||
|
||||
<h3>Step 2.</h3>
|
||||
<p>In Step 2, Fast Vector Highlighter generates {@link org.apache.lucene.search.vectorhighlight.FieldTermStack}. Fast Vector Highlighter uses {@link org.apache.lucene.index.TermFreqVector} data
|
||||
<p>In Step 2, Fast Vector Highlighter generates {@link org.apache.lucene.search.vectorhighlight.FieldTermStack}. Fast Vector Highlighter uses term vector data
|
||||
(must be stored {@link org.apache.lucene.document.FieldType#setStoreTermVectorOffsets(boolean)} and {@link org.apache.lucene.document.FieldType#setStoreTermVectorPositions(boolean)})
|
||||
to generate it. <code>FieldTermStack</code> keeps the terms in the user query.
|
||||
Therefore, in this sample case, Fast Vector Highlighter generates the following <code>FieldTermStack</code>:</p>
|
||||
|
|
|
@ -34,7 +34,6 @@ import org.apache.lucene.index.CorruptIndexException;
|
|||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.TermPositionVector;
|
||||
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
|
||||
import org.apache.lucene.search.Collector;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
|
@ -85,7 +84,7 @@ public class HighlighterPhraseTest extends LuceneTestCase {
|
|||
new QueryScorer(phraseQuery));
|
||||
|
||||
final TokenStream tokenStream = TokenSources
|
||||
.getTokenStream((TermPositionVector) indexReader.getTermFreqVector(
|
||||
.getTokenStream(indexReader.getTermVector(
|
||||
0, FIELD), false);
|
||||
assertEquals(highlighter.getBestFragment(new TokenStreamConcurrent(),
|
||||
TEXT), highlighter.getBestFragment(tokenStream, TEXT));
|
||||
|
@ -160,7 +159,7 @@ public class HighlighterPhraseTest extends LuceneTestCase {
|
|||
.nextSetBit(position + 1)) {
|
||||
assertEquals(0, position);
|
||||
final TokenStream tokenStream = TokenSources.getTokenStream(
|
||||
(TermPositionVector) indexReader.getTermFreqVector(position,
|
||||
indexReader.getTermVector(position,
|
||||
FIELD), false);
|
||||
assertEquals(highlighter.getBestFragment(new TokenStreamConcurrent(),
|
||||
TEXT), highlighter.getBestFragment(tokenStream, TEXT));
|
||||
|
@ -207,7 +206,7 @@ public class HighlighterPhraseTest extends LuceneTestCase {
|
|||
new SimpleHTMLFormatter(), new SimpleHTMLEncoder(),
|
||||
new QueryScorer(phraseQuery));
|
||||
final TokenStream tokenStream = TokenSources
|
||||
.getTokenStream((TermPositionVector) indexReader.getTermFreqVector(
|
||||
.getTokenStream(indexReader.getTermVector(
|
||||
0, FIELD), false);
|
||||
assertEquals(
|
||||
highlighter.getBestFragment(new TokenStreamSparse(), TEXT),
|
||||
|
@ -253,7 +252,7 @@ public class HighlighterPhraseTest extends LuceneTestCase {
|
|||
new SimpleHTMLFormatter(), new SimpleHTMLEncoder(),
|
||||
new QueryScorer(phraseQuery));
|
||||
final TokenStream tokenStream = TokenSources.getTokenStream(
|
||||
(TermPositionVector) indexReader.getTermFreqVector(0, FIELD), true);
|
||||
indexReader.getTermVector(0, FIELD), true);
|
||||
assertEquals("the fox <B>did</B> not <B>jump</B>", highlighter
|
||||
.getBestFragment(tokenStream, TEXT));
|
||||
} finally {
|
||||
|
@ -297,7 +296,7 @@ public class HighlighterPhraseTest extends LuceneTestCase {
|
|||
new SimpleHTMLFormatter(), new SimpleHTMLEncoder(),
|
||||
new QueryScorer(phraseQuery));
|
||||
final TokenStream tokenStream = TokenSources
|
||||
.getTokenStream((TermPositionVector) indexReader.getTermFreqVector(
|
||||
.getTokenStream(indexReader.getTermVector(
|
||||
0, FIELD), false);
|
||||
assertEquals(
|
||||
highlighter.getBestFragment(new TokenStreamSparse(), TEXT),
|
||||
|
|
|
@ -32,7 +32,6 @@ import org.apache.lucene.index.CorruptIndexException;
|
|||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.TermPositionVector;
|
||||
import org.apache.lucene.search.DisjunctionMaxQuery;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.Query;
|
||||
|
@ -133,7 +132,7 @@ public class TokenSourcesTest extends LuceneTestCase {
|
|||
new QueryScorer(query));
|
||||
final TokenStream tokenStream = TokenSources
|
||||
.getTokenStream(
|
||||
(TermPositionVector) indexReader.getTermFreqVector(0, FIELD),
|
||||
indexReader.getTermVector(0, FIELD),
|
||||
false);
|
||||
assertEquals("<B>the fox</B> did not jump",
|
||||
highlighter.getBestFragment(tokenStream, TEXT));
|
||||
|
@ -182,7 +181,7 @@ public class TokenSourcesTest extends LuceneTestCase {
|
|||
new QueryScorer(query));
|
||||
final TokenStream tokenStream = TokenSources
|
||||
.getTokenStream(
|
||||
(TermPositionVector) indexReader.getTermFreqVector(0, FIELD),
|
||||
indexReader.getTermVector(0, FIELD),
|
||||
false);
|
||||
assertEquals("<B>the fox</B> did not jump",
|
||||
highlighter.getBestFragment(tokenStream, TEXT));
|
||||
|
@ -230,7 +229,7 @@ public class TokenSourcesTest extends LuceneTestCase {
|
|||
new QueryScorer(phraseQuery));
|
||||
final TokenStream tokenStream = TokenSources
|
||||
.getTokenStream(
|
||||
(TermPositionVector) indexReader.getTermFreqVector(0, FIELD),
|
||||
indexReader.getTermVector(0, FIELD),
|
||||
false);
|
||||
assertEquals("<B>the fox</B> did not jump",
|
||||
highlighter.getBestFragment(tokenStream, TEXT));
|
||||
|
@ -279,7 +278,7 @@ public class TokenSourcesTest extends LuceneTestCase {
|
|||
new QueryScorer(phraseQuery));
|
||||
final TokenStream tokenStream = TokenSources
|
||||
.getTokenStream(
|
||||
(TermPositionVector) indexReader.getTermFreqVector(0, FIELD),
|
||||
indexReader.getTermVector(0, FIELD),
|
||||
false);
|
||||
assertEquals("<B>the fox</B> did not jump",
|
||||
highlighter.getBestFragment(tokenStream, TEXT));
|
||||
|
|
|
@ -1,29 +0,0 @@
|
|||
<?xml version="1.0"?>
|
||||
|
||||
<!--
|
||||
Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
contributor license agreements. See the NOTICE file distributed with
|
||||
this work for additional information regarding copyright ownership.
|
||||
The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
the "License"); you may not use this file except in compliance with
|
||||
the License. You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
-->
|
||||
|
||||
<project name="instantiated" default="default">
|
||||
|
||||
<description>
|
||||
InstantiatedIndex, an alternative RAM store.
|
||||
</description>
|
||||
|
||||
<import file="../contrib-build.xml"/>
|
||||
|
||||
|
||||
</project>
|
|
@ -1,62 +0,0 @@
|
|||
package org.apache.lucene.store.instantiated;
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
/**
|
||||
* For non package access see {@link org.apache.lucene.index.IndexReader#getFieldNames(org.apache.lucene.index.IndexReader.FieldOption)}
|
||||
*/
|
||||
class FieldSetting {
|
||||
String fieldName;
|
||||
|
||||
boolean storeTermVector = false;
|
||||
boolean storeOffsetWithTermVector = false;
|
||||
boolean storePositionWithTermVector = false;
|
||||
boolean storePayloads = false;
|
||||
|
||||
boolean stored = false;
|
||||
boolean indexed = false;
|
||||
boolean tokenized = false;
|
||||
|
||||
FieldSetting() {
|
||||
}
|
||||
|
||||
|
||||
FieldSetting(String fieldName) {
|
||||
this.fieldName = fieldName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o)
|
||||
return true;
|
||||
if (o == null || getClass() != o.getClass())
|
||||
return false;
|
||||
|
||||
final FieldSetting that = (FieldSetting) o;
|
||||
|
||||
return fieldName.equals(that.fieldName);
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return fieldName.hashCode();
|
||||
}
|
||||
|
||||
|
||||
}
|
|
@ -1,92 +0,0 @@
|
|||
package org.apache.lucene.store.instantiated;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Collection;
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
/**
|
||||
* Essentially a Map<FieldName, {@link org.apache.lucene.store.instantiated.FieldSetting}>
|
||||
*/
|
||||
class FieldSettings {
|
||||
|
||||
|
||||
FieldSettings() {
|
||||
}
|
||||
|
||||
private Map</** field name */String, FieldSetting> fieldSettings = new HashMap<String, FieldSetting>();
|
||||
|
||||
synchronized FieldSetting merge(FieldSetting fieldSetting) {
|
||||
FieldSetting setting = fieldSettings.get(fieldSetting.fieldName);
|
||||
|
||||
if (setting == null) {
|
||||
setting = new FieldSetting(fieldSetting.fieldName);
|
||||
fieldSettings.put(fieldSetting.fieldName, setting);
|
||||
}
|
||||
|
||||
if (fieldSetting.stored) {
|
||||
setting.stored = true;
|
||||
}
|
||||
|
||||
if ("b3".equals(fieldSetting.fieldName)) {
|
||||
System.currentTimeMillis();
|
||||
}
|
||||
if (fieldSetting.indexed) {
|
||||
setting.indexed = true;
|
||||
}
|
||||
if (fieldSetting.tokenized) {
|
||||
setting.tokenized = true;
|
||||
}
|
||||
|
||||
if (fieldSetting.storeTermVector) {
|
||||
setting.storeTermVector = true;
|
||||
}
|
||||
if (fieldSetting.storeOffsetWithTermVector) {
|
||||
setting.storeOffsetWithTermVector = true;
|
||||
}
|
||||
if (fieldSetting.storePositionWithTermVector) {
|
||||
setting.storePositionWithTermVector = true;
|
||||
}
|
||||
|
||||
if (fieldSetting.storePayloads) {
|
||||
setting.storePayloads = true;
|
||||
}
|
||||
|
||||
return setting;
|
||||
|
||||
}
|
||||
|
||||
FieldSetting get(String name) {
|
||||
return fieldSettings.get(name);
|
||||
}
|
||||
|
||||
FieldSetting get(String name, boolean create) {
|
||||
FieldSetting fieldSetting = fieldSettings.get(name);
|
||||
if (create && fieldSetting == null) {
|
||||
fieldSetting = new FieldSetting(name);
|
||||
fieldSettings.put(name, fieldSetting);
|
||||
}
|
||||
return fieldSetting;
|
||||
}
|
||||
|
||||
Collection<FieldSetting> values() {
|
||||
return fieldSettings.values();
|
||||
}
|
||||
|
||||
}
|
|
@ -1,101 +0,0 @@
|
|||
package org.apache.lucene.store.instantiated;
|
||||
|
||||
/**
|
||||
* Copyright 2006 The Apache Software Foundation
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.index.DocsAndPositionsEnum;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
public class InstantiatedDocsAndPositionsEnum extends DocsAndPositionsEnum {
|
||||
private int upto;
|
||||
private int posUpto;
|
||||
private Bits liveDocs;
|
||||
private InstantiatedTerm term;
|
||||
protected InstantiatedTermDocumentInformation currentDoc;
|
||||
private final BytesRef payload = new BytesRef();
|
||||
|
||||
public InstantiatedDocsAndPositionsEnum reset(Bits liveDocs, InstantiatedTerm term) {
|
||||
this.liveDocs = liveDocs;
|
||||
this.term = term;
|
||||
upto = -1;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return currentDoc.getDocument().getDocumentNumber();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextDoc() {
|
||||
upto++;
|
||||
if (upto >= term.getAssociatedDocuments().length) {
|
||||
return NO_MORE_DOCS;
|
||||
} else {
|
||||
currentDoc = term.getAssociatedDocuments()[upto];
|
||||
if (liveDocs == null || liveDocs.get(currentDoc.getDocument().getDocumentNumber())) {
|
||||
posUpto = -1;
|
||||
return docID();
|
||||
} else {
|
||||
return nextDoc();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) {
|
||||
if (currentDoc.getDocument().getDocumentNumber() >= target) {
|
||||
return nextDoc();
|
||||
}
|
||||
|
||||
int startOffset = upto >= 0 ? upto : 0;
|
||||
upto = term.seekCeilingDocumentInformationIndex(target, startOffset);
|
||||
if (upto == -1) {
|
||||
return NO_MORE_DOCS;
|
||||
}
|
||||
currentDoc = term.getAssociatedDocuments()[upto];
|
||||
|
||||
if (liveDocs != null && !liveDocs.get(currentDoc.getDocument().getDocumentNumber())) {
|
||||
return nextDoc();
|
||||
} else {
|
||||
posUpto = -1;
|
||||
return docID();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int freq() {
|
||||
return currentDoc.getTermPositions().length;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextPosition() {
|
||||
return currentDoc.getTermPositions()[++posUpto];
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasPayload() {
|
||||
return currentDoc.getPayloads()[posUpto] != null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getPayload() {
|
||||
payload.bytes = currentDoc.getPayloads()[posUpto];
|
||||
payload.length = payload.bytes.length;
|
||||
return payload;
|
||||
}
|
||||
}
|
|
@ -1,79 +0,0 @@
|
|||
package org.apache.lucene.store.instantiated;
|
||||
|
||||
/**
|
||||
* Copyright 2006 The Apache Software Foundation
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.index.DocsEnum;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
||||
public class InstantiatedDocsEnum extends DocsEnum {
|
||||
private int upto;
|
||||
private Bits liveDocs;
|
||||
private InstantiatedTerm term;
|
||||
protected InstantiatedTermDocumentInformation currentDoc;
|
||||
|
||||
public InstantiatedDocsEnum reset(Bits liveDocs, InstantiatedTerm term) {
|
||||
this.liveDocs = liveDocs;
|
||||
this.term = term;
|
||||
upto = -1;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return currentDoc.getDocument().getDocumentNumber();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextDoc() {
|
||||
upto++;
|
||||
if (upto >= term.getAssociatedDocuments().length) {
|
||||
return NO_MORE_DOCS;
|
||||
} else {
|
||||
currentDoc = term.getAssociatedDocuments()[upto];
|
||||
if (liveDocs == null || liveDocs.get(currentDoc.getDocument().getDocumentNumber())) {
|
||||
return docID();
|
||||
} else {
|
||||
return nextDoc();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) {
|
||||
if (currentDoc != null && currentDoc.getDocument().getDocumentNumber() >= target) {
|
||||
return nextDoc();
|
||||
}
|
||||
|
||||
int startOffset = upto >= 0 ? upto : 0;
|
||||
upto = term.seekCeilingDocumentInformationIndex(target, startOffset);
|
||||
if (upto == -1) {
|
||||
return NO_MORE_DOCS;
|
||||
}
|
||||
currentDoc = term.getAssociatedDocuments()[upto];
|
||||
|
||||
if (liveDocs != null && !liveDocs.get(currentDoc.getDocument().getDocumentNumber())) {
|
||||
return nextDoc();
|
||||
} else {
|
||||
return docID();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int freq() {
|
||||
return currentDoc.getTermPositions().length;
|
||||
}
|
||||
}
|
|
@ -1,75 +0,0 @@
|
|||
package org.apache.lucene.store.instantiated;
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.document.Document;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* A document in the instantiated index object graph, optionally coupled to the vector space view.
|
||||
*
|
||||
* @see org.apache.lucene.document.Document
|
||||
*/
|
||||
public class InstantiatedDocument {
|
||||
|
||||
private Document document;
|
||||
|
||||
public InstantiatedDocument() {
|
||||
this.document = new Document();
|
||||
}
|
||||
|
||||
|
||||
public InstantiatedDocument(Document document) {
|
||||
this.document = document;
|
||||
}
|
||||
|
||||
/** this is the unsafe index order document number. */
|
||||
private Integer documentNumber;
|
||||
|
||||
/** this is the term vector space view */
|
||||
private Map<String /*field name*/, List<InstantiatedTermDocumentInformation>> vectorSpace;
|
||||
|
||||
/**
|
||||
* @return position of document in the index.
|
||||
*/
|
||||
public Integer getDocumentNumber() {
|
||||
return documentNumber;
|
||||
}
|
||||
|
||||
void setDocumentNumber(Integer documentNumber) {
|
||||
this.documentNumber = documentNumber;
|
||||
}
|
||||
|
||||
public Map</*field name*/ String, List<InstantiatedTermDocumentInformation>> getVectorSpace() {
|
||||
return vectorSpace;
|
||||
}
|
||||
|
||||
public void setVectorSpace(Map</*field name*/ String, List<InstantiatedTermDocumentInformation>> vectorSpace) {
|
||||
this.vectorSpace = vectorSpace;
|
||||
}
|
||||
|
||||
public Document getDocument() {
|
||||
return document;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return document.toString();
|
||||
}
|
||||
}
|
|
@ -1,384 +0,0 @@
|
|||
package org.apache.lucene.store.instantiated;
|
||||
|
||||
/**
|
||||
* Copyright 2006 The Apache Software Foundation
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexableField;
|
||||
import org.apache.lucene.index.MultiNorms;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.index.Fields;
|
||||
import org.apache.lucene.index.FieldsEnum;
|
||||
import org.apache.lucene.index.DocsEnum;
|
||||
import org.apache.lucene.index.MultiFields;
|
||||
import org.apache.lucene.index.TermPositionVector;
|
||||
import org.apache.lucene.index.DocsAndPositionsEnum;
|
||||
import org.apache.lucene.util.BitVector;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CharsRef;
|
||||
|
||||
/**
|
||||
* Represented as a coupled graph of class instances, this
|
||||
* all-in-memory index store implementation delivers search
|
||||
* results up to a 100 times faster than the file-centric RAMDirectory
|
||||
* at the cost of greater RAM consumption.
|
||||
* <p>
|
||||
* @lucene.experimental
|
||||
* <p>
|
||||
* There are no read and write locks in this store.
|
||||
* {@link InstantiatedIndexReader} {@link InstantiatedIndexReader#isCurrent()} all the time
|
||||
* and {@link org.apache.lucene.store.instantiated.InstantiatedIndexWriter}
|
||||
* will attempt to update instances of the object graph in memory
|
||||
* at the same time as a searcher is reading from it.
|
||||
*
|
||||
* Consider using InstantiatedIndex as if it was immutable.
|
||||
*/
|
||||
public class InstantiatedIndex
|
||||
implements Closeable {
|
||||
|
||||
private long version = System.currentTimeMillis();
|
||||
|
||||
private InstantiatedDocument[] documentsByNumber;
|
||||
|
||||
private BitVector deletedDocuments;
|
||||
|
||||
private Map<String, Map<String, InstantiatedTerm>> termsByFieldAndText;
|
||||
private InstantiatedTerm[] orderedTerms;
|
||||
|
||||
private Map<String, byte[]> normsByFieldNameAndDocumentNumber;
|
||||
|
||||
private FieldSettings fieldSettings;
|
||||
|
||||
/**
|
||||
* Creates an empty instantiated index for you to fill with data using an {@link org.apache.lucene.store.instantiated.InstantiatedIndexWriter}.
|
||||
*/
|
||||
public InstantiatedIndex() {
|
||||
initialize();
|
||||
}
|
||||
|
||||
void initialize() {
|
||||
// todo: clear index without loosing memory (uncouple stuff)
|
||||
termsByFieldAndText = new HashMap<String, Map<String, InstantiatedTerm>>();
|
||||
fieldSettings = new FieldSettings();
|
||||
orderedTerms = new InstantiatedTerm[0];
|
||||
documentsByNumber = new InstantiatedDocument[0];
|
||||
normsByFieldNameAndDocumentNumber = new HashMap<String, byte[]>();
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Creates a new instantiated index that looks just like the index in a specific state as represented by a reader.
|
||||
*
|
||||
* @param sourceIndexReader the source index this new instantiated index will be copied from.
|
||||
* @throws IOException if the source index is not single-segment, or when accessing the source.
|
||||
*/
|
||||
public InstantiatedIndex(IndexReader sourceIndexReader) throws IOException {
|
||||
this(sourceIndexReader, null);
|
||||
}
|
||||
|
||||
|
||||
|
||||
/**
|
||||
* Creates a new instantiated index that looks just like the index in a specific state as represented by a reader.
|
||||
*
|
||||
* @param sourceIndexReader the source index this new instantiated index will be copied from.
|
||||
* @param fields fields to be added, or null for all
|
||||
* @throws IOException if the source index is not single-segment, or when accessing the source.
|
||||
*/
|
||||
public InstantiatedIndex(IndexReader sourceIndexReader, Set<String> fields) throws IOException {
|
||||
|
||||
if (sourceIndexReader.getSequentialSubReaders().length != 1) {
|
||||
System.out.println(("Source index has more than one segment."));
|
||||
//throw new IOException("Source index has more than one segment.");
|
||||
}
|
||||
|
||||
|
||||
initialize();
|
||||
|
||||
Collection<String> allFieldNames = sourceIndexReader.getFieldNames(IndexReader.FieldOption.ALL);
|
||||
|
||||
// load field options
|
||||
|
||||
Collection<String> indexedNames = sourceIndexReader.getFieldNames(IndexReader.FieldOption.INDEXED);
|
||||
for (String name : indexedNames) {
|
||||
FieldSetting setting = fieldSettings.get(name, true);
|
||||
setting.indexed = true;
|
||||
}
|
||||
Collection<String> indexedNoVecNames = sourceIndexReader.getFieldNames(IndexReader.FieldOption.INDEXED_NO_TERMVECTOR);
|
||||
for (String name : indexedNoVecNames) {
|
||||
FieldSetting setting = fieldSettings.get(name, true);
|
||||
setting.storeTermVector = false;
|
||||
setting.indexed = true;
|
||||
}
|
||||
Collection<String> indexedVecNames = sourceIndexReader.getFieldNames(IndexReader.FieldOption.INDEXED_WITH_TERMVECTOR);
|
||||
for (String name : indexedVecNames) {
|
||||
FieldSetting setting = fieldSettings.get(name, true);
|
||||
setting.storeTermVector = true;
|
||||
setting.indexed = true;
|
||||
}
|
||||
Collection<String> payloadNames = sourceIndexReader.getFieldNames(IndexReader.FieldOption.STORES_PAYLOADS);
|
||||
for (String name : payloadNames) {
|
||||
FieldSetting setting = fieldSettings.get(name, true);
|
||||
setting.storePayloads = true;
|
||||
}
|
||||
Collection<String> termVecNames = sourceIndexReader.getFieldNames(IndexReader.FieldOption.TERMVECTOR);
|
||||
for (String name : termVecNames) {
|
||||
FieldSetting setting = fieldSettings.get(name, true);
|
||||
setting.storeTermVector = true;
|
||||
}
|
||||
Collection<String> termVecOffsetNames = sourceIndexReader.getFieldNames(IndexReader.FieldOption.TERMVECTOR_WITH_OFFSET);
|
||||
for (String name : termVecOffsetNames) {
|
||||
FieldSetting setting = fieldSettings.get(name, true);
|
||||
setting.storeOffsetWithTermVector = true;
|
||||
}
|
||||
Collection<String> termVecPosNames = sourceIndexReader.getFieldNames(IndexReader.FieldOption.TERMVECTOR_WITH_POSITION);
|
||||
for (String name : termVecPosNames) {
|
||||
FieldSetting setting = fieldSettings.get(name, true);
|
||||
setting.storePositionWithTermVector = true;
|
||||
}
|
||||
Collection<String> termVecPosOffNames = sourceIndexReader.getFieldNames(IndexReader.FieldOption.TERMVECTOR_WITH_POSITION_OFFSET);
|
||||
for (String name : termVecPosOffNames) {
|
||||
FieldSetting setting = fieldSettings.get(name, true);
|
||||
setting.storeOffsetWithTermVector = true;
|
||||
setting.storePositionWithTermVector = true;
|
||||
}
|
||||
Collection<String> unindexedNames = sourceIndexReader.getFieldNames(IndexReader.FieldOption.UNINDEXED);
|
||||
for (String name : unindexedNames) {
|
||||
FieldSetting setting = fieldSettings.get(name, true);
|
||||
setting.indexed = false;
|
||||
}
|
||||
|
||||
|
||||
documentsByNumber = new InstantiatedDocument[sourceIndexReader.maxDoc()];
|
||||
|
||||
if (sourceIndexReader.hasDeletions()) {
|
||||
deletedDocuments = new BitVector(sourceIndexReader.maxDoc());
|
||||
}
|
||||
|
||||
// create documents
|
||||
final Bits liveDocs = MultiFields.getLiveDocs(sourceIndexReader);
|
||||
for (int i = 0; i < sourceIndexReader.maxDoc(); i++) {
|
||||
if (liveDocs != null && !liveDocs.get(i)) {
|
||||
deletedDocuments.set(i);
|
||||
} else {
|
||||
InstantiatedDocument document = new InstantiatedDocument();
|
||||
// copy stored fields from source reader
|
||||
Document sourceDocument = sourceIndexReader.document(i);
|
||||
for (IndexableField field : sourceDocument) {
|
||||
if (fields == null || fields.contains(field.name())) {
|
||||
document.getDocument().add(field);
|
||||
}
|
||||
}
|
||||
document.setDocumentNumber(i);
|
||||
documentsByNumber[i] = document;
|
||||
for (IndexableField field : document.getDocument()) {
|
||||
if (fields == null || fields.contains(field.name())) {
|
||||
if (field.fieldType().storeTermVectors()) {
|
||||
if (document.getVectorSpace() == null) {
|
||||
document.setVectorSpace(new HashMap<String, List<InstantiatedTermDocumentInformation>>());
|
||||
}
|
||||
document.getVectorSpace().put(field.name(), new ArrayList<InstantiatedTermDocumentInformation>());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// create norms
|
||||
for (String fieldName : allFieldNames) {
|
||||
if (fields == null || fields.contains(fieldName)) {
|
||||
byte norms[] = MultiNorms.norms(sourceIndexReader, fieldName);
|
||||
getNormsByFieldNameAndDocumentNumber().put(fieldName, norms);
|
||||
}
|
||||
}
|
||||
|
||||
// create terms
|
||||
for (String fieldName : allFieldNames) {
|
||||
if (fields == null || fields.contains(fieldName)) {
|
||||
getTermsByFieldAndText().put(fieldName, new HashMap<String, InstantiatedTerm>(5000));
|
||||
}
|
||||
}
|
||||
List<InstantiatedTerm> terms = new ArrayList<InstantiatedTerm>(5000 * getTermsByFieldAndText().size());
|
||||
Fields fieldsC = MultiFields.getFields(sourceIndexReader);
|
||||
if (fieldsC != null) {
|
||||
FieldsEnum fieldsEnum = fieldsC.iterator();
|
||||
String field;
|
||||
final CharsRef spare = new CharsRef();
|
||||
while((field = fieldsEnum.next()) != null) {
|
||||
if (fields == null || fields.contains(field)) {
|
||||
TermsEnum termsEnum = fieldsEnum.terms();
|
||||
BytesRef text;
|
||||
while((text = termsEnum.next()) != null) {
|
||||
String termText = text.utf8ToChars(spare).toString();
|
||||
InstantiatedTerm instantiatedTerm = new InstantiatedTerm(field, termText);
|
||||
final long totalTermFreq = termsEnum.totalTermFreq();
|
||||
if (totalTermFreq != -1) {
|
||||
instantiatedTerm.addPositionsCount(totalTermFreq);
|
||||
}
|
||||
getTermsByFieldAndText().get(field).put(termText, instantiatedTerm);
|
||||
instantiatedTerm.setTermIndex(terms.size());
|
||||
terms.add(instantiatedTerm);
|
||||
instantiatedTerm.setAssociatedDocuments(new InstantiatedTermDocumentInformation[termsEnum.docFreq()]);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
orderedTerms = terms.toArray(new InstantiatedTerm[terms.size()]);
|
||||
|
||||
// create term-document informations
|
||||
for (InstantiatedTerm term : orderedTerms) {
|
||||
DocsAndPositionsEnum termPositions = MultiFields.getTermPositionsEnum(sourceIndexReader,
|
||||
MultiFields.getLiveDocs(sourceIndexReader),
|
||||
term.getTerm().field(),
|
||||
new BytesRef(term.getTerm().text()));
|
||||
int position = 0;
|
||||
while (termPositions.nextDoc() != DocsEnum.NO_MORE_DOCS) {
|
||||
InstantiatedDocument document = documentsByNumber[termPositions.docID()];
|
||||
|
||||
byte[][] payloads = new byte[termPositions.freq()][];
|
||||
int[] positions = new int[termPositions.freq()];
|
||||
for (int i = 0; i < termPositions.freq(); i++) {
|
||||
positions[i] = termPositions.nextPosition();
|
||||
|
||||
if (termPositions.hasPayload()) {
|
||||
BytesRef br = termPositions.getPayload();
|
||||
payloads[i] = new byte[br.length];
|
||||
System.arraycopy(br.bytes, br.offset, payloads[i], 0, br.length);
|
||||
}
|
||||
}
|
||||
|
||||
InstantiatedTermDocumentInformation termDocumentInformation = new InstantiatedTermDocumentInformation(term, document, positions, payloads);
|
||||
term.getAssociatedDocuments()[position++] = termDocumentInformation;
|
||||
|
||||
if (document.getVectorSpace() != null
|
||||
&& document.getVectorSpace().containsKey(term.field())) {
|
||||
document.getVectorSpace().get(term.field()).add(termDocumentInformation);
|
||||
}
|
||||
|
||||
// termDocumentInformation.setIndexFromTerm(indexFromTerm++);
|
||||
}
|
||||
}
|
||||
|
||||
// load offsets to term-document informations
|
||||
for (InstantiatedDocument document : getDocumentsByNumber()) {
|
||||
if (document == null) {
|
||||
continue; // deleted
|
||||
}
|
||||
for (IndexableField field : document.getDocument()) {
|
||||
if (field.fieldType().storeTermVectors() && field.fieldType().storeTermVectorOffsets()) {
|
||||
TermPositionVector termPositionVector = (TermPositionVector) sourceIndexReader.getTermFreqVector(document.getDocumentNumber(), field.name());
|
||||
if (termPositionVector != null) {
|
||||
for (int i = 0; i < termPositionVector.getTerms().length; i++) {
|
||||
String token = termPositionVector.getTerms()[i].utf8ToString();
|
||||
InstantiatedTerm term = findTerm(field.name(), token);
|
||||
InstantiatedTermDocumentInformation termDocumentInformation = term.getAssociatedDocument(document.getDocumentNumber());
|
||||
termDocumentInformation.setTermOffsets(termPositionVector.getOffsets(i));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public InstantiatedIndexWriter indexWriterFactory(Analyzer analyzer, boolean create) throws IOException {
|
||||
return new InstantiatedIndexWriter(this, analyzer, create);
|
||||
}
|
||||
|
||||
public InstantiatedIndexReader indexReaderFactory() throws IOException {
|
||||
return new InstantiatedIndexReader(this);
|
||||
}
|
||||
|
||||
public void close() throws IOException {
|
||||
// todo: decouple everything
|
||||
}
|
||||
|
||||
InstantiatedTerm findTerm(Term term) {
|
||||
return findTerm(term.field(), term.text());
|
||||
}
|
||||
|
||||
InstantiatedTerm findTerm(String field, String text) {
|
||||
Map<String, InstantiatedTerm> termsByField = termsByFieldAndText.get(field);
|
||||
if (termsByField == null) {
|
||||
return null;
|
||||
} else {
|
||||
return termsByField.get(text);
|
||||
}
|
||||
}
|
||||
|
||||
public Map<String, Map<String, InstantiatedTerm>> getTermsByFieldAndText() {
|
||||
return termsByFieldAndText;
|
||||
}
|
||||
|
||||
|
||||
public InstantiatedTerm[] getOrderedTerms() {
|
||||
return orderedTerms;
|
||||
}
|
||||
|
||||
public InstantiatedDocument[] getDocumentsByNumber() {
|
||||
return documentsByNumber;
|
||||
}
|
||||
|
||||
public Map<String, byte[]> getNormsByFieldNameAndDocumentNumber() {
|
||||
return normsByFieldNameAndDocumentNumber;
|
||||
}
|
||||
|
||||
void setNormsByFieldNameAndDocumentNumber(Map<String, byte[]> normsByFieldNameAndDocumentNumber) {
|
||||
this.normsByFieldNameAndDocumentNumber = normsByFieldNameAndDocumentNumber;
|
||||
}
|
||||
|
||||
public BitVector getDeletedDocuments() {
|
||||
return deletedDocuments;
|
||||
}
|
||||
|
||||
void setDeletedDocuments(BitVector deletedDocuments) {
|
||||
this.deletedDocuments = deletedDocuments;
|
||||
}
|
||||
|
||||
void setOrderedTerms(InstantiatedTerm[] orderedTerms) {
|
||||
this.orderedTerms = orderedTerms;
|
||||
}
|
||||
|
||||
void setDocumentsByNumber(InstantiatedDocument[] documentsByNumber) {
|
||||
this.documentsByNumber = documentsByNumber;
|
||||
}
|
||||
|
||||
|
||||
public long getVersion() {
|
||||
return version;
|
||||
}
|
||||
|
||||
void setVersion(long version) {
|
||||
this.version = version;
|
||||
}
|
||||
|
||||
|
||||
FieldSettings getFieldSettings() {
|
||||
return fieldSettings;
|
||||
}
|
||||
}
|
|
@ -1,472 +0,0 @@
|
|||
package org.apache.lucene.store.instantiated;
|
||||
|
||||
/**
|
||||
* Copyright 2006 The Apache Software Foundation
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.Comparator;
|
||||
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.index.*;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BitVector;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
||||
/**
|
||||
* An InstantiatedIndexReader is not a snapshot in time, it is completely in
|
||||
* sync with the latest commit to the store!
|
||||
* <p>
|
||||
* Consider using InstantiatedIndex as if it was immutable.
|
||||
*/
|
||||
public class InstantiatedIndexReader extends IndexReader {
|
||||
|
||||
private final InstantiatedIndex index;
|
||||
private ReaderContext context = new AtomicReaderContext(this);
|
||||
|
||||
|
||||
public InstantiatedIndexReader(InstantiatedIndex index) {
|
||||
super();
|
||||
this.index = index;
|
||||
readerFinishedListeners = Collections.synchronizedSet(new HashSet<ReaderFinishedListener>());
|
||||
}
|
||||
|
||||
/**
|
||||
* An InstantiatedIndexReader is not a snapshot in time, it is completely in
|
||||
* sync with the latest commit to the store!
|
||||
*
|
||||
* @return output from {@link InstantiatedIndex#getVersion()} in associated instantiated index.
|
||||
*/
|
||||
@Override
|
||||
public long getVersion() {
|
||||
return index.getVersion();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Directory directory() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
/**
|
||||
* An InstantiatedIndexReader is always current!
|
||||
*
|
||||
* Check whether this IndexReader is still using the current (i.e., most
|
||||
* recently committed) version of the index. If a writer has committed any
|
||||
* changes to the index since this reader was opened, this will return
|
||||
* <code>false</code>, in which case you must open a new IndexReader in
|
||||
* order to see the changes. See the description of the <a
|
||||
* href="IndexWriter.html#autoCommit"><code>autoCommit</code></a> flag
|
||||
* which controls when the {@link IndexWriter} actually commits changes to the
|
||||
* index.
|
||||
*
|
||||
* @return always true
|
||||
* @throws CorruptIndexException if the index is corrupt
|
||||
* @throws IOException if there is a low-level IO error
|
||||
* @throws UnsupportedOperationException unless overridden in subclass
|
||||
*/
|
||||
@Override
|
||||
public boolean isCurrent() throws IOException {
|
||||
return true;
|
||||
}
|
||||
|
||||
public InstantiatedIndex getIndex() {
|
||||
return index;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bits getLiveDocs() {
|
||||
return new Bits() {
|
||||
public boolean get(int n) {
|
||||
return !(index.getDeletedDocuments() != null && index.getDeletedDocuments().get(n))
|
||||
&& !(uncommittedDeletedDocuments != null && uncommittedDeletedDocuments.get(n));
|
||||
}
|
||||
|
||||
public int length() {
|
||||
return maxDoc();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
private BitVector uncommittedDeletedDocuments;
|
||||
|
||||
private Map<String,List<NormUpdate>> uncommittedNormsByFieldNameAndDocumentNumber = null;
|
||||
|
||||
private static class NormUpdate {
|
||||
private int doc;
|
||||
private byte value;
|
||||
|
||||
public NormUpdate(int doc, byte value) {
|
||||
this.doc = doc;
|
||||
this.value = value;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int numDocs() {
|
||||
// todo i suppose this value could be cached, but array#length and bitvector#count is fast.
|
||||
int numDocs = getIndex().getDocumentsByNumber().length;
|
||||
if (uncommittedDeletedDocuments != null) {
|
||||
numDocs -= uncommittedDeletedDocuments.count();
|
||||
}
|
||||
if (index.getDeletedDocuments() != null) {
|
||||
numDocs -= index.getDeletedDocuments().count();
|
||||
}
|
||||
return numDocs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int maxDoc() {
|
||||
return getIndex().getDocumentsByNumber().length;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasDeletions() {
|
||||
return index.getDeletedDocuments() != null || uncommittedDeletedDocuments != null;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doDelete(int docNum) throws IOException {
|
||||
|
||||
// dont delete if already deleted
|
||||
if ((index.getDeletedDocuments() != null && index.getDeletedDocuments().get(docNum))
|
||||
|| (uncommittedDeletedDocuments != null && uncommittedDeletedDocuments.get(docNum))) {
|
||||
return;
|
||||
}
|
||||
|
||||
if (uncommittedDeletedDocuments == null) {
|
||||
uncommittedDeletedDocuments = new BitVector(maxDoc());
|
||||
}
|
||||
|
||||
uncommittedDeletedDocuments.set(docNum);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doUndeleteAll() throws IOException {
|
||||
// todo: read/write lock
|
||||
uncommittedDeletedDocuments = null;
|
||||
// todo: read/write unlock
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doCommit(Map<String,String> commitUserData) throws IOException {
|
||||
// todo: read/write lock
|
||||
|
||||
// 1. update norms
|
||||
if (uncommittedNormsByFieldNameAndDocumentNumber != null) {
|
||||
for (Map.Entry<String,List<NormUpdate>> e : uncommittedNormsByFieldNameAndDocumentNumber.entrySet()) {
|
||||
byte[] norms = getIndex().getNormsByFieldNameAndDocumentNumber().get(e.getKey());
|
||||
for (NormUpdate normUpdate : e.getValue()) {
|
||||
norms[normUpdate.doc] = normUpdate.value;
|
||||
}
|
||||
}
|
||||
uncommittedNormsByFieldNameAndDocumentNumber = null;
|
||||
}
|
||||
|
||||
// 2. remove deleted documents
|
||||
if (uncommittedDeletedDocuments != null) {
|
||||
if (index.getDeletedDocuments() == null) {
|
||||
index.setDeletedDocuments(uncommittedDeletedDocuments);
|
||||
} else {
|
||||
for (int d = 0; d< uncommittedDeletedDocuments.size(); d++) {
|
||||
if (uncommittedDeletedDocuments.get(d)) {
|
||||
index.getDeletedDocuments().set(d);
|
||||
}
|
||||
}
|
||||
}
|
||||
uncommittedDeletedDocuments = null;
|
||||
}
|
||||
|
||||
// todo unlock read/writelock
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doClose() throws IOException {
|
||||
// ignored
|
||||
// todo perhaps release all associated instances?
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<String> getFieldNames(FieldOption fieldOption) {
|
||||
Set<String> fieldSet = new HashSet<String>();
|
||||
for (FieldSetting fi : index.getFieldSettings().values()) {
|
||||
if (fieldOption == IndexReader.FieldOption.ALL) {
|
||||
fieldSet.add(fi.fieldName);
|
||||
} else if (!fi.indexed && fieldOption == IndexReader.FieldOption.UNINDEXED) {
|
||||
fieldSet.add(fi.fieldName);
|
||||
} else if (fi.storePayloads && fieldOption == IndexReader.FieldOption.STORES_PAYLOADS) {
|
||||
fieldSet.add(fi.fieldName);
|
||||
} else if (fi.indexed && fieldOption == IndexReader.FieldOption.INDEXED) {
|
||||
fieldSet.add(fi.fieldName);
|
||||
} else if (fi.indexed && fi.storeTermVector == false && fieldOption == IndexReader.FieldOption.INDEXED_NO_TERMVECTOR) {
|
||||
fieldSet.add(fi.fieldName);
|
||||
} else if (fi.storeTermVector == true && fi.storePositionWithTermVector == false && fi.storeOffsetWithTermVector == false
|
||||
&& fieldOption == IndexReader.FieldOption.TERMVECTOR) {
|
||||
fieldSet.add(fi.fieldName);
|
||||
} else if (fi.indexed && fi.storeTermVector && fieldOption == IndexReader.FieldOption.INDEXED_WITH_TERMVECTOR) {
|
||||
fieldSet.add(fi.fieldName);
|
||||
} else if (fi.storePositionWithTermVector && fi.storeOffsetWithTermVector == false
|
||||
&& fieldOption == IndexReader.FieldOption.TERMVECTOR_WITH_POSITION) {
|
||||
fieldSet.add(fi.fieldName);
|
||||
} else if (fi.storeOffsetWithTermVector && fi.storePositionWithTermVector == false
|
||||
&& fieldOption == IndexReader.FieldOption.TERMVECTOR_WITH_OFFSET) {
|
||||
fieldSet.add(fi.fieldName);
|
||||
} else if ((fi.storeOffsetWithTermVector && fi.storePositionWithTermVector)
|
||||
&& fieldOption == IndexReader.FieldOption.TERMVECTOR_WITH_POSITION_OFFSET) {
|
||||
fieldSet.add(fi.fieldName);
|
||||
}
|
||||
}
|
||||
return fieldSet;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the stored fields of the <code>n</code><sup>th</sup>
|
||||
* <code>Document</code> in this index.
|
||||
* <p>
|
||||
* <b>Warning!</b>
|
||||
* The resulting document is the actual stored document instance
|
||||
* and not a deserialized clone as retuned by an IndexReader
|
||||
* over a {@link org.apache.lucene.store.Directory}.
|
||||
* I.e., if you need to touch the document, clone it first!
|
||||
* <p>
|
||||
* This can also be seen as a feature for live changes of stored values,
|
||||
* but be careful! Adding a field with an name unknown to the index
|
||||
* or to a field with previously no stored values will make
|
||||
* {@link org.apache.lucene.store.instantiated.InstantiatedIndexReader#getFieldNames(org.apache.lucene.index.IndexReader.FieldOption)}
|
||||
* out of sync, causing problems for instance when merging the
|
||||
* instantiated index to another index.
|
||||
*
|
||||
* @throws CorruptIndexException if the index is corrupt
|
||||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
|
||||
@Override
|
||||
public Document document(int n) throws IOException {
|
||||
return getIndex().getDocumentsByNumber()[n].getDocument();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void document(int docID, StoredFieldVisitor visitor) throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
/**
|
||||
* never ever touch these values. it is the true values, unless norms have
|
||||
* been touched.
|
||||
*/
|
||||
@Override
|
||||
public byte[] norms(String field) throws IOException {
|
||||
byte[] norms = getIndex().getNormsByFieldNameAndDocumentNumber().get(field);
|
||||
if (norms == null) {
|
||||
return new byte[0]; // todo a static final zero length attribute?
|
||||
}
|
||||
if (uncommittedNormsByFieldNameAndDocumentNumber != null) {
|
||||
norms = norms.clone();
|
||||
List<NormUpdate> updated = uncommittedNormsByFieldNameAndDocumentNumber.get(field);
|
||||
if (updated != null) {
|
||||
for (NormUpdate normUpdate : updated) {
|
||||
norms[normUpdate.doc] = normUpdate.value;
|
||||
}
|
||||
}
|
||||
}
|
||||
return norms;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doSetNorm(int doc, String field, byte value) throws IOException {
|
||||
if (uncommittedNormsByFieldNameAndDocumentNumber == null) {
|
||||
uncommittedNormsByFieldNameAndDocumentNumber = new HashMap<String,List<NormUpdate>>(getIndex().getNormsByFieldNameAndDocumentNumber().size());
|
||||
}
|
||||
List<NormUpdate> list = uncommittedNormsByFieldNameAndDocumentNumber.get(field);
|
||||
if (list == null) {
|
||||
list = new LinkedList<NormUpdate>();
|
||||
uncommittedNormsByFieldNameAndDocumentNumber.put(field, list);
|
||||
}
|
||||
list.add(new NormUpdate(doc, value));
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docFreq(Term t) throws IOException {
|
||||
InstantiatedTerm term = getIndex().findTerm(t);
|
||||
if (term == null) {
|
||||
return 0;
|
||||
} else {
|
||||
return term.getAssociatedDocuments().length;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Fields fields() {
|
||||
if (getIndex().getOrderedTerms().length == 0) {
|
||||
return null;
|
||||
}
|
||||
|
||||
return new Fields() {
|
||||
@Override
|
||||
public FieldsEnum iterator() {
|
||||
final InstantiatedTerm[] orderedTerms = getIndex().getOrderedTerms();
|
||||
|
||||
return new FieldsEnum() {
|
||||
int upto = -1;
|
||||
String currentField;
|
||||
|
||||
@Override
|
||||
public String next() {
|
||||
do {
|
||||
upto++;
|
||||
if (upto >= orderedTerms.length) {
|
||||
return null;
|
||||
}
|
||||
} while(orderedTerms[upto].field().equals(currentField));
|
||||
|
||||
currentField = orderedTerms[upto].field();
|
||||
return currentField;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermsEnum terms() {
|
||||
return new InstantiatedTermsEnum(orderedTerms, upto, currentField);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Terms terms(final String field) {
|
||||
final InstantiatedTerm[] orderedTerms = getIndex().getOrderedTerms();
|
||||
int i = Arrays.binarySearch(orderedTerms, new Term(field), InstantiatedTerm.termComparator);
|
||||
if (i < 0) {
|
||||
i = -i - 1;
|
||||
}
|
||||
if (i >= orderedTerms.length || !orderedTerms[i].field().equals(field)) {
|
||||
// field does not exist
|
||||
return null;
|
||||
}
|
||||
final int startLoc = i;
|
||||
|
||||
// TODO: heavy to do this here; would be better to
|
||||
// do it up front & cache
|
||||
long sum = 0;
|
||||
int upto = i;
|
||||
while(upto < orderedTerms.length && orderedTerms[i].field().equals(field)) {
|
||||
sum += orderedTerms[i].getTotalTermFreq();
|
||||
upto++;
|
||||
}
|
||||
final long sumTotalTermFreq = sum;
|
||||
|
||||
return new Terms() {
|
||||
@Override
|
||||
public TermsEnum iterator() {
|
||||
return new InstantiatedTermsEnum(orderedTerms, startLoc, field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getSumTotalTermFreq() {
|
||||
return sumTotalTermFreq;
|
||||
}
|
||||
|
||||
// TODO: support this?
|
||||
@Override
|
||||
public long getSumDocFreq() {
|
||||
return -1;
|
||||
}
|
||||
|
||||
// TODO: support this?
|
||||
@Override
|
||||
public int getDocCount() throws IOException {
|
||||
return -1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getUniqueTermCount() throws IOException {
|
||||
return -1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Comparator<BytesRef> getComparator() {
|
||||
return BytesRef.getUTF8SortedAsUnicodeComparator();
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReaderContext getTopReaderContext() {
|
||||
return context;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermFreqVector[] getTermFreqVectors(int docNumber) throws IOException {
|
||||
InstantiatedDocument doc = getIndex().getDocumentsByNumber()[docNumber];
|
||||
if (doc.getVectorSpace() == null) {
|
||||
return null;
|
||||
}
|
||||
TermFreqVector[] ret = new TermFreqVector[doc.getVectorSpace().size()];
|
||||
Iterator<String> it = doc.getVectorSpace().keySet().iterator();
|
||||
for (int i = 0; i < ret.length; i++) {
|
||||
ret[i] = new InstantiatedTermPositionVector(getIndex().getDocumentsByNumber()[docNumber], it.next());
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermFreqVector getTermFreqVector(int docNumber, String field) throws IOException {
|
||||
InstantiatedDocument doc = getIndex().getDocumentsByNumber()[docNumber];
|
||||
if (doc.getVectorSpace() == null || doc.getVectorSpace().get(field) == null) {
|
||||
return null;
|
||||
} else {
|
||||
return new InstantiatedTermPositionVector(doc, field);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getTermFreqVector(int docNumber, String field, TermVectorMapper mapper) throws IOException {
|
||||
InstantiatedDocument doc = getIndex().getDocumentsByNumber()[docNumber];
|
||||
if (doc.getVectorSpace() != null && doc.getVectorSpace().get(field) == null) {
|
||||
List<InstantiatedTermDocumentInformation> tv = doc.getVectorSpace().get(field);
|
||||
mapper.setExpectations(field, tv.size(), true, true);
|
||||
for (InstantiatedTermDocumentInformation tdi : tv) {
|
||||
mapper.map(tdi.getTerm().getTerm().bytes(), tdi.getTermPositions().length, tdi.getTermOffsets(), tdi.getTermPositions());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getTermFreqVector(int docNumber, TermVectorMapper mapper) throws IOException {
|
||||
InstantiatedDocument doc = getIndex().getDocumentsByNumber()[docNumber];
|
||||
for (Map.Entry<String, List<InstantiatedTermDocumentInformation>> e : doc.getVectorSpace().entrySet()) {
|
||||
mapper.setExpectations(e.getKey(), e.getValue().size(), true, true);
|
||||
for (InstantiatedTermDocumentInformation tdi : e.getValue()) {
|
||||
mapper.map(tdi.getTerm().getTerm().bytes(), tdi.getTermPositions().length, tdi.getTermOffsets(), tdi.getTermPositions());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocValues perDocValues() throws IOException {
|
||||
return null;
|
||||
}
|
||||
}
|
|
@ -1,693 +0,0 @@
|
|||
package org.apache.lucene.store.instantiated;
|
||||
|
||||
/**
|
||||
* Copyright 2006 The Apache Software Foundation
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
import java.io.StringReader;
|
||||
import java.io.Closeable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.Token;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.index.FieldInvertState;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexableField;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.TermVectorOffsetInfo;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.similarities.SimilarityProvider;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.CollectionUtil;
|
||||
import org.apache.lucene.util.AttributeImpl;
|
||||
import org.apache.lucene.util.BitVector;
|
||||
|
||||
/**
|
||||
* This class, similar to {@link org.apache.lucene.index.IndexWriter}, has no locking mechanism.
|
||||
* <p>
|
||||
* {@link org.apache.lucene.store.instantiated.InstantiatedIndexReader} is navigating
|
||||
* the same instances in memory as this writer is updating so searchers active while
|
||||
* you are committing are bound to throw exceptions.
|
||||
* <p>
|
||||
* Consider using InstantiatedIndex as if it was immutable.
|
||||
*
|
||||
* @see org.apache.lucene.index.IndexWriter
|
||||
*/
|
||||
public class InstantiatedIndexWriter implements Closeable {
|
||||
|
||||
private PrintStream infoStream = null;
|
||||
|
||||
private final InstantiatedIndex index;
|
||||
private final Analyzer analyzer;
|
||||
|
||||
private SimilarityProvider similarityProvider = IndexSearcher.getDefaultSimilarityProvider(); // how to normalize;
|
||||
|
||||
private transient Set<String> fieldNameBuffer;
|
||||
/**
|
||||
* linked to ensure chronological order
|
||||
*/
|
||||
private Map<InstantiatedDocument, Map<FieldSetting, Map<String /*text*/, TermDocumentInformationFactory>>> termDocumentInformationFactoryByDocument = new LinkedHashMap<InstantiatedDocument, Map<FieldSetting, Map<String /*text*/, TermDocumentInformationFactory>>>(2000);
|
||||
|
||||
private Set<InstantiatedDocument> unflushedDocuments = new HashSet<InstantiatedDocument>();
|
||||
|
||||
public InstantiatedIndexWriter(InstantiatedIndex index) throws IOException {
|
||||
this(index, null);
|
||||
}
|
||||
|
||||
public InstantiatedIndexWriter(InstantiatedIndex index, Analyzer analyzer) throws IOException {
|
||||
this(index, analyzer, false);
|
||||
}
|
||||
|
||||
public InstantiatedIndexWriter(InstantiatedIndex index, Analyzer analyzer, boolean create) throws IOException {
|
||||
this.index = index;
|
||||
this.analyzer = analyzer;
|
||||
fieldNameBuffer = new HashSet<String>();
|
||||
if (create) {
|
||||
this.index.initialize();
|
||||
}
|
||||
}
|
||||
|
||||
private int mergeFactor = 2500;
|
||||
|
||||
/**
|
||||
* The sweetspot for this implementation is somewhere around 2500 at 2K text large documents.
|
||||
* <p/>
|
||||
* Benchmark output:
|
||||
* <pre>
|
||||
* ------------> Report sum by Prefix (MAddDocs) and Round (8 about 8 out of 160153)
|
||||
* Operation round mrg buf cmpnd runCnt recsPerRun rec/s elapsedSec avgUsedMem avgTotalMem
|
||||
* MAddDocs_20000 0 10 10 true 1 20000 81,4 245,68 200 325 152 268 156 928
|
||||
* MAddDocs_20000 - 1 1000 10 true - - 1 - - 20000 - - 494,1 - - 40,47 - 247 119 072 - 347 025 408
|
||||
* MAddDocs_20000 2 10 100 true 1 20000 104,8 190,81 233 895 552 363 720 704
|
||||
* MAddDocs_20000 - 3 2000 100 true - - 1 - - 20000 - - 527,2 - - 37,94 - 266 136 448 - 378 273 792
|
||||
* MAddDocs_20000 4 10 10 false 1 20000 103,2 193,75 222 089 792 378 273 792
|
||||
* MAddDocs_20000 - 5 3000 10 false - - 1 - - 20000 - - 545,2 - - 36,69 - 237 917 152 - 378 273 792
|
||||
* MAddDocs_20000 6 10 100 false 1 20000 102,7 194,67 237 018 976 378 273 792
|
||||
* MAddDocs_20000 - 7 4000 100 false - - 1 - - 20000 - - 535,8 - - 37,33 - 309 680 640 - 501 968 896
|
||||
* </pre>
|
||||
*
|
||||
* @see org.apache.lucene.index.LogMergePolicy#setMergeFactor(int)
|
||||
*/
|
||||
public void setMergeFactor(int mergeFactor) {
|
||||
this.mergeFactor = mergeFactor;
|
||||
}
|
||||
|
||||
/**
|
||||
* @see org.apache.lucene.index.LogMergePolicy#getMergeFactor()
|
||||
*/
|
||||
public int getMergeFactor() {
|
||||
return mergeFactor;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* If non-null, information about merges and a message when
|
||||
* maxFieldLength is reached could be printed to this -- currently
|
||||
* not yet implemented.
|
||||
*/
|
||||
public void setInfoStream(PrintStream infoStream) {
|
||||
this.infoStream = infoStream;
|
||||
}
|
||||
|
||||
|
||||
public void abort() throws IOException {
|
||||
// what not
|
||||
}
|
||||
|
||||
|
||||
public void addIndexes(IndexReader[] readers) {
|
||||
throw new RuntimeException("Not implemented");
|
||||
}
|
||||
|
||||
|
||||
public PrintStream getInfoStream() {
|
||||
return infoStream;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Flushes all changes to an index and closes all associated files.
|
||||
*/
|
||||
public void close() throws IOException {
|
||||
commit();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the number of documents currently in this index.
|
||||
*/
|
||||
public int docCount() {
|
||||
// todo: not certain. see http://www.nabble.com/IndexWriter.docCount-tf3128882.html#a8669483
|
||||
return index.getDocumentsByNumber().length /* - index.getDeletedDocuments().size() */ + unflushedDocuments.size();
|
||||
}
|
||||
|
||||
/**
|
||||
* Locks the index and commits the buffered documents.
|
||||
*/
|
||||
public void commit() throws IOException {
|
||||
|
||||
// todo write lock, unless held by caller
|
||||
|
||||
boolean orderedTermsDirty = false;
|
||||
Set<InstantiatedTerm> dirtyTerms = new HashSet<InstantiatedTerm>(1000);
|
||||
|
||||
Map<String, FieldSetting> fieldSettingsByFieldName = new HashMap<String, FieldSetting>();
|
||||
for (String fieldName : fieldNameBuffer) {
|
||||
fieldSettingsByFieldName.put(fieldName, new FieldSetting(fieldName));
|
||||
}
|
||||
|
||||
InstantiatedDocument[] documentsByNumber = new InstantiatedDocument[index.getDocumentsByNumber().length + termDocumentInformationFactoryByDocument.size()];
|
||||
System.arraycopy(index.getDocumentsByNumber(), 0, documentsByNumber, 0, index.getDocumentsByNumber().length);
|
||||
int documentNumber = index.getDocumentsByNumber().length;
|
||||
|
||||
List<InstantiatedTerm> orderedTerms = new ArrayList<InstantiatedTerm>(index.getOrderedTerms().length + 5000);
|
||||
for (InstantiatedTerm instantiatedTerm : index.getOrderedTerms()) {
|
||||
orderedTerms.add(instantiatedTerm);
|
||||
}
|
||||
|
||||
// update norm array with fake values for new documents
|
||||
Map<String, byte[]> normsByFieldNameAndDocumentNumber = new HashMap<String, byte[]>(index.getTermsByFieldAndText().size());
|
||||
Set<String> fieldNames = new HashSet<String>(20);
|
||||
fieldNames.addAll(index.getNormsByFieldNameAndDocumentNumber().keySet());
|
||||
fieldNames.addAll(fieldNameBuffer);
|
||||
for (String field : index.getTermsByFieldAndText().keySet()) {
|
||||
byte[] norms = new byte[index.getDocumentsByNumber().length + termDocumentInformationFactoryByDocument.size()];
|
||||
byte[] oldNorms = index.getNormsByFieldNameAndDocumentNumber().get(field);
|
||||
if (oldNorms != null) {
|
||||
System.arraycopy(oldNorms, 0, norms, 0, oldNorms.length);
|
||||
Arrays.fill(norms, oldNorms.length, norms.length, (byte) 0);
|
||||
} else {
|
||||
Arrays.fill(norms, 0, norms.length, (byte) 0);
|
||||
}
|
||||
normsByFieldNameAndDocumentNumber.put(field, norms);
|
||||
fieldNames.remove(field);
|
||||
}
|
||||
for (String field : fieldNames) {
|
||||
//System.out.println(field);
|
||||
byte[] norms = new byte[index.getDocumentsByNumber().length + termDocumentInformationFactoryByDocument.size()];
|
||||
Arrays.fill(norms, 0, norms.length, (byte) 0);
|
||||
normsByFieldNameAndDocumentNumber.put(field, norms);
|
||||
}
|
||||
fieldNames.clear();
|
||||
index.setNormsByFieldNameAndDocumentNumber(normsByFieldNameAndDocumentNumber);
|
||||
|
||||
for (Map.Entry<InstantiatedDocument, Map<FieldSetting, Map<String /*text*/, TermDocumentInformationFactory>>> eDocumentTermDocInfoByTermTextAndField : termDocumentInformationFactoryByDocument.entrySet()) {
|
||||
|
||||
InstantiatedDocument document = eDocumentTermDocInfoByTermTextAndField.getKey();
|
||||
|
||||
// assign document number
|
||||
document.setDocumentNumber(documentNumber++);
|
||||
documentsByNumber[document.getDocumentNumber()] = document;
|
||||
|
||||
// set norms, prepare document and create optimized size collections.
|
||||
|
||||
int numFieldsWithTermVectorsInDocument = 0;
|
||||
int termsInDocument = 0;
|
||||
for (Map.Entry<FieldSetting, Map<String /*text*/, TermDocumentInformationFactory>> eFieldTermDocInfoFactoriesByTermText : eDocumentTermDocInfoByTermTextAndField.getValue().entrySet()) {
|
||||
if (eFieldTermDocInfoFactoriesByTermText.getKey().storeTermVector) {
|
||||
numFieldsWithTermVectorsInDocument += eFieldTermDocInfoFactoriesByTermText.getValue().size();
|
||||
}
|
||||
termsInDocument += eFieldTermDocInfoFactoriesByTermText.getValue().size();
|
||||
|
||||
if (eFieldTermDocInfoFactoriesByTermText.getKey().indexed && !eFieldTermDocInfoFactoriesByTermText.getKey().omitNorms) {
|
||||
final String fieldName = eFieldTermDocInfoFactoriesByTermText.getKey().fieldName;
|
||||
final FieldInvertState invertState = new FieldInvertState();
|
||||
invertState.setBoost(eFieldTermDocInfoFactoriesByTermText.getKey().boost);
|
||||
invertState.setLength(eFieldTermDocInfoFactoriesByTermText.getKey().fieldLength);
|
||||
normsByFieldNameAndDocumentNumber.get(fieldName)[document.getDocumentNumber()] = similarityProvider.get(fieldName).computeNorm(invertState);
|
||||
} else {
|
||||
System.currentTimeMillis();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/** used for term vectors only, i think.. */
|
||||
Map<InstantiatedTerm, InstantiatedTermDocumentInformation> informationByTermOfCurrentDocument = new HashMap<InstantiatedTerm, InstantiatedTermDocumentInformation>(termsInDocument);
|
||||
|
||||
|
||||
Map<String, FieldSetting> documentFieldSettingsByFieldName = new HashMap<String, FieldSetting>(eDocumentTermDocInfoByTermTextAndField.getValue().size());
|
||||
|
||||
// terms...
|
||||
for (Map.Entry<FieldSetting, Map<String /*text*/, TermDocumentInformationFactory>> eFieldSetting_TermDocInfoFactoriesByTermText : eDocumentTermDocInfoByTermTextAndField.getValue().entrySet()) {
|
||||
documentFieldSettingsByFieldName.put(eFieldSetting_TermDocInfoFactoriesByTermText.getKey().fieldName, eFieldSetting_TermDocInfoFactoriesByTermText.getKey());
|
||||
|
||||
// find or create term
|
||||
for (Map.Entry<String /*text*/, TermDocumentInformationFactory> eTermText_TermDocInfoFactory : eFieldSetting_TermDocInfoFactoriesByTermText.getValue().entrySet()) {
|
||||
|
||||
// get term..
|
||||
InstantiatedTerm term;
|
||||
Map<String, InstantiatedTerm> termsByText = index.getTermsByFieldAndText().get(eFieldSetting_TermDocInfoFactoriesByTermText.getKey().fieldName);
|
||||
if (termsByText == null) {
|
||||
termsByText = new HashMap<String, InstantiatedTerm>(1000);
|
||||
index.getTermsByFieldAndText().put(eFieldSetting_TermDocInfoFactoriesByTermText.getKey().fieldName, termsByText);
|
||||
term = new InstantiatedTerm(eFieldSetting_TermDocInfoFactoriesByTermText.getKey().fieldName, eTermText_TermDocInfoFactory.getKey());
|
||||
termsByText.put(eTermText_TermDocInfoFactory.getKey(), term);
|
||||
int pos = Collections.binarySearch(orderedTerms, term, InstantiatedTerm.comparator);
|
||||
pos = -1 - pos;
|
||||
orderedTerms.add(pos, term);
|
||||
orderedTermsDirty = true;
|
||||
} else {
|
||||
term = termsByText.get(eTermText_TermDocInfoFactory.getKey());
|
||||
if (term == null) {
|
||||
term = new InstantiatedTerm(eFieldSetting_TermDocInfoFactoriesByTermText.getKey().fieldName, eTermText_TermDocInfoFactory.getKey());
|
||||
termsByText.put(eTermText_TermDocInfoFactory.getKey(), term);
|
||||
int pos = Collections.binarySearch(orderedTerms, term, InstantiatedTerm.comparator);
|
||||
pos = -1 - pos;
|
||||
orderedTerms.add(pos, term);
|
||||
orderedTermsDirty = true;
|
||||
}
|
||||
}
|
||||
|
||||
// create association term document information
|
||||
//
|
||||
// [Term]-- {0..*} | {0..* ordered} --(field)[Document]
|
||||
//
|
||||
// |
|
||||
// [TermDocumentInformation]
|
||||
|
||||
int[] positions = new int[eTermText_TermDocInfoFactory.getValue().termPositions.size()];
|
||||
for (int i = 0; i < positions.length; i++) {
|
||||
positions[i] = eTermText_TermDocInfoFactory.getValue().termPositions.get(i);
|
||||
}
|
||||
|
||||
byte[][] payloads = new byte[eTermText_TermDocInfoFactory.getValue().payloads.size()][];
|
||||
for (int i = 0; i < payloads.length; i++) {
|
||||
payloads[i] = eTermText_TermDocInfoFactory.getValue().payloads.get(i);
|
||||
}
|
||||
|
||||
// couple
|
||||
|
||||
InstantiatedTermDocumentInformation info = new InstantiatedTermDocumentInformation(term, document, /*eTermText_TermDocInfoFactory.getValue().termFrequency,*/ positions, payloads);
|
||||
|
||||
// todo optimize, this should be cached and updated to array in batches rather than appending the array once for every position!
|
||||
InstantiatedTermDocumentInformation[] associatedDocuments;
|
||||
if (term.getAssociatedDocuments() != null) {
|
||||
associatedDocuments = new InstantiatedTermDocumentInformation[term.getAssociatedDocuments().length + 1];
|
||||
System.arraycopy(term.getAssociatedDocuments(), 0, associatedDocuments, 0, term.getAssociatedDocuments().length);
|
||||
} else {
|
||||
associatedDocuments = new InstantiatedTermDocumentInformation[1];
|
||||
}
|
||||
associatedDocuments[associatedDocuments.length - 1] = info;
|
||||
term.setAssociatedDocuments(associatedDocuments);
|
||||
term.addPositionsCount(positions.length);
|
||||
|
||||
// todo optimize, only if term vector?
|
||||
informationByTermOfCurrentDocument.put(term, info);
|
||||
|
||||
|
||||
dirtyTerms.add(term);
|
||||
}
|
||||
|
||||
// term vector offsets
|
||||
if (eFieldSetting_TermDocInfoFactoriesByTermText.getKey().storeOffsetWithTermVector) {
|
||||
for (Map.Entry<InstantiatedTerm, InstantiatedTermDocumentInformation> e : informationByTermOfCurrentDocument.entrySet()) {
|
||||
if (eFieldSetting_TermDocInfoFactoriesByTermText.getKey().fieldName.equals(e.getKey().field())) {
|
||||
TermDocumentInformationFactory factory = eFieldSetting_TermDocInfoFactoriesByTermText.getValue().get(e.getKey().text());
|
||||
e.getValue().setTermOffsets(factory.termOffsets.toArray(new TermVectorOffsetInfo[factory.termOffsets.size()]));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Map<String, List<InstantiatedTermDocumentInformation>> termDocumentInformationsByField = new HashMap<String, List<InstantiatedTermDocumentInformation>>();
|
||||
for (Map.Entry<InstantiatedTerm, InstantiatedTermDocumentInformation> eTerm_TermDocumentInformation : informationByTermOfCurrentDocument.entrySet()) {
|
||||
List<InstantiatedTermDocumentInformation> termDocumentInformations = termDocumentInformationsByField.get(eTerm_TermDocumentInformation.getKey().field());
|
||||
if (termDocumentInformations == null) {
|
||||
termDocumentInformations = new ArrayList<InstantiatedTermDocumentInformation>();
|
||||
termDocumentInformationsByField.put(eTerm_TermDocumentInformation.getKey().field(), termDocumentInformations);
|
||||
}
|
||||
termDocumentInformations.add(eTerm_TermDocumentInformation.getValue());
|
||||
}
|
||||
|
||||
for (Map.Entry<String, List<InstantiatedTermDocumentInformation>> eField_TermDocInfos : termDocumentInformationsByField.entrySet()) {
|
||||
|
||||
CollectionUtil.quickSort(eField_TermDocInfos.getValue(), tdComp);
|
||||
|
||||
// add term vector
|
||||
if (documentFieldSettingsByFieldName.get(eField_TermDocInfos.getKey()).storeTermVector) {
|
||||
if (document.getVectorSpace() == null) {
|
||||
document.setVectorSpace(new HashMap<String, List<InstantiatedTermDocumentInformation>>(documentFieldSettingsByFieldName.size()));
|
||||
}
|
||||
document.getVectorSpace().put(eField_TermDocInfos.getKey(), eField_TermDocInfos.getValue());
|
||||
}
|
||||
|
||||
}
|
||||
fieldSettingsByFieldName.putAll(documentFieldSettingsByFieldName);
|
||||
}
|
||||
|
||||
// order document informations in dirty terms
|
||||
for (InstantiatedTerm term : dirtyTerms) {
|
||||
// todo optimize, i believe this is useless, that the natural order is document number?
|
||||
ArrayUtil.mergeSort(term.getAssociatedDocuments(), InstantiatedTermDocumentInformation.documentNumberComparator);
|
||||
|
||||
// // update association class reference for speedy skipTo()
|
||||
// for (int i = 0; i < term.getAssociatedDocuments().length; i++) {
|
||||
// term.getAssociatedDocuments()[i].setIndexFromTerm(i);
|
||||
// }
|
||||
}
|
||||
|
||||
|
||||
// flush to writer
|
||||
index.setDocumentsByNumber(documentsByNumber);
|
||||
index.setOrderedTerms(orderedTerms.toArray(new InstantiatedTerm[orderedTerms.size()]));
|
||||
|
||||
for (FieldSetting fieldSetting : fieldSettingsByFieldName.values()) {
|
||||
index.getFieldSettings().merge(fieldSetting);
|
||||
}
|
||||
// set term index
|
||||
if (orderedTermsDirty) {
|
||||
// todo optimize, only update from start position
|
||||
for (int i = 0; i < index.getOrderedTerms().length; i++) {
|
||||
index.getOrderedTerms()[i].setTermIndex(i);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
// remove deleted documents
|
||||
IndexReader indexDeleter = index.indexReaderFactory();
|
||||
if (unflushedDeletions.size() > 0) {
|
||||
for (Term term : unflushedDeletions) {
|
||||
indexDeleter.deleteDocuments(term);
|
||||
}
|
||||
unflushedDeletions.clear();
|
||||
}
|
||||
|
||||
|
||||
// all done, clear buffers
|
||||
unflushedDocuments.clear();
|
||||
termDocumentInformationFactoryByDocument.clear();
|
||||
fieldNameBuffer.clear();
|
||||
|
||||
|
||||
// update deleted documents bitset
|
||||
if (index.getDeletedDocuments() != null) {
|
||||
BitVector deletedDocuments = new BitVector(index.getDocumentsByNumber().length);
|
||||
for (int i = 0; i < index.getDeletedDocuments().size(); i++) {
|
||||
if (index.getDeletedDocuments().get(i)) {
|
||||
deletedDocuments.set(i);
|
||||
}
|
||||
}
|
||||
index.setDeletedDocuments(deletedDocuments);
|
||||
}
|
||||
|
||||
index.setVersion(System.currentTimeMillis());
|
||||
|
||||
// todo unlock
|
||||
|
||||
indexDeleter.close();
|
||||
|
||||
}
|
||||
|
||||
private static final Comparator<InstantiatedTermDocumentInformation> tdComp = new Comparator<InstantiatedTermDocumentInformation>() {
|
||||
public int compare(InstantiatedTermDocumentInformation instantiatedTermDocumentInformation, InstantiatedTermDocumentInformation instantiatedTermDocumentInformation1) {
|
||||
return instantiatedTermDocumentInformation.getTerm().getTerm().compareTo(instantiatedTermDocumentInformation1.getTerm().getTerm());
|
||||
}
|
||||
};
|
||||
|
||||
/**
|
||||
* Adds a document to this index.
|
||||
*/
|
||||
public void addDocument(Document doc) throws IOException {
|
||||
addDocument(doc, getAnalyzer());
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a document to this index, using the provided analyzer instead of the
|
||||
* value of {@link #getAnalyzer()}.
|
||||
*
|
||||
* @param doc
|
||||
* @param analyzer
|
||||
* @throws IOException
|
||||
*/
|
||||
public void addDocument(Document doc, Analyzer analyzer) throws IOException {
|
||||
addDocument(new InstantiatedDocument(doc), analyzer);
|
||||
}
|
||||
|
||||
/**
|
||||
* Tokenizes a document and adds it to the buffer.
|
||||
* Try to do all calculations in this method rather than in commit, as this is a non locking method.
|
||||
* Remember, this index implementation expects unlimited memory for maximum speed.
|
||||
*
|
||||
* @param document
|
||||
* @param analyzer
|
||||
* @throws IOException
|
||||
*/
|
||||
protected void addDocument(InstantiatedDocument document, Analyzer analyzer) throws IOException {
|
||||
|
||||
if (document.getDocumentNumber() != null) {
|
||||
throw new RuntimeException("Document number already set! Are you trying to add a document that already is bound to this or another index?");
|
||||
}
|
||||
|
||||
// todo: write lock
|
||||
|
||||
// normalize settings per field name in document
|
||||
|
||||
Map<String /* field name */, FieldSetting> fieldSettingsByFieldName = new HashMap<String, FieldSetting>();
|
||||
for (IndexableField field : document.getDocument()) {
|
||||
FieldSetting fieldSetting = fieldSettingsByFieldName.get(field.name());
|
||||
if (fieldSetting == null) {
|
||||
fieldSetting = new FieldSetting();
|
||||
fieldSetting.fieldName = field.name();
|
||||
fieldSettingsByFieldName.put(fieldSetting.fieldName, fieldSetting);
|
||||
fieldNameBuffer.add(fieldSetting.fieldName);
|
||||
}
|
||||
|
||||
// todo: fixme: multiple fields with the same name does not mean field boost += more boost.
|
||||
fieldSetting.boost *= field.boost();
|
||||
//fieldSettings.dimensions++;
|
||||
|
||||
|
||||
// once fieldSettings, always fieldSettings.
|
||||
if (field.fieldType().omitNorms()) {
|
||||
fieldSetting.omitNorms = true;
|
||||
}
|
||||
if (field.fieldType().indexed() ) {
|
||||
fieldSetting.indexed = true;
|
||||
}
|
||||
if (field.fieldType().tokenized()) {
|
||||
fieldSetting.tokenized = true;
|
||||
}
|
||||
if (field.fieldType().stored()) {
|
||||
fieldSetting.stored = true;
|
||||
}
|
||||
if (field.binaryValue() != null) {
|
||||
fieldSetting.isBinary = true;
|
||||
}
|
||||
if (field.fieldType().storeTermVectors()) {
|
||||
fieldSetting.storeTermVector = true;
|
||||
}
|
||||
if (field.fieldType().storeTermVectorPositions()) {
|
||||
fieldSetting.storePositionWithTermVector = true;
|
||||
}
|
||||
if (field.fieldType().storeTermVectorOffsets()) {
|
||||
fieldSetting.storeOffsetWithTermVector = true;
|
||||
}
|
||||
}
|
||||
|
||||
Map<IndexableField, LinkedList<Token>> tokensByField = new LinkedHashMap<IndexableField, LinkedList<Token>>(20);
|
||||
|
||||
// tokenize indexed fields.
|
||||
for (Iterator<IndexableField> it = document.getDocument().iterator(); it.hasNext();) {
|
||||
|
||||
IndexableField field = it.next();
|
||||
|
||||
FieldSetting fieldSetting = fieldSettingsByFieldName.get(field.name());
|
||||
|
||||
if (field.fieldType().indexed()) {
|
||||
|
||||
LinkedList<Token> tokens = new LinkedList<Token>();
|
||||
tokensByField.put(field, tokens);
|
||||
|
||||
if (field.fieldType().tokenized()) {
|
||||
final TokenStream tokenStream = field.tokenStream(analyzer);
|
||||
// reset the TokenStream to the first token
|
||||
tokenStream.reset();
|
||||
|
||||
while (tokenStream.incrementToken()) {
|
||||
// TODO: this is a simple workaround to still work with tokens, not very effective, but as far as I know, this writer should get removed soon:
|
||||
final Token token = new Token();
|
||||
for (Iterator<AttributeImpl> atts = tokenStream.getAttributeImplsIterator(); atts.hasNext();) {
|
||||
final AttributeImpl att = atts.next();
|
||||
try {
|
||||
att.copyTo(token);
|
||||
} catch (Exception e) {
|
||||
// ignore unsupported attributes,
|
||||
// this may fail to copy some attributes, if a special combined AttributeImpl is used, that
|
||||
// implements basic attributes supported by Token and also other customized ones in one class.
|
||||
}
|
||||
}
|
||||
tokens.add(token); // the vector will be built on commit.
|
||||
fieldSetting.fieldLength++;
|
||||
}
|
||||
tokenStream.end();
|
||||
tokenStream.close();
|
||||
} else {
|
||||
// untokenized
|
||||
String fieldVal = field.stringValue();
|
||||
Token token = new Token(0, fieldVal.length(), "untokenized");
|
||||
token.setEmpty().append(fieldVal);
|
||||
tokens.add(token);
|
||||
fieldSetting.fieldLength++;
|
||||
}
|
||||
}
|
||||
|
||||
if (!field.fieldType().stored()) {
|
||||
//it.remove();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
Map<FieldSetting, Map<String /*text*/, TermDocumentInformationFactory>> termDocumentInformationFactoryByTermTextAndFieldSetting = new HashMap<FieldSetting, Map<String /*text*/, TermDocumentInformationFactory>>();
|
||||
termDocumentInformationFactoryByDocument.put(document, termDocumentInformationFactoryByTermTextAndFieldSetting);
|
||||
|
||||
// build term vector, term positions and term offsets
|
||||
for (Map.Entry<IndexableField, LinkedList<Token>> eField_Tokens : tokensByField.entrySet()) {
|
||||
FieldSetting fieldSetting = fieldSettingsByFieldName.get(eField_Tokens.getKey().name());
|
||||
|
||||
Map<String, TermDocumentInformationFactory> termDocumentInformationFactoryByTermText = termDocumentInformationFactoryByTermTextAndFieldSetting.get(fieldSettingsByFieldName.get(eField_Tokens.getKey().name()));
|
||||
if (termDocumentInformationFactoryByTermText == null) {
|
||||
termDocumentInformationFactoryByTermText = new HashMap<String /*text*/, TermDocumentInformationFactory>();
|
||||
termDocumentInformationFactoryByTermTextAndFieldSetting.put(fieldSettingsByFieldName.get(eField_Tokens.getKey().name()), termDocumentInformationFactoryByTermText);
|
||||
}
|
||||
|
||||
int lastOffset = 0;
|
||||
|
||||
// for each new field, move positions a bunch.
|
||||
if (fieldSetting.position > 0) {
|
||||
// todo what if no analyzer set, multiple fields with same name and index without tokenization?
|
||||
fieldSetting.position += analyzer.getPositionIncrementGap(fieldSetting.fieldName);
|
||||
}
|
||||
|
||||
for (Token token : eField_Tokens.getValue()) {
|
||||
|
||||
TermDocumentInformationFactory termDocumentInformationFactory = termDocumentInformationFactoryByTermText.get(token.toString());
|
||||
if (termDocumentInformationFactory == null) {
|
||||
termDocumentInformationFactory = new TermDocumentInformationFactory();
|
||||
termDocumentInformationFactoryByTermText.put(token.toString(), termDocumentInformationFactory);
|
||||
}
|
||||
//termDocumentInformationFactory.termFrequency++;
|
||||
|
||||
fieldSetting.position += (token.getPositionIncrement() - 1);
|
||||
termDocumentInformationFactory.termPositions.add(fieldSetting.position++);
|
||||
|
||||
if (token.getPayload() != null && token.getPayload().length() > 0) {
|
||||
termDocumentInformationFactory.payloads.add(token.getPayload().toByteArray());
|
||||
fieldSetting.storePayloads = true;
|
||||
} else {
|
||||
termDocumentInformationFactory.payloads.add(null);
|
||||
}
|
||||
|
||||
if (eField_Tokens.getKey().fieldType().storeTermVectorOffsets()) {
|
||||
|
||||
termDocumentInformationFactory.termOffsets.add(new TermVectorOffsetInfo(fieldSetting.offset + token.startOffset(), fieldSetting.offset + token.endOffset()));
|
||||
lastOffset = fieldSetting.offset + token.endOffset();
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
if (eField_Tokens.getKey().fieldType().storeTermVectorOffsets()) {
|
||||
fieldSetting.offset = lastOffset + 1;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
unflushedDocuments.add(document);
|
||||
|
||||
// if too many documents in buffer, commit.
|
||||
if (unflushedDocuments.size() >= getMergeFactor()) {
|
||||
commit(/*lock*/);
|
||||
}
|
||||
|
||||
// todo: unlock write lock
|
||||
|
||||
}
|
||||
|
||||
|
||||
private Set<Term> unflushedDeletions = new HashSet<Term>();
|
||||
|
||||
public void deleteDocuments(Term term) throws IOException {
|
||||
unflushedDeletions.add(term);
|
||||
}
|
||||
|
||||
public void deleteDocuments(Term[] terms) throws IOException {
|
||||
for (Term term : terms) {
|
||||
deleteDocuments(term);
|
||||
}
|
||||
}
|
||||
|
||||
public void updateDocument(Term term, Document doc) throws IOException {
|
||||
updateDocument(term, doc, getAnalyzer());
|
||||
}
|
||||
|
||||
public void updateDocument(Term term, Document doc, Analyzer analyzer) throws IOException {
|
||||
deleteDocuments(term);
|
||||
addDocument(doc, analyzer);
|
||||
}
|
||||
|
||||
public SimilarityProvider getSimilarityProvider() {
|
||||
return similarityProvider;
|
||||
}
|
||||
|
||||
public void setSimilarityProvider(SimilarityProvider similarityProvider) {
|
||||
this.similarityProvider = similarityProvider;
|
||||
}
|
||||
|
||||
public Analyzer getAnalyzer() {
|
||||
return analyzer;
|
||||
}
|
||||
|
||||
private class TermDocumentInformationFactory {
|
||||
private LinkedList<byte[]> payloads = new LinkedList<byte[]>();
|
||||
private LinkedList<Integer> termPositions = new LinkedList<Integer>();
|
||||
private LinkedList<TermVectorOffsetInfo> termOffsets = new LinkedList<TermVectorOffsetInfo>();
|
||||
}
|
||||
|
||||
|
||||
static class FieldSetting extends org.apache.lucene.store.instantiated.FieldSetting {
|
||||
|
||||
float boost = 1;
|
||||
int position = 0;
|
||||
int offset;
|
||||
int fieldLength = 0;
|
||||
|
||||
boolean omitNorms = false;
|
||||
boolean isBinary = false;
|
||||
|
||||
private FieldSetting() {
|
||||
}
|
||||
|
||||
private FieldSetting(String fieldName) {
|
||||
super(fieldName);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
}
|
|
@ -1,255 +0,0 @@
|
|||
package org.apache.lucene.store.instantiated;
|
||||
|
||||
/**
|
||||
* Copyright 2006 The Apache Software Foundation
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.util.Comparator;
|
||||
|
||||
import org.apache.lucene.index.Term;
|
||||
|
||||
/**
|
||||
* A term in the inverted index, coupled to the documents it occurs in.
|
||||
*
|
||||
* @see org.apache.lucene.index.Term
|
||||
*/
|
||||
public class InstantiatedTerm {
|
||||
|
||||
public static final Comparator<InstantiatedTerm> comparator = new Comparator<InstantiatedTerm>() {
|
||||
public int compare(InstantiatedTerm instantiatedTerm, InstantiatedTerm instantiatedTerm1) {
|
||||
return instantiatedTerm.getTerm().compareTo(instantiatedTerm1.getTerm());
|
||||
}
|
||||
};
|
||||
|
||||
public static final Comparator<Object> termComparator = new Comparator<Object>() {
|
||||
public int compare(Object o, Object o1) {
|
||||
return ((InstantiatedTerm)o).getTerm().compareTo((Term)o1);
|
||||
}
|
||||
};
|
||||
|
||||
private Term term;
|
||||
|
||||
private long totalTermFreq;
|
||||
|
||||
/**
|
||||
* index of term in InstantiatedIndex
|
||||
* @see org.apache.lucene.store.instantiated.InstantiatedIndex#getOrderedTerms() */
|
||||
private int termIndex;
|
||||
|
||||
/**
|
||||
* @return Term associated with this entry of the index object graph
|
||||
*/
|
||||
public Term getTerm() {
|
||||
return term;
|
||||
}
|
||||
|
||||
InstantiatedTerm(String field, String text) {
|
||||
this.term = new Term(field, text);
|
||||
}
|
||||
|
||||
// this could speed up TermDocs.skipTo even more
|
||||
// private Map</** document number*/Integer, /** index in associatedDocuments */Integer> associatedDocumentIndexByDocumentNumber = new HashMap<Integer, Integer>();
|
||||
//
|
||||
// public Map</** document number*/Integer, /** index in associatedDocuments */Integer> getAssociatedDocumentIndexByDocumentNumber() {
|
||||
// return associatedDocumentIndexByDocumentNumber;
|
||||
// }
|
||||
|
||||
/** Ordered by document number */
|
||||
private InstantiatedTermDocumentInformation[] associatedDocuments;
|
||||
|
||||
/**
|
||||
* Meta data per document in which this term is occurring.
|
||||
* Ordered by document number.
|
||||
*
|
||||
* @return Meta data per document in which this term is occurring.
|
||||
*/
|
||||
public InstantiatedTermDocumentInformation[] getAssociatedDocuments() {
|
||||
return associatedDocuments;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Meta data per document in which this term is occurring.
|
||||
* Ordered by document number.
|
||||
*
|
||||
* @param associatedDocuments meta data per document in which this term is occurring, ordered by document number
|
||||
*/
|
||||
void setAssociatedDocuments(InstantiatedTermDocumentInformation[] associatedDocuments) {
|
||||
this.associatedDocuments = associatedDocuments;
|
||||
}
|
||||
|
||||
void addPositionsCount(long count) {
|
||||
totalTermFreq += count;
|
||||
}
|
||||
|
||||
public long getTotalTermFreq() {
|
||||
return totalTermFreq;
|
||||
}
|
||||
|
||||
/**
|
||||
* Finds index to the first beyond the current whose document number is
|
||||
* greater than or equal to <i>target</i>, -1 if there is no such element.
|
||||
*
|
||||
* @param target the document number to match
|
||||
* @return -1 if there is no such element
|
||||
*/
|
||||
public int seekCeilingDocumentInformationIndex(int target) {
|
||||
return seekCeilingDocumentInformationIndex(target, 0, getAssociatedDocuments().length);
|
||||
}
|
||||
|
||||
/**
|
||||
* Finds index to the first beyond the current whose document number is
|
||||
* greater than or equal to <i>target</i>, -1 if there is no such element.
|
||||
*
|
||||
* @param target the document number to match
|
||||
* @param startOffset associated documents index start offset
|
||||
* @return -1 if there is no such element
|
||||
*/
|
||||
public int seekCeilingDocumentInformationIndex(int target, int startOffset) {
|
||||
return seekCeilingDocumentInformationIndex(target, startOffset, getAssociatedDocuments().length);
|
||||
}
|
||||
|
||||
/**
|
||||
* Finds index to the first beyond the current whose document number is
|
||||
* greater than or equal to <i>target</i>, -1 if there is no such element.
|
||||
*
|
||||
* @param target the document number to match
|
||||
* @param startOffset associated documents index start offset
|
||||
* @param endPosition associated documents index end position
|
||||
* @return -1 if there is no such element
|
||||
*/
|
||||
public int seekCeilingDocumentInformationIndex(int target, int startOffset, int endPosition) {
|
||||
|
||||
int pos = binarySearchAssociatedDocuments(target, startOffset, endPosition - startOffset);
|
||||
|
||||
// int pos = Arrays.binarySearch(getAssociatedDocuments(), target, InstantiatedTermDocumentInformation.doumentNumberIntegerComparator);
|
||||
|
||||
if (pos < 0) {
|
||||
pos = -1 - pos;
|
||||
}
|
||||
if (getAssociatedDocuments().length <= pos) {
|
||||
return -1;
|
||||
} else {
|
||||
return pos;
|
||||
}
|
||||
}
|
||||
|
||||
public int binarySearchAssociatedDocuments(int target) {
|
||||
return binarySearchAssociatedDocuments(target, 0);
|
||||
}
|
||||
|
||||
public int binarySearchAssociatedDocuments(int target, int offset) {
|
||||
return binarySearchAssociatedDocuments(target, offset, associatedDocuments.length - offset);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param target value to search for in the array
|
||||
* @param offset index of the first valid value in the array
|
||||
* @param length number of valid values in the array
|
||||
* @return index of an occurrence of key in array, or -(insertionIndex + 1) if key is not contained in array (<i>insertionIndex</i> is then the index at which key could be inserted).
|
||||
*/
|
||||
public int binarySearchAssociatedDocuments(int target, int offset, int length) {
|
||||
|
||||
// implementation originally from http://ochafik.free.fr/blog/?p=106
|
||||
|
||||
if (length == 0) {
|
||||
return -1 - offset;
|
||||
}
|
||||
int min = offset, max = offset + length - 1;
|
||||
int minVal = getAssociatedDocuments()[min].getDocument().getDocumentNumber();
|
||||
int maxVal = getAssociatedDocuments()[max].getDocument().getDocumentNumber();
|
||||
|
||||
|
||||
int nPreviousSteps = 0;
|
||||
|
||||
for (; ;) {
|
||||
|
||||
// be careful not to compute key - minVal, for there might be an integer overflow.
|
||||
if (target <= minVal) return target == minVal ? min : -1 - min;
|
||||
if (target >= maxVal) return target == maxVal ? max : -2 - max;
|
||||
|
||||
assert min != max;
|
||||
|
||||
int pivot;
|
||||
// A typical binarySearch algorithm uses pivot = (min + max) / 2.
|
||||
// The pivot we use here tries to be smarter and to choose a pivot close to the expectable location of the key.
|
||||
// This reduces dramatically the number of steps needed to get to the key.
|
||||
// However, it does not work well with a logarithmic distribution of values, for instance.
|
||||
// When the key is not found quickly the smart way, we switch to the standard pivot.
|
||||
if (nPreviousSteps > 2) {
|
||||
pivot = (min + max) >> 1;
|
||||
// stop increasing nPreviousSteps from now on
|
||||
} else {
|
||||
// NOTE: We cannot do the following operations in int precision, because there might be overflows.
|
||||
// long operations are slower than float operations with the hardware this was tested on (intel core duo 2, JVM 1.6.0).
|
||||
// Overall, using float proved to be the safest and fastest approach.
|
||||
pivot = min + (int) ((target - (float) minVal) / (maxVal - (float) minVal) * (max - min));
|
||||
nPreviousSteps++;
|
||||
}
|
||||
|
||||
int pivotVal = getAssociatedDocuments()[pivot].getDocument().getDocumentNumber();
|
||||
|
||||
// NOTE: do not store key - pivotVal because of overflows
|
||||
if (target > pivotVal) {
|
||||
min = pivot + 1;
|
||||
max--;
|
||||
} else if (target == pivotVal) {
|
||||
return pivot;
|
||||
} else {
|
||||
min++;
|
||||
max = pivot - 1;
|
||||
}
|
||||
maxVal = getAssociatedDocuments()[max].getDocument().getDocumentNumber();
|
||||
minVal = getAssociatedDocuments()[min].getDocument().getDocumentNumber();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Navigates to the view of this occurrences of this term in a specific document.
|
||||
*
|
||||
* This method is only used by InstantiatedIndex(IndexReader) and
|
||||
* should not be optimized for less CPU at the cost of more RAM.
|
||||
*
|
||||
* @param documentNumber the n:th document in the index
|
||||
* @return view of this term from specified document
|
||||
*/
|
||||
public InstantiatedTermDocumentInformation getAssociatedDocument(int documentNumber) {
|
||||
int pos = binarySearchAssociatedDocuments(documentNumber);
|
||||
return pos < 0 ? null : getAssociatedDocuments()[pos];
|
||||
}
|
||||
|
||||
public final String field() {
|
||||
return term.field();
|
||||
}
|
||||
|
||||
public final String text() {
|
||||
return term.text();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return term.toString();
|
||||
}
|
||||
|
||||
|
||||
public int getTermIndex() {
|
||||
return termIndex;
|
||||
}
|
||||
|
||||
public void setTermIndex(int termIndex) {
|
||||
this.termIndex = termIndex;
|
||||
}
|
||||
}
|
|
@ -1,135 +0,0 @@
|
|||
package org.apache.lucene.store.instantiated;
|
||||
|
||||
import org.apache.lucene.index.TermVectorOffsetInfo;
|
||||
|
||||
import java.util.Comparator;
|
||||
|
||||
/**
|
||||
* Copyright 2006 The Apache Software Foundation
|
||||
*
|
||||
* Licensed 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.
|
||||
*/
|
||||
|
||||
/**
|
||||
* There is one instance of this class per indexed term in a document
|
||||
* and it contains the meta data about each occurrence of a term in a document.
|
||||
*
|
||||
* It is the inner glue of the inverted index.
|
||||
*
|
||||
* <pre>
|
||||
* [Term]-- {0..*} | {0..*} --(field)[Document]
|
||||
* <<ordered>>
|
||||
* |
|
||||
* [TermDocumentInformation]
|
||||
* +payloads
|
||||
* +termPositions
|
||||
* +termOffsets
|
||||
* </pre>
|
||||
*
|
||||
*/
|
||||
public class InstantiatedTermDocumentInformation {
|
||||
|
||||
public static final Comparator<InstantiatedTermDocumentInformation> termComparator = new Comparator<InstantiatedTermDocumentInformation>() {
|
||||
public int compare(InstantiatedTermDocumentInformation instantiatedTermDocumentInformation, InstantiatedTermDocumentInformation instantiatedTermDocumentInformation1) {
|
||||
return instantiatedTermDocumentInformation.getTerm().getTerm().compareTo(instantiatedTermDocumentInformation1.getTerm().getTerm());
|
||||
}
|
||||
};
|
||||
|
||||
public static final Comparator<InstantiatedTermDocumentInformation> documentNumberComparator = new Comparator<InstantiatedTermDocumentInformation>() {
|
||||
public int compare(InstantiatedTermDocumentInformation instantiatedTermDocumentInformation, InstantiatedTermDocumentInformation instantiatedTermDocumentInformation1) {
|
||||
return instantiatedTermDocumentInformation.getDocument().getDocumentNumber().compareTo(instantiatedTermDocumentInformation1.getDocument().getDocumentNumber());
|
||||
}
|
||||
};
|
||||
|
||||
public static final Comparator doumentNumberIntegerComparator = new Comparator() {
|
||||
public int compare(Object o1, Object o2) {
|
||||
InstantiatedTermDocumentInformation di = (InstantiatedTermDocumentInformation) o1;
|
||||
Integer i = (Integer) o2;
|
||||
return di.getDocument().getDocumentNumber().compareTo(i);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
private byte[][] payloads;
|
||||
private int[] termPositions;
|
||||
private InstantiatedTerm term;
|
||||
private InstantiatedDocument document;
|
||||
private TermVectorOffsetInfo[] termOffsets;
|
||||
|
||||
|
||||
|
||||
public InstantiatedTermDocumentInformation(InstantiatedTerm term, InstantiatedDocument document, int[] termPositions, byte[][] payloads) {
|
||||
this.term = term;
|
||||
this.document = document;
|
||||
this.termPositions = termPositions;
|
||||
this.payloads = payloads;
|
||||
}
|
||||
|
||||
|
||||
// not quite sure why I wanted this.
|
||||
// /**
|
||||
// * [Term]--- {0..* ordered} ->[Info]
|
||||
// */
|
||||
// private int indexFromTerm;
|
||||
|
||||
|
||||
// public int getIndexFromTerm() {
|
||||
// return indexFromTerm;
|
||||
// }
|
||||
//
|
||||
// void setIndexFromTerm(int indexFromTerm) {
|
||||
// this.indexFromTerm = indexFromTerm;
|
||||
// }
|
||||
|
||||
|
||||
public int[] getTermPositions() {
|
||||
return termPositions;
|
||||
}
|
||||
|
||||
|
||||
public byte[][] getPayloads() {
|
||||
return payloads;
|
||||
}
|
||||
|
||||
public InstantiatedDocument getDocument() {
|
||||
return document;
|
||||
}
|
||||
|
||||
|
||||
|
||||
public InstantiatedTerm getTerm() {
|
||||
return term;
|
||||
}
|
||||
|
||||
|
||||
void setTermPositions(int[] termPositions) {
|
||||
this.termPositions = termPositions;
|
||||
}
|
||||
|
||||
|
||||
void setTerm(InstantiatedTerm term) {
|
||||
this.term = term;
|
||||
}
|
||||
|
||||
void setDocument(InstantiatedDocument document) {
|
||||
this.document = document;
|
||||
}
|
||||
|
||||
public TermVectorOffsetInfo[] getTermOffsets() {
|
||||
return termOffsets;
|
||||
}
|
||||
|
||||
void setTermOffsets(TermVectorOffsetInfo[] termOffsets) {
|
||||
this.termOffsets = termOffsets;
|
||||
}
|
||||
}
|
|
@ -1,111 +0,0 @@
|
|||
package org.apache.lucene.store.instantiated;
|
||||
|
||||
import org.apache.lucene.index.TermFreqVector;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Copyright 2006 The Apache Software Foundation
|
||||
*
|
||||
* Licensed 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.
|
||||
*/
|
||||
|
||||
/**
|
||||
* Vector space view of a document in an {@link InstantiatedIndexReader}.
|
||||
*
|
||||
* @see org.apache.lucene.index.TermFreqVector
|
||||
*/
|
||||
public class InstantiatedTermFreqVector
|
||||
implements TermFreqVector {
|
||||
|
||||
private final List<InstantiatedTermDocumentInformation> termDocumentInformations;
|
||||
private final String field;
|
||||
private final BytesRef terms[];
|
||||
private final int termFrequencies[];
|
||||
|
||||
public InstantiatedTermFreqVector(InstantiatedDocument document, String field) {
|
||||
this.field = field;
|
||||
termDocumentInformations = document.getVectorSpace().get(field);
|
||||
terms = new BytesRef[termDocumentInformations.size()];
|
||||
termFrequencies = new int[termDocumentInformations.size()];
|
||||
|
||||
for (int i = 0; i < termDocumentInformations.size(); i++) {
|
||||
InstantiatedTermDocumentInformation termDocumentInformation = termDocumentInformations.get(i);
|
||||
terms[i] = termDocumentInformation.getTerm().getTerm().bytes();
|
||||
termFrequencies[i] = termDocumentInformation.getTermPositions().length;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The number of the field this vector is associated with
|
||||
*/
|
||||
public String getField() {
|
||||
return field;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
sb.append('{');
|
||||
sb.append(field).append(": ");
|
||||
if (terms != null) {
|
||||
for (int i = 0; i < terms.length; i++) {
|
||||
if (i > 0) sb.append(", ");
|
||||
sb.append(terms[i]).append('/').append(termFrequencies[i]);
|
||||
}
|
||||
}
|
||||
sb.append('}');
|
||||
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
public int size() {
|
||||
return terms == null ? 0 : terms.length;
|
||||
}
|
||||
|
||||
public BytesRef[] getTerms() {
|
||||
return terms;
|
||||
}
|
||||
|
||||
public int[] getTermFrequencies() {
|
||||
return termFrequencies;
|
||||
}
|
||||
|
||||
public int indexOf(BytesRef termText) {
|
||||
if (terms == null)
|
||||
return -1;
|
||||
int res = Arrays.binarySearch(terms, termText);
|
||||
return res >= 0 ? res : -1;
|
||||
}
|
||||
|
||||
public int[] indexesOf(BytesRef[] termNumbers, int start, int len) {
|
||||
// TODO: there must be a more efficient way of doing this.
|
||||
// At least, we could advance the lower bound of the terms array
|
||||
// as we find valid indices. Also, it might be possible to leverage
|
||||
// this even more by starting in the middle of the termNumbers array
|
||||
// and thus dividing the terms array maybe in half with each found index.
|
||||
int res[] = new int[len];
|
||||
|
||||
for (int i = 0; i < len; i++) {
|
||||
res[i] = indexOf(termNumbers[start + i]);
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
public List<InstantiatedTermDocumentInformation> getTermDocumentInformations() {
|
||||
return termDocumentInformations;
|
||||
}
|
||||
|
||||
}
|
|
@ -1,43 +0,0 @@
|
|||
package org.apache.lucene.store.instantiated;
|
||||
|
||||
/**
|
||||
* Copyright 2006 The Apache Software Foundation
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.index.TermPositionVector;
|
||||
import org.apache.lucene.index.TermVectorOffsetInfo;
|
||||
|
||||
/**
|
||||
* Extended vector space view of a document in an {@link InstantiatedIndexReader}.
|
||||
*
|
||||
* @see org.apache.lucene.index.TermPositionVector
|
||||
*/
|
||||
public class InstantiatedTermPositionVector
|
||||
extends InstantiatedTermFreqVector
|
||||
implements TermPositionVector {
|
||||
|
||||
public InstantiatedTermPositionVector(InstantiatedDocument document, String field) {
|
||||
super(document, field);
|
||||
}
|
||||
|
||||
public int[] getTermPositions(int index) {
|
||||
return getTermDocumentInformations().get(index).getTermPositions();
|
||||
}
|
||||
|
||||
public TermVectorOffsetInfo[] getOffsets(int index) {
|
||||
return getTermDocumentInformations().get(index).getTermOffsets();
|
||||
}
|
||||
|
||||
}
|
|
@ -1,154 +0,0 @@
|
|||
package org.apache.lucene.store.instantiated;
|
||||
|
||||
/**
|
||||
* Copyright 2006 The Apache Software Foundation
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.index.OrdTermState;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.TermState;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.index.DocsEnum;
|
||||
import org.apache.lucene.index.DocsAndPositionsEnum;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
|
||||
public class InstantiatedTermsEnum extends TermsEnum {
|
||||
private final String field;
|
||||
private final InstantiatedTerm[] terms;
|
||||
private final BytesRef br = new BytesRef();
|
||||
private final int start;
|
||||
private int upto;
|
||||
|
||||
public InstantiatedTermsEnum(InstantiatedTerm[] terms, int start, String field) {
|
||||
this.start = start;
|
||||
upto = start-1;
|
||||
this.field = field;
|
||||
this.terms = terms;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean seekExact(BytesRef text, boolean useCache) {
|
||||
final Term t = new Term(field, text);
|
||||
int loc = Arrays.binarySearch(terms, t, InstantiatedTerm.termComparator);
|
||||
if (loc < 0) {
|
||||
return false;
|
||||
} else {
|
||||
upto = loc;
|
||||
br.copy(text);
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public SeekStatus seekCeil(BytesRef text, boolean useCache) {
|
||||
final Term t = new Term(field, text);
|
||||
int loc = Arrays.binarySearch(terms, t, InstantiatedTerm.termComparator);
|
||||
if (loc < 0) {
|
||||
upto = -loc - 1;
|
||||
if (upto >= terms.length) {
|
||||
return SeekStatus.END;
|
||||
} else {
|
||||
br.copy(terms[upto].getTerm().bytes());
|
||||
return SeekStatus.NOT_FOUND;
|
||||
}
|
||||
} else {
|
||||
upto = loc;
|
||||
br.copy(text);
|
||||
return SeekStatus.FOUND;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void seekExact(long ord) {
|
||||
assert (start + (int) ord) < terms.length;
|
||||
upto = start + (int) ord;
|
||||
assert field.equals(terms[upto].field());
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef next() {
|
||||
upto++;
|
||||
if (upto >= terms.length) {
|
||||
return null;
|
||||
}
|
||||
if (terms[upto].field().equals(field)) {
|
||||
br.copy(terms[upto].getTerm().text());
|
||||
return br;
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef term() {
|
||||
return br;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long ord() {
|
||||
return upto - start;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docFreq() {
|
||||
return terms[upto].getAssociatedDocuments().length;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long totalTermFreq() {
|
||||
final long v = terms[upto].getTotalTermFreq();
|
||||
return v == 0 ? -1 : v;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocsEnum docs(Bits liveDocs, DocsEnum reuse) {
|
||||
if (reuse == null || !(reuse instanceof InstantiatedDocsEnum)) {
|
||||
reuse = new InstantiatedDocsEnum();
|
||||
}
|
||||
return ((InstantiatedDocsEnum) reuse).reset(liveDocs, terms[upto]);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse) {
|
||||
if (reuse == null || !(reuse instanceof InstantiatedDocsAndPositionsEnum)) {
|
||||
reuse = new InstantiatedDocsAndPositionsEnum();
|
||||
}
|
||||
return ((InstantiatedDocsAndPositionsEnum) reuse).reset(liveDocs, terms[upto]);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Comparator<BytesRef> getComparator() {
|
||||
return BytesRef.getUTF8SortedAsUnicodeComparator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermState termState() throws IOException {
|
||||
final OrdTermState state = new OrdTermState();
|
||||
state.ord = upto - start;
|
||||
return state;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void seekExact(BytesRef term, TermState state) throws IOException {
|
||||
assert state != null && state instanceof OrdTermState;
|
||||
seekExact(((OrdTermState)state).ord); // just use the ord for simplicity
|
||||
}
|
||||
}
|
||||
|
Binary file not shown.
Before Width: | Height: | Size: 156 KiB |
Binary file not shown.
Before Width: | Height: | Size: 61 KiB |
|
@ -1,50 +0,0 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?><umlet_diagram><help_text>//Uncomment the following line to change the fontsize:
|
||||
//fontsize=14
|
||||
|
||||
//Welcome to UMLet!
|
||||
|
||||
// *Double-click on UML elements to add them to the diagram.
|
||||
// *Edit element properties by modifying the text in this panel.
|
||||
// *Edit the files in the 'palettes' directory to store your own element palettes.
|
||||
// *Press Del or Backspace to remove elements from the diagram.
|
||||
// *Hold down Ctrl key to select multiple elements.
|
||||
// *Press c to copy the UML diagram to the system clipboard.
|
||||
// * This text will be stored with each diagram. Feel free to use the area for notes.
|
||||
</help_text><element><type>com.umlet.element.base.Class</type><coordinates><x>310</x><y>540</y><w>310</w><h>110</h></coordinates><panel_attributes>bg=#eeeeee
|
||||
fg=#000000
|
||||
InstantiatedTermDocumentInformation
|
||||
--
|
||||
+payloads:byte[][]
|
||||
+termPositions:int[]
|
||||
+termOffsets:TermVectorOffsetInfo[]
|
||||
+indexFromTerm:int
|
||||
--</panel_attributes><additional_attributes></additional_attributes></element><element><type>com.umlet.element.base.Relation</type><coordinates><x>460</x><y>380</y><w>40</w><h>180</h></coordinates><panel_attributes>lt=.</panel_attributes><additional_attributes>20;20;20;160</additional_attributes></element><element><type>com.umlet.element.base.Relation</type><coordinates><x>460</x><y>40</y><w>61</w><h>160</h></coordinates><panel_attributes>lt=<-
|
||||
q2=field
|
||||
m1=0..1</panel_attributes><additional_attributes>30;20;30;140</additional_attributes></element><element><type>com.umlet.element.base.Class</type><coordinates><x>430</x><y>30</y><w>120</w><h>30</h></coordinates><panel_attributes>bg=#eeeeee
|
||||
fg=#000099
|
||||
_norm: byte[][]_</panel_attributes><additional_attributes></additional_attributes></element><element><type>com.umlet.element.base.Class</type><coordinates><x>80</x><y>390</y><w>100</w><h>30</h></coordinates><panel_attributes>bg=#eeeeee
|
||||
fg=#000099
|
||||
Term</panel_attributes><additional_attributes></additional_attributes></element><element><type>com.umlet.element.base.Relation</type><coordinates><x>770</x><y>380</y><w>120</w><h>40</h></coordinates><panel_attributes>lt=->
|
||||
m2=1</panel_attributes><additional_attributes>20;20;100;20</additional_attributes></element><element><type>com.umlet.element.base.Class</type><coordinates><x>870</x><y>390</y><w>100</w><h>30</h></coordinates><panel_attributes>bg=#eeeeee
|
||||
fg=#000099
|
||||
Document</panel_attributes><additional_attributes></additional_attributes></element><element><type>com.umlet.element.base.Class</type><coordinates><x>590</x><y>370</y><w>200</w><h>60</h></coordinates><panel_attributes>bg=#eeeeee
|
||||
fg=#000000
|
||||
InstantiatedDocument
|
||||
--
|
||||
+documentNumber:int
|
||||
--</panel_attributes><additional_attributes></additional_attributes></element><element><type>com.umlet.element.base.Relation</type><coordinates><x>520</x><y>190</y><w>170</w><h>200</h></coordinates><panel_attributes>lt=<-
|
||||
m1=0..*
|
||||
<<ordered>></panel_attributes><additional_attributes>150;180;20;20</additional_attributes></element><element><type>com.umlet.element.base.Relation</type><coordinates><x>290</x><y>190</y><w>140</w><h>220</h></coordinates><panel_attributes>lt=<-
|
||||
m1=0..*
|
||||
<<ordered>></panel_attributes><additional_attributes>20;200;120;20</additional_attributes></element><element><type>com.umlet.element.base.Class</type><coordinates><x>380</x><y>180</y><w>200</w><h>30</h></coordinates><panel_attributes>bg=#eeeeee
|
||||
fg=#000000
|
||||
InstantiatedIndex</panel_attributes><additional_attributes></additional_attributes></element><element><type>com.umlet.element.base.Relation</type><coordinates><x>160</x><y>380</y><w>110</w><h>40</h></coordinates><panel_attributes>lt=->
|
||||
m2=1</panel_attributes><additional_attributes>90;20;20;20</additional_attributes></element><element><type>com.umlet.element.base.Class</type><coordinates><x>250</x><y>390</y><w>160</w><h>30</h></coordinates><panel_attributes>bg=#eeeeee
|
||||
fg=#000000
|
||||
InstantiatedTerm
|
||||
</panel_attributes><additional_attributes></additional_attributes></element><element><type>com.umlet.element.base.Relation</type><coordinates><x>380</x><y>190</y><w>146</w><h>220</h></coordinates><panel_attributes>lt=<-
|
||||
q2=field, term
|
||||
m1=0..1</panel_attributes><additional_attributes>20;200;100;20</additional_attributes></element><element><type>com.umlet.element.base.Relation</type><coordinates><x>390</x><y>380</y><w>220</w><h>40</h></coordinates><panel_attributes>lt=-
|
||||
q2=field
|
||||
m2=0..*
|
||||
m1=0..*</panel_attributes><additional_attributes>20;20;200;20</additional_attributes></element></umlet_diagram>
|
|
@ -1,95 +0,0 @@
|
|||
<!DOCTYPE HTML PUBLIC "-//W3C//DTD HTML 4.01//EN" "http://www.w3.org/TR/html4/strict.dtd">
|
||||
<html>
|
||||
<!--
|
||||
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.
|
||||
-->
|
||||
<head>
|
||||
<title>InstantiatedIndex</title>
|
||||
</head>
|
||||
<body>
|
||||
<p>InstantiatedIndex, alternative RAM store for small corpora.</p>
|
||||
|
||||
<p>@lucene.experimental</p>
|
||||
<h2>Abstract</h2>
|
||||
|
||||
<p>
|
||||
Represented as a coupled graph of class instances, this
|
||||
all-in-memory index store implementation delivers search
|
||||
results up to a 100 times faster than the file-centric RAMDirectory
|
||||
at the cost of greater RAM consumption.
|
||||
</p>
|
||||
|
||||
<h2>API</h2>
|
||||
|
||||
<p>
|
||||
Just as the default store implementation, InstantiatedIndex
|
||||
comes with an IndexReader and IndexWriter. The latter share
|
||||
many method signatures with the file-centric IndexWriter.
|
||||
</p>
|
||||
|
||||
<p>
|
||||
It is also possible to load the content of another index
|
||||
by passing an IndexReader to the InstantiatedIndex constructor.
|
||||
</p>
|
||||
|
||||
<h2>Performance</h2>
|
||||
|
||||
<p>
|
||||
At a few thousand ~160 characters long documents
|
||||
InstantiatedIndex outperforms RAMDirectory some 50x,
|
||||
15x at 100 documents of 2000 characters length,
|
||||
and is linear to RAMDirectory at 10,000 documents of 2000 characters length.
|
||||
</p>
|
||||
|
||||
<p>Mileage may vary depending on term saturation.</p>
|
||||
|
||||
<img src="doc-files/HitCollectionBench.jpg" alt="benchmark"/>
|
||||
|
||||
<p>
|
||||
Populated with a single document InstantiatedIndex is almost, but not quite, as fast as MemoryIndex.
|
||||
</p>
|
||||
|
||||
<p>
|
||||
It takes more or less the same time to populate an InstantiatedIndex
|
||||
as it takes to populate a RAMDirectory. Hardly any effort has been put
|
||||
in to optimizing the InstantiatedIndexWriter, only minimizing the amount
|
||||
of time needed to write-lock the index has been considered.
|
||||
</p>
|
||||
|
||||
<h2>Caveats</h2>
|
||||
<ul>
|
||||
<li>No locks! Consider using InstantiatedIndex as if it was immutable.</li>
|
||||
<li>No documents with fields containing readers.</li>
|
||||
<li>No field selection when retrieving documents, as all stored field are available in memory.</li>
|
||||
<li>Any document returned must cloned if they are to be touched.</li>
|
||||
<li>Norms array returned must not be touched.</li>
|
||||
</ul>
|
||||
|
||||
<h2>Use cases</h2>
|
||||
|
||||
<p>
|
||||
Could replace any small index that could do with greater response time.
|
||||
spell check a priori index,
|
||||
the index of new documents exposed to user search agent queries,
|
||||
to compile classifiers in machine learning environments, etc.
|
||||
</p>
|
||||
|
||||
<h2>Class diagram</h2>
|
||||
<a href="doc-files/classdiagram.png"><img width="640px" height="480px" src="doc-files/classdiagram.png" alt="class diagram"></a>
|
||||
<br/>
|
||||
<a href="doc-files/classdiagram.uxf">Diagram</a> rendered using <a href="http://umlet.com">UMLet</a> 7.1.
|
||||
</body>
|
||||
</html>
|
|
@ -1,26 +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.
|
||||
-->
|
||||
<html>
|
||||
<head>
|
||||
<title>
|
||||
instantiated
|
||||
</title>
|
||||
</head>
|
||||
<body>
|
||||
instantiated
|
||||
</body>
|
||||
</html>
|
|
@ -1,98 +0,0 @@
|
|||
/**
|
||||
* Copyright 2006 The Apache Software Foundation
|
||||
*
|
||||
* Licensed 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.store.instantiated;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.MultiFields;
|
||||
import org.apache.lucene.index.MultiNorms;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.search.TopDocs;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
||||
public class TestEmptyIndex extends LuceneTestCase {
|
||||
|
||||
public void testSearch() throws Exception {
|
||||
|
||||
InstantiatedIndex ii = new InstantiatedIndex();
|
||||
|
||||
IndexReader r = new InstantiatedIndexReader(ii);
|
||||
IndexSearcher s = newSearcher(r);
|
||||
|
||||
TopDocs td = s.search(new TermQuery(new Term("foo", "bar")), 1);
|
||||
|
||||
assertEquals(0, td.totalHits);
|
||||
|
||||
s.close();
|
||||
r.close();
|
||||
ii.close();
|
||||
|
||||
}
|
||||
|
||||
public void testNorms() throws Exception {
|
||||
|
||||
InstantiatedIndex ii = new InstantiatedIndex();
|
||||
IndexReader r = new InstantiatedIndexReader(ii);
|
||||
testNorms(r);
|
||||
r.close();
|
||||
ii.close();
|
||||
|
||||
// make sure a Directory acts the same
|
||||
Directory d = newDirectory();
|
||||
new IndexWriter(d, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random))).close();
|
||||
r = IndexReader.open(d, false);
|
||||
testNorms(r);
|
||||
r.close();
|
||||
d.close();
|
||||
|
||||
}
|
||||
|
||||
private void testNorms(IndexReader r) throws IOException {
|
||||
byte[] norms = MultiNorms.norms(r, "foo");
|
||||
if (norms != null) {
|
||||
assertEquals(0, norms.length);
|
||||
}
|
||||
}
|
||||
|
||||
public void testTermsEnum() throws Exception {
|
||||
|
||||
InstantiatedIndex ii = new InstantiatedIndex();
|
||||
IndexReader r = new InstantiatedIndexReader(ii);
|
||||
termsEnumTest(r);
|
||||
r.close();
|
||||
ii.close();
|
||||
|
||||
// make sure a Directory acts the same
|
||||
Directory d = newDirectory();
|
||||
new IndexWriter(d, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random))).close();
|
||||
r = IndexReader.open(d, false);
|
||||
termsEnumTest(r);
|
||||
r.close();
|
||||
d.close();
|
||||
}
|
||||
|
||||
public void termsEnumTest(IndexReader r) throws Exception {
|
||||
assertNull(MultiFields.getFields(r));
|
||||
}
|
||||
|
||||
}
|
|
@ -1,583 +0,0 @@
|
|||
package org.apache.lucene.store.instantiated;
|
||||
/**
|
||||
* Copyright 2006 The Apache Software Foundation
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.lucene.analysis.Token;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.MultiNorms;
|
||||
import org.apache.lucene.index.Payload;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.DocsEnum;
|
||||
import org.apache.lucene.index.DocsAndPositionsEnum;
|
||||
import org.apache.lucene.index.MultiFields;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.index.FieldsEnum;
|
||||
import org.apache.lucene.index.TermFreqVector;
|
||||
import org.apache.lucene.index.TermPositionVector;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.AttributeImpl;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
||||
/**
|
||||
* Asserts equality of content and behaviour of two index readers.
|
||||
*/
|
||||
public class TestIndicesEquals extends LuceneTestCase {
|
||||
|
||||
// public void test2() throws Exception {
|
||||
// FSDirectory fsdir = FSDirectory.open(new File("/tmp/fatcorpus"));
|
||||
// IndexReader ir = IndexReader.open(fsdir, false);
|
||||
// InstantiatedIndex ii = new InstantiatedIndex(ir);
|
||||
// ir.close();
|
||||
// testEquals(fsdir, ii);
|
||||
// }
|
||||
|
||||
|
||||
public void testLoadIndexReader() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
|
||||
// create dir data
|
||||
IndexWriter indexWriter = new IndexWriter(dir, newIndexWriterConfig(
|
||||
TEST_VERSION_CURRENT, new MockAnalyzer(random)).setMergePolicy(newLogMergePolicy()));
|
||||
|
||||
for (int i = 0; i < 20; i++) {
|
||||
Document document = new Document();
|
||||
assembleDocument(document, i);
|
||||
indexWriter.addDocument(document);
|
||||
}
|
||||
indexWriter.close();
|
||||
|
||||
// test load ii from index reader
|
||||
IndexReader ir = IndexReader.open(dir, false);
|
||||
InstantiatedIndex ii = new InstantiatedIndex(ir);
|
||||
ir.close();
|
||||
|
||||
testEqualBehaviour(dir, ii);
|
||||
dir.close();
|
||||
}
|
||||
|
||||
|
||||
public void testInstantiatedIndexWriter() throws Exception {
|
||||
|
||||
Directory dir = newDirectory();
|
||||
InstantiatedIndex ii = new InstantiatedIndex();
|
||||
|
||||
// we need to pass the "same" random to both, so they surely index the same payload data.
|
||||
long seed = random.nextLong();
|
||||
|
||||
// create dir data
|
||||
IndexWriter indexWriter = new IndexWriter(dir, newIndexWriterConfig(
|
||||
TEST_VERSION_CURRENT, new MockAnalyzer(new Random(seed))).setMergePolicy(newLogMergePolicy()));
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: make test index");
|
||||
}
|
||||
for (int i = 0; i < 500; i++) {
|
||||
Document document = new Document();
|
||||
assembleDocument(document, i);
|
||||
indexWriter.addDocument(document);
|
||||
}
|
||||
indexWriter.close();
|
||||
|
||||
// test ii writer
|
||||
InstantiatedIndexWriter instantiatedIndexWriter = ii.indexWriterFactory(new MockAnalyzer(new Random(seed)), true);
|
||||
for (int i = 0; i < 500; i++) {
|
||||
Document document = new Document();
|
||||
assembleDocument(document, i);
|
||||
instantiatedIndexWriter.addDocument(document);
|
||||
}
|
||||
instantiatedIndexWriter.close();
|
||||
|
||||
|
||||
testEqualBehaviour(dir, ii);
|
||||
|
||||
dir.close();
|
||||
|
||||
}
|
||||
|
||||
|
||||
private void testTermDocsSomeMore(Directory aprioriIndex, InstantiatedIndex testIndex) throws Exception {
|
||||
|
||||
IndexReader aprioriReader = IndexReader.open(aprioriIndex, false);
|
||||
IndexReader testReader = testIndex.indexReaderFactory();
|
||||
|
||||
// test seek
|
||||
|
||||
Term t = new Term("c", "danny");
|
||||
TermsEnum aprioriTermEnum = MultiFields.getTerms(aprioriReader, t.field()).iterator();
|
||||
aprioriTermEnum.seekCeil(new BytesRef(t.text()));
|
||||
TermsEnum testTermEnum = MultiFields.getTerms(testReader, t.field()).iterator();
|
||||
testTermEnum.seekCeil(new BytesRef(t.text()));
|
||||
assertEquals(aprioriTermEnum.term(), testTermEnum.term());
|
||||
|
||||
DocsEnum aprioriTermDocs = aprioriTermEnum.docs(MultiFields.getLiveDocs(aprioriReader), null);
|
||||
DocsEnum testTermDocs = testTermEnum.docs(MultiFields.getLiveDocs(testReader), null);
|
||||
|
||||
assertEquals(aprioriTermDocs.nextDoc(), testTermDocs.nextDoc());
|
||||
assertEquals(aprioriTermDocs.freq(), testTermDocs.freq());
|
||||
|
||||
if (aprioriTermDocs.advance(4) != DocsEnum.NO_MORE_DOCS) {
|
||||
assertTrue(testTermDocs.advance(4) != DocsEnum.NO_MORE_DOCS);
|
||||
assertEquals(aprioriTermDocs.freq(), testTermDocs.freq());
|
||||
assertEquals(aprioriTermDocs.docID(), testTermDocs.docID());
|
||||
} else {
|
||||
assertEquals(DocsEnum.NO_MORE_DOCS, testTermDocs.advance(4));
|
||||
}
|
||||
|
||||
if (aprioriTermDocs.nextDoc() != DocsEnum.NO_MORE_DOCS) {
|
||||
assertTrue(testTermDocs.nextDoc() != DocsEnum.NO_MORE_DOCS);
|
||||
assertEquals(aprioriTermDocs.freq(), testTermDocs.freq());
|
||||
assertEquals(aprioriTermDocs.docID(), testTermDocs.docID());
|
||||
} else {
|
||||
assertEquals(DocsEnum.NO_MORE_DOCS, testTermDocs.nextDoc());
|
||||
}
|
||||
|
||||
|
||||
// beyond this point all next and skipto will return false
|
||||
|
||||
if (aprioriTermDocs.advance(100) != DocsEnum.NO_MORE_DOCS) {
|
||||
assertTrue(testTermDocs.advance(100) != DocsEnum.NO_MORE_DOCS);
|
||||
assertEquals(aprioriTermDocs.freq(), testTermDocs.freq());
|
||||
assertEquals(aprioriTermDocs.docID(), testTermDocs.docID());
|
||||
} else {
|
||||
assertEquals(DocsEnum.NO_MORE_DOCS, testTermDocs.advance(100));
|
||||
}
|
||||
|
||||
// start using the API the way one is supposed to use it
|
||||
|
||||
t = new Term("", "");
|
||||
FieldsEnum apFieldsEnum = MultiFields.getFields(aprioriReader).iterator();
|
||||
String apFirstField = apFieldsEnum.next();
|
||||
|
||||
FieldsEnum testFieldsEnum = MultiFields.getFields(testReader).iterator();
|
||||
String testFirstField = testFieldsEnum.next();
|
||||
assertEquals(apFirstField, testFirstField);
|
||||
|
||||
aprioriTermEnum = apFieldsEnum.terms();
|
||||
testTermEnum = testFieldsEnum.terms();
|
||||
|
||||
assertEquals(aprioriTermEnum.next(), testTermEnum.next());
|
||||
|
||||
aprioriTermDocs = aprioriTermEnum.docs(MultiFields.getLiveDocs(aprioriReader), aprioriTermDocs);
|
||||
testTermDocs = testTermEnum.docs(MultiFields.getLiveDocs(testReader), testTermDocs);
|
||||
|
||||
while (aprioriTermDocs.nextDoc() != DocsEnum.NO_MORE_DOCS) {
|
||||
assertTrue(testTermDocs.nextDoc() != DocsEnum.NO_MORE_DOCS);
|
||||
assertEquals(aprioriTermDocs.freq(), testTermDocs.freq());
|
||||
assertEquals(aprioriTermDocs.docID(), testTermDocs.docID());
|
||||
}
|
||||
assertEquals(DocsEnum.NO_MORE_DOCS, testTermDocs.nextDoc());
|
||||
|
||||
// clean up
|
||||
aprioriReader.close();
|
||||
testReader.close();
|
||||
|
||||
}
|
||||
|
||||
|
||||
private void assembleDocument(Document document, int i) {
|
||||
FieldType customType = new FieldType(TextField.TYPE_STORED);
|
||||
customType.setStoreTermVectors(true);
|
||||
customType.setStoreTermVectorOffsets(true);
|
||||
customType.setStoreTermVectorPositions(true);
|
||||
//document.add(new Field("a", i + " Do you really want to go and live in that house all winter?", Field.Store.YES, Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
|
||||
document.add(new Field("a", i + " Do you really want to go and live in that house all winter?", customType));
|
||||
if (i > 0) {
|
||||
//document.add(new Field("b0", i + " All work and no play makes Jack a dull boy", Field.Store.YES, Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
|
||||
document.add(new Field("b0", i + " All work and no play makes Jack a dull boy", customType));
|
||||
|
||||
//document.add(new Field("b1", i + " All work and no play makes Jack a dull boy", Field.Store.YES, Field.Index.NOT_ANALYZED_NO_NORMS, Field.TermVector.NO));
|
||||
FieldType customType2 = new FieldType(TextField.TYPE_STORED);
|
||||
customType2.setTokenized(false);
|
||||
customType2.setOmitNorms(true);
|
||||
document.add(new Field("b1", i + " All work and no play makes Jack a dull boy", customType2));
|
||||
|
||||
//document.add(new Field("b2", i + " All work and no play makes Jack a dull boy", Field.Store.NO, Field.Index.NOT_ANALYZED, Field.TermVector.NO));
|
||||
FieldType customType3 = new FieldType(TextField.TYPE_UNSTORED);
|
||||
customType3.setTokenized(false);
|
||||
document.add(new Field("b1", i + " All work and no play makes Jack a dull boy", customType3));
|
||||
|
||||
//document.add(new Field("b3", i + " All work and no play makes Jack a dull boy", Field.Store.YES, Field.Index.NO, Field.TermVector.NO));
|
||||
FieldType customType4 = new FieldType(TextField.TYPE_STORED);
|
||||
customType4.setIndexed(false);
|
||||
customType4.setTokenized(false);
|
||||
document.add(new Field("b1", i + " All work and no play makes Jack a dull boy", customType4));
|
||||
if (i > 1) {
|
||||
//document.add(new Field("c", i + " Redrum redrum", Field.Store.YES, Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
|
||||
document.add(new Field("c", i + " Redrum redrum", customType));
|
||||
if (i > 2) {
|
||||
//document.add(new Field("d", i + " Hello Danny, come and play with us... forever and ever. and ever.", Field.Store.YES, Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS));
|
||||
document.add(new Field("d", i + " Hello Danny, come and play with us... forever and ever. and ever.", customType));
|
||||
if (i > 3) {
|
||||
//Field f = new Field("e", i + " Heres Johnny!", Field.Store.YES, Field.Index.ANALYZED, Field.TermVector.WITH_POSITIONS_OFFSETS);
|
||||
//f.setOmitNorms(true);
|
||||
FieldType customType5 = new FieldType(TextField.TYPE_UNSTORED);
|
||||
customType5.setOmitNorms(true);
|
||||
Field f = new Field("e", i + " Heres Johnny!", customType5);
|
||||
document.add(f);
|
||||
if (i > 4) {
|
||||
final List<Token> tokens = new ArrayList<Token>(2);
|
||||
Token t = createToken("the", 0, 2, "text");
|
||||
t.setPayload(new Payload(new byte[]{1, 2, 3}));
|
||||
tokens.add(t);
|
||||
t = createToken("end", 3, 5, "text");
|
||||
t.setPayload(new Payload(new byte[]{2}));
|
||||
tokens.add(t);
|
||||
tokens.add(createToken("fin", 7, 9));
|
||||
TokenStream ts = new TokenStream(Token.TOKEN_ATTRIBUTE_FACTORY) {
|
||||
final AttributeImpl reusableToken = (AttributeImpl) addAttribute(CharTermAttribute.class);
|
||||
Iterator<Token> it = tokens.iterator();
|
||||
|
||||
@Override
|
||||
public final boolean incrementToken() throws IOException {
|
||||
if (!it.hasNext()) {
|
||||
return false;
|
||||
}
|
||||
clearAttributes();
|
||||
it.next().copyTo(reusableToken);
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void reset() throws IOException {
|
||||
it = tokens.iterator();
|
||||
}
|
||||
};
|
||||
|
||||
//document.add(new Field("f", ts));
|
||||
document.add(new TextField("f", ts));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Asserts that the content of two index readers equal each other.
|
||||
*
|
||||
* @param aprioriIndex the index that is known to be correct
|
||||
* @param testIndex the index that is supposed to equals the apriori index.
|
||||
* @throws Exception
|
||||
*/
|
||||
protected void testEqualBehaviour(Directory aprioriIndex, InstantiatedIndex testIndex) throws Exception {
|
||||
|
||||
|
||||
testEquals(aprioriIndex, testIndex);
|
||||
|
||||
// delete a few documents
|
||||
IndexReader air = IndexReader.open(aprioriIndex, false);
|
||||
InstantiatedIndexReader tir = testIndex.indexReaderFactory();
|
||||
|
||||
assertEquals(air.isCurrent(), tir.isCurrent());
|
||||
assertEquals(air.hasDeletions(), tir.hasDeletions());
|
||||
assertEquals(air.maxDoc(), tir.maxDoc());
|
||||
assertEquals(air.numDocs(), tir.numDocs());
|
||||
assertEquals(air.numDeletedDocs(), tir.numDeletedDocs());
|
||||
|
||||
air.deleteDocument(3);
|
||||
tir.deleteDocument(3);
|
||||
|
||||
assertEquals(air.isCurrent(), tir.isCurrent());
|
||||
assertEquals(air.hasDeletions(), tir.hasDeletions());
|
||||
assertEquals(air.maxDoc(), tir.maxDoc());
|
||||
assertEquals(air.numDocs(), tir.numDocs());
|
||||
assertEquals(air.numDeletedDocs(), tir.numDeletedDocs());
|
||||
|
||||
air.deleteDocument(8);
|
||||
tir.deleteDocument(8);
|
||||
|
||||
assertEquals(air.isCurrent(), tir.isCurrent());
|
||||
assertEquals(air.hasDeletions(), tir.hasDeletions());
|
||||
assertEquals(air.maxDoc(), tir.maxDoc());
|
||||
assertEquals(air.numDocs(), tir.numDocs());
|
||||
assertEquals(air.numDeletedDocs(), tir.numDeletedDocs());
|
||||
|
||||
// this (in 3.0) commits the deletions
|
||||
air.close();
|
||||
tir.close();
|
||||
|
||||
air = IndexReader.open(aprioriIndex, false);
|
||||
tir = testIndex.indexReaderFactory();
|
||||
|
||||
assertEquals(air.isCurrent(), tir.isCurrent());
|
||||
assertEquals(air.hasDeletions(), tir.hasDeletions());
|
||||
assertEquals(air.maxDoc(), tir.maxDoc());
|
||||
assertEquals(air.numDocs(), tir.numDocs());
|
||||
assertEquals(air.numDeletedDocs(), tir.numDeletedDocs());
|
||||
|
||||
final Bits aLiveDocs = MultiFields.getLiveDocs(air);
|
||||
final Bits tLiveDocs = MultiFields.getLiveDocs(tir);
|
||||
assertTrue((aLiveDocs != null && tLiveDocs != null) ||
|
||||
(aLiveDocs == null && tLiveDocs == null));
|
||||
if (aLiveDocs != null) {
|
||||
for (int d =0; d<air.maxDoc(); d++) {
|
||||
assertEquals(aLiveDocs.get(d), tLiveDocs.get(d));
|
||||
}
|
||||
}
|
||||
|
||||
air.close();
|
||||
tir.close();
|
||||
|
||||
|
||||
// make sure they still equal
|
||||
testEquals(aprioriIndex, testIndex);
|
||||
}
|
||||
|
||||
protected void testEquals(Directory aprioriIndex, InstantiatedIndex testIndex) throws Exception {
|
||||
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: testEquals");
|
||||
}
|
||||
testTermDocsSomeMore(aprioriIndex, testIndex);
|
||||
|
||||
IndexReader aprioriReader = IndexReader.open(aprioriIndex, false);
|
||||
IndexReader testReader = testIndex.indexReaderFactory();
|
||||
|
||||
assertEquals(aprioriReader.numDocs(), testReader.numDocs());
|
||||
|
||||
// assert field options
|
||||
assertEquals(aprioriReader.getFieldNames(IndexReader.FieldOption.INDEXED), testReader.getFieldNames(IndexReader.FieldOption.INDEXED));
|
||||
assertEquals(aprioriReader.getFieldNames(IndexReader.FieldOption.INDEXED_NO_TERMVECTOR), testReader.getFieldNames(IndexReader.FieldOption.INDEXED_NO_TERMVECTOR));
|
||||
assertEquals(aprioriReader.getFieldNames(IndexReader.FieldOption.INDEXED_WITH_TERMVECTOR), testReader.getFieldNames(IndexReader.FieldOption.INDEXED_WITH_TERMVECTOR));
|
||||
assertEquals(aprioriReader.getFieldNames(IndexReader.FieldOption.STORES_PAYLOADS), testReader.getFieldNames(IndexReader.FieldOption.STORES_PAYLOADS));
|
||||
assertEquals(aprioriReader.getFieldNames(IndexReader.FieldOption.TERMVECTOR), testReader.getFieldNames(IndexReader.FieldOption.TERMVECTOR));
|
||||
assertEquals(aprioriReader.getFieldNames(IndexReader.FieldOption.TERMVECTOR_WITH_OFFSET), testReader.getFieldNames(IndexReader.FieldOption.TERMVECTOR_WITH_OFFSET));
|
||||
assertEquals(aprioriReader.getFieldNames(IndexReader.FieldOption.TERMVECTOR_WITH_POSITION), testReader.getFieldNames(IndexReader.FieldOption.TERMVECTOR_WITH_POSITION));
|
||||
assertEquals(aprioriReader.getFieldNames(IndexReader.FieldOption.TERMVECTOR_WITH_POSITION_OFFSET), testReader.getFieldNames(IndexReader.FieldOption.TERMVECTOR_WITH_POSITION_OFFSET));
|
||||
assertEquals(aprioriReader.getFieldNames(IndexReader.FieldOption.UNINDEXED), testReader.getFieldNames(IndexReader.FieldOption.UNINDEXED));
|
||||
|
||||
for (Object field : aprioriReader.getFieldNames(IndexReader.FieldOption.ALL)) {
|
||||
|
||||
// test norms as used by normal use
|
||||
|
||||
byte[] aprioriNorms = MultiNorms.norms(aprioriReader, (String) field);
|
||||
byte[] testNorms = MultiNorms.norms(testReader, (String) field);
|
||||
|
||||
if (aprioriNorms != null) {
|
||||
assertEquals(aprioriNorms.length, testNorms.length);
|
||||
|
||||
for (int i = 0; i < aprioriNorms.length; i++) {
|
||||
assertEquals("norms does not equals for field " + field + " in document " + i, aprioriNorms[i], testNorms[i]);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
final Bits apLiveDocs = MultiFields.getLiveDocs(aprioriReader);
|
||||
final Bits testLiveDocs = MultiFields.getLiveDocs(testReader);
|
||||
assertTrue((apLiveDocs != null && testLiveDocs != null) ||
|
||||
(apLiveDocs == null && testLiveDocs == null));
|
||||
if (apLiveDocs != null) {
|
||||
for (int docIndex = 0; docIndex < aprioriReader.numDocs(); docIndex++) {
|
||||
assertEquals(apLiveDocs.get(docIndex), testLiveDocs.get(docIndex));
|
||||
}
|
||||
}
|
||||
|
||||
// compare term enumeration stepping
|
||||
|
||||
FieldsEnum aprioriFieldsEnum = MultiFields.getFields(aprioriReader).iterator();
|
||||
FieldsEnum testFieldsEnum = MultiFields.getFields(testReader).iterator();
|
||||
|
||||
String aprioriField;
|
||||
while((aprioriField = aprioriFieldsEnum.next()) != null) {
|
||||
String testField = testFieldsEnum.next();
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: verify field=" + testField);
|
||||
}
|
||||
assertEquals(aprioriField, testField);
|
||||
|
||||
TermsEnum aprioriTermEnum = aprioriFieldsEnum.terms();
|
||||
TermsEnum testTermEnum = testFieldsEnum.terms();
|
||||
|
||||
BytesRef aprioriText;
|
||||
while((aprioriText = aprioriTermEnum.next()) != null) {
|
||||
assertEquals(aprioriText, testTermEnum.next());
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: verify term=" + aprioriText.utf8ToString());
|
||||
}
|
||||
|
||||
assertTrue(aprioriTermEnum.docFreq() == testTermEnum.docFreq());
|
||||
final long totalTermFreq = aprioriTermEnum.totalTermFreq();
|
||||
if (totalTermFreq != -1) {
|
||||
assertEquals(totalTermFreq, testTermEnum.totalTermFreq());
|
||||
}
|
||||
|
||||
// compare termDocs seeking
|
||||
|
||||
DocsEnum aprioriTermDocs = aprioriTermEnum.docs(MultiFields.getLiveDocs(aprioriReader), null);
|
||||
DocsEnum testTermDocs = testTermEnum.docs(MultiFields.getLiveDocs(testReader), null);
|
||||
|
||||
while (aprioriTermDocs.nextDoc() != DocsEnum.NO_MORE_DOCS) {
|
||||
assertTrue(testTermDocs.advance(aprioriTermDocs.docID()) != DocsEnum.NO_MORE_DOCS);
|
||||
assertEquals(aprioriTermDocs.docID(), testTermDocs.docID());
|
||||
}
|
||||
|
||||
// compare documents per term
|
||||
|
||||
assertEquals(aprioriReader.docFreq(aprioriField, aprioriTermEnum.term()), testReader.docFreq(aprioriField, testTermEnum.term()));
|
||||
|
||||
aprioriTermDocs = aprioriTermEnum.docs(MultiFields.getLiveDocs(aprioriReader), aprioriTermDocs);
|
||||
testTermDocs = testTermEnum.docs(MultiFields.getLiveDocs(testReader), testTermDocs);
|
||||
|
||||
while (true) {
|
||||
if (aprioriTermDocs.nextDoc() == DocsEnum.NO_MORE_DOCS) {
|
||||
assertEquals(DocsEnum.NO_MORE_DOCS, testTermDocs.nextDoc());
|
||||
break;
|
||||
}
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: verify doc=" + aprioriTermDocs.docID());
|
||||
}
|
||||
|
||||
assertTrue(testTermDocs.nextDoc() != DocsEnum.NO_MORE_DOCS);
|
||||
|
||||
assertEquals(aprioriTermDocs.docID(), testTermDocs.docID());
|
||||
assertEquals(aprioriTermDocs.freq(), testTermDocs.freq());
|
||||
}
|
||||
|
||||
// compare term positions
|
||||
|
||||
DocsAndPositionsEnum aprioriTermPositions = aprioriTermEnum.docsAndPositions(MultiFields.getLiveDocs(aprioriReader), null);
|
||||
DocsAndPositionsEnum testTermPositions = testTermEnum.docsAndPositions(MultiFields.getLiveDocs(testReader), null);
|
||||
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: enum1=" + aprioriTermPositions + " enum2=" + testTermPositions);
|
||||
}
|
||||
if (aprioriTermPositions != null) {
|
||||
|
||||
for (int docIndex = 0; docIndex < aprioriReader.maxDoc(); docIndex++) {
|
||||
boolean hasNext = aprioriTermPositions.nextDoc() != DocsEnum.NO_MORE_DOCS;
|
||||
if (hasNext) {
|
||||
assertTrue(testTermPositions.nextDoc() != DocsEnum.NO_MORE_DOCS);
|
||||
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: verify doc=" + aprioriTermPositions.docID());
|
||||
}
|
||||
|
||||
assertEquals(aprioriTermPositions.freq(), testTermPositions.freq());
|
||||
|
||||
for (int termPositionIndex = 0; termPositionIndex < aprioriTermPositions.freq(); termPositionIndex++) {
|
||||
int aprioriPos = aprioriTermPositions.nextPosition();
|
||||
int testPos = testTermPositions.nextPosition();
|
||||
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: verify pos=" + aprioriPos);
|
||||
}
|
||||
|
||||
assertEquals(aprioriPos, testPos);
|
||||
|
||||
assertEquals(aprioriTermPositions.hasPayload(), testTermPositions.hasPayload());
|
||||
if (aprioriTermPositions.hasPayload()) {
|
||||
BytesRef apPayload = aprioriTermPositions.getPayload();
|
||||
BytesRef testPayload = testTermPositions.getPayload();
|
||||
assertEquals(apPayload, testPayload);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
assertNull(testTermEnum.next());
|
||||
}
|
||||
assertNull(testFieldsEnum.next());
|
||||
|
||||
// compare term vectors and position vectors
|
||||
|
||||
for (int documentNumber = 0; documentNumber < aprioriReader.numDocs(); documentNumber++) {
|
||||
|
||||
if (documentNumber > 0) {
|
||||
assertNotNull(aprioriReader.getTermFreqVector(documentNumber, "b0"));
|
||||
assertNull(aprioriReader.getTermFreqVector(documentNumber, "b1"));
|
||||
|
||||
assertNotNull(testReader.getTermFreqVector(documentNumber, "b0"));
|
||||
assertNull(testReader.getTermFreqVector(documentNumber, "b1"));
|
||||
|
||||
}
|
||||
|
||||
TermFreqVector[] aprioriFreqVectors = aprioriReader.getTermFreqVectors(documentNumber);
|
||||
TermFreqVector[] testFreqVectors = testReader.getTermFreqVectors(documentNumber);
|
||||
|
||||
if (aprioriFreqVectors != null && testFreqVectors != null) {
|
||||
|
||||
Arrays.sort(aprioriFreqVectors, new Comparator<TermFreqVector>() {
|
||||
public int compare(TermFreqVector termFreqVector, TermFreqVector termFreqVector1) {
|
||||
return termFreqVector.getField().compareTo(termFreqVector1.getField());
|
||||
}
|
||||
});
|
||||
Arrays.sort(testFreqVectors, new Comparator<TermFreqVector>() {
|
||||
public int compare(TermFreqVector termFreqVector, TermFreqVector termFreqVector1) {
|
||||
return termFreqVector.getField().compareTo(termFreqVector1.getField());
|
||||
}
|
||||
});
|
||||
|
||||
assertEquals("document " + documentNumber + " vectors does not match", aprioriFreqVectors.length, testFreqVectors.length);
|
||||
|
||||
for (int freqVectorIndex = 0; freqVectorIndex < aprioriFreqVectors.length; freqVectorIndex++) {
|
||||
assertTrue(Arrays.equals(aprioriFreqVectors[freqVectorIndex].getTermFrequencies(), testFreqVectors[freqVectorIndex].getTermFrequencies()));
|
||||
assertTrue(Arrays.equals(aprioriFreqVectors[freqVectorIndex].getTerms(), testFreqVectors[freqVectorIndex].getTerms()));
|
||||
|
||||
if (aprioriFreqVectors[freqVectorIndex] instanceof TermPositionVector) {
|
||||
TermPositionVector aprioriTermPositionVector = (TermPositionVector) aprioriFreqVectors[freqVectorIndex];
|
||||
TermPositionVector testTermPositionVector = (TermPositionVector) testFreqVectors[freqVectorIndex];
|
||||
|
||||
for (int positionVectorIndex = 0; positionVectorIndex < aprioriFreqVectors[freqVectorIndex].getTerms().length; positionVectorIndex++)
|
||||
{
|
||||
if (aprioriTermPositionVector.getOffsets(positionVectorIndex) != null) {
|
||||
assertTrue(Arrays.equals(aprioriTermPositionVector.getOffsets(positionVectorIndex), testTermPositionVector.getOffsets(positionVectorIndex)));
|
||||
}
|
||||
|
||||
if (aprioriTermPositionVector.getTermPositions(positionVectorIndex) != null) {
|
||||
assertTrue(Arrays.equals(aprioriTermPositionVector.getTermPositions(positionVectorIndex), testTermPositionVector.getTermPositions(positionVectorIndex)));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
aprioriReader.close();
|
||||
testReader.close();
|
||||
}
|
||||
|
||||
private static Token createToken(String term, int start, int offset)
|
||||
{
|
||||
return new Token(term, start, offset);
|
||||
}
|
||||
|
||||
private static Token createToken(String term, int start, int offset, String type)
|
||||
{
|
||||
return new Token(term, start, offset, type);
|
||||
}
|
||||
|
||||
|
||||
}
|
|
@ -1,72 +0,0 @@
|
|||
package org.apache.lucene.store.instantiated;
|
||||
|
||||
/*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.TextField;
|
||||
|
||||
/**
|
||||
* @since 2009-mar-30 13:15:49
|
||||
*/
|
||||
public class TestMultiSegmentReaderOnConstructor extends LuceneTestCase {
|
||||
|
||||
public void test() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)));
|
||||
addDocument(iw, "Hello, world!");
|
||||
addDocument(iw, "All work and no play makes jack a dull boy");
|
||||
iw.close();
|
||||
|
||||
iw = new IndexWriter(dir, new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setOpenMode(OpenMode.APPEND));
|
||||
addDocument(iw, "Hello, tellus!");
|
||||
addDocument(iw, "All work and no play makes danny a dull boy");
|
||||
iw.close();
|
||||
|
||||
iw = new IndexWriter(dir, new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).setOpenMode(OpenMode.APPEND));
|
||||
addDocument(iw, "Hello, earth!");
|
||||
addDocument(iw, "All work and no play makes wendy a dull girl");
|
||||
iw.close();
|
||||
|
||||
IndexReader multiSegReader = IndexReader.open(dir, false);
|
||||
multiSegReader.deleteDocument(2);
|
||||
|
||||
try {
|
||||
new InstantiatedIndex(multiSegReader);
|
||||
} catch (Exception e) {
|
||||
e.printStackTrace(System.out);
|
||||
fail("No exceptions when loading a multi-seg reader!");
|
||||
}
|
||||
|
||||
// todo some assertations.
|
||||
multiSegReader.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
private void addDocument(IndexWriter iw, String text) throws IOException {
|
||||
Document doc = new Document();
|
||||
doc.add(new TextField("field", text));
|
||||
iw.addDocument(doc);
|
||||
}
|
||||
}
|
|
@ -1,79 +0,0 @@
|
|||
/**
|
||||
* Copyright 2006 The Apache Software Foundation
|
||||
*
|
||||
* Licensed 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.store.instantiated;
|
||||
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.StringField;
|
||||
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
||||
/**
|
||||
* Assert that the content of an index
|
||||
* is instantly available
|
||||
* for all open searchers
|
||||
* also after a commit.
|
||||
*/
|
||||
public class TestRealTime extends LuceneTestCase {
|
||||
|
||||
public void test() throws Exception {
|
||||
|
||||
InstantiatedIndex index = new InstantiatedIndex();
|
||||
InstantiatedIndexReader reader = new InstantiatedIndexReader(index);
|
||||
IndexSearcher searcher = newSearcher(reader, false);
|
||||
InstantiatedIndexWriter writer = new InstantiatedIndexWriter(index);
|
||||
|
||||
Document doc;
|
||||
Collector collector;
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new StringField("f", "a"));
|
||||
writer.addDocument(doc);
|
||||
writer.commit();
|
||||
|
||||
collector = new Collector();
|
||||
searcher.search(new TermQuery(new Term("f", "a")), collector);
|
||||
assertEquals(1, collector.hits);
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new StringField("f", "a"));
|
||||
writer.addDocument(doc);
|
||||
writer.commit();
|
||||
|
||||
collector = new Collector();
|
||||
searcher.search(new TermQuery(new Term("f", "a")), collector);
|
||||
assertEquals(2, collector.hits);
|
||||
|
||||
}
|
||||
|
||||
public static class Collector extends org.apache.lucene.search.Collector {
|
||||
private int hits = 0;
|
||||
@Override
|
||||
public void setScorer(Scorer scorer) {}
|
||||
@Override
|
||||
public void setNextReader(AtomicReaderContext context) {}
|
||||
@Override
|
||||
public boolean acceptsDocsOutOfOrder() { return true; }
|
||||
@Override
|
||||
public void collect(int doc) {
|
||||
hits++;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -45,10 +45,7 @@ import org.apache.lucene.index.IndexReader;
|
|||
import org.apache.lucene.index.OrdTermState;
|
||||
import org.apache.lucene.index.StoredFieldVisitor;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.TermFreqVector;
|
||||
import org.apache.lucene.index.TermPositionVector;
|
||||
import org.apache.lucene.index.TermState;
|
||||
import org.apache.lucene.index.TermVectorMapper;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
|
@ -782,80 +779,85 @@ public class MemoryIndex {
|
|||
public ReaderContext getTopReaderContext() {
|
||||
return readerInfos;
|
||||
}
|
||||
|
||||
private class MemoryFields extends Fields {
|
||||
@Override
|
||||
public FieldsEnum iterator() {
|
||||
return new FieldsEnum() {
|
||||
int upto = -1;
|
||||
|
||||
@Override
|
||||
public String next() {
|
||||
upto++;
|
||||
if (upto >= sortedFields.length) {
|
||||
return null;
|
||||
}
|
||||
return sortedFields[upto].getKey();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Terms terms() {
|
||||
return MemoryFields.this.terms(sortedFields[upto].getKey());
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Terms terms(final String field) {
|
||||
int i = Arrays.binarySearch(sortedFields, field, termComparator);
|
||||
if (i < 0) {
|
||||
return null;
|
||||
} else {
|
||||
final Info info = getInfo(i);
|
||||
info.sortTerms();
|
||||
|
||||
return new Terms() {
|
||||
@Override
|
||||
public TermsEnum iterator(TermsEnum reuse) {
|
||||
return new MemoryTermsEnum(info);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Comparator<BytesRef> getComparator() {
|
||||
return BytesRef.getUTF8SortedAsUnicodeComparator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getUniqueTermCount() {
|
||||
return info.sortedTerms.length;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getSumTotalTermFreq() {
|
||||
return info.getSumTotalTermFreq();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getSumDocFreq() throws IOException {
|
||||
// each term has df=1
|
||||
return info.sortedTerms.length;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getDocCount() throws IOException {
|
||||
return info.sortedTerms.length > 0 ? 1 : 0;
|
||||
}
|
||||
|
||||
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getUniqueFieldCount() {
|
||||
return sortedFields.length;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Fields fields() {
|
||||
|
||||
sortFields();
|
||||
|
||||
return new Fields() {
|
||||
@Override
|
||||
public FieldsEnum iterator() {
|
||||
return new FieldsEnum() {
|
||||
int upto = -1;
|
||||
|
||||
@Override
|
||||
public String next() {
|
||||
upto++;
|
||||
if (upto >= sortedFields.length) {
|
||||
return null;
|
||||
}
|
||||
return sortedFields[upto].getKey();
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermsEnum terms() {
|
||||
return new MemoryTermsEnum(sortedFields[upto].getValue());
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Terms terms(final String field) {
|
||||
int i = Arrays.binarySearch(sortedFields, field, termComparator);
|
||||
if (i < 0) {
|
||||
return null;
|
||||
} else {
|
||||
final Info info = getInfo(i);
|
||||
info.sortTerms();
|
||||
|
||||
return new Terms() {
|
||||
@Override
|
||||
public TermsEnum iterator() {
|
||||
return new MemoryTermsEnum(info);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Comparator<BytesRef> getComparator() {
|
||||
return BytesRef.getUTF8SortedAsUnicodeComparator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getUniqueTermCount() {
|
||||
return info.sortedTerms.length;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getSumTotalTermFreq() {
|
||||
return info.getSumTotalTermFreq();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getSumDocFreq() throws IOException {
|
||||
// each term has df=1
|
||||
return info.sortedTerms.length;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getDocCount() throws IOException {
|
||||
return info.sortedTerms.length > 0 ? 1 : 0;
|
||||
}
|
||||
|
||||
|
||||
};
|
||||
}
|
||||
}
|
||||
};
|
||||
return new MemoryFields();
|
||||
}
|
||||
|
||||
private class MemoryTermsEnum extends TermsEnum {
|
||||
|
@ -1062,128 +1064,12 @@ public class MemoryIndex {
|
|||
}
|
||||
|
||||
@Override
|
||||
public TermFreqVector[] getTermFreqVectors(int docNumber) {
|
||||
if (DEBUG) System.err.println("MemoryIndexReader.getTermFreqVectors");
|
||||
TermFreqVector[] vectors = new TermFreqVector[fields.size()];
|
||||
// if (vectors.length == 0) return null;
|
||||
Iterator<String> iter = fields.keySet().iterator();
|
||||
for (int i=0; i < vectors.length; i++) {
|
||||
vectors[i] = getTermFreqVector(docNumber, iter.next());
|
||||
public Fields getTermVectors(int docID) {
|
||||
if (docID == 0) {
|
||||
return fields();
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
return vectors;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getTermFreqVector(int docNumber, TermVectorMapper mapper) throws IOException
|
||||
{
|
||||
if (DEBUG) System.err.println("MemoryIndexReader.getTermFreqVectors");
|
||||
|
||||
// if (vectors.length == 0) return null;
|
||||
for (final String fieldName : fields.keySet())
|
||||
{
|
||||
getTermFreqVector(docNumber, fieldName, mapper);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getTermFreqVector(int docNumber, String field, TermVectorMapper mapper) throws IOException
|
||||
{
|
||||
if (DEBUG) System.err.println("MemoryIndexReader.getTermFreqVector");
|
||||
final Info info = getInfo(field);
|
||||
if (info == null){
|
||||
return;
|
||||
}
|
||||
info.sortTerms();
|
||||
mapper.setExpectations(field, info.sortedTerms.length, stride != 1, true);
|
||||
for (int i = info.sortedTerms.length; --i >=0;){
|
||||
|
||||
ArrayIntList positions = info.sortedTerms[i].getValue();
|
||||
int size = positions.size();
|
||||
org.apache.lucene.index.TermVectorOffsetInfo[] offsets =
|
||||
new org.apache.lucene.index.TermVectorOffsetInfo[size / stride];
|
||||
|
||||
for (int k=0, j=1; j < size; k++, j += stride) {
|
||||
int start = positions.get(j);
|
||||
int end = positions.get(j+1);
|
||||
offsets[k] = new org.apache.lucene.index.TermVectorOffsetInfo(start, end);
|
||||
}
|
||||
mapper.map(info.sortedTerms[i].getKey(),
|
||||
numPositions(info.sortedTerms[i].getValue()),
|
||||
offsets, (info.sortedTerms[i].getValue()).toArray(stride));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermFreqVector getTermFreqVector(int docNumber, final String fieldName) {
|
||||
if (DEBUG) System.err.println("MemoryIndexReader.getTermFreqVector");
|
||||
final Info info = getInfo(fieldName);
|
||||
if (info == null) return null; // TODO: or return empty vector impl???
|
||||
info.sortTerms();
|
||||
|
||||
return new TermPositionVector() {
|
||||
|
||||
private final Map.Entry<BytesRef,ArrayIntList>[] sortedTerms = info.sortedTerms;
|
||||
|
||||
public String getField() {
|
||||
return fieldName;
|
||||
}
|
||||
|
||||
public int size() {
|
||||
return sortedTerms.length;
|
||||
}
|
||||
|
||||
public BytesRef[] getTerms() {
|
||||
BytesRef[] terms = new BytesRef[sortedTerms.length];
|
||||
for (int i=sortedTerms.length; --i >= 0; ) {
|
||||
terms[i] = sortedTerms[i].getKey();
|
||||
}
|
||||
return terms;
|
||||
}
|
||||
|
||||
public int[] getTermFrequencies() {
|
||||
int[] freqs = new int[sortedTerms.length];
|
||||
for (int i=sortedTerms.length; --i >= 0; ) {
|
||||
freqs[i] = numPositions(sortedTerms[i].getValue());
|
||||
}
|
||||
return freqs;
|
||||
}
|
||||
|
||||
public int indexOf(BytesRef term) {
|
||||
int i = Arrays.binarySearch(sortedTerms, term, termComparator);
|
||||
return i >= 0 ? i : -1;
|
||||
}
|
||||
|
||||
public int[] indexesOf(BytesRef[] terms, int start, int len) {
|
||||
int[] indexes = new int[len];
|
||||
for (int i=0; i < len; i++) {
|
||||
indexes[i] = indexOf(terms[start++]);
|
||||
}
|
||||
return indexes;
|
||||
}
|
||||
|
||||
// lucene >= 1.4.3
|
||||
public int[] getTermPositions(int index) {
|
||||
return sortedTerms[index].getValue().toArray(stride);
|
||||
}
|
||||
|
||||
// lucene >= 1.9 (remove this method for lucene-1.4.3)
|
||||
public org.apache.lucene.index.TermVectorOffsetInfo[] getOffsets(int index) {
|
||||
if (stride == 1) return null; // no offsets stored
|
||||
|
||||
ArrayIntList positions = sortedTerms[index].getValue();
|
||||
int size = positions.size();
|
||||
org.apache.lucene.index.TermVectorOffsetInfo[] offsets =
|
||||
new org.apache.lucene.index.TermVectorOffsetInfo[size / stride];
|
||||
|
||||
for (int i=0, j=1; j < size; i++, j += stride) {
|
||||
int start = positions.get(j);
|
||||
int end = positions.get(j+1);
|
||||
offsets[i] = new org.apache.lucene.index.TermVectorOffsetInfo(start, end);
|
||||
}
|
||||
return offsets;
|
||||
}
|
||||
|
||||
};
|
||||
}
|
||||
|
||||
private SimilarityProvider getSimilarityProvider() {
|
||||
|
|
|
@ -57,7 +57,7 @@ public class BalancedSegmentMergePolicy extends LogByteSizeMergePolicy {
|
|||
|
||||
@Override
|
||||
protected long size(SegmentInfo info) throws IOException {
|
||||
long byteSize = info.sizeInBytes(true);
|
||||
long byteSize = info.sizeInBytes();
|
||||
float delRatio = (info.docCount <= 0 ? 0.0f : ((float)info.getDelCount() / (float)info.docCount));
|
||||
return (info.docCount <= 0 ? byteSize : (long)((1.0f - delRatio) * byteSize));
|
||||
}
|
||||
|
|
|
@ -127,7 +127,7 @@ public class FieldNormModifier {
|
|||
if (fields != null) {
|
||||
Terms terms = fields.terms(field);
|
||||
if (terms != null) {
|
||||
TermsEnum termsEnum = terms.iterator();
|
||||
TermsEnum termsEnum = terms.iterator(null);
|
||||
DocsEnum docs = null;
|
||||
while(termsEnum.next() != null) {
|
||||
docs = termsEnum.docs(liveDocs, docs);
|
||||
|
|
|
@ -105,7 +105,7 @@ public class IndexSplitter {
|
|||
DecimalFormat formatter = new DecimalFormat("###,###.###");
|
||||
for (int x = 0; x < infos.size(); x++) {
|
||||
SegmentInfo info = infos.info(x);
|
||||
String sizeStr = formatter.format(info.sizeInBytes(true));
|
||||
String sizeStr = formatter.format(info.sizeInBytes());
|
||||
System.out.println(info.name + " " + sizeStr);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,182 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*
|
||||
*/
|
||||
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
|
||||
/**
|
||||
* Transparent access to the vector space model,
|
||||
* either via TermFreqVector or by resolving it from the inverted index.
|
||||
* <p/>
|
||||
* Resolving a term vector from a large index can be a time consuming process.
|
||||
* <p/>
|
||||
* Warning! This class is not thread safe!
|
||||
*/
|
||||
public class TermVectorAccessor {
|
||||
|
||||
public TermVectorAccessor() {
|
||||
}
|
||||
|
||||
/**
|
||||
* Instance reused to save garbage collector some time
|
||||
*/
|
||||
private TermVectorMapperDecorator decoratedMapper = new TermVectorMapperDecorator();
|
||||
|
||||
|
||||
/**
|
||||
* Visits the TermVectorMapper and populates it with terms available for a given document,
|
||||
* either via a vector created at index time or by resolving them from the inverted index.
|
||||
*
|
||||
* @param indexReader Index source
|
||||
* @param documentNumber Source document to access
|
||||
* @param fieldName Field to resolve
|
||||
* @param mapper Mapper to be mapped with data
|
||||
* @throws IOException
|
||||
*/
|
||||
public void accept(IndexReader indexReader, int documentNumber, String fieldName, TermVectorMapper mapper) throws IOException {
|
||||
decoratedMapper.decorated = mapper;
|
||||
decoratedMapper.termVectorStored = false;
|
||||
|
||||
indexReader.getTermFreqVector(documentNumber, fieldName, decoratedMapper);
|
||||
|
||||
if (!decoratedMapper.termVectorStored) {
|
||||
mapper.setDocumentNumber(documentNumber);
|
||||
build(indexReader, fieldName, mapper, documentNumber);
|
||||
}
|
||||
}
|
||||
|
||||
/** Instance reused to save garbage collector some time */
|
||||
private List<BytesRef> tokens;
|
||||
|
||||
/** Instance reused to save garbage collector some time */
|
||||
private List<int[]> positions;
|
||||
|
||||
/** Instance reused to save garbage collector some time */
|
||||
private List<Integer> frequencies;
|
||||
|
||||
|
||||
/**
|
||||
* Populates the mapper with terms available for the given field in a document
|
||||
* by resolving the inverted index.
|
||||
*
|
||||
* @param indexReader
|
||||
* @param field interned field name
|
||||
* @param mapper
|
||||
* @param documentNumber
|
||||
* @throws IOException
|
||||
*/
|
||||
private void build(IndexReader indexReader, String field, TermVectorMapper mapper, int documentNumber) throws IOException {
|
||||
|
||||
if (tokens == null) {
|
||||
tokens = new ArrayList<BytesRef>(500);
|
||||
positions = new ArrayList<int[]>(500);
|
||||
frequencies = new ArrayList<Integer>(500);
|
||||
} else {
|
||||
tokens.clear();
|
||||
frequencies.clear();
|
||||
positions.clear();
|
||||
}
|
||||
|
||||
final Bits liveDocs = MultiFields.getLiveDocs(indexReader);
|
||||
|
||||
Terms terms = MultiFields.getTerms(indexReader, field);
|
||||
boolean anyTerms = false;
|
||||
if (terms != null) {
|
||||
TermsEnum termsEnum = terms.iterator();
|
||||
DocsEnum docs = null;
|
||||
DocsAndPositionsEnum postings = null;
|
||||
while(true) {
|
||||
BytesRef text = termsEnum.next();
|
||||
if (text != null) {
|
||||
anyTerms = true;
|
||||
if (!mapper.isIgnoringPositions()) {
|
||||
docs = postings = termsEnum.docsAndPositions(liveDocs, postings);
|
||||
} else {
|
||||
docs = termsEnum.docs(liveDocs, docs);
|
||||
}
|
||||
|
||||
int docID = docs.advance(documentNumber);
|
||||
if (docID == documentNumber) {
|
||||
|
||||
frequencies.add(Integer.valueOf(docs.freq()));
|
||||
tokens.add(new BytesRef(text));
|
||||
|
||||
if (!mapper.isIgnoringPositions()) {
|
||||
int[] positions = new int[docs.freq()];
|
||||
for (int i = 0; i < positions.length; i++) {
|
||||
positions[i] = postings.nextPosition();
|
||||
}
|
||||
this.positions.add(positions);
|
||||
} else {
|
||||
positions.add(null);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (anyTerms) {
|
||||
mapper.setDocumentNumber(documentNumber);
|
||||
mapper.setExpectations(field, tokens.size(), false, !mapper.isIgnoringPositions());
|
||||
for (int i = 0; i < tokens.size(); i++) {
|
||||
mapper.map(tokens.get(i), frequencies.get(i).intValue(), (TermVectorOffsetInfo[]) null, positions.get(i));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private static class TermVectorMapperDecorator extends TermVectorMapper {
|
||||
|
||||
private TermVectorMapper decorated;
|
||||
|
||||
@Override
|
||||
public boolean isIgnoringPositions() {
|
||||
return decorated.isIgnoringPositions();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isIgnoringOffsets() {
|
||||
return decorated.isIgnoringOffsets();
|
||||
}
|
||||
|
||||
private boolean termVectorStored = false;
|
||||
|
||||
@Override
|
||||
public void setExpectations(String field, int numTerms, boolean storeOffsets, boolean storePositions) {
|
||||
decorated.setExpectations(field, numTerms, storeOffsets, storePositions);
|
||||
termVectorStored = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void map(BytesRef term, int frequency, TermVectorOffsetInfo[] offsets, int[] positions) {
|
||||
decorated.map(term, frequency, offsets, positions);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setDocumentNumber(int documentNumber) {
|
||||
decorated.setDocumentNumber(documentNumber);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -19,11 +19,15 @@ package org.apache.lucene.index.codecs.appending;
|
|||
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.DefaultDocValuesFormat;
|
||||
import org.apache.lucene.index.codecs.DefaultFieldInfosFormat;
|
||||
import org.apache.lucene.index.codecs.DefaultStoredFieldsFormat;
|
||||
import org.apache.lucene.index.codecs.DefaultTermVectorsFormat;
|
||||
import org.apache.lucene.index.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.index.codecs.FieldInfosFormat;
|
||||
import org.apache.lucene.index.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.index.codecs.PostingsFormat;
|
||||
import org.apache.lucene.index.codecs.SegmentInfosFormat;
|
||||
import org.apache.lucene.index.codecs.TermVectorsFormat;
|
||||
import org.apache.lucene.index.codecs.lucene40.Lucene40Codec;
|
||||
|
||||
/**
|
||||
|
@ -45,6 +49,8 @@ public class AppendingCodec extends Codec {
|
|||
private final PostingsFormat postings = new AppendingPostingsFormat();
|
||||
private final SegmentInfosFormat infos = new AppendingSegmentInfosFormat();
|
||||
private final StoredFieldsFormat fields = new DefaultStoredFieldsFormat();
|
||||
private final FieldInfosFormat fieldInfos = new DefaultFieldInfosFormat();
|
||||
private final TermVectorsFormat vectors = new DefaultTermVectorsFormat();
|
||||
private final DocValuesFormat docValues = new DefaultDocValuesFormat();
|
||||
|
||||
@Override
|
||||
|
@ -56,6 +62,11 @@ public class AppendingCodec extends Codec {
|
|||
public StoredFieldsFormat storedFieldsFormat() {
|
||||
return fields;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermVectorsFormat termVectorsFormat() {
|
||||
return vectors;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesFormat docValuesFormat() {
|
||||
|
@ -66,4 +77,9 @@ public class AppendingCodec extends Codec {
|
|||
public SegmentInfosFormat segmentInfosFormat() {
|
||||
return infos;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldInfosFormat fieldInfosFormat() {
|
||||
return fieldInfos;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -117,7 +117,7 @@ public class HighFreqTerms {
|
|||
}
|
||||
Terms terms = fields.terms(field);
|
||||
if (terms != null) {
|
||||
TermsEnum termsEnum = terms.iterator();
|
||||
TermsEnum termsEnum = terms.iterator(null);
|
||||
tiq = new TermStatsQueue(numTerms);
|
||||
fillQueue(termsEnum, tiq, field);
|
||||
}
|
||||
|
@ -131,8 +131,10 @@ public class HighFreqTerms {
|
|||
while (true) {
|
||||
field = fieldsEnum.next();
|
||||
if (field != null) {
|
||||
TermsEnum terms = fieldsEnum.terms();
|
||||
fillQueue(terms, tiq, field);
|
||||
Terms terms = fieldsEnum.terms();
|
||||
if (terms != null) {
|
||||
fillQueue(terms.iterator(null), tiq, field);
|
||||
}
|
||||
} else {
|
||||
break;
|
||||
}
|
||||
|
@ -185,7 +187,7 @@ public class HighFreqTerms {
|
|||
return 0;
|
||||
}
|
||||
|
||||
TermsEnum termsEnum = terms.iterator();
|
||||
TermsEnum termsEnum = terms.iterator(null);
|
||||
if (termsEnum.seekCeil(termText) != TermsEnum.SeekStatus.FOUND) {
|
||||
return 0;
|
||||
}
|
||||
|
|
|
@ -70,7 +70,7 @@ public class TestMultiPassIndexSplitter extends LuceneTestCase {
|
|||
assertTrue(ir.numDocs() - NUM_DOCS / 3 <= 1); // rounding error
|
||||
Document doc = ir.document(0);
|
||||
assertEquals("0", doc.get("id"));
|
||||
TermsEnum te = MultiFields.getTerms(ir, "id").iterator();
|
||||
TermsEnum te = MultiFields.getTerms(ir, "id").iterator(null);
|
||||
assertEquals(TermsEnum.SeekStatus.NOT_FOUND, te.seekCeil(new BytesRef("1")));
|
||||
assertNotSame("1", te.term().utf8ToString());
|
||||
ir.close();
|
||||
|
@ -78,7 +78,7 @@ public class TestMultiPassIndexSplitter extends LuceneTestCase {
|
|||
assertTrue(ir.numDocs() - NUM_DOCS / 3 <= 1);
|
||||
doc = ir.document(0);
|
||||
assertEquals("1", doc.get("id"));
|
||||
te = MultiFields.getTerms(ir, "id").iterator();
|
||||
te = MultiFields.getTerms(ir, "id").iterator(null);
|
||||
assertEquals(TermsEnum.SeekStatus.NOT_FOUND, te.seekCeil(new BytesRef("0")));
|
||||
|
||||
assertNotSame("0", te.term().utf8ToString());
|
||||
|
@ -88,7 +88,7 @@ public class TestMultiPassIndexSplitter extends LuceneTestCase {
|
|||
doc = ir.document(0);
|
||||
assertEquals("2", doc.get("id"));
|
||||
|
||||
te = MultiFields.getTerms(ir, "id").iterator();
|
||||
te = MultiFields.getTerms(ir, "id").iterator(null);
|
||||
assertEquals(TermsEnum.SeekStatus.NOT_FOUND, te.seekCeil(new BytesRef("1")));
|
||||
assertNotSame("1", te.term());
|
||||
|
||||
|
@ -128,7 +128,7 @@ public class TestMultiPassIndexSplitter extends LuceneTestCase {
|
|||
doc = ir.document(0);
|
||||
assertEquals(start + "", doc.get("id"));
|
||||
// make sure the deleted doc is not here
|
||||
TermsEnum te = MultiFields.getTerms(ir, "id").iterator();
|
||||
TermsEnum te = MultiFields.getTerms(ir, "id").iterator(null);
|
||||
Term t = new Term("id", (NUM_DOCS - 1) + "");
|
||||
assertEquals(TermsEnum.SeekStatus.NOT_FOUND, te.seekCeil(new BytesRef(t.text())));
|
||||
assertNotSame(t.text(), te.term().utf8ToString());
|
||||
|
|
|
@ -1,115 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
||||
/*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*
|
||||
*/
|
||||
|
||||
public class TestTermVectorAccessor extends LuceneTestCase {
|
||||
|
||||
public void test() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)));
|
||||
|
||||
Document doc = new Document();
|
||||
FieldType customType = new FieldType(TextField.TYPE_UNSTORED);
|
||||
customType.setStoreTermVectors(true);
|
||||
customType.setStoreTermVectorPositions(true);
|
||||
customType.setStoreTermVectorOffsets(true);
|
||||
doc.add(newField("a", "a b a c a d a e a f a g a h a", customType));
|
||||
doc.add(newField("b", "a b c b d b e b f b g b h b", customType));
|
||||
doc.add(newField("c", "a c b c d c e c f c g c h c", customType));
|
||||
iw.addDocument(doc);
|
||||
|
||||
doc = new Document();
|
||||
FieldType customType2 = new FieldType(TextField.TYPE_UNSTORED);
|
||||
customType2.setStoreTermVectors(true);
|
||||
customType2.setStoreTermVectorPositions(true);
|
||||
doc.add(newField("a", "a b a c a d a e a f a g a h a", customType2));
|
||||
doc.add(newField("b", "a b c b d b e b f b g b h b", customType2));
|
||||
doc.add(newField("c", "a c b c d c e c f c g c h c", customType2));
|
||||
iw.addDocument(doc);
|
||||
|
||||
doc = new Document();
|
||||
FieldType customType3 = new FieldType(TextField.TYPE_UNSTORED);
|
||||
customType3.setStoreTermVectors(true);
|
||||
doc.add(newField("a", "a b a c a d a e a f a g a h a", customType3));
|
||||
doc.add(newField("b", "a b c b d b e b f b g b h b", customType3));
|
||||
doc.add(newField("c", "a c b c d c e c f c g c h c", customType3));
|
||||
iw.addDocument(doc);
|
||||
|
||||
doc = new Document();
|
||||
doc.add(newField("a", "a b a c a d a e a f a g a h a", TextField.TYPE_UNSTORED));
|
||||
doc.add(newField("b", "a b c b d b e b f b g b h b", TextField.TYPE_UNSTORED));
|
||||
doc.add(newField("c", "a c b c d c e c f c g c h c", TextField.TYPE_UNSTORED));
|
||||
iw.addDocument(doc);
|
||||
|
||||
doc = new Document();
|
||||
doc.add(newField("a", "a b a c a d a e a f a g a h a", customType));
|
||||
doc.add(newField("b", "a b c b d b e b f b g b h b", TextField.TYPE_UNSTORED));
|
||||
doc.add(newField("c", "a c b c d c e c f c g c h c", customType3));
|
||||
iw.addDocument(doc);
|
||||
|
||||
iw.close();
|
||||
|
||||
IndexReader ir = IndexReader.open(dir, false);
|
||||
|
||||
TermVectorAccessor accessor = new TermVectorAccessor();
|
||||
|
||||
ParallelArrayTermVectorMapper mapper;
|
||||
TermFreqVector tfv;
|
||||
|
||||
for (int i = 0; i < ir.maxDoc(); i++) {
|
||||
|
||||
mapper = new ParallelArrayTermVectorMapper();
|
||||
accessor.accept(ir, i, "a", mapper);
|
||||
tfv = mapper.materializeVector();
|
||||
assertEquals("doc " + i, "a", tfv.getTerms()[0].utf8ToString());
|
||||
assertEquals("doc " + i, 8, tfv.getTermFrequencies()[0]);
|
||||
|
||||
mapper = new ParallelArrayTermVectorMapper();
|
||||
accessor.accept(ir, i, "b", mapper);
|
||||
tfv = mapper.materializeVector();
|
||||
assertEquals("doc " + i, 8, tfv.getTermFrequencies().length);
|
||||
assertEquals("doc " + i, "b", tfv.getTerms()[1].utf8ToString());
|
||||
assertEquals("doc " + i, 7, tfv.getTermFrequencies()[1]);
|
||||
|
||||
mapper = new ParallelArrayTermVectorMapper();
|
||||
accessor.accept(ir, i, "c", mapper);
|
||||
tfv = mapper.materializeVector();
|
||||
assertEquals("doc " + i, 8, tfv.getTermFrequencies().length);
|
||||
assertEquals("doc " + i, "c", tfv.getTerms()[2].utf8ToString());
|
||||
assertEquals("doc " + i, 7, tfv.getTermFrequencies()[2]);
|
||||
|
||||
mapper = new ParallelArrayTermVectorMapper();
|
||||
accessor.accept(ir, i, "q", mapper);
|
||||
tfv = mapper.materializeVector();
|
||||
assertNull("doc " + i, tfv);
|
||||
|
||||
}
|
||||
|
||||
ir.close();
|
||||
|
||||
dir.close();
|
||||
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -130,7 +130,7 @@ public class TestAppendingCodec extends LuceneTestCase {
|
|||
Fields fields = MultiFields.getFields(reader);
|
||||
Terms terms = fields.terms("f");
|
||||
assertNotNull(terms);
|
||||
TermsEnum te = terms.iterator();
|
||||
TermsEnum te = terms.iterator(null);
|
||||
assertEquals(SeekStatus.FOUND, te.seekCeil(new BytesRef("quick")));
|
||||
assertEquals(SeekStatus.FOUND, te.seekCeil(new BytesRef("brown")));
|
||||
assertEquals(SeekStatus.FOUND, te.seekCeil(new BytesRef("fox")));
|
||||
|
|
|
@ -86,7 +86,7 @@ public class DuplicateFilter extends Filter {
|
|||
return bits;
|
||||
}
|
||||
|
||||
TermsEnum termsEnum = terms.iterator();
|
||||
TermsEnum termsEnum = terms.iterator(null);
|
||||
DocsEnum docs = null;
|
||||
while (true) {
|
||||
BytesRef currTerm = termsEnum.next();
|
||||
|
@ -124,7 +124,7 @@ public class DuplicateFilter extends Filter {
|
|||
return bits;
|
||||
}
|
||||
|
||||
TermsEnum termsEnum = terms.iterator();
|
||||
TermsEnum termsEnum = terms.iterator(null);
|
||||
DocsEnum docs = null;
|
||||
while (true) {
|
||||
BytesRef currTerm = termsEnum.next();
|
||||
|
|
|
@ -101,7 +101,7 @@ public class SlowCollatedTermRangeQuery extends MultiTermQuery {
|
|||
return TermsEnum.EMPTY;
|
||||
}
|
||||
|
||||
TermsEnum tenum = terms.iterator();
|
||||
TermsEnum tenum = terms.iterator(null);
|
||||
|
||||
if (lowerTerm == null && upperTerm == null) {
|
||||
return tenum;
|
||||
|
|
|
@ -69,7 +69,7 @@ public class RegexQuery extends MultiTermQuery implements RegexQueryCapable {
|
|||
|
||||
@Override
|
||||
protected FilteredTermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
|
||||
return new RegexTermsEnum(terms.iterator(), term, regexImpl);
|
||||
return new RegexTermsEnum(terms.iterator(null), term, regexImpl);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -373,7 +373,7 @@ class BufferedDeletesStream {
|
|||
currentField = term.field();
|
||||
Terms terms = fields.terms(currentField);
|
||||
if (terms != null) {
|
||||
termsEnum = terms.iterator();
|
||||
termsEnum = terms.iterator(null);
|
||||
} else {
|
||||
termsEnum = null;
|
||||
}
|
||||
|
|
|
@ -495,7 +495,7 @@ public class CheckIndex {
|
|||
segInfoStat.hasProx = info.getHasProx();
|
||||
msg(" numFiles=" + info.files().size());
|
||||
segInfoStat.numFiles = info.files().size();
|
||||
segInfoStat.sizeMB = info.sizeInBytes(true)/(1024.*1024.);
|
||||
segInfoStat.sizeMB = info.sizeInBytes()/(1024.*1024.);
|
||||
msg(" size (MB)=" + nf.format(segInfoStat.sizeMB));
|
||||
Map<String,String> diagnostics = info.getDiagnostics();
|
||||
segInfoStat.diagnostics = diagnostics;
|
||||
|
@ -676,6 +676,7 @@ public class CheckIndex {
|
|||
infoStream.print(" test: terms, freq, prox...");
|
||||
}
|
||||
|
||||
int computedFieldCount = 0;
|
||||
final Fields fields = reader.fields();
|
||||
if (fields == null) {
|
||||
msg("OK [no fields/terms]");
|
||||
|
@ -691,9 +692,20 @@ public class CheckIndex {
|
|||
if (field == null) {
|
||||
break;
|
||||
}
|
||||
|
||||
// TODO: really the codec should not return a field
|
||||
// from FieldsEnum if it has to Terms... but we do
|
||||
// this today:
|
||||
// assert fields.terms(field) != null;
|
||||
computedFieldCount++;
|
||||
|
||||
final TermsEnum terms = fieldsEnum.terms();
|
||||
assert terms != null;
|
||||
final Terms terms = fieldsEnum.terms();
|
||||
if (terms == null) {
|
||||
continue;
|
||||
}
|
||||
|
||||
final TermsEnum termsEnum = terms.iterator(null);
|
||||
|
||||
boolean hasOrd = true;
|
||||
final long termCountStart = status.termCount;
|
||||
|
||||
|
@ -706,7 +718,7 @@ public class CheckIndex {
|
|||
FixedBitSet visitedDocs = new FixedBitSet(reader.maxDoc());
|
||||
while(true) {
|
||||
|
||||
final BytesRef term = terms.next();
|
||||
final BytesRef term = termsEnum.next();
|
||||
if (term == null) {
|
||||
break;
|
||||
}
|
||||
|
@ -722,20 +734,20 @@ public class CheckIndex {
|
|||
lastTerm.copy(term);
|
||||
}
|
||||
|
||||
final int docFreq = terms.docFreq();
|
||||
final int docFreq = termsEnum.docFreq();
|
||||
if (docFreq <= 0) {
|
||||
throw new RuntimeException("docfreq: " + docFreq + " is out of bounds");
|
||||
}
|
||||
status.totFreq += docFreq;
|
||||
sumDocFreq += docFreq;
|
||||
|
||||
docs = terms.docs(liveDocs, docs);
|
||||
postings = terms.docsAndPositions(liveDocs, postings);
|
||||
docs = termsEnum.docs(liveDocs, docs);
|
||||
postings = termsEnum.docsAndPositions(liveDocs, postings);
|
||||
|
||||
if (hasOrd) {
|
||||
long ord = -1;
|
||||
try {
|
||||
ord = terms.ord();
|
||||
ord = termsEnum.ord();
|
||||
} catch (UnsupportedOperationException uoe) {
|
||||
hasOrd = false;
|
||||
}
|
||||
|
@ -804,12 +816,12 @@ public class CheckIndex {
|
|||
}
|
||||
}
|
||||
|
||||
final long totalTermFreq2 = terms.totalTermFreq();
|
||||
final long totalTermFreq2 = termsEnum.totalTermFreq();
|
||||
final boolean hasTotalTermFreq = postings != null && totalTermFreq2 != -1;
|
||||
|
||||
// Re-count if there are deleted docs:
|
||||
if (reader.hasDeletions()) {
|
||||
final DocsEnum docsNoDel = terms.docs(null, docs);
|
||||
final DocsEnum docsNoDel = termsEnum.docs(null, docs);
|
||||
docCount = 0;
|
||||
totalTermFreq = 0;
|
||||
while(docsNoDel.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
|
||||
|
@ -836,7 +848,7 @@ public class CheckIndex {
|
|||
if (hasPositions) {
|
||||
for(int idx=0;idx<7;idx++) {
|
||||
final int skipDocID = (int) (((idx+1)*(long) maxDoc)/8);
|
||||
postings = terms.docsAndPositions(liveDocs, postings);
|
||||
postings = termsEnum.docsAndPositions(liveDocs, postings);
|
||||
final int docID = postings.advance(skipDocID);
|
||||
if (docID == DocsEnum.NO_MORE_DOCS) {
|
||||
break;
|
||||
|
@ -872,7 +884,7 @@ public class CheckIndex {
|
|||
} else {
|
||||
for(int idx=0;idx<7;idx++) {
|
||||
final int skipDocID = (int) (((idx+1)*(long) maxDoc)/8);
|
||||
docs = terms.docs(liveDocs, docs);
|
||||
docs = termsEnum.docs(liveDocs, docs);
|
||||
final int docID = docs.advance(skipDocID);
|
||||
if (docID == DocsEnum.NO_MORE_DOCS) {
|
||||
break;
|
||||
|
@ -900,7 +912,8 @@ public class CheckIndex {
|
|||
// no terms, eg there used to be terms but all
|
||||
// docs got deleted and then merged away):
|
||||
// make sure TermsEnum is empty:
|
||||
if (fieldsEnum.terms().next() != null) {
|
||||
final Terms fieldTerms2 = fieldsEnum.terms();
|
||||
if (fieldTerms2 != null && fieldTerms2.iterator(null).next() != null) {
|
||||
throw new RuntimeException("Fields.terms(field=" + field + ") returned null yet the field appears to have terms");
|
||||
}
|
||||
} else {
|
||||
|
@ -927,16 +940,16 @@ public class CheckIndex {
|
|||
}
|
||||
}
|
||||
|
||||
if (fieldTerms != null) {
|
||||
final int v = fieldTerms.getDocCount();
|
||||
if (v != -1 && visitedDocs.cardinality() != v) {
|
||||
throw new RuntimeException("docCount for field " + field + "=" + v + " != recomputed docCount=" + visitedDocs.cardinality());
|
||||
if (fieldTerms != null) {
|
||||
final int v = fieldTerms.getDocCount();
|
||||
if (v != -1 && visitedDocs.cardinality() != v) {
|
||||
throw new RuntimeException("docCount for field " + field + "=" + v + " != recomputed docCount=" + visitedDocs.cardinality());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Test seek to last term:
|
||||
if (lastTerm != null) {
|
||||
if (terms.seekCeil(lastTerm) != TermsEnum.SeekStatus.FOUND) {
|
||||
if (termsEnum.seekCeil(lastTerm) != TermsEnum.SeekStatus.FOUND) {
|
||||
throw new RuntimeException("seek to last term " + lastTerm + " failed");
|
||||
}
|
||||
|
||||
|
@ -963,18 +976,18 @@ public class CheckIndex {
|
|||
// Seek by ord
|
||||
for(int i=seekCount-1;i>=0;i--) {
|
||||
long ord = i*(termCount/seekCount);
|
||||
terms.seekExact(ord);
|
||||
seekTerms[i] = new BytesRef(terms.term());
|
||||
termsEnum.seekExact(ord);
|
||||
seekTerms[i] = new BytesRef(termsEnum.term());
|
||||
}
|
||||
|
||||
// Seek by term
|
||||
long totDocCount = 0;
|
||||
for(int i=seekCount-1;i>=0;i--) {
|
||||
if (terms.seekCeil(seekTerms[i]) != TermsEnum.SeekStatus.FOUND) {
|
||||
if (termsEnum.seekCeil(seekTerms[i]) != TermsEnum.SeekStatus.FOUND) {
|
||||
throw new RuntimeException("seek to existing term " + seekTerms[i] + " failed");
|
||||
}
|
||||
|
||||
docs = terms.docs(liveDocs, docs);
|
||||
docs = termsEnum.docs(liveDocs, docs);
|
||||
if (docs == null) {
|
||||
throw new RuntimeException("null DocsEnum from to existing term " + seekTerms[i]);
|
||||
}
|
||||
|
@ -997,6 +1010,17 @@ public class CheckIndex {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
int fieldCount = fields.getUniqueFieldCount();
|
||||
|
||||
if (fieldCount != -1) {
|
||||
if (fieldCount < 0) {
|
||||
throw new RuntimeException("invalid fieldCount: " + fieldCount);
|
||||
}
|
||||
if (fieldCount != computedFieldCount) {
|
||||
throw new RuntimeException("fieldCount mismatch " + fieldCount + " vs recomputed field count " + computedFieldCount);
|
||||
}
|
||||
}
|
||||
|
||||
// for most implementations, this is boring (just the sum across all fields)
|
||||
// but codecs that don't work per-field like preflex actually implement this,
|
||||
|
@ -1137,18 +1161,136 @@ public class CheckIndex {
|
|||
private Status.TermVectorStatus testTermVectors(SegmentInfo info, SegmentReader reader, NumberFormat format) {
|
||||
final Status.TermVectorStatus status = new Status.TermVectorStatus();
|
||||
|
||||
TermsEnum termsEnum = null;
|
||||
try {
|
||||
if (infoStream != null) {
|
||||
infoStream.print(" test: term vectors........");
|
||||
}
|
||||
|
||||
// TODO: maybe we can factor out testTermIndex and reuse here?
|
||||
DocsEnum docs = null;
|
||||
DocsAndPositionsEnum postings = null;
|
||||
final Bits liveDocs = reader.getLiveDocs();
|
||||
for (int j = 0; j < info.docCount; ++j) {
|
||||
if (liveDocs == null || liveDocs.get(j)) {
|
||||
status.docCount++;
|
||||
TermFreqVector[] tfv = reader.getTermFreqVectors(j);
|
||||
Fields tfv = reader.getTermVectors(j);
|
||||
if (tfv != null) {
|
||||
status.totVectors += tfv.length;
|
||||
int tfvComputedFieldCount = 0;
|
||||
long tfvComputedTermCount = 0;
|
||||
|
||||
FieldsEnum fieldsEnum = tfv.iterator();
|
||||
String field = null;
|
||||
String lastField = null;
|
||||
while((field = fieldsEnum.next()) != null) {
|
||||
status.totVectors++;
|
||||
tfvComputedFieldCount++;
|
||||
|
||||
if (lastField == null) {
|
||||
lastField = field;
|
||||
} else if (lastField.compareTo(field) > 0) {
|
||||
throw new RuntimeException("vector fields are out of order: lastField=" + lastField + " field=" + field + " doc=" + j);
|
||||
}
|
||||
|
||||
Terms terms = tfv.terms(field);
|
||||
termsEnum = terms.iterator(termsEnum);
|
||||
|
||||
long tfvComputedTermCountForField = 0;
|
||||
long tfvComputedSumTotalTermFreq = 0;
|
||||
|
||||
BytesRef term = null;
|
||||
while ((term = termsEnum.next()) != null) {
|
||||
tfvComputedTermCountForField++;
|
||||
|
||||
if (termsEnum.docFreq() != 1) {
|
||||
throw new RuntimeException("vector docFreq for doc " + j + ", field " + field + ", term" + term + " != 1");
|
||||
}
|
||||
|
||||
long totalTermFreq = termsEnum.totalTermFreq();
|
||||
|
||||
if (totalTermFreq != -1 && totalTermFreq <= 0) {
|
||||
throw new RuntimeException("totalTermFreq: " + totalTermFreq + " is out of bounds");
|
||||
}
|
||||
|
||||
DocsEnum docsEnum;
|
||||
DocsAndPositionsEnum dp = termsEnum.docsAndPositions(null, postings);
|
||||
if (dp == null) {
|
||||
DocsEnum d = termsEnum.docs(null, docs);
|
||||
docsEnum = docs = d;
|
||||
} else {
|
||||
docsEnum = postings = dp;
|
||||
}
|
||||
|
||||
final int doc = docsEnum.nextDoc();
|
||||
|
||||
if (doc != 0) {
|
||||
throw new RuntimeException("vector for doc " + j + " didn't return docID=0: got docID=" + doc);
|
||||
}
|
||||
|
||||
final int tf = docsEnum.freq();
|
||||
tfvComputedSumTotalTermFreq += tf;
|
||||
|
||||
if (tf <= 0) {
|
||||
throw new RuntimeException("vector freq " + tf + " is out of bounds");
|
||||
}
|
||||
|
||||
if (totalTermFreq != -1 && totalTermFreq != tf) {
|
||||
throw new RuntimeException("vector totalTermFreq " + totalTermFreq + " != tf " + tf);
|
||||
}
|
||||
|
||||
if (dp != null) {
|
||||
int lastPosition = -1;
|
||||
for (int i = 0; i < tf; i++) {
|
||||
int pos = dp.nextPosition();
|
||||
if (pos != -1 && pos < 0) {
|
||||
throw new RuntimeException("vector position " + pos + " is out of bounds");
|
||||
}
|
||||
|
||||
if (pos < lastPosition) {
|
||||
throw new RuntimeException("vector position " + pos + " < lastPos " + lastPosition);
|
||||
}
|
||||
|
||||
lastPosition = pos;
|
||||
}
|
||||
}
|
||||
|
||||
if (docsEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
|
||||
throw new RuntimeException("vector for doc " + j + " references multiple documents!");
|
||||
}
|
||||
}
|
||||
|
||||
long uniqueTermCount = terms.getUniqueTermCount();
|
||||
if (uniqueTermCount != -1 && uniqueTermCount != tfvComputedTermCountForField) {
|
||||
throw new RuntimeException("vector term count for doc " + j + ", field " + field + " = " + uniqueTermCount + " != recomputed term count=" + tfvComputedTermCountForField);
|
||||
}
|
||||
|
||||
int docCount = terms.getDocCount();
|
||||
if (docCount != -1 && docCount != 1) {
|
||||
throw new RuntimeException("vector doc count for doc " + j + ", field " + field + " = " + docCount + " != 1");
|
||||
}
|
||||
|
||||
long sumDocFreq = terms.getSumDocFreq();
|
||||
if (sumDocFreq != -1 && sumDocFreq != tfvComputedTermCountForField) {
|
||||
throw new RuntimeException("vector postings count for doc " + j + ", field " + field + " = " + sumDocFreq + " != recomputed postings count=" + tfvComputedTermCountForField);
|
||||
}
|
||||
|
||||
long sumTotalTermFreq = terms.getSumTotalTermFreq();
|
||||
if (sumTotalTermFreq != -1 && sumTotalTermFreq != tfvComputedSumTotalTermFreq) {
|
||||
throw new RuntimeException("vector sumTotalTermFreq for doc " + j + ", field " + field + " = " + sumTotalTermFreq + " != recomputed sumTotalTermFreq=" + tfvComputedSumTotalTermFreq);
|
||||
}
|
||||
|
||||
tfvComputedTermCount += tfvComputedTermCountForField;
|
||||
}
|
||||
|
||||
int tfvUniqueFieldCount = tfv.getUniqueFieldCount();
|
||||
if (tfvUniqueFieldCount != -1 && tfvUniqueFieldCount != tfvComputedFieldCount) {
|
||||
throw new RuntimeException("vector field count for doc " + j + "=" + tfvUniqueFieldCount + " != recomputed uniqueFieldCount=" + tfvComputedFieldCount);
|
||||
}
|
||||
|
||||
long tfvUniqueTermCount = tfv.getUniqueTermCount();
|
||||
if (tfvUniqueTermCount != -1 && tfvUniqueTermCount != tfvComputedTermCount) {
|
||||
throw new RuntimeException("vector term count for doc " + j + "=" + tfvUniqueTermCount + " != recomputed uniqueTermCount=" + tfvComputedTermCount);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -494,33 +494,10 @@ class DirectoryReader extends IndexReader implements Cloneable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public TermFreqVector[] getTermFreqVectors(int n) throws IOException {
|
||||
public Fields getTermVectors(int docID) throws IOException {
|
||||
ensureOpen();
|
||||
int i = readerIndex(n); // find segment num
|
||||
return subReaders[i].getTermFreqVectors(n - starts[i]); // dispatch to segment
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermFreqVector getTermFreqVector(int n, String field)
|
||||
throws IOException {
|
||||
ensureOpen();
|
||||
int i = readerIndex(n); // find segment num
|
||||
return subReaders[i].getTermFreqVector(n - starts[i], field);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void getTermFreqVector(int docNumber, String field, TermVectorMapper mapper) throws IOException {
|
||||
ensureOpen();
|
||||
int i = readerIndex(docNumber); // find segment num
|
||||
subReaders[i].getTermFreqVector(docNumber - starts[i], field, mapper);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getTermFreqVector(int docNumber, TermVectorMapper mapper) throws IOException {
|
||||
ensureOpen();
|
||||
int i = readerIndex(docNumber); // find segment num
|
||||
subReaders[i].getTermFreqVector(docNumber - starts[i], mapper);
|
||||
int i = readerIndex(docID); // find segment num
|
||||
return subReaders[i].getTermVectors(docID - starts[i]); // dispatch to segment
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -25,10 +25,13 @@ import java.util.HashSet;
|
|||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.index.DocumentsWriterPerThread.DocState;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.index.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.index.codecs.FieldInfosWriter;
|
||||
import org.apache.lucene.index.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.index.values.PerDocFieldValues;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
|
@ -45,6 +48,7 @@ final class DocFieldProcessor extends DocConsumer {
|
|||
|
||||
final DocFieldConsumer consumer;
|
||||
final StoredFieldsConsumer fieldsWriter;
|
||||
final Codec codec;
|
||||
|
||||
// Holds all fields seen in current doc
|
||||
DocFieldProcessorPerField[] fields = new DocFieldProcessorPerField[1];
|
||||
|
@ -61,6 +65,7 @@ final class DocFieldProcessor extends DocConsumer {
|
|||
|
||||
public DocFieldProcessor(DocumentsWriterPerThread docWriter, DocFieldConsumer consumer) {
|
||||
this.docState = docWriter.docState;
|
||||
this.codec = docWriter.codec;
|
||||
this.consumer = consumer;
|
||||
fieldsWriter = new StoredFieldsConsumer(docWriter);
|
||||
}
|
||||
|
@ -81,8 +86,8 @@ final class DocFieldProcessor extends DocConsumer {
|
|||
// consumer can alter the FieldInfo* if necessary. EG,
|
||||
// FreqProxTermsWriter does this with
|
||||
// FieldInfo.storePayload.
|
||||
final String fileName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.FIELD_INFOS_EXTENSION);
|
||||
state.fieldInfos.write(state.directory, fileName);
|
||||
FieldInfosWriter infosWriter = codec.fieldInfosFormat().getFieldInfosWriter();
|
||||
infosWriter.write(state.directory, state.segmentName, state.fieldInfos, IOContext.DEFAULT);
|
||||
for (DocValuesConsumerAndDocID consumers : docValues.values()) {
|
||||
consumers.docValuesConsumer.finish(state.numDocs);
|
||||
}
|
||||
|
|
|
@ -200,7 +200,7 @@ public class DocTermOrds {
|
|||
//System.out.println("GET normal enum");
|
||||
final Terms terms = MultiFields.getTerms(reader, field);
|
||||
if (terms != null) {
|
||||
return terms.iterator();
|
||||
return terms.iterator(null);
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
|
@ -234,7 +234,7 @@ public class DocTermOrds {
|
|||
return;
|
||||
}
|
||||
|
||||
final TermsEnum te = terms.iterator();
|
||||
final TermsEnum te = terms.iterator(null);
|
||||
final BytesRef seekStart = termPrefix != null ? termPrefix : new BytesRef();
|
||||
//System.out.println("seekStart=" + seekStart.utf8ToString());
|
||||
if (te.seekCeil(seekStart) == TermsEnum.SeekStatus.END) {
|
||||
|
@ -644,7 +644,7 @@ public class DocTermOrds {
|
|||
public OrdWrappedTermsEnum(IndexReader reader) throws IOException {
|
||||
this.reader = reader;
|
||||
assert indexedTermsArray != null;
|
||||
termsEnum = MultiFields.getTerms(reader, field).iterator();
|
||||
termsEnum = MultiFields.getTerms(reader, field).iterator(null);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -46,7 +46,7 @@ import org.apache.lucene.util.InfoStream;
|
|||
* which in turn processes the document and interacts with
|
||||
* other consumers in the indexing chain. Certain
|
||||
* consumers, like {@link StoredFieldsConsumer} and {@link
|
||||
* TermVectorsTermsWriter}, digest a document and
|
||||
* TermVectorsConsumer}, digest a document and
|
||||
* immediately write bytes to the "doc store" files (ie,
|
||||
* they do not consume RAM per document, except while they
|
||||
* are processing the document).
|
||||
|
|
|
@ -74,7 +74,7 @@ public class DocumentsWriterPerThread {
|
|||
|
||||
// Build up indexing chain:
|
||||
|
||||
final TermsHashConsumer termVectorsWriter = new TermVectorsTermsWriter(documentsWriterPerThread);
|
||||
final TermsHashConsumer termVectorsWriter = new TermVectorsConsumer(documentsWriterPerThread);
|
||||
final TermsHashConsumer freqProxWriter = new FreqProxTermsWriter();
|
||||
|
||||
final InvertedDocConsumer termsHash = new TermsHash(documentsWriterPerThread, freqProxWriter, true,
|
||||
|
@ -414,7 +414,7 @@ public class DocumentsWriterPerThread {
|
|||
private void doAfterFlush() throws IOException {
|
||||
segment = null;
|
||||
consumer.doAfterFlush();
|
||||
fieldInfos = new FieldInfos(fieldInfos);
|
||||
fieldInfos = FieldInfos.from(fieldInfos);
|
||||
parent.subtractFlushedNumDocs(numDocsInRAM);
|
||||
numDocsInRAM = 0;
|
||||
}
|
||||
|
|
|
@ -51,12 +51,15 @@ public final class FieldInfo {
|
|||
DOCS_AND_FREQS_AND_POSITIONS
|
||||
};
|
||||
|
||||
FieldInfo(String na, boolean tk, int nu, boolean storeTermVector,
|
||||
/**
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public FieldInfo(String name, boolean isIndexed, int number, boolean storeTermVector,
|
||||
boolean storePositionWithTermVector, boolean storeOffsetWithTermVector,
|
||||
boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, ValueType docValues) {
|
||||
name = na;
|
||||
isIndexed = tk;
|
||||
number = nu;
|
||||
this.name = name;
|
||||
this.isIndexed = isIndexed;
|
||||
this.number = number;
|
||||
this.docValues = docValues;
|
||||
if (isIndexed) {
|
||||
this.storeTermVector = storeTermVector;
|
||||
|
|
|
@ -31,7 +31,6 @@ import java.util.Map.Entry;
|
|||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||
import org.apache.lucene.index.values.ValueType;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.CodecUtil;
|
||||
|
@ -185,57 +184,27 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
private final HashMap<String,FieldInfo> byName = new HashMap<String,FieldInfo>();
|
||||
private final FieldNumberBiMap globalFieldNumbers;
|
||||
|
||||
// First used in 2.9; prior to 2.9 there was no format header
|
||||
public static final int FORMAT_START = -2;
|
||||
// First used in 3.4: omit only positional information
|
||||
public static final int FORMAT_OMIT_POSITIONS = -3;
|
||||
// per-field codec support, records index values for fields
|
||||
public static final int FORMAT_FLEX = -4;
|
||||
|
||||
// whenever you add a new format, make it 1 smaller (negative version logic)!
|
||||
static final int FORMAT_CURRENT = FORMAT_FLEX;
|
||||
|
||||
static final int FORMAT_MINIMUM = FORMAT_START;
|
||||
|
||||
static final byte IS_INDEXED = 0x1;
|
||||
static final byte STORE_TERMVECTOR = 0x2;
|
||||
static final byte STORE_POSITIONS_WITH_TERMVECTOR = 0x4;
|
||||
static final byte STORE_OFFSET_WITH_TERMVECTOR = 0x8;
|
||||
static final byte OMIT_NORMS = 0x10;
|
||||
static final byte STORE_PAYLOADS = 0x20;
|
||||
static final byte OMIT_TERM_FREQ_AND_POSITIONS = 0x40;
|
||||
static final byte OMIT_POSITIONS = -128;
|
||||
|
||||
private int format;
|
||||
private boolean hasFreq; // only set if readonly
|
||||
private boolean hasProx; // only set if readonly
|
||||
private boolean hasVectors; // only set if readonly
|
||||
private long version; // internal use to track changes
|
||||
|
||||
|
||||
/**
|
||||
* Creates a new {@link FieldInfos} instance with a private
|
||||
* {@link org.apache.lucene.index.FieldInfos.FieldNumberBiMap}
|
||||
* <p>
|
||||
* Note: this ctor should not be used during indexing use
|
||||
* {@link FieldInfos#FieldInfos(FieldInfos)} or
|
||||
* {@link FieldInfos#FieldInfos(FieldNumberBiMap)}
|
||||
* instead.
|
||||
*/
|
||||
public FieldInfos() {
|
||||
this(new FieldNumberBiMap());
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new {@link FieldInfo} instance from the given instance. If the given instance is
|
||||
* read-only this instance will be read-only too.
|
||||
* Creates a new read-only FieldInfos: only public to be accessible
|
||||
* from the codecs package
|
||||
*
|
||||
* @see #isReadOnly()
|
||||
* @lucene.internal
|
||||
*/
|
||||
FieldInfos(FieldInfos other) {
|
||||
this(other.globalFieldNumbers);
|
||||
public FieldInfos(FieldInfo[] infos, boolean hasFreq, boolean hasProx, boolean hasVectors) {
|
||||
this(null);
|
||||
this.hasFreq = hasFreq;
|
||||
this.hasProx = hasProx;
|
||||
this.hasVectors = hasVectors;
|
||||
for (FieldInfo info : infos) {
|
||||
putInternal(info);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Creates a new FieldInfos instance with the given {@link FieldNumberBiMap}.
|
||||
* If the {@link FieldNumberBiMap} is <code>null</code> this instance will be read-only.
|
||||
|
@ -244,26 +213,6 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
FieldInfos(FieldNumberBiMap globalFieldNumbers) {
|
||||
this.globalFieldNumbers = globalFieldNumbers;
|
||||
}
|
||||
|
||||
/**
|
||||
* Construct a FieldInfos object using the directory and the name of the file
|
||||
* IndexInput.
|
||||
* <p>
|
||||
* Note: The created instance will be read-only
|
||||
*
|
||||
* @param d The directory to open the IndexInput from
|
||||
* @param name The name of the file to open the IndexInput from in the Directory
|
||||
* @throws IOException
|
||||
*/
|
||||
public FieldInfos(Directory d, String name) throws IOException {
|
||||
this((FieldNumberBiMap)null); // use null here to make this FIs Read-Only
|
||||
final IndexInput input = d.openInput(name, IOContext.READONCE);
|
||||
try {
|
||||
read(input, name);
|
||||
} finally {
|
||||
input.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* adds the given field to this FieldInfos name / number mapping. The given FI
|
||||
|
@ -293,7 +242,6 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
@Override
|
||||
synchronized public Object clone() {
|
||||
FieldInfos fis = new FieldInfos(globalFieldNumbers);
|
||||
fis.format = format;
|
||||
fis.hasFreq = hasFreq;
|
||||
fis.hasProx = hasProx;
|
||||
fis.hasVectors = hasVectors;
|
||||
|
@ -549,19 +497,10 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
return false;
|
||||
}
|
||||
|
||||
public void write(Directory d, String name) throws IOException {
|
||||
IndexOutput output = d.createOutput(name, IOContext.READONCE);
|
||||
try {
|
||||
write(output);
|
||||
} finally {
|
||||
output.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns <code>true</code> iff this instance is not backed by a
|
||||
* {@link org.apache.lucene.index.FieldInfos.FieldNumberBiMap}. Instances read from a directory via
|
||||
* {@link FieldInfos#FieldInfos(Directory, String)} will always be read-only
|
||||
* {@link FieldInfos#FieldInfos(FieldInfo[], boolean, boolean, boolean)} will always be read-only
|
||||
* since no {@link org.apache.lucene.index.FieldInfos.FieldNumberBiMap} is supplied, otherwise
|
||||
* <code>false</code>.
|
||||
*/
|
||||
|
@ -572,182 +511,6 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
synchronized final long getVersion() {
|
||||
return version;
|
||||
}
|
||||
|
||||
public void write(IndexOutput output) throws IOException {
|
||||
output.writeVInt(FORMAT_CURRENT);
|
||||
output.writeVInt(size());
|
||||
for (FieldInfo fi : this) {
|
||||
assert fi.indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS || !fi.storePayloads;
|
||||
byte bits = 0x0;
|
||||
if (fi.isIndexed) bits |= IS_INDEXED;
|
||||
if (fi.storeTermVector) bits |= STORE_TERMVECTOR;
|
||||
if (fi.storePositionWithTermVector) bits |= STORE_POSITIONS_WITH_TERMVECTOR;
|
||||
if (fi.storeOffsetWithTermVector) bits |= STORE_OFFSET_WITH_TERMVECTOR;
|
||||
if (fi.omitNorms) bits |= OMIT_NORMS;
|
||||
if (fi.storePayloads) bits |= STORE_PAYLOADS;
|
||||
if (fi.indexOptions == IndexOptions.DOCS_ONLY)
|
||||
bits |= OMIT_TERM_FREQ_AND_POSITIONS;
|
||||
else if (fi.indexOptions == IndexOptions.DOCS_AND_FREQS)
|
||||
bits |= OMIT_POSITIONS;
|
||||
output.writeString(fi.name);
|
||||
output.writeInt(fi.number);
|
||||
output.writeByte(bits);
|
||||
|
||||
final byte b;
|
||||
|
||||
if (fi.docValues == null) {
|
||||
b = 0;
|
||||
} else {
|
||||
switch(fi.docValues) {
|
||||
case VAR_INTS:
|
||||
b = 1;
|
||||
break;
|
||||
case FLOAT_32:
|
||||
b = 2;
|
||||
break;
|
||||
case FLOAT_64:
|
||||
b = 3;
|
||||
break;
|
||||
case BYTES_FIXED_STRAIGHT:
|
||||
b = 4;
|
||||
break;
|
||||
case BYTES_FIXED_DEREF:
|
||||
b = 5;
|
||||
break;
|
||||
case BYTES_VAR_STRAIGHT:
|
||||
b = 6;
|
||||
break;
|
||||
case BYTES_VAR_DEREF:
|
||||
b = 7;
|
||||
break;
|
||||
case FIXED_INTS_16:
|
||||
b = 8;
|
||||
break;
|
||||
case FIXED_INTS_32:
|
||||
b = 9;
|
||||
break;
|
||||
case FIXED_INTS_64:
|
||||
b = 10;
|
||||
break;
|
||||
case FIXED_INTS_8:
|
||||
b = 11;
|
||||
break;
|
||||
case BYTES_FIXED_SORTED:
|
||||
b = 12;
|
||||
break;
|
||||
case BYTES_VAR_SORTED:
|
||||
b = 13;
|
||||
break;
|
||||
default:
|
||||
throw new IllegalStateException("unhandled indexValues type " + fi.docValues);
|
||||
}
|
||||
}
|
||||
output.writeByte(b);
|
||||
}
|
||||
}
|
||||
|
||||
private void read(IndexInput input, String fileName) throws IOException {
|
||||
format = input.readVInt();
|
||||
|
||||
if (format > FORMAT_MINIMUM) {
|
||||
throw new IndexFormatTooOldException(input, format, FORMAT_MINIMUM, FORMAT_CURRENT);
|
||||
}
|
||||
if (format < FORMAT_CURRENT) {
|
||||
throw new IndexFormatTooNewException(input, format, FORMAT_MINIMUM, FORMAT_CURRENT);
|
||||
}
|
||||
|
||||
final int size = input.readVInt(); //read in the size
|
||||
|
||||
for (int i = 0; i < size; i++) {
|
||||
String name = input.readString();
|
||||
final int fieldNumber = format <= FORMAT_FLEX? input.readInt():i;
|
||||
byte bits = input.readByte();
|
||||
boolean isIndexed = (bits & IS_INDEXED) != 0;
|
||||
boolean storeTermVector = (bits & STORE_TERMVECTOR) != 0;
|
||||
boolean storePositionsWithTermVector = (bits & STORE_POSITIONS_WITH_TERMVECTOR) != 0;
|
||||
boolean storeOffsetWithTermVector = (bits & STORE_OFFSET_WITH_TERMVECTOR) != 0;
|
||||
boolean omitNorms = (bits & OMIT_NORMS) != 0;
|
||||
boolean storePayloads = (bits & STORE_PAYLOADS) != 0;
|
||||
final IndexOptions indexOptions;
|
||||
if ((bits & OMIT_TERM_FREQ_AND_POSITIONS) != 0) {
|
||||
indexOptions = IndexOptions.DOCS_ONLY;
|
||||
} else if ((bits & OMIT_POSITIONS) != 0) {
|
||||
if (format <= FORMAT_OMIT_POSITIONS) {
|
||||
indexOptions = IndexOptions.DOCS_AND_FREQS;
|
||||
} else {
|
||||
throw new CorruptIndexException("Corrupt fieldinfos, OMIT_POSITIONS set but format=" + format + " (resource: " + input + ")");
|
||||
}
|
||||
} else {
|
||||
indexOptions = IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
|
||||
}
|
||||
|
||||
// LUCENE-3027: past indices were able to write
|
||||
// storePayloads=true when omitTFAP is also true,
|
||||
// which is invalid. We correct that, here:
|
||||
if (indexOptions != IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
|
||||
storePayloads = false;
|
||||
}
|
||||
hasVectors |= storeTermVector;
|
||||
hasProx |= isIndexed && indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
|
||||
hasFreq |= isIndexed && indexOptions != IndexOptions.DOCS_ONLY;
|
||||
ValueType docValuesType = null;
|
||||
if (format <= FORMAT_FLEX) {
|
||||
final byte b = input.readByte();
|
||||
switch(b) {
|
||||
case 0:
|
||||
docValuesType = null;
|
||||
break;
|
||||
case 1:
|
||||
docValuesType = ValueType.VAR_INTS;
|
||||
break;
|
||||
case 2:
|
||||
docValuesType = ValueType.FLOAT_32;
|
||||
break;
|
||||
case 3:
|
||||
docValuesType = ValueType.FLOAT_64;
|
||||
break;
|
||||
case 4:
|
||||
docValuesType = ValueType.BYTES_FIXED_STRAIGHT;
|
||||
break;
|
||||
case 5:
|
||||
docValuesType = ValueType.BYTES_FIXED_DEREF;
|
||||
break;
|
||||
case 6:
|
||||
docValuesType = ValueType.BYTES_VAR_STRAIGHT;
|
||||
break;
|
||||
case 7:
|
||||
docValuesType = ValueType.BYTES_VAR_DEREF;
|
||||
break;
|
||||
case 8:
|
||||
docValuesType = ValueType.FIXED_INTS_16;
|
||||
break;
|
||||
case 9:
|
||||
docValuesType = ValueType.FIXED_INTS_32;
|
||||
break;
|
||||
case 10:
|
||||
docValuesType = ValueType.FIXED_INTS_64;
|
||||
break;
|
||||
case 11:
|
||||
docValuesType = ValueType.FIXED_INTS_8;
|
||||
break;
|
||||
case 12:
|
||||
docValuesType = ValueType.BYTES_FIXED_SORTED;
|
||||
break;
|
||||
case 13:
|
||||
docValuesType = ValueType.BYTES_VAR_SORTED;
|
||||
break;
|
||||
|
||||
default:
|
||||
throw new IllegalStateException("unhandled indexValues type " + b);
|
||||
}
|
||||
}
|
||||
addInternal(name, fieldNumber, isIndexed, storeTermVector, storePositionsWithTermVector, storeOffsetWithTermVector, omitNorms, storePayloads, indexOptions, docValuesType);
|
||||
}
|
||||
|
||||
if (input.getFilePointer() != input.length()) {
|
||||
throw new CorruptIndexException("did not read all bytes from file \"" + fileName + "\": read " + input.getFilePointer() + " vs size " + input.length() + " (resource: " + input + ")");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Reverts all uncommitted changes
|
||||
|
@ -783,4 +546,14 @@ public final class FieldInfos implements Iterable<FieldInfo> {
|
|||
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new {@link FieldInfo} instance from the given instance. If the given instance is
|
||||
* read-only this instance will be read-only too.
|
||||
*
|
||||
* @see #isReadOnly()
|
||||
*/
|
||||
static FieldInfos from(FieldInfos other) {
|
||||
return new FieldInfos(other.globalFieldNumbers);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,74 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
import java.util.*;
|
||||
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/**
|
||||
* Copyright 2007 The Apache Software Foundation
|
||||
* <p/>
|
||||
* Licensed 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
|
||||
* <p/>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p/>
|
||||
* 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.
|
||||
*/
|
||||
|
||||
/**
|
||||
* For each Field, store a sorted collection of {@link TermVectorEntry}s
|
||||
* <p/>
|
||||
* This is not thread-safe.
|
||||
*/
|
||||
public class FieldSortedTermVectorMapper extends TermVectorMapper{
|
||||
private Map<String,SortedSet<TermVectorEntry>> fieldToTerms = new HashMap<String,SortedSet<TermVectorEntry>>();
|
||||
private SortedSet<TermVectorEntry> currentSet;
|
||||
private String currentField;
|
||||
private Comparator<TermVectorEntry> comparator;
|
||||
|
||||
/**
|
||||
*
|
||||
* @param comparator A Comparator for sorting {@link TermVectorEntry}s
|
||||
*/
|
||||
public FieldSortedTermVectorMapper(Comparator<TermVectorEntry> comparator) {
|
||||
this(false, false, comparator);
|
||||
}
|
||||
|
||||
|
||||
public FieldSortedTermVectorMapper(boolean ignoringPositions, boolean ignoringOffsets, Comparator<TermVectorEntry> comparator) {
|
||||
super(ignoringPositions, ignoringOffsets);
|
||||
this.comparator = comparator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void map(BytesRef term, int frequency, TermVectorOffsetInfo[] offsets, int[] positions) {
|
||||
TermVectorEntry entry = new TermVectorEntry(currentField, term, frequency, offsets, positions);
|
||||
currentSet.add(entry);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setExpectations(String field, int numTerms, boolean storeOffsets, boolean storePositions) {
|
||||
currentSet = new TreeSet<TermVectorEntry>(comparator);
|
||||
currentField = field;
|
||||
fieldToTerms.put(field, currentSet);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the mapping between fields and terms, sorted by the comparator
|
||||
*
|
||||
* @return A map between field names and {@link java.util.SortedSet}s per field. SortedSet entries are {@link TermVectorEntry}
|
||||
*/
|
||||
public Map<String,SortedSet<TermVectorEntry>> getFieldToTerms() {
|
||||
return fieldToTerms;
|
||||
}
|
||||
|
||||
|
||||
public Comparator<TermVectorEntry> getComparator() {
|
||||
return comparator;
|
||||
}
|
||||
}
|
|
@ -31,6 +31,12 @@ public abstract class Fields {
|
|||
/** Get the {@link Terms} for this field. This will return
|
||||
* null if the field does not exist. */
|
||||
public abstract Terms terms(String field) throws IOException;
|
||||
|
||||
/** Returns the number of terms for all fields, or -1 if this
|
||||
* measure isn't stored by the codec. Note that, just like
|
||||
* other term measures, this measure does not take deleted
|
||||
* documents into account. */
|
||||
public abstract int getUniqueFieldCount() throws IOException;
|
||||
|
||||
/** Returns the number of terms for all fields, or -1 if this
|
||||
* measure isn't stored by the codec. Note that, just like
|
||||
|
|
|
@ -47,12 +47,19 @@ public abstract class FieldsEnum {
|
|||
* null when there are no more fields.*/
|
||||
public abstract String next() throws IOException;
|
||||
|
||||
/** Get {@link TermsEnum} for the current field. You
|
||||
* should not call {@link #next} until you're done using
|
||||
* this {@link TermsEnum}. After {@link #next} returns
|
||||
* null this method should not be called. This method
|
||||
* will not return null. */
|
||||
public abstract TermsEnum terms() throws IOException;
|
||||
// TODO: would be nice to require/fix all impls so they
|
||||
// never return null here... we have to fix the writers to
|
||||
// never write 0-terms fields... or maybe allow a non-null
|
||||
// Terms instance in just this case
|
||||
|
||||
/** Get {@link Terms} for the current field. After {@link #next} returns
|
||||
* null this method should not be called. This method may
|
||||
* return null in some cases, which means the provided
|
||||
* field does not have any terms. */
|
||||
public abstract Terms terms() throws IOException;
|
||||
|
||||
// TODO: should we allow pulling Terms as well? not just
|
||||
// the iterator?
|
||||
|
||||
public final static FieldsEnum[] EMPTY_ARRAY = new FieldsEnum[0];
|
||||
|
||||
|
@ -65,7 +72,7 @@ public abstract class FieldsEnum {
|
|||
}
|
||||
|
||||
@Override
|
||||
public TermsEnum terms() {
|
||||
public Terms terms() {
|
||||
throw new IllegalStateException("this method should never be called");
|
||||
}
|
||||
};
|
||||
|
|
|
@ -58,6 +58,11 @@ public class FilterIndexReader extends IndexReader {
|
|||
public Terms terms(String field) throws IOException {
|
||||
return in.terms(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getUniqueFieldCount() throws IOException {
|
||||
return in.getUniqueFieldCount();
|
||||
}
|
||||
}
|
||||
|
||||
/** Base class for filtering {@link Terms}
|
||||
|
@ -70,8 +75,8 @@ public class FilterIndexReader extends IndexReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public TermsEnum iterator() throws IOException {
|
||||
return in.iterator();
|
||||
public TermsEnum iterator(TermsEnum reuse) throws IOException {
|
||||
return in.iterator(reuse);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -128,7 +133,7 @@ public class FilterIndexReader extends IndexReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public TermsEnum terms() throws IOException {
|
||||
public Terms terms() throws IOException {
|
||||
return in.terms();
|
||||
}
|
||||
}
|
||||
|
@ -316,30 +321,10 @@ public class FilterIndexReader extends IndexReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public TermFreqVector[] getTermFreqVectors(int docNumber)
|
||||
public Fields getTermVectors(int docID)
|
||||
throws IOException {
|
||||
ensureOpen();
|
||||
return in.getTermFreqVectors(docNumber);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermFreqVector getTermFreqVector(int docNumber, String field)
|
||||
throws IOException {
|
||||
ensureOpen();
|
||||
return in.getTermFreqVector(docNumber, field);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void getTermFreqVector(int docNumber, String field, TermVectorMapper mapper) throws IOException {
|
||||
ensureOpen();
|
||||
in.getTermFreqVector(docNumber, field, mapper);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getTermFreqVector(int docNumber, TermVectorMapper mapper) throws IOException {
|
||||
ensureOpen();
|
||||
in.getTermFreqVector(docNumber, mapper);
|
||||
return in.getTermVectors(docID);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.lucene.index.codecs.PostingsFormat; // for javadocs
|
||||
import org.apache.lucene.index.codecs.Codec; // for javadocs
|
||||
|
||||
// TODO: put all files under codec and remove all the static extensions here
|
||||
|
||||
|
@ -33,7 +33,7 @@ import org.apache.lucene.index.codecs.PostingsFormat; // for javadocs
|
|||
* {@link #segmentFileName(String, String, String) segmentFileName}).
|
||||
*
|
||||
* <p><b>NOTE</b>: extensions used by codecs are not
|
||||
* listed here. You must interact with the {@link PostingsFormat}
|
||||
* listed here. You must interact with the {@link Codec}
|
||||
* directly.
|
||||
*
|
||||
* @lucene.internal
|
||||
|
@ -53,15 +53,6 @@ public final class IndexFileNames {
|
|||
/** Extension of norms file */
|
||||
public static final String NORMS_EXTENSION = "nrm";
|
||||
|
||||
/** Extension of vectors fields file */
|
||||
public static final String VECTORS_FIELDS_EXTENSION = "tvf";
|
||||
|
||||
/** Extension of vectors documents file */
|
||||
public static final String VECTORS_DOCUMENTS_EXTENSION = "tvd";
|
||||
|
||||
/** Extension of vectors index file */
|
||||
public static final String VECTORS_INDEX_EXTENSION = "tvx";
|
||||
|
||||
/** Extension of compound file */
|
||||
public static final String COMPOUND_FILE_EXTENSION = "cfs";
|
||||
|
||||
|
@ -74,9 +65,6 @@ public final class IndexFileNames {
|
|||
/** Extension of deletes */
|
||||
public static final String DELETES_EXTENSION = "del";
|
||||
|
||||
/** Extension of field infos */
|
||||
public static final String FIELD_INFOS_EXTENSION = "fnm";
|
||||
|
||||
/** Extension of separate norms */
|
||||
public static final String SEPARATE_NORMS_EXTENSION = "s";
|
||||
|
||||
|
@ -93,38 +81,16 @@ public final class IndexFileNames {
|
|||
public static final String INDEX_EXTENSIONS[] = new String[] {
|
||||
COMPOUND_FILE_EXTENSION,
|
||||
COMPOUND_FILE_ENTRIES_EXTENSION,
|
||||
FIELD_INFOS_EXTENSION,
|
||||
DELETES_EXTENSION,
|
||||
VECTORS_INDEX_EXTENSION,
|
||||
VECTORS_DOCUMENTS_EXTENSION,
|
||||
VECTORS_FIELDS_EXTENSION,
|
||||
GEN_EXTENSION,
|
||||
NORMS_EXTENSION,
|
||||
COMPOUND_FILE_STORE_EXTENSION,
|
||||
GLOBAL_FIELD_NUM_MAP_EXTENSION,
|
||||
};
|
||||
|
||||
public static final String[] STORE_INDEX_EXTENSIONS = new String[] {
|
||||
VECTORS_INDEX_EXTENSION,
|
||||
VECTORS_FIELDS_EXTENSION,
|
||||
VECTORS_DOCUMENTS_EXTENSION,
|
||||
};
|
||||
|
||||
public static final String[] NON_STORE_INDEX_EXTENSIONS = new String[] {
|
||||
FIELD_INFOS_EXTENSION,
|
||||
NORMS_EXTENSION
|
||||
};
|
||||
|
||||
static final String COMPOUND_EXTENSIONS_NOT_CODEC[] = new String[] {
|
||||
FIELD_INFOS_EXTENSION,
|
||||
};
|
||||
|
||||
/** File extensions for term vector support */
|
||||
public static final String VECTOR_EXTENSIONS[] = new String[] {
|
||||
VECTORS_INDEX_EXTENSION,
|
||||
VECTORS_DOCUMENTS_EXTENSION,
|
||||
VECTORS_FIELDS_EXTENSION
|
||||
};
|
||||
|
||||
/**
|
||||
* Computes the full file name from base, extension and generation. If the
|
||||
|
@ -156,21 +122,6 @@ public final class IndexFileNames {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns true if the provided filename is one of the doc store files (ends
|
||||
* with an extension in {@link #STORE_INDEX_EXTENSIONS}).
|
||||
*/
|
||||
// TODO: this method is stupid.
|
||||
public static boolean isDocStoreFile(String fileName) {
|
||||
if (fileName.endsWith(COMPOUND_FILE_STORE_EXTENSION))
|
||||
return true;
|
||||
for (String ext : STORE_INDEX_EXTENSIONS) {
|
||||
if (fileName.endsWith(ext))
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a file name that includes the given segment name, your own custom
|
||||
* name and extension. The format of the filename is:
|
||||
|
|
|
@ -803,58 +803,25 @@ public abstract class IndexReader implements Cloneable,Closeable {
|
|||
throw new UnsupportedOperationException("This reader does not support this method.");
|
||||
}
|
||||
|
||||
/**
|
||||
* Return an array of term frequency vectors for the specified document.
|
||||
* The array contains a vector for each vectorized field in the document.
|
||||
* Each vector contains terms and frequencies for all terms in a given vectorized field.
|
||||
* If no such fields existed, the method returns null. The term vectors that are
|
||||
* returned may either be of type {@link TermFreqVector}
|
||||
* or of type {@link TermPositionVector} if
|
||||
* positions or offsets have been stored.
|
||||
*
|
||||
* @param docNumber document for which term frequency vectors are returned
|
||||
* @return array of term frequency vectors. May be null if no term vectors have been
|
||||
* stored for the specified document.
|
||||
* @throws IOException if index cannot be accessed
|
||||
*/
|
||||
abstract public TermFreqVector[] getTermFreqVectors(int docNumber)
|
||||
/** Retrieve term vectors for this document, or null if
|
||||
* term vectors were not indexed. The returned Fields
|
||||
* instance acts like a single-document inverted index
|
||||
* (the docID will be 0). */
|
||||
abstract public Fields getTermVectors(int docID)
|
||||
throws IOException;
|
||||
|
||||
|
||||
/**
|
||||
* Return a term frequency vector for the specified document and field. The
|
||||
* returned vector contains terms and frequencies for the terms in
|
||||
* the specified field of this document, if the field had the storeTermVector
|
||||
* flag set. If termvectors had been stored with positions or offsets, a
|
||||
* {@link TermPositionVector} is returned.
|
||||
*
|
||||
* @param docNumber document for which the term frequency vector is returned
|
||||
* @param field field for which the term frequency vector is returned.
|
||||
* @return term frequency vector May be null if field does not exist in the specified
|
||||
* document or term vector was not stored.
|
||||
* @throws IOException if index cannot be accessed
|
||||
*/
|
||||
abstract public TermFreqVector getTermFreqVector(int docNumber, String field)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Load the Term Vector into a user-defined data structure instead of relying on the parallel arrays of
|
||||
* the {@link TermFreqVector}.
|
||||
* @param docNumber The number of the document to load the vector for
|
||||
* @param field The name of the field to load
|
||||
* @param mapper The {@link TermVectorMapper} to process the vector. Must not be null
|
||||
* @throws IOException if term vectors cannot be accessed or if they do not exist on the field and doc. specified.
|
||||
*
|
||||
*/
|
||||
abstract public void getTermFreqVector(int docNumber, String field, TermVectorMapper mapper) throws IOException;
|
||||
|
||||
/**
|
||||
* Map all the term vectors for all fields in a Document
|
||||
* @param docNumber The number of the document to load the vector for
|
||||
* @param mapper The {@link TermVectorMapper} to process the vector. Must not be null
|
||||
* @throws IOException if term vectors cannot be accessed or if they do not exist on the field and doc. specified.
|
||||
*/
|
||||
abstract public void getTermFreqVector(int docNumber, TermVectorMapper mapper) throws IOException;
|
||||
/** Retrieve term vector for this document and field, or
|
||||
* null if term vectors were not indexed. The returned
|
||||
* Fields instance acts like a single-document inverted
|
||||
* index (the docID will be 0). */
|
||||
public Terms getTermVector(int docID, String field)
|
||||
throws IOException {
|
||||
Fields vectors = getTermVectors(docID);
|
||||
if (vectors == null) {
|
||||
return null;
|
||||
}
|
||||
return vectors.terms(field);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns <code>true</code> if an index exists at the specified directory.
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.lucene.analysis.Analyzer;
|
|||
import org.apache.lucene.index.DocumentsWriterPerThread.FlushedSegment;
|
||||
import org.apache.lucene.index.FieldInfos.FieldNumberBiMap;
|
||||
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
|
||||
import org.apache.lucene.index.MergeState.CheckAbort;
|
||||
import org.apache.lucene.index.PayloadProcessorProvider.DirPayloadProcessor;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.search.Query;
|
||||
|
@ -2177,7 +2178,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
setDiagnostics(newSegment, "flush");
|
||||
|
||||
IOContext context = new IOContext(new FlushInfo(newSegment.docCount, newSegment.sizeInBytes(true)));
|
||||
IOContext context = new IOContext(new FlushInfo(newSegment.docCount, newSegment.sizeInBytes()));
|
||||
|
||||
boolean success = false;
|
||||
try {
|
||||
|
@ -2409,7 +2410,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
&& versionComparator.compare(info.getVersion(), "3.1") >= 0;
|
||||
}
|
||||
|
||||
IOContext context = new IOContext(new MergeInfo(info.docCount, info.sizeInBytes(true), true, -1));
|
||||
IOContext context = new IOContext(new MergeInfo(info.docCount, info.sizeInBytes(), true, -1));
|
||||
|
||||
if (createCFS) {
|
||||
copySegmentIntoCFS(info, newSegName, context);
|
||||
|
@ -2498,7 +2499,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
// Now create the compound file if needed
|
||||
if (useCompoundFile) {
|
||||
merger.createCompoundFile(IndexFileNames.segmentFileName(mergedName, "", IndexFileNames.COMPOUND_FILE_EXTENSION), info, context);
|
||||
createCompoundFile(directory, IndexFileNames.segmentFileName(mergedName, "", IndexFileNames.COMPOUND_FILE_EXTENSION), MergeState.CheckAbort.NONE, info, context);
|
||||
|
||||
// delete new non cfs files directly: they were never
|
||||
// registered with IFD
|
||||
|
@ -2572,10 +2573,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
newDsName = segName;
|
||||
}
|
||||
|
||||
Set<String> codecDocStoreFiles = info.codecDocStoreFiles();
|
||||
// Copy the segment files
|
||||
for (String file: info.files()) {
|
||||
final String newFileName;
|
||||
if (IndexFileNames.isDocStoreFile(file)) {
|
||||
if (codecDocStoreFiles.contains(file) || file.endsWith(IndexFileNames.COMPOUND_FILE_STORE_EXTENSION)) {
|
||||
newFileName = newDsName + IndexFileNames.stripSegmentName(file);
|
||||
if (dsFilesCopied.contains(newFileName)) {
|
||||
continue;
|
||||
|
@ -3412,7 +3414,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
final int delCount = numDeletedDocs(info);
|
||||
assert delCount <= info.docCount;
|
||||
final double delRatio = ((double) delCount)/info.docCount;
|
||||
merge.estimatedMergeBytes += info.sizeInBytes(true) * (1.0 - delRatio);
|
||||
merge.estimatedMergeBytes += info.sizeInBytes() * (1.0 - delRatio);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -3614,7 +3616,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
if (infoStream != null) {
|
||||
infoStream.message("IW", "create compound file " + compoundFileName);
|
||||
}
|
||||
merger.createCompoundFile(compoundFileName, merge.info, new IOContext(merge.getMergeInfo()));
|
||||
createCompoundFile(directory, compoundFileName, checkAbort, merge.info, new IOContext(merge.getMergeInfo()));
|
||||
success = true;
|
||||
} catch (IOException ioe) {
|
||||
synchronized(this) {
|
||||
|
@ -3663,7 +3665,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
}
|
||||
|
||||
if (infoStream != null) {
|
||||
infoStream.message("IW", String.format("merged segment size=%.3f MB vs estimate=%.3f MB", merge.info.sizeInBytes(true)/1024./1024., merge.estimatedMergeBytes/1024/1024.));
|
||||
infoStream.message("IW", String.format("merged segment size=%.3f MB vs estimate=%.3f MB", merge.info.sizeInBytes()/1024./1024., merge.estimatedMergeBytes/1024/1024.));
|
||||
}
|
||||
|
||||
final IndexReaderWarmer mergedSegmentWarmer = config.getMergedSegmentWarmer();
|
||||
|
@ -4060,4 +4062,32 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
ensureOpen();
|
||||
return payloadProcessorProvider;
|
||||
}
|
||||
|
||||
/**
|
||||
* NOTE: this method creates a compound file for all files returned by
|
||||
* info.files(). While, generally, this may include separate norms and
|
||||
* deletion files, this SegmentInfo must not reference such files when this
|
||||
* method is called, because they are not allowed within a compound file.
|
||||
*/
|
||||
static final Collection<String> createCompoundFile(Directory directory, String fileName, CheckAbort checkAbort, final SegmentInfo info, IOContext context)
|
||||
throws IOException {
|
||||
|
||||
// Now merge all added files
|
||||
Collection<String> files = info.files();
|
||||
CompoundFileDirectory cfsDir = new CompoundFileDirectory(directory, fileName, context, true);
|
||||
try {
|
||||
for (String file : files) {
|
||||
assert !IndexFileNames.matchesExtension(file, IndexFileNames.DELETES_EXTENSION)
|
||||
: ".del file is not allowed in .cfs: " + file;
|
||||
assert !IndexFileNames.isSeparateNormsFile(file)
|
||||
: "separate norms file (.s[0-9]+) is not allowed in .cfs: " + file;
|
||||
directory.copy(cfsDir, file, file, context);
|
||||
checkAbort.work(directory.fileLength(file));
|
||||
}
|
||||
} finally {
|
||||
cfsDir.close();
|
||||
}
|
||||
|
||||
return files;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -196,7 +196,7 @@ public abstract class LogMergePolicy extends MergePolicy {
|
|||
}
|
||||
|
||||
protected long sizeBytes(SegmentInfo info) throws IOException {
|
||||
long byteSize = info.sizeInBytes(true);
|
||||
long byteSize = info.sizeInBytes();
|
||||
if (calibrateSizeByDeletes) {
|
||||
int delCount = writer.get().numDeletedDocs(info);
|
||||
double delRatio = (info.docCount <= 0 ? 0.0f : ((float)delCount / (float)info.docCount));
|
||||
|
|
|
@ -173,7 +173,7 @@ public abstract class MergePolicy implements java.io.Closeable {
|
|||
public long totalBytesSize() throws IOException {
|
||||
long total = 0;
|
||||
for (SegmentInfo info : segments) {
|
||||
total += info.sizeInBytes(true);
|
||||
total += info.sizeInBytes();
|
||||
}
|
||||
return total;
|
||||
}
|
||||
|
|
|
@ -193,7 +193,8 @@ public final class MultiFields extends Fields {
|
|||
if (fieldsEnums.size() == 0) {
|
||||
return FieldsEnum.EMPTY;
|
||||
} else {
|
||||
return new MultiFieldsEnum(fieldsEnums.toArray(FieldsEnum.EMPTY_ARRAY),
|
||||
return new MultiFieldsEnum(this,
|
||||
fieldsEnums.toArray(FieldsEnum.EMPTY_ARRAY),
|
||||
fieldsSlices.toArray(ReaderUtil.Slice.EMPTY_ARRAY));
|
||||
}
|
||||
}
|
||||
|
@ -231,5 +232,10 @@ public final class MultiFields extends Fields {
|
|||
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getUniqueFieldCount() {
|
||||
return terms.size();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -47,12 +47,14 @@ public final class MultiFieldsEnum extends FieldsEnum {
|
|||
private final MultiTermsEnum terms;
|
||||
private final MultiIndexDocValues docValues;
|
||||
|
||||
private final Fields fields;
|
||||
|
||||
private String currentField;
|
||||
|
||||
/** The subs array must be newly initialized FieldsEnum
|
||||
* (ie, {@link FieldsEnum#next} has not been called. */
|
||||
public MultiFieldsEnum(FieldsEnum[] subs, ReaderUtil.Slice[] subSlices) throws IOException {
|
||||
public MultiFieldsEnum(MultiFields fields, FieldsEnum[] subs, ReaderUtil.Slice[] subSlices) throws IOException {
|
||||
this.fields = fields;
|
||||
terms = new MultiTermsEnum(subSlices);
|
||||
queue = new FieldMergeQueue(subs.length);
|
||||
docValues = new MultiIndexDocValues();
|
||||
|
@ -107,20 +109,9 @@ public final class MultiFieldsEnum extends FieldsEnum {
|
|||
}
|
||||
|
||||
@Override
|
||||
public TermsEnum terms() throws IOException {
|
||||
final List<MultiTermsEnum.TermsEnumIndex> termsEnums = new ArrayList<MultiTermsEnum.TermsEnumIndex>();
|
||||
for(int i=0;i<numTop;i++) {
|
||||
final TermsEnum terms = top[i].fields.terms();
|
||||
if (terms != null) {
|
||||
termsEnums.add(new MultiTermsEnum.TermsEnumIndex(terms, top[i].index));
|
||||
}
|
||||
}
|
||||
|
||||
if (termsEnums.size() == 0) {
|
||||
return TermsEnum.EMPTY;
|
||||
} else {
|
||||
return terms.reset(termsEnums.toArray(MultiTermsEnum.TermsEnumIndex.EMPTY_ARRAY));
|
||||
}
|
||||
public Terms terms() throws IOException {
|
||||
// Ask our parent MultiFields:
|
||||
return fields.terms(currentField);
|
||||
}
|
||||
|
||||
public final static class FieldsEnumWithSlice {
|
||||
|
|
|
@ -204,33 +204,10 @@ public class MultiReader extends IndexReader implements Cloneable {
|
|||
}
|
||||
|
||||
@Override
|
||||
public TermFreqVector[] getTermFreqVectors(int n) throws IOException {
|
||||
public Fields getTermVectors(int docID) throws IOException {
|
||||
ensureOpen();
|
||||
int i = readerIndex(n); // find segment num
|
||||
return subReaders[i].getTermFreqVectors(n - starts[i]); // dispatch to segment
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermFreqVector getTermFreqVector(int n, String field)
|
||||
throws IOException {
|
||||
ensureOpen();
|
||||
int i = readerIndex(n); // find segment num
|
||||
return subReaders[i].getTermFreqVector(n - starts[i], field);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void getTermFreqVector(int docNumber, String field, TermVectorMapper mapper) throws IOException {
|
||||
ensureOpen();
|
||||
int i = readerIndex(docNumber); // find segment num
|
||||
subReaders[i].getTermFreqVector(docNumber - starts[i], field, mapper);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getTermFreqVector(int docNumber, TermVectorMapper mapper) throws IOException {
|
||||
ensureOpen();
|
||||
int i = readerIndex(docNumber); // find segment num
|
||||
subReaders[i].getTermFreqVector(docNumber - starts[i], mapper);
|
||||
int i = readerIndex(docID); // find segment num
|
||||
return subReaders[i].getTermVectors(docID - starts[i]); // dispatch to segment
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -78,11 +78,11 @@ public final class MultiTerms extends Terms {
|
|||
}
|
||||
|
||||
@Override
|
||||
public TermsEnum iterator() throws IOException {
|
||||
public TermsEnum iterator(TermsEnum reuse) throws IOException {
|
||||
|
||||
final List<MultiTermsEnum.TermsEnumIndex> termsEnums = new ArrayList<MultiTermsEnum.TermsEnumIndex>();
|
||||
for(int i=0;i<subs.length;i++) {
|
||||
final TermsEnum termsEnum = subs[i].iterator();
|
||||
final TermsEnum termsEnum = subs[i].iterator(null);
|
||||
if (termsEnum != null) {
|
||||
termsEnums.add(new MultiTermsEnum.TermsEnumIndex(termsEnum, i));
|
||||
}
|
||||
|
|
|
@ -131,9 +131,9 @@ public class ParallelReader extends IndexReader {
|
|||
for (final String field : fields) { // update fieldToReader map
|
||||
if (fieldToReader.get(field) == null) {
|
||||
fieldToReader.put(field, reader);
|
||||
this.fields.addField(field, MultiFields.getFields(reader).terms(field));
|
||||
this.perDocs.addField(field, reader);
|
||||
}
|
||||
this.fields.addField(field, reader);
|
||||
this.perDocs.addField(field, reader);
|
||||
}
|
||||
|
||||
if (!ignoreStoredFields)
|
||||
|
@ -151,10 +151,11 @@ public class ParallelReader extends IndexReader {
|
|||
|
||||
private class ParallelFieldsEnum extends FieldsEnum {
|
||||
String currentField;
|
||||
IndexReader currentReader;
|
||||
Iterator<String> keys;
|
||||
private final Fields fields;
|
||||
|
||||
ParallelFieldsEnum() {
|
||||
ParallelFieldsEnum(Fields fields) {
|
||||
this.fields = fields;
|
||||
keys = fieldToReader.keySet().iterator();
|
||||
}
|
||||
|
||||
|
@ -162,23 +163,15 @@ public class ParallelReader extends IndexReader {
|
|||
public String next() throws IOException {
|
||||
if (keys.hasNext()) {
|
||||
currentField = keys.next();
|
||||
currentReader = fieldToReader.get(currentField);
|
||||
} else {
|
||||
currentField = null;
|
||||
currentReader = null;
|
||||
}
|
||||
return currentField;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermsEnum terms() throws IOException {
|
||||
assert currentReader != null;
|
||||
Terms terms = MultiFields.getTerms(currentReader, currentField);
|
||||
if (terms != null) {
|
||||
return terms.iterator();
|
||||
} else {
|
||||
return TermsEnum.EMPTY;
|
||||
}
|
||||
public Terms terms() throws IOException {
|
||||
return fields.terms(currentField);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -187,19 +180,24 @@ public class ParallelReader extends IndexReader {
|
|||
private class ParallelFields extends Fields {
|
||||
final HashMap<String,Terms> fields = new HashMap<String,Terms>();
|
||||
|
||||
public void addField(String field, IndexReader r) throws IOException {
|
||||
Fields multiFields = MultiFields.getFields(r);
|
||||
fields.put(field, multiFields.terms(field));
|
||||
public void addField(String fieldName, Terms terms) throws IOException {
|
||||
fields.put(fieldName, terms);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldsEnum iterator() throws IOException {
|
||||
return new ParallelFieldsEnum();
|
||||
return new ParallelFieldsEnum(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Terms terms(String field) throws IOException {
|
||||
return fields.get(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getUniqueFieldCount() throws IOException {
|
||||
return fields.size();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -362,49 +360,18 @@ public class ParallelReader extends IndexReader {
|
|||
|
||||
// get all vectors
|
||||
@Override
|
||||
public TermFreqVector[] getTermFreqVectors(int n) throws IOException {
|
||||
public Fields getTermVectors(int docID) throws IOException {
|
||||
ensureOpen();
|
||||
ArrayList<TermFreqVector> results = new ArrayList<TermFreqVector>();
|
||||
for (final Map.Entry<String,IndexReader> e: fieldToReader.entrySet()) {
|
||||
|
||||
String field = e.getKey();
|
||||
IndexReader reader = e.getValue();
|
||||
TermFreqVector vector = reader.getTermFreqVector(n, field);
|
||||
if (vector != null)
|
||||
results.add(vector);
|
||||
}
|
||||
return results.toArray(new TermFreqVector[results.size()]);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermFreqVector getTermFreqVector(int n, String field)
|
||||
throws IOException {
|
||||
ensureOpen();
|
||||
IndexReader reader = fieldToReader.get(field);
|
||||
return reader==null ? null : reader.getTermFreqVector(n, field);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void getTermFreqVector(int docNumber, String field, TermVectorMapper mapper) throws IOException {
|
||||
ensureOpen();
|
||||
IndexReader reader = fieldToReader.get(field);
|
||||
if (reader != null) {
|
||||
reader.getTermFreqVector(docNumber, field, mapper);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getTermFreqVector(int docNumber, TermVectorMapper mapper) throws IOException {
|
||||
ensureOpen();
|
||||
|
||||
for (final Map.Entry<String,IndexReader> e : fieldToReader.entrySet()) {
|
||||
|
||||
String field = e.getKey();
|
||||
IndexReader reader = e.getValue();
|
||||
reader.getTermFreqVector(docNumber, field, mapper);
|
||||
ParallelFields fields = new ParallelFields();
|
||||
for (Map.Entry<String,IndexReader> ent : fieldToReader.entrySet()) {
|
||||
String fieldName = ent.getKey();
|
||||
Terms vector = ent.getValue().getTermVector(docID, fieldName);
|
||||
if (vector != null) {
|
||||
fields.addField(fieldName, vector);
|
||||
}
|
||||
}
|
||||
|
||||
return fields;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -1,172 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
/**
|
||||
* Copyright 2007 The Apache Software Foundation
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/**
|
||||
* For each Field, store position by position information. It ignores frequency information
|
||||
* <p/>
|
||||
* This is not thread-safe.
|
||||
*/
|
||||
public class PositionBasedTermVectorMapper extends TermVectorMapper{
|
||||
private Map<String, Map<Integer,TVPositionInfo>> fieldToTerms;
|
||||
|
||||
private String currentField;
|
||||
/**
|
||||
* A Map of Integer and TVPositionInfo
|
||||
*/
|
||||
private Map<Integer,TVPositionInfo> currentPositions;
|
||||
private boolean storeOffsets;
|
||||
|
||||
|
||||
|
||||
|
||||
/**
|
||||
*
|
||||
*
|
||||
*/
|
||||
public PositionBasedTermVectorMapper() {
|
||||
super(false, false);
|
||||
}
|
||||
|
||||
public PositionBasedTermVectorMapper(boolean ignoringOffsets)
|
||||
{
|
||||
super(false, ignoringOffsets);
|
||||
}
|
||||
|
||||
/**
|
||||
* Never ignores positions. This mapper doesn't make much sense unless there are positions
|
||||
* @return false
|
||||
*/
|
||||
@Override
|
||||
public boolean isIgnoringPositions() {
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Callback for the TermVectorReader.
|
||||
* @param term
|
||||
* @param frequency
|
||||
* @param offsets
|
||||
* @param positions
|
||||
*/
|
||||
@Override
|
||||
public void map(BytesRef term, int frequency, TermVectorOffsetInfo[] offsets, int[] positions) {
|
||||
for (int i = 0; i < positions.length; i++) {
|
||||
Integer posVal = Integer.valueOf(positions[i]);
|
||||
TVPositionInfo pos = currentPositions.get(posVal);
|
||||
if (pos == null) {
|
||||
pos = new TVPositionInfo(positions[i], storeOffsets);
|
||||
currentPositions.put(posVal, pos);
|
||||
}
|
||||
pos.addTerm(term, offsets != null ? offsets[i] : null);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Callback mechanism used by the TermVectorReader
|
||||
* @param field The field being read
|
||||
* @param numTerms The number of terms in the vector
|
||||
* @param storeOffsets Whether offsets are available
|
||||
* @param storePositions Whether positions are available
|
||||
*/
|
||||
@Override
|
||||
public void setExpectations(String field, int numTerms, boolean storeOffsets, boolean storePositions) {
|
||||
if (storePositions == false)
|
||||
{
|
||||
throw new RuntimeException("You must store positions in order to use this Mapper");
|
||||
}
|
||||
if (storeOffsets == true)
|
||||
{
|
||||
//ignoring offsets
|
||||
}
|
||||
fieldToTerms = new HashMap<String,Map<Integer,TVPositionInfo>>(numTerms);
|
||||
this.storeOffsets = storeOffsets;
|
||||
currentField = field;
|
||||
currentPositions = new HashMap<Integer,TVPositionInfo>();
|
||||
fieldToTerms.put(currentField, currentPositions);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the mapping between fields and terms, sorted by the comparator
|
||||
*
|
||||
* @return A map between field names and a Map. The sub-Map key is the position as the integer, the value is {@link org.apache.lucene.index.PositionBasedTermVectorMapper.TVPositionInfo}.
|
||||
*/
|
||||
public Map<String,Map<Integer,TVPositionInfo>> getFieldToTerms() {
|
||||
return fieldToTerms;
|
||||
}
|
||||
|
||||
/**
|
||||
* Container for a term at a position
|
||||
*/
|
||||
public static class TVPositionInfo{
|
||||
private int position;
|
||||
|
||||
private List<BytesRef> terms;
|
||||
|
||||
private List<TermVectorOffsetInfo> offsets;
|
||||
|
||||
|
||||
public TVPositionInfo(int position, boolean storeOffsets) {
|
||||
this.position = position;
|
||||
terms = new ArrayList<BytesRef>();
|
||||
if (storeOffsets) {
|
||||
offsets = new ArrayList<TermVectorOffsetInfo>();
|
||||
}
|
||||
}
|
||||
|
||||
void addTerm(BytesRef term, TermVectorOffsetInfo info)
|
||||
{
|
||||
terms.add(term);
|
||||
if (offsets != null) {
|
||||
offsets.add(info);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @return The position of the term
|
||||
*/
|
||||
public int getPosition() {
|
||||
return position;
|
||||
}
|
||||
|
||||
/**
|
||||
* Note, there may be multiple terms at the same position
|
||||
* @return A List of BytesRefs
|
||||
*/
|
||||
public List<BytesRef> getTerms() {
|
||||
return terms;
|
||||
}
|
||||
|
||||
/**
|
||||
* Parallel list (to {@link #getTerms()}) of TermVectorOffsetInfo objects. There may be multiple entries since there may be multiple terms at a position
|
||||
* @return A List of TermVectorOffsetInfo objects, if offsets are stored.
|
||||
*/
|
||||
public List<TermVectorOffsetInfo> getOffsets() {
|
||||
return offsets;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
}
|
|
@ -25,6 +25,7 @@ import org.apache.lucene.index.codecs.PostingsFormat;
|
|||
import org.apache.lucene.index.codecs.FieldsProducer;
|
||||
import org.apache.lucene.index.codecs.StoredFieldsReader;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.index.codecs.TermVectorsReader;
|
||||
import org.apache.lucene.store.CompoundFileDirectory;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
|
@ -165,11 +166,10 @@ final class SegmentCoreReaders {
|
|||
assert storeDir != null;
|
||||
}
|
||||
|
||||
final String storesSegment = si.getDocStoreSegment();
|
||||
fieldsReaderOrig = si.getCodec().storedFieldsFormat().fieldsReader(storeDir, si, fieldInfos, context);
|
||||
|
||||
if (si.getHasVectors()) { // open term vector files only as needed
|
||||
termVectorsReaderOrig = new TermVectorsReader(storeDir, storesSegment, fieldInfos, context, si.getDocStoreOffset(), si.docCount);
|
||||
termVectorsReaderOrig = si.getCodec().termVectorsFormat().vectorsReader(storeDir, si, fieldInfos, context);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,12 +28,10 @@ import java.util.Map.Entry;
|
|||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
|
||||
import org.apache.lucene.index.codecs.FieldInfosReader;
|
||||
import org.apache.lucene.store.CompoundFileDirectory;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.Constants;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
|
||||
|
@ -45,9 +43,11 @@ import org.apache.lucene.util.StringHelper;
|
|||
*/
|
||||
public final class SegmentInfo implements Cloneable {
|
||||
// TODO: remove with hasVector and hasProx
|
||||
private static final int CHECK_FIELDINFO = -2;
|
||||
static final int NO = -1; // e.g. no norms; no deletes;
|
||||
static final int YES = 1; // e.g. have norms; have deletes;
|
||||
public static final int CHECK_FIELDINFO = -2;
|
||||
|
||||
// TODO: remove these from this class, for now this is the representation
|
||||
public static final int NO = -1; // e.g. no norms; no deletes;
|
||||
public static final int YES = 1; // e.g. have norms; have deletes;
|
||||
static final int WITHOUT_GEN = 0; // a file name that has no GEN in it.
|
||||
|
||||
public String name; // unique name in dir
|
||||
|
@ -167,100 +167,30 @@ public final class SegmentInfo implements Cloneable {
|
|||
}
|
||||
|
||||
/**
|
||||
* Construct a new SegmentInfo instance by reading a
|
||||
* previously saved SegmentInfo from input.
|
||||
* Construct a new complete SegmentInfo instance from input.
|
||||
* <p>Note: this is public only to allow access from
|
||||
* the codecs package.</p>
|
||||
*
|
||||
* @param dir directory to load from
|
||||
* @param format format of the segments info file
|
||||
* @param input input handle to read segment info from
|
||||
*/
|
||||
public SegmentInfo(Directory dir, int format, IndexInput input) throws IOException {
|
||||
public SegmentInfo(Directory dir, String version, String name, int docCount, long delGen, int docStoreOffset,
|
||||
String docStoreSegment, boolean docStoreIsCompoundFile, Map<Integer,Long> normGen, boolean isCompoundFile,
|
||||
int delCount, int hasProx, Codec codec, Map<String,String> diagnostics, int hasVectors) {
|
||||
this.dir = dir;
|
||||
if (format <= DefaultSegmentInfosWriter.FORMAT_3_1) {
|
||||
version = input.readString();
|
||||
}
|
||||
name = input.readString();
|
||||
docCount = input.readInt();
|
||||
delGen = input.readLong();
|
||||
docStoreOffset = input.readInt();
|
||||
if (docStoreOffset != -1) {
|
||||
docStoreSegment = input.readString();
|
||||
docStoreIsCompoundFile = input.readByte() == YES;
|
||||
} else {
|
||||
docStoreSegment = name;
|
||||
docStoreIsCompoundFile = false;
|
||||
}
|
||||
|
||||
if (format > DefaultSegmentInfosWriter.FORMAT_4_0) {
|
||||
// pre-4.0 indexes write a byte if there is a single norms file
|
||||
byte b = input.readByte();
|
||||
assert 1 == b;
|
||||
}
|
||||
|
||||
int numNormGen = input.readInt();
|
||||
if (numNormGen == NO) {
|
||||
normGen = null;
|
||||
} else {
|
||||
normGen = new HashMap<Integer, Long>();
|
||||
for(int j=0;j<numNormGen;j++) {
|
||||
int fieldNumber = j;
|
||||
if (format <= DefaultSegmentInfosWriter.FORMAT_4_0) {
|
||||
fieldNumber = input.readInt();
|
||||
}
|
||||
|
||||
normGen.put(fieldNumber, input.readLong());
|
||||
}
|
||||
}
|
||||
isCompoundFile = input.readByte() == YES;
|
||||
|
||||
delCount = input.readInt();
|
||||
assert delCount <= docCount;
|
||||
|
||||
hasProx = input.readByte();
|
||||
|
||||
|
||||
// System.out.println(Thread.currentThread().getName() + ": si.read hasProx=" + hasProx + " seg=" + name);
|
||||
// note: if the codec is not available: Codec.forName will throw an exception.
|
||||
if (format <= DefaultSegmentInfosWriter.FORMAT_4_0) {
|
||||
codec = Codec.forName(input.readString());
|
||||
} else {
|
||||
codec = Codec.forName("Lucene3x");
|
||||
}
|
||||
diagnostics = input.readStringStringMap();
|
||||
|
||||
if (format <= DefaultSegmentInfosWriter.FORMAT_HAS_VECTORS) {
|
||||
hasVectors = input.readByte();
|
||||
} else {
|
||||
final String storesSegment;
|
||||
final String ext;
|
||||
final boolean isCompoundFile;
|
||||
if (docStoreOffset != -1) {
|
||||
storesSegment = docStoreSegment;
|
||||
isCompoundFile = docStoreIsCompoundFile;
|
||||
ext = IndexFileNames.COMPOUND_FILE_STORE_EXTENSION;
|
||||
} else {
|
||||
storesSegment = name;
|
||||
isCompoundFile = getUseCompoundFile();
|
||||
ext = IndexFileNames.COMPOUND_FILE_EXTENSION;
|
||||
}
|
||||
final Directory dirToTest;
|
||||
if (isCompoundFile) {
|
||||
dirToTest = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(storesSegment, "", ext), IOContext.READONCE, false);
|
||||
} else {
|
||||
dirToTest = dir;
|
||||
}
|
||||
try {
|
||||
hasVectors = dirToTest.fileExists(IndexFileNames.segmentFileName(storesSegment, "", IndexFileNames.VECTORS_INDEX_EXTENSION)) ? YES : NO;
|
||||
} finally {
|
||||
if (isCompoundFile) {
|
||||
dirToTest.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
this.version = version;
|
||||
this.name = name;
|
||||
this.docCount = docCount;
|
||||
this.delGen = delGen;
|
||||
this.docStoreOffset = docStoreOffset;
|
||||
this.docStoreSegment = docStoreSegment;
|
||||
this.docStoreIsCompoundFile = docStoreIsCompoundFile;
|
||||
this.normGen = normGen;
|
||||
this.isCompoundFile = isCompoundFile;
|
||||
this.delCount = delCount;
|
||||
this.hasProx = hasProx;
|
||||
this.codec = codec;
|
||||
this.diagnostics = diagnostics;
|
||||
this.hasVectors = hasVectors;
|
||||
}
|
||||
|
||||
|
||||
synchronized void loadFieldInfos(Directory dir, boolean checkCompoundFile) throws IOException {
|
||||
if (fieldInfos == null) {
|
||||
Directory dir0 = dir;
|
||||
|
@ -269,8 +199,8 @@ public final class SegmentInfo implements Cloneable {
|
|||
"", IndexFileNames.COMPOUND_FILE_EXTENSION), IOContext.READONCE, false);
|
||||
}
|
||||
try {
|
||||
fieldInfos = new FieldInfos(dir0, IndexFileNames.segmentFileName(name,
|
||||
"", IndexFileNames.FIELD_INFOS_EXTENSION));
|
||||
FieldInfosReader reader = codec.fieldInfosFormat().getFieldInfosReader();
|
||||
fieldInfos = reader.read(dir0, name, IOContext.READONCE);
|
||||
} finally {
|
||||
if (dir != dir0) {
|
||||
dir0.close();
|
||||
|
@ -279,12 +209,24 @@ public final class SegmentInfo implements Cloneable {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns total size in bytes of all of files used by this segment
|
||||
*/
|
||||
public long sizeInBytes() throws IOException {
|
||||
return sizeInBytes(true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns total size in bytes of all of files used by this segment (if
|
||||
* {@code includeDocStores} is true), or the size of all files except the
|
||||
* store files otherwise.
|
||||
* <p>
|
||||
* NOTE: includeDocStores=false should only be used for debugging.
|
||||
* Theoretically a codec could combine its files however it wants (after-
|
||||
* the-fact or something), and this calculation is not particularly
|
||||
* efficient.
|
||||
*/
|
||||
public long sizeInBytes(boolean includeDocStores) throws IOException {
|
||||
long sizeInBytes(boolean includeDocStores) throws IOException {
|
||||
// TODO: based on how this is used, can't we just forget about all this docstore crap?
|
||||
// its really an abstraction violation into the codec
|
||||
if (includeDocStores) {
|
||||
|
@ -317,11 +259,17 @@ public final class SegmentInfo implements Cloneable {
|
|||
}
|
||||
}
|
||||
|
||||
// TODO: a little messy, but sizeInBytes above that uses this is the real problem.
|
||||
private boolean isDocStoreFile(String fileName) throws IOException {
|
||||
Set<String> codecDocStoreFiles() throws IOException {
|
||||
Set<String> docStoreFiles = new HashSet<String>();
|
||||
codec.storedFieldsFormat().files(dir, this, docStoreFiles);
|
||||
return IndexFileNames.isDocStoreFile(fileName) || docStoreFiles.contains(fileName);
|
||||
codec.termVectorsFormat().files(dir, this, docStoreFiles);
|
||||
return docStoreFiles;
|
||||
}
|
||||
|
||||
// TODO: a little messy, but sizeInBytes above that uses this is the real problem.
|
||||
private boolean isDocStoreFile(String fileName) throws IOException {
|
||||
Set<String> docStoreFiles = codecDocStoreFiles();
|
||||
return fileName.endsWith(IndexFileNames.COMPOUND_FILE_STORE_EXTENSION) || docStoreFiles.contains(fileName);
|
||||
}
|
||||
|
||||
public boolean getHasVectors() throws IOException {
|
||||
|
@ -551,41 +499,6 @@ public final class SegmentInfo implements Cloneable {
|
|||
this.docStoreSegment = docStoreSegment;
|
||||
}
|
||||
|
||||
/** Save this segment's info. */
|
||||
public void write(IndexOutput output)
|
||||
throws IOException {
|
||||
assert delCount <= docCount: "delCount=" + delCount + " docCount=" + docCount + " segment=" + name;
|
||||
// Write the Lucene version that created this segment, since 3.1
|
||||
output.writeString(version);
|
||||
output.writeString(name);
|
||||
output.writeInt(docCount);
|
||||
output.writeLong(delGen);
|
||||
|
||||
output.writeInt(docStoreOffset);
|
||||
if (docStoreOffset != -1) {
|
||||
output.writeString(docStoreSegment);
|
||||
output.writeByte((byte) (docStoreIsCompoundFile ? 1:0));
|
||||
}
|
||||
|
||||
|
||||
if (normGen == null) {
|
||||
output.writeInt(NO);
|
||||
} else {
|
||||
output.writeInt(normGen.size());
|
||||
for (Entry<Integer,Long> entry : normGen.entrySet()) {
|
||||
output.writeInt(entry.getKey());
|
||||
output.writeLong(entry.getValue());
|
||||
}
|
||||
}
|
||||
|
||||
output.writeByte((byte) (isCompoundFile ? YES : NO));
|
||||
output.writeInt(delCount);
|
||||
output.writeByte((byte) (hasProx));
|
||||
output.writeString(codec.getName());
|
||||
output.writeStringStringMap(diagnostics);
|
||||
output.writeByte((byte) (hasVectors));
|
||||
}
|
||||
|
||||
public boolean getHasProx() throws IOException {
|
||||
return hasProx == CHECK_FIELDINFO ? getFieldInfos().hasProx() : hasProx == YES;
|
||||
}
|
||||
|
@ -599,7 +512,7 @@ public final class SegmentInfo implements Cloneable {
|
|||
this.codec = codec;
|
||||
}
|
||||
|
||||
Codec getCodec() {
|
||||
public Codec getCodec() {
|
||||
return codec;
|
||||
}
|
||||
|
||||
|
@ -646,18 +559,6 @@ public final class SegmentInfo implements Cloneable {
|
|||
// TODO: push this out into preflex fieldsFormat?
|
||||
if (docStoreIsCompoundFile) {
|
||||
fileSet.add(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.COMPOUND_FILE_STORE_EXTENSION));
|
||||
} else {
|
||||
if (getHasVectors()) {
|
||||
fileSet.add(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.VECTORS_INDEX_EXTENSION));
|
||||
fileSet.add(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION));
|
||||
fileSet.add(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.VECTORS_FIELDS_EXTENSION));
|
||||
}
|
||||
}
|
||||
} else if (!useCompoundFile) {
|
||||
if (getHasVectors()) {
|
||||
fileSet.add(IndexFileNames.segmentFileName(name, "", IndexFileNames.VECTORS_INDEX_EXTENSION));
|
||||
fileSet.add(IndexFileNames.segmentFileName(name, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION));
|
||||
fileSet.add(IndexFileNames.segmentFileName(name, "", IndexFileNames.VECTORS_FIELDS_EXTENSION));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -795,4 +696,27 @@ public final class SegmentInfo implements Cloneable {
|
|||
void setBufferedDeletesGen(long v) {
|
||||
bufferedDeletesGen = v;
|
||||
}
|
||||
|
||||
/** @lucene.internal */
|
||||
public long getDelGen() {
|
||||
return delGen;
|
||||
}
|
||||
|
||||
/** @lucene.internal */
|
||||
public Map<Integer,Long> getNormGen() {
|
||||
return normGen;
|
||||
}
|
||||
|
||||
// TODO: clean up this SI/FI stuff here
|
||||
/** returns the 'real' value for hasProx (doesn't consult fieldinfos)
|
||||
* @lucene.internal */
|
||||
public int getHasProxInternal() {
|
||||
return hasProx;
|
||||
}
|
||||
|
||||
/** returns the 'real' value for hasVectors (doesn't consult fieldinfos)
|
||||
* @lucene.internal */
|
||||
public int getHasVectorsInternal() {
|
||||
return hasVectors;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,12 +25,12 @@ import java.util.List;
|
|||
|
||||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||
import org.apache.lucene.index.IndexReader.FieldOption;
|
||||
import org.apache.lucene.index.MergePolicy.MergeAbortedException;
|
||||
import org.apache.lucene.index.codecs.Codec;
|
||||
import org.apache.lucene.index.codecs.FieldInfosWriter;
|
||||
import org.apache.lucene.index.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.index.codecs.StoredFieldsWriter;
|
||||
import org.apache.lucene.index.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.store.CompoundFileDirectory;
|
||||
import org.apache.lucene.index.codecs.TermVectorsWriter;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
|
@ -52,10 +52,6 @@ final class SegmentMerger {
|
|||
private final String segment;
|
||||
private final int termIndexInterval;
|
||||
|
||||
/** Maximum number of contiguous documents to bulk-copy
|
||||
when merging term vectors */
|
||||
private final static int MAX_RAW_MERGE_DOCS = 4192;
|
||||
|
||||
private final Codec codec;
|
||||
|
||||
private final IOContext context;
|
||||
|
@ -128,41 +124,15 @@ final class SegmentMerger {
|
|||
mergeNorms();
|
||||
|
||||
if (mergeState.fieldInfos.hasVectors()) {
|
||||
mergeVectors(segmentWriteState);
|
||||
int numMerged = mergeVectors();
|
||||
assert numMerged == mergeState.mergedDocCount;
|
||||
}
|
||||
// write FIS once merge is done. IDV might change types or drops fields
|
||||
mergeState.fieldInfos.write(directory, segment + "." + IndexFileNames.FIELD_INFOS_EXTENSION);
|
||||
FieldInfosWriter fieldInfosWriter = codec.fieldInfosFormat().getFieldInfosWriter();
|
||||
fieldInfosWriter.write(directory, segment, mergeState.fieldInfos, context);
|
||||
return mergeState;
|
||||
}
|
||||
|
||||
/**
|
||||
* NOTE: this method creates a compound file for all files returned by
|
||||
* info.files(). While, generally, this may include separate norms and
|
||||
* deletion files, this SegmentInfo must not reference such files when this
|
||||
* method is called, because they are not allowed within a compound file.
|
||||
*/
|
||||
final Collection<String> createCompoundFile(String fileName, final SegmentInfo info, IOContext context)
|
||||
throws IOException {
|
||||
|
||||
// Now merge all added files
|
||||
Collection<String> files = info.files();
|
||||
CompoundFileDirectory cfsDir = new CompoundFileDirectory(directory, fileName, context, true);
|
||||
try {
|
||||
for (String file : files) {
|
||||
assert !IndexFileNames.matchesExtension(file, IndexFileNames.DELETES_EXTENSION)
|
||||
: ".del file is not allowed in .cfs: " + file;
|
||||
assert !IndexFileNames.isSeparateNormsFile(file)
|
||||
: "separate norms file (.s[0-9]+) is not allowed in .cfs: " + file;
|
||||
directory.copy(cfsDir, file, file, context);
|
||||
mergeState.checkAbort.work(directory.fileLength(file));
|
||||
}
|
||||
} finally {
|
||||
cfsDir.close();
|
||||
}
|
||||
|
||||
return files;
|
||||
}
|
||||
|
||||
private static void addIndexed(IndexReader reader, FieldInfos fInfos,
|
||||
Collection<String> names, boolean storeTermVectors,
|
||||
boolean storePositionWithTermVector, boolean storeOffsetWithTermVector,
|
||||
|
@ -244,138 +214,27 @@ final class SegmentMerger {
|
|||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
private int mergeFields() throws CorruptIndexException, IOException {
|
||||
int docCount = 0;
|
||||
|
||||
final StoredFieldsWriter fieldsWriter = codec.storedFieldsFormat().fieldsWriter(directory, segment, context);
|
||||
|
||||
try {
|
||||
docCount = fieldsWriter.merge(mergeState);
|
||||
return fieldsWriter.merge(mergeState);
|
||||
} finally {
|
||||
fieldsWriter.close();
|
||||
}
|
||||
|
||||
return docCount;
|
||||
}
|
||||
|
||||
/**
|
||||
* Merge the TermVectors from each of the segments into the new one.
|
||||
* @throws IOException
|
||||
*/
|
||||
private final void mergeVectors(SegmentWriteState segmentWriteState) throws IOException {
|
||||
TermVectorsWriter termVectorsWriter = new TermVectorsWriter(directory, segment, mergeState.fieldInfos, context);
|
||||
// Used for bulk-reading raw bytes for term vectors
|
||||
int rawDocLengths[] = new int[MAX_RAW_MERGE_DOCS];
|
||||
int rawDocLengths2[] = new int[MAX_RAW_MERGE_DOCS];
|
||||
private final int mergeVectors() throws IOException {
|
||||
final TermVectorsWriter termVectorsWriter = codec.termVectorsFormat().vectorsWriter(directory, segment, context);
|
||||
|
||||
try {
|
||||
int idx = 0;
|
||||
for (final MergeState.IndexReaderAndLiveDocs reader : mergeState.readers) {
|
||||
final SegmentReader matchingSegmentReader = mergeState.matchingSegmentReaders[idx++];
|
||||
TermVectorsReader matchingVectorsReader = null;
|
||||
if (matchingSegmentReader != null) {
|
||||
TermVectorsReader vectorsReader = matchingSegmentReader.getTermVectorsReader();
|
||||
|
||||
// If the TV* files are an older format then they cannot read raw docs:
|
||||
if (vectorsReader != null && vectorsReader.canReadRawDocs()) {
|
||||
matchingVectorsReader = vectorsReader;
|
||||
}
|
||||
}
|
||||
if (reader.liveDocs != null) {
|
||||
copyVectorsWithDeletions(termVectorsWriter, matchingVectorsReader, reader, rawDocLengths, rawDocLengths2);
|
||||
} else {
|
||||
copyVectorsNoDeletions(termVectorsWriter, matchingVectorsReader, reader, rawDocLengths, rawDocLengths2);
|
||||
}
|
||||
}
|
||||
return termVectorsWriter.merge(mergeState);
|
||||
} finally {
|
||||
termVectorsWriter.close();
|
||||
}
|
||||
|
||||
final String fileName = IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_INDEX_EXTENSION);
|
||||
final long tvxSize = directory.fileLength(fileName);
|
||||
final int mergedDocs = segmentWriteState.numDocs;
|
||||
|
||||
if (4+((long) mergedDocs)*16 != tvxSize)
|
||||
// This is most likely a bug in Sun JRE 1.6.0_04/_05;
|
||||
// we detect that the bug has struck, here, and
|
||||
// throw an exception to prevent the corruption from
|
||||
// entering the index. See LUCENE-1282 for
|
||||
// details.
|
||||
throw new RuntimeException("mergeVectors produced an invalid result: mergedDocs is " + mergedDocs + " but tvx size is " + tvxSize + " file=" + fileName + " file exists?=" + directory.fileExists(fileName) + "; now aborting this merge to prevent index corruption");
|
||||
}
|
||||
|
||||
private void copyVectorsWithDeletions(final TermVectorsWriter termVectorsWriter,
|
||||
final TermVectorsReader matchingVectorsReader,
|
||||
final MergeState.IndexReaderAndLiveDocs reader,
|
||||
int rawDocLengths[],
|
||||
int rawDocLengths2[])
|
||||
throws IOException, MergeAbortedException {
|
||||
final int maxDoc = reader.reader.maxDoc();
|
||||
final Bits liveDocs = reader.liveDocs;
|
||||
if (matchingVectorsReader != null) {
|
||||
// We can bulk-copy because the fieldInfos are "congruent"
|
||||
for (int docNum = 0; docNum < maxDoc;) {
|
||||
if (!liveDocs.get(docNum)) {
|
||||
// skip deleted docs
|
||||
++docNum;
|
||||
continue;
|
||||
}
|
||||
// We can optimize this case (doing a bulk byte copy) since the field
|
||||
// numbers are identical
|
||||
int start = docNum, numDocs = 0;
|
||||
do {
|
||||
docNum++;
|
||||
numDocs++;
|
||||
if (docNum >= maxDoc) break;
|
||||
if (!liveDocs.get(docNum)) {
|
||||
docNum++;
|
||||
break;
|
||||
}
|
||||
} while(numDocs < MAX_RAW_MERGE_DOCS);
|
||||
|
||||
matchingVectorsReader.rawDocs(rawDocLengths, rawDocLengths2, start, numDocs);
|
||||
termVectorsWriter.addRawDocuments(matchingVectorsReader, rawDocLengths, rawDocLengths2, numDocs);
|
||||
mergeState.checkAbort.work(300 * numDocs);
|
||||
}
|
||||
} else {
|
||||
for (int docNum = 0; docNum < maxDoc; docNum++) {
|
||||
if (!liveDocs.get(docNum)) {
|
||||
// skip deleted docs
|
||||
continue;
|
||||
}
|
||||
|
||||
// NOTE: it's very important to first assign to vectors then pass it to
|
||||
// termVectorsWriter.addAllDocVectors; see LUCENE-1282
|
||||
TermFreqVector[] vectors = reader.reader.getTermFreqVectors(docNum);
|
||||
termVectorsWriter.addAllDocVectors(vectors);
|
||||
mergeState.checkAbort.work(300);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void copyVectorsNoDeletions(final TermVectorsWriter termVectorsWriter,
|
||||
final TermVectorsReader matchingVectorsReader,
|
||||
final MergeState.IndexReaderAndLiveDocs reader,
|
||||
int rawDocLengths[],
|
||||
int rawDocLengths2[])
|
||||
throws IOException, MergeAbortedException {
|
||||
final int maxDoc = reader.reader.maxDoc();
|
||||
if (matchingVectorsReader != null) {
|
||||
// We can bulk-copy because the fieldInfos are "congruent"
|
||||
int docCount = 0;
|
||||
while (docCount < maxDoc) {
|
||||
int len = Math.min(MAX_RAW_MERGE_DOCS, maxDoc - docCount);
|
||||
matchingVectorsReader.rawDocs(rawDocLengths, rawDocLengths2, docCount, len);
|
||||
termVectorsWriter.addRawDocuments(matchingVectorsReader, rawDocLengths, rawDocLengths2, len);
|
||||
docCount += len;
|
||||
mergeState.checkAbort.work(300 * len);
|
||||
}
|
||||
} else {
|
||||
for (int docNum = 0; docNum < maxDoc; docNum++) {
|
||||
// NOTE: it's very important to first assign to vectors then pass it to
|
||||
// termVectorsWriter.addAllDocVectors; see LUCENE-1282
|
||||
TermFreqVector[] vectors = reader.reader.getTermFreqVectors(docNum);
|
||||
termVectorsWriter.addAllDocVectors(vectors);
|
||||
mergeState.checkAbort.work(300);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private final void mergeTerms(SegmentWriteState segmentWriteState) throws CorruptIndexException, IOException {
|
||||
|
|
|
@ -32,6 +32,7 @@ import org.apache.lucene.store.IndexInput;
|
|||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||
import org.apache.lucene.index.codecs.StoredFieldsReader;
|
||||
import org.apache.lucene.index.codecs.PerDocValues;
|
||||
import org.apache.lucene.index.codecs.TermVectorsReader;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.BitVector;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
@ -659,19 +660,16 @@ public class SegmentReader extends IndexReader implements Cloneable {
|
|||
/**
|
||||
* Create a clone from the initial TermVectorsReader and store it in the ThreadLocal.
|
||||
* @return TermVectorsReader
|
||||
* @lucene.internal
|
||||
*/
|
||||
TermVectorsReader getTermVectorsReader() {
|
||||
public TermVectorsReader getTermVectorsReader() {
|
||||
TermVectorsReader tvReader = termVectorsLocal.get();
|
||||
if (tvReader == null) {
|
||||
TermVectorsReader orig = core.getTermVectorsReaderOrig();
|
||||
if (orig == null) {
|
||||
return null;
|
||||
} else {
|
||||
try {
|
||||
tvReader = (TermVectorsReader) orig.clone();
|
||||
} catch (CloneNotSupportedException cnse) {
|
||||
return null;
|
||||
}
|
||||
tvReader = orig.clone();
|
||||
}
|
||||
termVectorsLocal.set(tvReader);
|
||||
}
|
||||
|
@ -689,67 +687,15 @@ public class SegmentReader extends IndexReader implements Cloneable {
|
|||
* @throws IOException
|
||||
*/
|
||||
@Override
|
||||
public TermFreqVector getTermFreqVector(int docNumber, String field) throws IOException {
|
||||
// Check if this field is invalid or has no stored term vector
|
||||
public Fields getTermVectors(int docID) throws IOException {
|
||||
ensureOpen();
|
||||
FieldInfo fi = core.fieldInfos.fieldInfo(field);
|
||||
if (fi == null || !fi.storeTermVector)
|
||||
return null;
|
||||
|
||||
TermVectorsReader termVectorsReader = getTermVectorsReader();
|
||||
if (termVectorsReader == null)
|
||||
return null;
|
||||
|
||||
return termVectorsReader.get(docNumber, field);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void getTermFreqVector(int docNumber, String field, TermVectorMapper mapper) throws IOException {
|
||||
ensureOpen();
|
||||
FieldInfo fi = core.fieldInfos.fieldInfo(field);
|
||||
if (fi == null || !fi.storeTermVector)
|
||||
return;
|
||||
|
||||
TermVectorsReader termVectorsReader = getTermVectorsReader();
|
||||
if (termVectorsReader == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
|
||||
termVectorsReader.get(docNumber, field, mapper);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void getTermFreqVector(int docNumber, TermVectorMapper mapper) throws IOException {
|
||||
ensureOpen();
|
||||
|
||||
TermVectorsReader termVectorsReader = getTermVectorsReader();
|
||||
if (termVectorsReader == null)
|
||||
return;
|
||||
|
||||
termVectorsReader.get(docNumber, mapper);
|
||||
}
|
||||
|
||||
/** Return an array of term frequency vectors for the specified document.
|
||||
* The array contains a vector for each vectorized field in the document.
|
||||
* Each vector vector contains term numbers and frequencies for all terms
|
||||
* in a given vectorized field.
|
||||
* If no such fields existed, the method returns null.
|
||||
* @throws IOException
|
||||
*/
|
||||
@Override
|
||||
public TermFreqVector[] getTermFreqVectors(int docNumber) throws IOException {
|
||||
ensureOpen();
|
||||
|
||||
TermVectorsReader termVectorsReader = getTermVectorsReader();
|
||||
if (termVectorsReader == null)
|
||||
return null;
|
||||
|
||||
return termVectorsReader.get(docNumber);
|
||||
}
|
||||
return termVectorsReader.get(docID);
|
||||
}
|
||||
|
||||
|
||||
/** {@inheritDoc} */
|
||||
@Override
|
||||
public String toString() {
|
||||
|
|
|
@ -1,67 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
class SegmentTermPositionVector extends SegmentTermVector implements TermPositionVector {
|
||||
protected int[][] positions;
|
||||
protected TermVectorOffsetInfo[][] offsets;
|
||||
public static final int[] EMPTY_TERM_POS = new int[0];
|
||||
|
||||
public SegmentTermPositionVector(String field, BytesRef terms[], int termFreqs[], int[][] positions, TermVectorOffsetInfo[][] offsets) {
|
||||
super(field, terms, termFreqs);
|
||||
this.offsets = offsets;
|
||||
this.positions = positions;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns an array of TermVectorOffsetInfo in which the term is found.
|
||||
*
|
||||
* @param index The position in the array to get the offsets from
|
||||
* @return An array of TermVectorOffsetInfo objects or the empty list
|
||||
* @see org.apache.lucene.analysis.Token
|
||||
*/
|
||||
public TermVectorOffsetInfo[] getOffsets(int index) {
|
||||
TermVectorOffsetInfo[] result = TermVectorOffsetInfo.EMPTY_OFFSET_INFO;
|
||||
if(offsets == null)
|
||||
return null;
|
||||
if (index >=0 && index < offsets.length)
|
||||
{
|
||||
result = offsets[index];
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns an array of positions in which the term is found.
|
||||
* Terms are identified by the index at which its number appears in the
|
||||
* term String array obtained from the <code>indexOf</code> method.
|
||||
*/
|
||||
public int[] getTermPositions(int index) {
|
||||
int[] result = EMPTY_TERM_POS;
|
||||
if(positions == null)
|
||||
return null;
|
||||
if (index >=0 && index < positions.length)
|
||||
{
|
||||
result = positions[index];
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
}
|
|
@ -1,92 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.util.*;
|
||||
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
|
||||
class SegmentTermVector implements TermFreqVector {
|
||||
private String field;
|
||||
private BytesRef terms[];
|
||||
private int termFreqs[];
|
||||
|
||||
SegmentTermVector(String field, BytesRef terms[], int termFreqs[]) {
|
||||
this.field = field;
|
||||
this.terms = terms;
|
||||
this.termFreqs = termFreqs;
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @return The number of the field this vector is associated with
|
||||
*/
|
||||
public String getField() {
|
||||
return field;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
sb.append('{');
|
||||
sb.append(field).append(": ");
|
||||
if(terms != null){
|
||||
for (int i=0; i<terms.length; i++) {
|
||||
if (i>0) sb.append(", ");
|
||||
sb.append(terms[i]).append('/').append(termFreqs[i]);
|
||||
}
|
||||
}
|
||||
sb.append('}');
|
||||
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
public int size() {
|
||||
return terms == null ? 0 : terms.length;
|
||||
}
|
||||
|
||||
public BytesRef [] getTerms() {
|
||||
return terms;
|
||||
}
|
||||
|
||||
public int[] getTermFrequencies() {
|
||||
return termFreqs;
|
||||
}
|
||||
|
||||
public int indexOf(BytesRef termBytes) {
|
||||
if(terms == null)
|
||||
return -1;
|
||||
int res = Arrays.binarySearch(terms, termBytes);
|
||||
return res >= 0 ? res : -1;
|
||||
}
|
||||
|
||||
public int[] indexesOf(BytesRef [] termNumbers, int start, int len) {
|
||||
// TODO: there must be a more efficient way of doing this.
|
||||
// At least, we could advance the lower bound of the terms array
|
||||
// as we find valid indexes. Also, it might be possible to leverage
|
||||
// this even more by starting in the middle of the termNumbers array
|
||||
// and thus dividing the terms array maybe in half with each found index.
|
||||
int res[] = new int[len];
|
||||
|
||||
for (int i=0; i < len; i++) {
|
||||
res[i] = indexOf(termNumbers[start+ i]);
|
||||
}
|
||||
return res;
|
||||
}
|
||||
}
|
|
@ -1,133 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
/**
|
||||
* Copyright 2007 The Apache Software Foundation
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.util.*;
|
||||
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/**
|
||||
* Store a sorted collection of {@link org.apache.lucene.index.TermVectorEntry}s. Collects all term information
|
||||
* into a single, SortedSet.
|
||||
* <br/>
|
||||
* NOTE: This Mapper ignores all Field information for the Document. This means that if you are using offset/positions you will not
|
||||
* know what Fields they correlate with.
|
||||
* <br/>
|
||||
* This is not thread-safe
|
||||
*/
|
||||
public class SortedTermVectorMapper extends TermVectorMapper{
|
||||
|
||||
|
||||
private SortedSet<TermVectorEntry> currentSet;
|
||||
private Map<BytesRef,TermVectorEntry> termToTVE = new HashMap<BytesRef,TermVectorEntry>();
|
||||
private boolean storeOffsets;
|
||||
private boolean storePositions;
|
||||
/**
|
||||
* Stand-in name for the field in {@link TermVectorEntry}.
|
||||
*/
|
||||
public static final String ALL = "_ALL_";
|
||||
|
||||
/**
|
||||
*
|
||||
* @param comparator A Comparator for sorting {@link TermVectorEntry}s
|
||||
*/
|
||||
public SortedTermVectorMapper(Comparator<TermVectorEntry> comparator) {
|
||||
this(false, false, comparator);
|
||||
}
|
||||
|
||||
|
||||
public SortedTermVectorMapper(boolean ignoringPositions, boolean ignoringOffsets, Comparator<TermVectorEntry> comparator) {
|
||||
super(ignoringPositions, ignoringOffsets);
|
||||
currentSet = new TreeSet<TermVectorEntry>(comparator);
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @param term The term to map
|
||||
* @param frequency The frequency of the term
|
||||
* @param offsets Offset information, may be null
|
||||
* @param positions Position information, may be null
|
||||
*/
|
||||
//We need to combine any previous mentions of the term
|
||||
@Override
|
||||
public void map(BytesRef term, int frequency, TermVectorOffsetInfo[] offsets, int[] positions) {
|
||||
TermVectorEntry entry = termToTVE.get(term);
|
||||
if (entry == null) {
|
||||
entry = new TermVectorEntry(ALL, term, frequency,
|
||||
storeOffsets == true ? offsets : null,
|
||||
storePositions == true ? positions : null);
|
||||
termToTVE.put(term, entry);
|
||||
currentSet.add(entry);
|
||||
} else {
|
||||
entry.setFrequency(entry.getFrequency() + frequency);
|
||||
if (storeOffsets)
|
||||
{
|
||||
TermVectorOffsetInfo [] existingOffsets = entry.getOffsets();
|
||||
//A few diff. cases here: offsets is null, existing offsets is null, both are null, same for positions
|
||||
if (existingOffsets != null && offsets != null && offsets.length > 0)
|
||||
{
|
||||
//copy over the existing offsets
|
||||
TermVectorOffsetInfo [] newOffsets = new TermVectorOffsetInfo[existingOffsets.length + offsets.length];
|
||||
System.arraycopy(existingOffsets, 0, newOffsets, 0, existingOffsets.length);
|
||||
System.arraycopy(offsets, 0, newOffsets, existingOffsets.length, offsets.length);
|
||||
entry.setOffsets(newOffsets);
|
||||
}
|
||||
else if (existingOffsets == null && offsets != null && offsets.length > 0)
|
||||
{
|
||||
entry.setOffsets(offsets);
|
||||
}
|
||||
//else leave it alone
|
||||
}
|
||||
if (storePositions)
|
||||
{
|
||||
int [] existingPositions = entry.getPositions();
|
||||
if (existingPositions != null && positions != null && positions.length > 0)
|
||||
{
|
||||
int [] newPositions = new int[existingPositions.length + positions.length];
|
||||
System.arraycopy(existingPositions, 0, newPositions, 0, existingPositions.length);
|
||||
System.arraycopy(positions, 0, newPositions, existingPositions.length, positions.length);
|
||||
entry.setPositions(newPositions);
|
||||
}
|
||||
else if (existingPositions == null && positions != null && positions.length > 0)
|
||||
{
|
||||
entry.setPositions(positions);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setExpectations(String field, int numTerms, boolean storeOffsets, boolean storePositions) {
|
||||
|
||||
this.storeOffsets = storeOffsets;
|
||||
this.storePositions = storePositions;
|
||||
}
|
||||
|
||||
/**
|
||||
* The TermVectorEntrySet. A SortedSet of {@link TermVectorEntry} objects. Sort is by the comparator passed into the constructor.
|
||||
*<br/>
|
||||
* This set will be empty until after the mapping process takes place.
|
||||
*
|
||||
* @return The SortedSet of {@link TermVectorEntry}.
|
||||
*/
|
||||
public SortedSet<TermVectorEntry> getTermVectorEntrySet()
|
||||
{
|
||||
return currentSet;
|
||||
}
|
||||
|
||||
}
|
|
@ -45,12 +45,12 @@ final class StoredFieldsConsumer {
|
|||
|
||||
private int numStoredFields;
|
||||
private IndexableField[] storedFields;
|
||||
private int[] fieldNumbers;
|
||||
private FieldInfo[] fieldInfos;
|
||||
|
||||
public void reset() {
|
||||
numStoredFields = 0;
|
||||
storedFields = new IndexableField[1];
|
||||
fieldNumbers = new int[1];
|
||||
fieldInfos = new FieldInfo[1];
|
||||
}
|
||||
|
||||
public void startDocument() {
|
||||
|
@ -116,7 +116,7 @@ final class StoredFieldsConsumer {
|
|||
if (fieldsWriter != null && numStoredFields > 0) {
|
||||
fieldsWriter.startDocument(numStoredFields);
|
||||
for (int i = 0; i < numStoredFields; i++) {
|
||||
fieldsWriter.writeField(fieldNumbers[i], storedFields[i]);
|
||||
fieldsWriter.writeField(fieldInfos[i], storedFields[i]);
|
||||
}
|
||||
lastDocID++;
|
||||
}
|
||||
|
@ -131,14 +131,14 @@ final class StoredFieldsConsumer {
|
|||
IndexableField[] newArray = new IndexableField[newSize];
|
||||
System.arraycopy(storedFields, 0, newArray, 0, numStoredFields);
|
||||
storedFields = newArray;
|
||||
}
|
||||
|
||||
if (numStoredFields == fieldNumbers.length) {
|
||||
fieldNumbers = ArrayUtil.grow(fieldNumbers);
|
||||
|
||||
FieldInfo[] newInfoArray = new FieldInfo[newSize];
|
||||
System.arraycopy(fieldInfos, 0, newInfoArray, 0, numStoredFields);
|
||||
fieldInfos = newInfoArray;
|
||||
}
|
||||
|
||||
storedFields[numStoredFields] = field;
|
||||
fieldNumbers[numStoredFields] = fieldInfo.number;
|
||||
fieldInfos[numStoredFields] = fieldInfo;
|
||||
numStoredFields++;
|
||||
|
||||
assert docState.testPoint("StoredFieldsWriterPerThread.processFields.writeField");
|
||||
|
|
|
@ -1,73 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
/** Provides access to stored term vector of
|
||||
* a document field. The vector consists of the name of the field, an array of the terms that occur in the field of the
|
||||
* {@link org.apache.lucene.document.Document} and a parallel array of frequencies. Thus, getTermFrequencies()[5] corresponds with the
|
||||
* frequency of getTerms()[5], assuming there are at least 5 terms in the Document.
|
||||
*/
|
||||
public interface TermFreqVector {
|
||||
/**
|
||||
* The {@link org.apache.lucene.index.IndexableField} name.
|
||||
* @return The name of the field this vector is associated with.
|
||||
*
|
||||
*/
|
||||
public String getField();
|
||||
|
||||
/**
|
||||
* @return The number of terms in the term vector.
|
||||
*/
|
||||
public int size();
|
||||
|
||||
/**
|
||||
* @return An Array of term texts in ascending order.
|
||||
*/
|
||||
public BytesRef[] getTerms();
|
||||
|
||||
|
||||
/** Array of term frequencies. Locations of the array correspond one to one
|
||||
* to the terms in the array obtained from <code>getTerms</code>
|
||||
* method. Each location in the array contains the number of times this
|
||||
* term occurs in the document or the document field.
|
||||
*/
|
||||
public int[] getTermFrequencies();
|
||||
|
||||
|
||||
/** Return an index in the term numbers array returned from
|
||||
* <code>getTerms</code> at which the term with the specified
|
||||
* <code>term</code> appears. If this term does not appear in the array,
|
||||
* return -1.
|
||||
*/
|
||||
public int indexOf(BytesRef term);
|
||||
|
||||
|
||||
/** Just like <code>indexOf(int)</code> but searches for a number of terms
|
||||
* at the same time. Returns an array that has the same size as the number
|
||||
* of terms searched for, each slot containing the result of searching for
|
||||
* that term number.
|
||||
*
|
||||
* @param terms array containing terms to look for
|
||||
* @param start index in the array where the list of terms starts
|
||||
* @param len the number of terms in the list
|
||||
*/
|
||||
public int[] indexesOf(BytesRef[] terms, int start, int len);
|
||||
|
||||
}
|
|
@ -1,43 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
/** Extends <code>TermFreqVector</code> to provide additional information about
|
||||
* positions in which each of the terms is found. A TermPositionVector not necessarily
|
||||
* contains both positions and offsets, but at least one of these arrays exists.
|
||||
*/
|
||||
public interface TermPositionVector extends TermFreqVector {
|
||||
|
||||
/** Returns an array of positions in which the term is found.
|
||||
* Terms are identified by the index at which its number appears in the
|
||||
* term String array obtained from the <code>indexOf</code> method.
|
||||
* May return null if positions have not been stored.
|
||||
*/
|
||||
public int[] getTermPositions(int index);
|
||||
|
||||
/**
|
||||
* Returns an array of TermVectorOffsetInfo in which the term is found.
|
||||
* May return null if offsets have not been stored.
|
||||
*
|
||||
* @see org.apache.lucene.analysis.Token
|
||||
*
|
||||
* @param index The position in the array to get the offsets from
|
||||
* @return An array of TermVectorOffsetInfo objects or the empty list
|
||||
*/
|
||||
public TermVectorOffsetInfo [] getOffsets(int index);
|
||||
}
|
|
@ -1,103 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/**
|
||||
* Copyright 2007 The Apache Software Foundation
|
||||
* <p/>
|
||||
* Licensed 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
|
||||
* <p/>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p/>
|
||||
* 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.
|
||||
*/
|
||||
|
||||
/**
|
||||
* Convenience class for holding TermVector information.
|
||||
*/
|
||||
public class TermVectorEntry {
|
||||
private String field;
|
||||
private BytesRef term;
|
||||
private int frequency;
|
||||
private TermVectorOffsetInfo [] offsets;
|
||||
int [] positions;
|
||||
|
||||
|
||||
public TermVectorEntry() {
|
||||
}
|
||||
|
||||
public TermVectorEntry(String field, BytesRef term, int frequency, TermVectorOffsetInfo[] offsets, int[] positions) {
|
||||
this.field = field;
|
||||
this.term = term;
|
||||
this.frequency = frequency;
|
||||
this.offsets = offsets;
|
||||
this.positions = positions;
|
||||
}
|
||||
|
||||
|
||||
public String getField() {
|
||||
return field;
|
||||
}
|
||||
|
||||
public int getFrequency() {
|
||||
return frequency;
|
||||
}
|
||||
|
||||
public TermVectorOffsetInfo[] getOffsets() {
|
||||
return offsets;
|
||||
}
|
||||
|
||||
public int[] getPositions() {
|
||||
return positions;
|
||||
}
|
||||
|
||||
public BytesRef getTerm() {
|
||||
return term;
|
||||
}
|
||||
|
||||
//Keep package local
|
||||
void setFrequency(int frequency) {
|
||||
this.frequency = frequency;
|
||||
}
|
||||
|
||||
void setOffsets(TermVectorOffsetInfo[] offsets) {
|
||||
this.offsets = offsets;
|
||||
}
|
||||
|
||||
void setPositions(int[] positions) {
|
||||
this.positions = positions;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
|
||||
TermVectorEntry that = (TermVectorEntry) o;
|
||||
|
||||
if (term != null ? !term.equals(that.term) : that.term != null) return false;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return (term != null ? term.hashCode() : 0);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "TermVectorEntry{" +
|
||||
"field='" + field + '\'' +
|
||||
", term='" + term + '\'' +
|
||||
", frequency=" + frequency +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -1,40 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
/**
|
||||
* Copyright 2007 The Apache Software Foundation
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
|
||||
import java.util.Comparator;
|
||||
|
||||
/**
|
||||
* Compares {@link org.apache.lucene.index.TermVectorEntry}s first by frequency and then by
|
||||
* the term (case-sensitive)
|
||||
*
|
||||
**/
|
||||
public class TermVectorEntryFreqSortedComparator implements Comparator<TermVectorEntry> {
|
||||
public int compare(TermVectorEntry entry, TermVectorEntry entry1) {
|
||||
int result = 0;
|
||||
result = entry1.getFrequency() - entry.getFrequency();
|
||||
if (result == 0)
|
||||
{
|
||||
result = entry.getTerm().compareTo(entry1.getTerm());
|
||||
if (result == 0)
|
||||
{
|
||||
result = entry.getField().compareTo(entry1.getField());
|
||||
}
|
||||
}
|
||||
return result;
|
||||
}
|
||||
}
|
|
@ -1,104 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/**
|
||||
* Copyright 2007 The Apache Software Foundation
|
||||
*
|
||||
* Licensed 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.
|
||||
*/
|
||||
|
||||
|
||||
/**
|
||||
* The TermVectorMapper can be used to map Term Vectors into your own
|
||||
* structure instead of the parallel array structure used by
|
||||
* {@link org.apache.lucene.index.IndexReader#getTermFreqVector(int,String)}.
|
||||
* <p/>
|
||||
* It is up to the implementation to make sure it is thread-safe.
|
||||
*
|
||||
*
|
||||
**/
|
||||
public abstract class TermVectorMapper {
|
||||
|
||||
private boolean ignoringPositions;
|
||||
private boolean ignoringOffsets;
|
||||
|
||||
|
||||
protected TermVectorMapper() {
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @param ignoringPositions true if this mapper should tell Lucene to ignore positions even if they are stored
|
||||
* @param ignoringOffsets similar to ignoringPositions
|
||||
*/
|
||||
protected TermVectorMapper(boolean ignoringPositions, boolean ignoringOffsets) {
|
||||
this.ignoringPositions = ignoringPositions;
|
||||
this.ignoringOffsets = ignoringOffsets;
|
||||
}
|
||||
|
||||
/**
|
||||
* Tell the mapper what to expect in regards to field, number of terms, offset and position storage.
|
||||
* This method will be called once before retrieving the vector for a field.
|
||||
*
|
||||
* This method will be called before {@link #map(BytesRef,int,TermVectorOffsetInfo[],int[])}.
|
||||
* @param field The field the vector is for
|
||||
* @param numTerms The number of terms that need to be mapped
|
||||
* @param storeOffsets true if the mapper should expect offset information
|
||||
* @param storePositions true if the mapper should expect positions info
|
||||
*/
|
||||
public abstract void setExpectations(String field, int numTerms, boolean storeOffsets, boolean storePositions);
|
||||
/**
|
||||
* Map the Term Vector information into your own structure
|
||||
* @param term The term to add to the vector
|
||||
* @param frequency The frequency of the term in the document
|
||||
* @param offsets null if the offset is not specified, otherwise the offset into the field of the term
|
||||
* @param positions null if the position is not specified, otherwise the position in the field of the term
|
||||
*/
|
||||
public abstract void map(BytesRef term, int frequency, TermVectorOffsetInfo [] offsets, int [] positions);
|
||||
|
||||
/**
|
||||
* Indicate to Lucene that even if there are positions stored, this mapper is not interested in them and they
|
||||
* can be skipped over. Derived classes should set this to true if they want to ignore positions. The default
|
||||
* is false, meaning positions will be loaded if they are stored.
|
||||
* @return false
|
||||
*/
|
||||
public boolean isIgnoringPositions()
|
||||
{
|
||||
return ignoringPositions;
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @see #isIgnoringPositions() Same principal as {@link #isIgnoringPositions()}, but applied to offsets. false by default.
|
||||
* @return false
|
||||
*/
|
||||
public boolean isIgnoringOffsets()
|
||||
{
|
||||
return ignoringOffsets;
|
||||
}
|
||||
|
||||
/**
|
||||
* Passes down the index of the document whose term vector is currently being mapped,
|
||||
* once for each top level call to a term vector reader.
|
||||
*<p/>
|
||||
* Default implementation IGNORES the document number. Override if your implementation needs the document number.
|
||||
* <p/>
|
||||
* NOTE: Document numbers are internal to Lucene and subject to change depending on indexing operations.
|
||||
*
|
||||
* @param documentNumber index of document currently being mapped
|
||||
*/
|
||||
public void setDocumentNumber(int documentNumber) {
|
||||
}
|
||||
|
||||
}
|
|
@ -1,91 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
/**
|
||||
* The TermVectorOffsetInfo class holds information pertaining to a Term in a {@link org.apache.lucene.index.TermPositionVector}'s
|
||||
* offset information. This offset information is the character offset as set during the Analysis phase (and thus may not be the actual offset in the
|
||||
* original content).
|
||||
*/
|
||||
public class TermVectorOffsetInfo {
|
||||
/**
|
||||
* Convenience declaration when creating a {@link org.apache.lucene.index.TermPositionVector} that stores only position information.
|
||||
*/
|
||||
public transient static final TermVectorOffsetInfo[] EMPTY_OFFSET_INFO = new TermVectorOffsetInfo[0];
|
||||
private int startOffset;
|
||||
private int endOffset;
|
||||
|
||||
public TermVectorOffsetInfo() {
|
||||
}
|
||||
|
||||
public TermVectorOffsetInfo(int startOffset, int endOffset) {
|
||||
this.endOffset = endOffset;
|
||||
this.startOffset = startOffset;
|
||||
}
|
||||
|
||||
/**
|
||||
* The accessor for the ending offset for the term
|
||||
* @return The offset
|
||||
*/
|
||||
public int getEndOffset() {
|
||||
return endOffset;
|
||||
}
|
||||
|
||||
public void setEndOffset(int endOffset) {
|
||||
this.endOffset = endOffset;
|
||||
}
|
||||
|
||||
/**
|
||||
* The accessor for the starting offset of the term.
|
||||
*
|
||||
* @return The offset
|
||||
*/
|
||||
public int getStartOffset() {
|
||||
return startOffset;
|
||||
}
|
||||
|
||||
public void setStartOffset(int startOffset) {
|
||||
this.startOffset = startOffset;
|
||||
}
|
||||
|
||||
/**
|
||||
* Two TermVectorOffsetInfos are equals if both the start and end offsets are the same
|
||||
* @param o The comparison Object
|
||||
* @return true if both {@link #getStartOffset()} and {@link #getEndOffset()} are the same for both objects.
|
||||
*/
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) return true;
|
||||
if (!(o instanceof TermVectorOffsetInfo)) return false;
|
||||
|
||||
final TermVectorOffsetInfo termVectorOffsetInfo = (TermVectorOffsetInfo) o;
|
||||
|
||||
if (endOffset != termVectorOffsetInfo.endOffset) return false;
|
||||
if (startOffset != termVectorOffsetInfo.startOffset) return false;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
int result;
|
||||
result = startOffset;
|
||||
result = 29 * result + endOffset;
|
||||
return result;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,184 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.index.codecs.TermVectorsWriter;
|
||||
import org.apache.lucene.store.FlushInfo;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
final class TermVectorsConsumer extends TermsHashConsumer {
|
||||
|
||||
TermVectorsWriter writer;
|
||||
final DocumentsWriterPerThread docWriter;
|
||||
int freeCount;
|
||||
int lastDocID;
|
||||
|
||||
final DocumentsWriterPerThread.DocState docState;
|
||||
final BytesRef flushTerm = new BytesRef();
|
||||
|
||||
// Used by perField when serializing the term vectors
|
||||
final ByteSliceReader vectorSliceReaderPos = new ByteSliceReader();
|
||||
final ByteSliceReader vectorSliceReaderOff = new ByteSliceReader();
|
||||
boolean hasVectors;
|
||||
|
||||
public TermVectorsConsumer(DocumentsWriterPerThread docWriter) {
|
||||
this.docWriter = docWriter;
|
||||
docState = docWriter.docState;
|
||||
}
|
||||
|
||||
@Override
|
||||
void flush(Map<FieldInfo, TermsHashConsumerPerField> fieldsToFlush, final SegmentWriteState state) throws IOException {
|
||||
if (writer != null) {
|
||||
// At least one doc in this run had term vectors enabled
|
||||
try {
|
||||
fill(state.numDocs);
|
||||
assert state.segmentName != null;
|
||||
writer.finish(state.numDocs);
|
||||
} finally {
|
||||
IOUtils.close(writer);
|
||||
writer = null;
|
||||
|
||||
lastDocID = 0;
|
||||
hasVectors = false;
|
||||
}
|
||||
}
|
||||
|
||||
for (final TermsHashConsumerPerField field : fieldsToFlush.values() ) {
|
||||
TermVectorsConsumerPerField perField = (TermVectorsConsumerPerField) field;
|
||||
perField.termsHashPerField.reset();
|
||||
perField.shrinkHash();
|
||||
}
|
||||
}
|
||||
|
||||
/** Fills in no-term-vectors for all docs we haven't seen
|
||||
* since the last doc that had term vectors. */
|
||||
void fill(int docID) throws IOException {
|
||||
while(lastDocID < docID) {
|
||||
writer.startDocument(0);
|
||||
lastDocID++;
|
||||
}
|
||||
}
|
||||
|
||||
private final void initTermVectorsWriter() throws IOException {
|
||||
if (writer == null) {
|
||||
IOContext context = new IOContext(new FlushInfo(docWriter.getNumDocsInRAM(), docWriter.bytesUsed()));
|
||||
writer = docWriter.codec.termVectorsFormat().vectorsWriter(docWriter.directory, docWriter.getSegment(), context);
|
||||
lastDocID = 0;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
void finishDocument(TermsHash termsHash) throws IOException {
|
||||
|
||||
assert docWriter.writer.testPoint("TermVectorsTermsWriter.finishDocument start");
|
||||
|
||||
if (!hasVectors) {
|
||||
return;
|
||||
}
|
||||
|
||||
initTermVectorsWriter();
|
||||
|
||||
fill(docState.docID);
|
||||
|
||||
// Append term vectors to the real outputs:
|
||||
writer.startDocument(numVectorFields);
|
||||
for (int i = 0; i < numVectorFields; i++) {
|
||||
perFields[i].finishDocument();
|
||||
}
|
||||
|
||||
assert lastDocID == docState.docID: "lastDocID=" + lastDocID + " docState.docID=" + docState.docID;
|
||||
|
||||
lastDocID++;
|
||||
|
||||
termsHash.reset();
|
||||
reset();
|
||||
assert docWriter.writer.testPoint("TermVectorsTermsWriter.finishDocument end");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort() {
|
||||
hasVectors = false;
|
||||
|
||||
if (writer != null) {
|
||||
writer.abort();
|
||||
writer = null;
|
||||
}
|
||||
|
||||
lastDocID = 0;
|
||||
|
||||
reset();
|
||||
}
|
||||
|
||||
int numVectorFields;
|
||||
|
||||
TermVectorsConsumerPerField[] perFields;
|
||||
|
||||
void reset() {
|
||||
numVectorFields = 0;
|
||||
perFields = new TermVectorsConsumerPerField[1];
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermsHashConsumerPerField addField(TermsHashPerField termsHashPerField, FieldInfo fieldInfo) {
|
||||
return new TermVectorsConsumerPerField(termsHashPerField, this, fieldInfo);
|
||||
}
|
||||
|
||||
void addFieldToFlush(TermVectorsConsumerPerField fieldToFlush) {
|
||||
if (numVectorFields == perFields.length) {
|
||||
int newSize = ArrayUtil.oversize(numVectorFields + 1, RamUsageEstimator.NUM_BYTES_OBJECT_REF);
|
||||
TermVectorsConsumerPerField[] newArray = new TermVectorsConsumerPerField[newSize];
|
||||
System.arraycopy(perFields, 0, newArray, 0, numVectorFields);
|
||||
perFields = newArray;
|
||||
}
|
||||
|
||||
perFields[numVectorFields++] = fieldToFlush;
|
||||
}
|
||||
|
||||
@Override
|
||||
void startDocument() throws IOException {
|
||||
assert clearLastVectorFieldName();
|
||||
reset();
|
||||
}
|
||||
|
||||
// Called only by assert
|
||||
final boolean clearLastVectorFieldName() {
|
||||
lastVectorFieldName = null;
|
||||
return true;
|
||||
}
|
||||
|
||||
// Called only by assert
|
||||
String lastVectorFieldName;
|
||||
final boolean vectorFieldsInOrder(FieldInfo fi) {
|
||||
try {
|
||||
if (lastVectorFieldName != null)
|
||||
return lastVectorFieldName.compareTo(fi.name) < 0;
|
||||
else
|
||||
return true;
|
||||
} finally {
|
||||
lastVectorFieldName = fi.name;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -20,15 +20,15 @@ package org.apache.lucene.index;
|
|||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.index.codecs.TermVectorsWriter;
|
||||
import org.apache.lucene.util.ByteBlockPool;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
final class TermVectorsTermsWriterPerField extends TermsHashConsumerPerField {
|
||||
final class TermVectorsConsumerPerField extends TermsHashConsumerPerField {
|
||||
|
||||
final TermsHashPerField termsHashPerField;
|
||||
final TermVectorsTermsWriter termsWriter;
|
||||
final TermVectorsConsumer termsWriter;
|
||||
final FieldInfo fieldInfo;
|
||||
final DocumentsWriterPerThread.DocState docState;
|
||||
final FieldInvertState fieldState;
|
||||
|
@ -40,7 +40,7 @@ final class TermVectorsTermsWriterPerField extends TermsHashConsumerPerField {
|
|||
int maxNumPostings;
|
||||
OffsetAttribute offsetAttribute = null;
|
||||
|
||||
public TermVectorsTermsWriterPerField(TermsHashPerField termsHashPerField, TermVectorsTermsWriter termsWriter, FieldInfo fieldInfo) {
|
||||
public TermVectorsConsumerPerField(TermsHashPerField termsHashPerField, TermVectorsConsumer termsWriter, FieldInfo fieldInfo) {
|
||||
this.termsHashPerField = termsHashPerField;
|
||||
this.termsWriter = termsWriter;
|
||||
this.fieldInfo = fieldInfo;
|
||||
|
@ -117,25 +117,17 @@ final class TermVectorsTermsWriterPerField extends TermsHashConsumerPerField {
|
|||
assert termsWriter.vectorFieldsInOrder(fieldInfo);
|
||||
|
||||
TermVectorsPostingsArray postings = (TermVectorsPostingsArray) termsHashPerField.postingsArray;
|
||||
final IndexOutput tvf = termsWriter.tvf;
|
||||
final TermVectorsWriter tv = termsWriter.writer;
|
||||
|
||||
// TODO: we may want to make this sort in same order
|
||||
// as Codec's terms dict?
|
||||
final int[] termIDs = termsHashPerField.sortPostings(BytesRef.getUTF8SortedAsUnicodeComparator());
|
||||
|
||||
tvf.writeVInt(numPostings);
|
||||
byte bits = 0x0;
|
||||
if (doVectorPositions)
|
||||
bits |= TermVectorsReader.STORE_POSITIONS_WITH_TERMVECTOR;
|
||||
if (doVectorOffsets)
|
||||
bits |= TermVectorsReader.STORE_OFFSET_WITH_TERMVECTOR;
|
||||
tvf.writeByte(bits);
|
||||
|
||||
int lastLen = 0;
|
||||
byte[] lastBytes = null;
|
||||
int lastStart = 0;
|
||||
|
||||
final ByteSliceReader reader = termsWriter.vectorSliceReader;
|
||||
tv.startField(fieldInfo, numPostings, doVectorPositions, doVectorOffsets);
|
||||
|
||||
final ByteSliceReader posReader = doVectorPositions ? termsWriter.vectorSliceReaderPos : null;
|
||||
final ByteSliceReader offReader = doVectorOffsets ? termsWriter.vectorSliceReaderOff : null;
|
||||
|
||||
final ByteBlockPool termBytePool = termsHashPerField.termBytePool;
|
||||
|
||||
for(int j=0;j<numPostings;j++) {
|
||||
|
@ -144,41 +136,22 @@ final class TermVectorsTermsWriterPerField extends TermsHashConsumerPerField {
|
|||
|
||||
// Get BytesRef
|
||||
termBytePool.setBytesRef(flushTerm, postings.textStarts[termID]);
|
||||
|
||||
// Compute common byte prefix between last term and
|
||||
// this term
|
||||
int prefix = 0;
|
||||
if (j > 0) {
|
||||
while(prefix < lastLen && prefix < flushTerm.length) {
|
||||
if (lastBytes[lastStart+prefix] != flushTerm.bytes[flushTerm.offset+prefix]) {
|
||||
break;
|
||||
}
|
||||
prefix++;
|
||||
tv.startTerm(flushTerm, freq);
|
||||
|
||||
if (doVectorPositions || doVectorOffsets) {
|
||||
if (posReader != null) {
|
||||
termsHashPerField.initReader(posReader, termID, 0);
|
||||
}
|
||||
}
|
||||
|
||||
lastLen = flushTerm.length;
|
||||
lastBytes = flushTerm.bytes;
|
||||
lastStart = flushTerm.offset;
|
||||
|
||||
final int suffix = flushTerm.length - prefix;
|
||||
tvf.writeVInt(prefix);
|
||||
tvf.writeVInt(suffix);
|
||||
tvf.writeBytes(flushTerm.bytes, lastStart+prefix, suffix);
|
||||
tvf.writeVInt(freq);
|
||||
|
||||
if (doVectorPositions) {
|
||||
termsHashPerField.initReader(reader, termID, 0);
|
||||
reader.writeTo(tvf);
|
||||
}
|
||||
|
||||
if (doVectorOffsets) {
|
||||
termsHashPerField.initReader(reader, termID, 1);
|
||||
reader.writeTo(tvf);
|
||||
if (offReader != null) {
|
||||
termsHashPerField.initReader(offReader, termID, 1);
|
||||
}
|
||||
tv.addProx(freq, posReader, offReader);
|
||||
}
|
||||
}
|
||||
|
||||
termsHashPerField.reset();
|
||||
// commit the termVectors once successful success - FI will otherwise reset them
|
||||
fieldInfo.commitVectors();
|
||||
}
|
||||
|
||||
void shrinkHash() {
|
|
@ -1,538 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
|
||||
class TermVectorsReader implements Cloneable, Closeable {
|
||||
|
||||
// NOTE: if you make a new format, it must be larger than
|
||||
// the current format
|
||||
|
||||
// Changed strings to UTF8 with length-in-bytes not length-in-chars
|
||||
static final int FORMAT_UTF8_LENGTH_IN_BYTES = 4;
|
||||
|
||||
// NOTE: always change this if you switch to a new format!
|
||||
// whenever you add a new format, make it 1 larger (positive version logic)!
|
||||
static final int FORMAT_CURRENT = FORMAT_UTF8_LENGTH_IN_BYTES;
|
||||
|
||||
// when removing support for old versions, leave the last supported version here
|
||||
static final int FORMAT_MINIMUM = FORMAT_UTF8_LENGTH_IN_BYTES;
|
||||
|
||||
//The size in bytes that the FORMAT_VERSION will take up at the beginning of each file
|
||||
static final int FORMAT_SIZE = 4;
|
||||
|
||||
static final byte STORE_POSITIONS_WITH_TERMVECTOR = 0x1;
|
||||
static final byte STORE_OFFSET_WITH_TERMVECTOR = 0x2;
|
||||
|
||||
private FieldInfos fieldInfos;
|
||||
|
||||
private IndexInput tvx;
|
||||
private IndexInput tvd;
|
||||
private IndexInput tvf;
|
||||
private int size;
|
||||
private int numTotalDocs;
|
||||
|
||||
// The docID offset where our docs begin in the index
|
||||
// file. This will be 0 if we have our own private file.
|
||||
private int docStoreOffset;
|
||||
|
||||
private final int format;
|
||||
|
||||
TermVectorsReader(Directory d, String segment, FieldInfos fieldInfos, IOContext context)
|
||||
throws CorruptIndexException, IOException {
|
||||
this(d, segment, fieldInfos, context, -1, 0);
|
||||
}
|
||||
|
||||
TermVectorsReader(Directory d, String segment, FieldInfos fieldInfos, IOContext context, int docStoreOffset, int size)
|
||||
throws CorruptIndexException, IOException {
|
||||
boolean success = false;
|
||||
|
||||
try {
|
||||
String idxName = IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_INDEX_EXTENSION);
|
||||
tvx = d.openInput(idxName, context);
|
||||
format = checkValidFormat(tvx, idxName);
|
||||
String fn = IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
|
||||
tvd = d.openInput(fn, context);
|
||||
final int tvdFormat = checkValidFormat(tvd, fn);
|
||||
fn = IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_FIELDS_EXTENSION);
|
||||
tvf = d.openInput(fn, context);
|
||||
final int tvfFormat = checkValidFormat(tvf, fn);
|
||||
|
||||
assert format == tvdFormat;
|
||||
assert format == tvfFormat;
|
||||
|
||||
numTotalDocs = (int) (tvx.length() >> 4);
|
||||
|
||||
if (-1 == docStoreOffset) {
|
||||
this.docStoreOffset = 0;
|
||||
this.size = numTotalDocs;
|
||||
assert size == 0 || numTotalDocs == size;
|
||||
} else {
|
||||
this.docStoreOffset = docStoreOffset;
|
||||
this.size = size;
|
||||
// Verify the file is long enough to hold all of our
|
||||
// docs
|
||||
assert numTotalDocs >= size + docStoreOffset: "numTotalDocs=" + numTotalDocs + " size=" + size + " docStoreOffset=" + docStoreOffset;
|
||||
}
|
||||
|
||||
this.fieldInfos = fieldInfos;
|
||||
success = true;
|
||||
} finally {
|
||||
// With lock-less commits, it's entirely possible (and
|
||||
// fine) to hit a FileNotFound exception above. In
|
||||
// this case, we want to explicitly close any subset
|
||||
// of things that were opened so that we don't have to
|
||||
// wait for a GC to do so.
|
||||
if (!success) {
|
||||
close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Used for bulk copy when merging
|
||||
IndexInput getTvdStream() {
|
||||
return tvd;
|
||||
}
|
||||
|
||||
// Used for bulk copy when merging
|
||||
IndexInput getTvfStream() {
|
||||
return tvf;
|
||||
}
|
||||
|
||||
private void seekTvx(final int docNum) throws IOException {
|
||||
tvx.seek((docNum + docStoreOffset) * 16L + FORMAT_SIZE);
|
||||
}
|
||||
|
||||
boolean canReadRawDocs() {
|
||||
// we can always read raw docs, unless the term vectors
|
||||
// didn't exist
|
||||
return format != 0;
|
||||
}
|
||||
|
||||
/** Retrieve the length (in bytes) of the tvd and tvf
|
||||
* entries for the next numDocs starting with
|
||||
* startDocID. This is used for bulk copying when
|
||||
* merging segments, if the field numbers are
|
||||
* congruent. Once this returns, the tvf & tvd streams
|
||||
* are seeked to the startDocID. */
|
||||
final void rawDocs(int[] tvdLengths, int[] tvfLengths, int startDocID, int numDocs) throws IOException {
|
||||
|
||||
if (tvx == null) {
|
||||
Arrays.fill(tvdLengths, 0);
|
||||
Arrays.fill(tvfLengths, 0);
|
||||
return;
|
||||
}
|
||||
|
||||
seekTvx(startDocID);
|
||||
|
||||
long tvdPosition = tvx.readLong();
|
||||
tvd.seek(tvdPosition);
|
||||
|
||||
long tvfPosition = tvx.readLong();
|
||||
tvf.seek(tvfPosition);
|
||||
|
||||
long lastTvdPosition = tvdPosition;
|
||||
long lastTvfPosition = tvfPosition;
|
||||
|
||||
int count = 0;
|
||||
while (count < numDocs) {
|
||||
final int docID = docStoreOffset + startDocID + count + 1;
|
||||
assert docID <= numTotalDocs;
|
||||
if (docID < numTotalDocs) {
|
||||
tvdPosition = tvx.readLong();
|
||||
tvfPosition = tvx.readLong();
|
||||
} else {
|
||||
tvdPosition = tvd.length();
|
||||
tvfPosition = tvf.length();
|
||||
assert count == numDocs-1;
|
||||
}
|
||||
tvdLengths[count] = (int) (tvdPosition-lastTvdPosition);
|
||||
tvfLengths[count] = (int) (tvfPosition-lastTvfPosition);
|
||||
count++;
|
||||
lastTvdPosition = tvdPosition;
|
||||
lastTvfPosition = tvfPosition;
|
||||
}
|
||||
}
|
||||
|
||||
private int checkValidFormat(IndexInput in, String fn) throws CorruptIndexException, IOException
|
||||
{
|
||||
int format = in.readInt();
|
||||
if (format < FORMAT_MINIMUM)
|
||||
throw new IndexFormatTooOldException(in, format, FORMAT_MINIMUM, FORMAT_CURRENT);
|
||||
if (format > FORMAT_CURRENT)
|
||||
throw new IndexFormatTooNewException(in, format, FORMAT_MINIMUM, FORMAT_CURRENT);
|
||||
return format;
|
||||
}
|
||||
|
||||
public void close() throws IOException {
|
||||
IOUtils.close(tvx, tvd, tvf);
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @return The number of documents in the reader
|
||||
*/
|
||||
int size() {
|
||||
return size;
|
||||
}
|
||||
|
||||
public void get(int docNum, String field, TermVectorMapper mapper) throws IOException {
|
||||
if (tvx != null) {
|
||||
int fieldNumber = fieldInfos.fieldNumber(field);
|
||||
//We need to account for the FORMAT_SIZE at when seeking in the tvx
|
||||
//We don't need to do this in other seeks because we already have the
|
||||
// file pointer
|
||||
//that was written in another file
|
||||
seekTvx(docNum);
|
||||
//System.out.println("TVX Pointer: " + tvx.getFilePointer());
|
||||
long tvdPosition = tvx.readLong();
|
||||
|
||||
tvd.seek(tvdPosition);
|
||||
int fieldCount = tvd.readVInt();
|
||||
//System.out.println("Num Fields: " + fieldCount);
|
||||
// There are only a few fields per document. We opt for a full scan
|
||||
// rather then requiring that they be ordered. We need to read through
|
||||
// all of the fields anyway to get to the tvf pointers.
|
||||
int number = 0;
|
||||
int found = -1;
|
||||
for (int i = 0; i < fieldCount; i++) {
|
||||
number = tvd.readVInt();
|
||||
if (number == fieldNumber)
|
||||
found = i;
|
||||
}
|
||||
|
||||
// This field, although valid in the segment, was not found in this
|
||||
// document
|
||||
if (found != -1) {
|
||||
// Compute position in the tvf file
|
||||
long position = tvx.readLong();
|
||||
for (int i = 1; i <= found; i++)
|
||||
position += tvd.readVLong();
|
||||
|
||||
mapper.setDocumentNumber(docNum);
|
||||
readTermVector(field, position, mapper);
|
||||
} else {
|
||||
//System.out.println("Fieldable not found");
|
||||
}
|
||||
} else {
|
||||
//System.out.println("No tvx file");
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
/**
|
||||
* Retrieve the term vector for the given document and field
|
||||
* @param docNum The document number to retrieve the vector for
|
||||
* @param field The field within the document to retrieve
|
||||
* @return The TermFreqVector for the document and field or null if there is no termVector for this field.
|
||||
* @throws IOException if there is an error reading the term vector files
|
||||
*/
|
||||
TermFreqVector get(int docNum, String field) throws IOException {
|
||||
// Check if no term vectors are available for this segment at all
|
||||
ParallelArrayTermVectorMapper mapper = new ParallelArrayTermVectorMapper();
|
||||
get(docNum, field, mapper);
|
||||
|
||||
return mapper.materializeVector();
|
||||
}
|
||||
|
||||
// Reads the String[] fields; you have to pre-seek tvd to
|
||||
// the right point
|
||||
private String[] readFields(int fieldCount) throws IOException {
|
||||
int number = 0;
|
||||
String[] fields = new String[fieldCount];
|
||||
|
||||
for (int i = 0; i < fieldCount; i++) {
|
||||
number = tvd.readVInt();
|
||||
fields[i] = fieldInfos.fieldName(number);
|
||||
}
|
||||
|
||||
return fields;
|
||||
}
|
||||
|
||||
// Reads the long[] offsets into TVF; you have to pre-seek
|
||||
// tvx/tvd to the right point
|
||||
private long[] readTvfPointers(int fieldCount) throws IOException {
|
||||
// Compute position in the tvf file
|
||||
long position = tvx.readLong();
|
||||
|
||||
long[] tvfPointers = new long[fieldCount];
|
||||
tvfPointers[0] = position;
|
||||
|
||||
for (int i = 1; i < fieldCount; i++) {
|
||||
position += tvd.readVLong();
|
||||
tvfPointers[i] = position;
|
||||
}
|
||||
|
||||
return tvfPointers;
|
||||
}
|
||||
|
||||
/**
|
||||
* Return all term vectors stored for this document or null if the could not be read in.
|
||||
*
|
||||
* @param docNum The document number to retrieve the vector for
|
||||
* @return All term frequency vectors
|
||||
* @throws IOException if there is an error reading the term vector files
|
||||
*/
|
||||
TermFreqVector[] get(int docNum) throws IOException {
|
||||
TermFreqVector[] result = null;
|
||||
if (tvx != null) {
|
||||
//We need to offset by
|
||||
seekTvx(docNum);
|
||||
long tvdPosition = tvx.readLong();
|
||||
|
||||
tvd.seek(tvdPosition);
|
||||
int fieldCount = tvd.readVInt();
|
||||
|
||||
// No fields are vectorized for this document
|
||||
if (fieldCount != 0) {
|
||||
final String[] fields = readFields(fieldCount);
|
||||
final long[] tvfPointers = readTvfPointers(fieldCount);
|
||||
result = readTermVectors(docNum, fields, tvfPointers);
|
||||
}
|
||||
} else {
|
||||
//System.out.println("No tvx file");
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
public void get(int docNumber, TermVectorMapper mapper) throws IOException {
|
||||
// Check if no term vectors are available for this segment at all
|
||||
if (tvx != null) {
|
||||
//We need to offset by
|
||||
|
||||
seekTvx(docNumber);
|
||||
long tvdPosition = tvx.readLong();
|
||||
|
||||
tvd.seek(tvdPosition);
|
||||
int fieldCount = tvd.readVInt();
|
||||
|
||||
// No fields are vectorized for this document
|
||||
if (fieldCount != 0) {
|
||||
final String[] fields = readFields(fieldCount);
|
||||
final long[] tvfPointers = readTvfPointers(fieldCount);
|
||||
mapper.setDocumentNumber(docNumber);
|
||||
readTermVectors(fields, tvfPointers, mapper);
|
||||
}
|
||||
} else {
|
||||
//System.out.println("No tvx file");
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private SegmentTermVector[] readTermVectors(int docNum, String fields[], long tvfPointers[])
|
||||
throws IOException {
|
||||
SegmentTermVector res[] = new SegmentTermVector[fields.length];
|
||||
for (int i = 0; i < fields.length; i++) {
|
||||
ParallelArrayTermVectorMapper mapper = new ParallelArrayTermVectorMapper();
|
||||
mapper.setDocumentNumber(docNum);
|
||||
readTermVector(fields[i], tvfPointers[i], mapper);
|
||||
res[i] = (SegmentTermVector) mapper.materializeVector();
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
private void readTermVectors(String fields[], long tvfPointers[], TermVectorMapper mapper)
|
||||
throws IOException {
|
||||
for (int i = 0; i < fields.length; i++) {
|
||||
readTermVector(fields[i], tvfPointers[i], mapper);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
*
|
||||
* @param field The field to read in
|
||||
* @param tvfPointer The pointer within the tvf file where we should start reading
|
||||
* @param mapper The mapper used to map the TermVector
|
||||
* @throws IOException
|
||||
*/
|
||||
private void readTermVector(String field, long tvfPointer, TermVectorMapper mapper)
|
||||
throws IOException {
|
||||
|
||||
// Now read the data from specified position
|
||||
//We don't need to offset by the FORMAT here since the pointer already includes the offset
|
||||
tvf.seek(tvfPointer);
|
||||
|
||||
int numTerms = tvf.readVInt();
|
||||
//System.out.println("Num Terms: " + numTerms);
|
||||
// If no terms - return a constant empty termvector. However, this should never occur!
|
||||
if (numTerms == 0)
|
||||
return;
|
||||
|
||||
boolean storePositions;
|
||||
boolean storeOffsets;
|
||||
|
||||
byte bits = tvf.readByte();
|
||||
storePositions = (bits & STORE_POSITIONS_WITH_TERMVECTOR) != 0;
|
||||
storeOffsets = (bits & STORE_OFFSET_WITH_TERMVECTOR) != 0;
|
||||
|
||||
mapper.setExpectations(field, numTerms, storeOffsets, storePositions);
|
||||
int start = 0;
|
||||
int deltaLength = 0;
|
||||
int totalLength = 0;
|
||||
byte[] byteBuffer;
|
||||
|
||||
// init the buffer
|
||||
byteBuffer = new byte[20];
|
||||
|
||||
for (int i = 0; i < numTerms; i++) {
|
||||
start = tvf.readVInt();
|
||||
deltaLength = tvf.readVInt();
|
||||
totalLength = start + deltaLength;
|
||||
|
||||
final BytesRef term = new BytesRef(totalLength);
|
||||
|
||||
// Term stored as utf8 bytes
|
||||
if (byteBuffer.length < totalLength) {
|
||||
byteBuffer = ArrayUtil.grow(byteBuffer, totalLength);
|
||||
}
|
||||
tvf.readBytes(byteBuffer, start, deltaLength);
|
||||
System.arraycopy(byteBuffer, 0, term.bytes, 0, totalLength);
|
||||
term.length = totalLength;
|
||||
int freq = tvf.readVInt();
|
||||
int [] positions = null;
|
||||
if (storePositions) { //read in the positions
|
||||
//does the mapper even care about positions?
|
||||
if (!mapper.isIgnoringPositions()) {
|
||||
positions = new int[freq];
|
||||
int prevPosition = 0;
|
||||
for (int j = 0; j < freq; j++)
|
||||
{
|
||||
positions[j] = prevPosition + tvf.readVInt();
|
||||
prevPosition = positions[j];
|
||||
}
|
||||
} else {
|
||||
//we need to skip over the positions. Since these are VInts, I don't believe there is anyway to know for sure how far to skip
|
||||
//
|
||||
for (int j = 0; j < freq; j++)
|
||||
{
|
||||
tvf.readVInt();
|
||||
}
|
||||
}
|
||||
}
|
||||
TermVectorOffsetInfo[] offsets = null;
|
||||
if (storeOffsets) {
|
||||
//does the mapper even care about offsets?
|
||||
if (!mapper.isIgnoringOffsets()) {
|
||||
offsets = new TermVectorOffsetInfo[freq];
|
||||
int prevOffset = 0;
|
||||
for (int j = 0; j < freq; j++) {
|
||||
int startOffset = prevOffset + tvf.readVInt();
|
||||
int endOffset = startOffset + tvf.readVInt();
|
||||
offsets[j] = new TermVectorOffsetInfo(startOffset, endOffset);
|
||||
prevOffset = endOffset;
|
||||
}
|
||||
} else {
|
||||
for (int j = 0; j < freq; j++){
|
||||
tvf.readVInt();
|
||||
tvf.readVInt();
|
||||
}
|
||||
}
|
||||
}
|
||||
mapper.map(term, freq, offsets, positions);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Object clone() throws CloneNotSupportedException {
|
||||
|
||||
final TermVectorsReader clone = (TermVectorsReader) super.clone();
|
||||
|
||||
// These are null when a TermVectorsReader was created
|
||||
// on a segment that did not have term vectors saved
|
||||
if (tvx != null && tvd != null && tvf != null) {
|
||||
clone.tvx = (IndexInput) tvx.clone();
|
||||
clone.tvd = (IndexInput) tvd.clone();
|
||||
clone.tvf = (IndexInput) tvf.clone();
|
||||
}
|
||||
|
||||
return clone;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Models the existing parallel array structure
|
||||
*/
|
||||
class ParallelArrayTermVectorMapper extends TermVectorMapper
|
||||
{
|
||||
|
||||
private BytesRef[] terms;
|
||||
private int[] termFreqs;
|
||||
private int positions[][];
|
||||
private TermVectorOffsetInfo offsets[][];
|
||||
private int currentPosition;
|
||||
private boolean storingOffsets;
|
||||
private boolean storingPositions;
|
||||
private String field;
|
||||
|
||||
@Override
|
||||
public void setExpectations(String field, int numTerms, boolean storeOffsets, boolean storePositions) {
|
||||
this.field = field;
|
||||
terms = new BytesRef[numTerms];
|
||||
termFreqs = new int[numTerms];
|
||||
this.storingOffsets = storeOffsets;
|
||||
this.storingPositions = storePositions;
|
||||
if(storePositions)
|
||||
this.positions = new int[numTerms][];
|
||||
if(storeOffsets)
|
||||
this.offsets = new TermVectorOffsetInfo[numTerms][];
|
||||
}
|
||||
|
||||
@Override
|
||||
public void map(BytesRef term, int frequency, TermVectorOffsetInfo[] offsets, int[] positions) {
|
||||
terms[currentPosition] = term;
|
||||
termFreqs[currentPosition] = frequency;
|
||||
if (storingOffsets)
|
||||
{
|
||||
this.offsets[currentPosition] = offsets;
|
||||
}
|
||||
if (storingPositions)
|
||||
{
|
||||
this.positions[currentPosition] = positions;
|
||||
}
|
||||
currentPosition++;
|
||||
}
|
||||
|
||||
/**
|
||||
* Construct the vector
|
||||
* @return The {@link TermFreqVector} based on the mappings.
|
||||
*/
|
||||
public TermFreqVector materializeVector() {
|
||||
SegmentTermVector tv = null;
|
||||
if (field != null && terms != null) {
|
||||
if (storingPositions || storingOffsets) {
|
||||
tv = new SegmentTermPositionVector(field, terms, termFreqs, positions, offsets);
|
||||
} else {
|
||||
tv = new SegmentTermVector(field, terms, termFreqs);
|
||||
}
|
||||
}
|
||||
return tv;
|
||||
}
|
||||
}
|
|
@ -1,242 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.store.FlushInfo;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.store.IOContext.Context;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
final class TermVectorsTermsWriter extends TermsHashConsumer {
|
||||
|
||||
final DocumentsWriterPerThread docWriter;
|
||||
int freeCount;
|
||||
IndexOutput tvx;
|
||||
IndexOutput tvd;
|
||||
IndexOutput tvf;
|
||||
int lastDocID;
|
||||
|
||||
final DocumentsWriterPerThread.DocState docState;
|
||||
final BytesRef flushTerm = new BytesRef();
|
||||
|
||||
// Used by perField when serializing the term vectors
|
||||
final ByteSliceReader vectorSliceReader = new ByteSliceReader();
|
||||
boolean hasVectors;
|
||||
|
||||
public TermVectorsTermsWriter(DocumentsWriterPerThread docWriter) {
|
||||
this.docWriter = docWriter;
|
||||
docState = docWriter.docState;
|
||||
}
|
||||
|
||||
@Override
|
||||
void flush(Map<FieldInfo, TermsHashConsumerPerField> fieldsToFlush, final SegmentWriteState state) throws IOException {
|
||||
if (tvx != null) {
|
||||
// At least one doc in this run had term vectors enabled
|
||||
fill(state.numDocs);
|
||||
assert state.segmentName != null;
|
||||
String idxName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.VECTORS_INDEX_EXTENSION);
|
||||
IOUtils.close(tvx, tvf, tvd);
|
||||
tvx = tvd = tvf = null;
|
||||
if (4+((long) state.numDocs)*16 != state.directory.fileLength(idxName)) {
|
||||
throw new RuntimeException("after flush: tvx size mismatch: " + state.numDocs + " docs vs " + state.directory.fileLength(idxName) + " length in bytes of " + idxName + " file exists?=" + state.directory.fileExists(idxName));
|
||||
}
|
||||
|
||||
lastDocID = 0;
|
||||
hasVectors = false;
|
||||
}
|
||||
|
||||
for (final TermsHashConsumerPerField field : fieldsToFlush.values() ) {
|
||||
TermVectorsTermsWriterPerField perField = (TermVectorsTermsWriterPerField) field;
|
||||
perField.termsHashPerField.reset();
|
||||
perField.shrinkHash();
|
||||
}
|
||||
}
|
||||
|
||||
/** Fills in no-term-vectors for all docs we haven't seen
|
||||
* since the last doc that had term vectors. */
|
||||
void fill(int docID) throws IOException {
|
||||
if (lastDocID < docID) {
|
||||
final long tvfPosition = tvf.getFilePointer();
|
||||
while(lastDocID < docID) {
|
||||
tvx.writeLong(tvd.getFilePointer());
|
||||
tvd.writeVInt(0);
|
||||
tvx.writeLong(tvfPosition);
|
||||
lastDocID++;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private final void initTermVectorsWriter() throws IOException {
|
||||
if (tvx == null) {
|
||||
boolean success = false;
|
||||
try {
|
||||
IOContext context = new IOContext(new FlushInfo(docWriter.getNumDocsInRAM(), docWriter.bytesUsed()));
|
||||
// If we hit an exception while init'ing the term
|
||||
// vector output files, we must abort this segment
|
||||
// because those files will be in an unknown
|
||||
// state:
|
||||
tvx = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_INDEX_EXTENSION), context);
|
||||
tvd = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION), context);
|
||||
tvf = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_FIELDS_EXTENSION), context);
|
||||
|
||||
tvx.writeInt(TermVectorsReader.FORMAT_CURRENT);
|
||||
tvd.writeInt(TermVectorsReader.FORMAT_CURRENT);
|
||||
tvf.writeInt(TermVectorsReader.FORMAT_CURRENT);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeWhileHandlingException(tvx, tvd, tvf);
|
||||
}
|
||||
}
|
||||
|
||||
lastDocID = 0;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
void finishDocument(TermsHash termsHash) throws IOException {
|
||||
|
||||
assert docWriter.writer.testPoint("TermVectorsTermsWriter.finishDocument start");
|
||||
|
||||
if (!hasVectors) {
|
||||
return;
|
||||
}
|
||||
|
||||
initTermVectorsWriter();
|
||||
|
||||
fill(docState.docID);
|
||||
|
||||
// Append term vectors to the real outputs:
|
||||
tvx.writeLong(tvd.getFilePointer());
|
||||
tvx.writeLong(tvf.getFilePointer());
|
||||
tvd.writeVInt(numVectorFields);
|
||||
if (numVectorFields > 0) {
|
||||
for(int i=0;i<numVectorFields;i++) {
|
||||
tvd.writeVInt(perFields[i].fieldInfo.number);
|
||||
}
|
||||
long lastPos = tvf.getFilePointer();
|
||||
perFields[0].finishDocument();
|
||||
perFields[0].fieldInfo.commitVectors();
|
||||
for(int i=1;i<numVectorFields;i++) {
|
||||
long pos = tvf.getFilePointer();
|
||||
tvd.writeVLong(pos-lastPos);
|
||||
lastPos = pos;
|
||||
perFields[i].finishDocument();
|
||||
// commit the termVectors once successful success - FI will otherwise reset them
|
||||
perFields[i].fieldInfo.commitVectors();
|
||||
}
|
||||
}
|
||||
|
||||
assert lastDocID == docState.docID: "lastDocID=" + lastDocID + " docState.docID=" + docState.docID;
|
||||
|
||||
lastDocID++;
|
||||
|
||||
termsHash.reset();
|
||||
reset();
|
||||
assert docWriter.writer.testPoint("TermVectorsTermsWriter.finishDocument end");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort() {
|
||||
hasVectors = false;
|
||||
try {
|
||||
IOUtils.closeWhileHandlingException(tvx, tvd, tvf);
|
||||
} catch (IOException e) {
|
||||
// cannot happen since we suppress exceptions
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
|
||||
try {
|
||||
docWriter.directory.deleteFile(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_INDEX_EXTENSION));
|
||||
} catch (IOException ignored) {
|
||||
}
|
||||
|
||||
try {
|
||||
docWriter.directory.deleteFile(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION));
|
||||
} catch (IOException ignored) {
|
||||
}
|
||||
|
||||
try {
|
||||
docWriter.directory.deleteFile(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_FIELDS_EXTENSION));
|
||||
} catch (IOException ignored) {
|
||||
}
|
||||
|
||||
tvx = tvd = tvf = null;
|
||||
lastDocID = 0;
|
||||
|
||||
reset();
|
||||
}
|
||||
|
||||
int numVectorFields;
|
||||
|
||||
TermVectorsTermsWriterPerField[] perFields;
|
||||
|
||||
void reset() {
|
||||
numVectorFields = 0;
|
||||
perFields = new TermVectorsTermsWriterPerField[1];
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermsHashConsumerPerField addField(TermsHashPerField termsHashPerField, FieldInfo fieldInfo) {
|
||||
return new TermVectorsTermsWriterPerField(termsHashPerField, this, fieldInfo);
|
||||
}
|
||||
|
||||
void addFieldToFlush(TermVectorsTermsWriterPerField fieldToFlush) {
|
||||
if (numVectorFields == perFields.length) {
|
||||
int newSize = ArrayUtil.oversize(numVectorFields + 1, RamUsageEstimator.NUM_BYTES_OBJECT_REF);
|
||||
TermVectorsTermsWriterPerField[] newArray = new TermVectorsTermsWriterPerField[newSize];
|
||||
System.arraycopy(perFields, 0, newArray, 0, numVectorFields);
|
||||
perFields = newArray;
|
||||
}
|
||||
|
||||
perFields[numVectorFields++] = fieldToFlush;
|
||||
}
|
||||
|
||||
@Override
|
||||
void startDocument() throws IOException {
|
||||
assert clearLastVectorFieldName();
|
||||
reset();
|
||||
}
|
||||
|
||||
// Called only by assert
|
||||
final boolean clearLastVectorFieldName() {
|
||||
lastVectorFieldName = null;
|
||||
return true;
|
||||
}
|
||||
|
||||
// Called only by assert
|
||||
String lastVectorFieldName;
|
||||
final boolean vectorFieldsInOrder(FieldInfo fi) {
|
||||
try {
|
||||
if (lastVectorFieldName != null)
|
||||
return lastVectorFieldName.compareTo(fi.name) < 0;
|
||||
else
|
||||
return true;
|
||||
} finally {
|
||||
lastVectorFieldName = fi.name;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -1,199 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
final class TermVectorsWriter {
|
||||
|
||||
private IndexOutput tvx = null, tvd = null, tvf = null;
|
||||
private FieldInfos fieldInfos;
|
||||
|
||||
public TermVectorsWriter(Directory directory, String segment,
|
||||
FieldInfos fieldInfos, IOContext context) throws IOException {
|
||||
boolean success = false;
|
||||
try {
|
||||
// Open files for TermVector storage
|
||||
tvx = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_INDEX_EXTENSION), context);
|
||||
tvx.writeInt(TermVectorsReader.FORMAT_CURRENT);
|
||||
tvd = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION), context);
|
||||
tvd.writeInt(TermVectorsReader.FORMAT_CURRENT);
|
||||
tvf = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_FIELDS_EXTENSION), context);
|
||||
tvf.writeInt(TermVectorsReader.FORMAT_CURRENT);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeWhileHandlingException(tvx, tvd, tvf);
|
||||
}
|
||||
}
|
||||
|
||||
this.fieldInfos = fieldInfos;
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a complete document specified by all its term vectors. If document has no
|
||||
* term vectors, add value for tvx.
|
||||
*
|
||||
* @param vectors
|
||||
* @throws IOException
|
||||
*/
|
||||
public final void addAllDocVectors(TermFreqVector[] vectors) throws IOException {
|
||||
|
||||
tvx.writeLong(tvd.getFilePointer());
|
||||
tvx.writeLong(tvf.getFilePointer());
|
||||
|
||||
if (vectors != null) {
|
||||
final int numFields = vectors.length;
|
||||
tvd.writeVInt(numFields);
|
||||
|
||||
long[] fieldPointers = new long[numFields];
|
||||
|
||||
for (int i=0; i<numFields; i++) {
|
||||
fieldPointers[i] = tvf.getFilePointer();
|
||||
|
||||
final int fieldNumber = fieldInfos.fieldNumber(vectors[i].getField());
|
||||
|
||||
// 1st pass: write field numbers to tvd
|
||||
tvd.writeVInt(fieldNumber);
|
||||
|
||||
final int numTerms = vectors[i].size();
|
||||
tvf.writeVInt(numTerms);
|
||||
|
||||
final TermPositionVector tpVector;
|
||||
|
||||
final byte bits;
|
||||
final boolean storePositions;
|
||||
final boolean storeOffsets;
|
||||
|
||||
if (vectors[i] instanceof TermPositionVector) {
|
||||
// May have positions & offsets
|
||||
tpVector = (TermPositionVector) vectors[i];
|
||||
storePositions = tpVector.size() > 0 && tpVector.getTermPositions(0) != null;
|
||||
storeOffsets = tpVector.size() > 0 && tpVector.getOffsets(0) != null;
|
||||
bits = (byte) ((storePositions ? TermVectorsReader.STORE_POSITIONS_WITH_TERMVECTOR : 0) +
|
||||
(storeOffsets ? TermVectorsReader.STORE_OFFSET_WITH_TERMVECTOR : 0));
|
||||
} else {
|
||||
tpVector = null;
|
||||
bits = 0;
|
||||
storePositions = false;
|
||||
storeOffsets = false;
|
||||
}
|
||||
|
||||
tvf.writeVInt(bits);
|
||||
|
||||
final BytesRef[] terms = vectors[i].getTerms();
|
||||
final int[] freqs = vectors[i].getTermFrequencies();
|
||||
|
||||
for (int j=0; j<numTerms; j++) {
|
||||
|
||||
int start = j == 0 ? 0 : StringHelper.bytesDifference(terms[j-1].bytes,
|
||||
terms[j-1].length,
|
||||
terms[j].bytes,
|
||||
terms[j].length);
|
||||
int length = terms[j].length - start;
|
||||
tvf.writeVInt(start); // write shared prefix length
|
||||
tvf.writeVInt(length); // write delta length
|
||||
tvf.writeBytes(terms[j].bytes, start, length); // write delta bytes
|
||||
|
||||
final int termFreq = freqs[j];
|
||||
|
||||
tvf.writeVInt(termFreq);
|
||||
|
||||
if (storePositions) {
|
||||
final int[] positions = tpVector.getTermPositions(j);
|
||||
if (positions == null)
|
||||
throw new IllegalStateException("Trying to write positions that are null!");
|
||||
assert positions.length == termFreq;
|
||||
|
||||
// use delta encoding for positions
|
||||
int lastPosition = 0;
|
||||
for(int k=0;k<positions.length;k++) {
|
||||
final int position = positions[k];
|
||||
tvf.writeVInt(position-lastPosition);
|
||||
lastPosition = position;
|
||||
}
|
||||
}
|
||||
|
||||
if (storeOffsets) {
|
||||
final TermVectorOffsetInfo[] offsets = tpVector.getOffsets(j);
|
||||
if (offsets == null)
|
||||
throw new IllegalStateException("Trying to write offsets that are null!");
|
||||
assert offsets.length == termFreq;
|
||||
|
||||
// use delta encoding for offsets
|
||||
int lastEndOffset = 0;
|
||||
for(int k=0;k<offsets.length;k++) {
|
||||
final int startOffset = offsets[k].getStartOffset();
|
||||
final int endOffset = offsets[k].getEndOffset();
|
||||
tvf.writeVInt(startOffset-lastEndOffset);
|
||||
tvf.writeVInt(endOffset-startOffset);
|
||||
lastEndOffset = endOffset;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// 2nd pass: write field pointers to tvd
|
||||
if (numFields > 1) {
|
||||
long lastFieldPointer = fieldPointers[0];
|
||||
for (int i=1; i<numFields; i++) {
|
||||
final long fieldPointer = fieldPointers[i];
|
||||
tvd.writeVLong(fieldPointer-lastFieldPointer);
|
||||
lastFieldPointer = fieldPointer;
|
||||
}
|
||||
}
|
||||
} else
|
||||
tvd.writeVInt(0);
|
||||
}
|
||||
|
||||
/**
|
||||
* Do a bulk copy of numDocs documents from reader to our
|
||||
* streams. This is used to expedite merging, if the
|
||||
* field numbers are congruent.
|
||||
*/
|
||||
final void addRawDocuments(TermVectorsReader reader, int[] tvdLengths, int[] tvfLengths, int numDocs) throws IOException {
|
||||
long tvdPosition = tvd.getFilePointer();
|
||||
long tvfPosition = tvf.getFilePointer();
|
||||
long tvdStart = tvdPosition;
|
||||
long tvfStart = tvfPosition;
|
||||
for(int i=0;i<numDocs;i++) {
|
||||
tvx.writeLong(tvdPosition);
|
||||
tvdPosition += tvdLengths[i];
|
||||
tvx.writeLong(tvfPosition);
|
||||
tvfPosition += tvfLengths[i];
|
||||
}
|
||||
tvd.copyBytes(reader.getTvdStream(), tvdPosition-tvdStart);
|
||||
tvf.copyBytes(reader.getTvfStream(), tvfPosition-tvfStart);
|
||||
assert tvd.getFilePointer() == tvdPosition;
|
||||
assert tvf.getFilePointer() == tvfPosition;
|
||||
}
|
||||
|
||||
/** Close all streams. */
|
||||
final void close() throws IOException {
|
||||
// make an effort to close all streams we can but remember and re-throw
|
||||
// the first exception encountered in this process
|
||||
IOUtils.close(tvx, tvd, tvf);
|
||||
}
|
||||
}
|
|
@ -37,8 +37,11 @@ public abstract class Terms {
|
|||
private final CloseableThreadLocal<TermsEnum> threadEnums = new CloseableThreadLocal<TermsEnum>();
|
||||
|
||||
/** Returns an iterator that will step through all
|
||||
* terms. This method will not return null.*/
|
||||
public abstract TermsEnum iterator() throws IOException;
|
||||
* terms. This method will not return null. If you have
|
||||
* a previous TermsEnum, for example from a different
|
||||
* field, you can pass it for possible reuse if the
|
||||
* implementation can do so. */
|
||||
public abstract TermsEnum iterator(TermsEnum reuse) throws IOException;
|
||||
|
||||
/** Returns a TermsEnum that iterates over all terms that
|
||||
* are accepted by the provided {@link
|
||||
|
@ -59,9 +62,9 @@ public abstract class Terms {
|
|||
throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead");
|
||||
}
|
||||
if (startTerm == null) {
|
||||
return new AutomatonTermsEnum(iterator(), compiled);
|
||||
return new AutomatonTermsEnum(iterator(null), compiled);
|
||||
} else {
|
||||
return new AutomatonTermsEnum(iterator(), compiled) {
|
||||
return new AutomatonTermsEnum(iterator(null), compiled) {
|
||||
@Override
|
||||
protected BytesRef nextSeekTerm(BytesRef term) throws IOException {
|
||||
if (term == null) {
|
||||
|
@ -186,7 +189,7 @@ public abstract class Terms {
|
|||
/**
|
||||
* Returns a thread-private {@link TermsEnum} instance. Obtaining
|
||||
* {@link TermsEnum} from this method might be more efficient than using
|
||||
* {@link #iterator()} directly since this method doesn't necessarily create a
|
||||
* {@link #iterator(TermsEnum)} directly since this method doesn't necessarily create a
|
||||
* new {@link TermsEnum} instance.
|
||||
* <p>
|
||||
* NOTE: {@link TermsEnum} instances obtained from this method must not be
|
||||
|
@ -201,7 +204,7 @@ public abstract class Terms {
|
|||
public TermsEnum getThreadTermsEnum() throws IOException {
|
||||
TermsEnum termsEnum = threadEnums.get();
|
||||
if (termsEnum == null) {
|
||||
termsEnum = iterator();
|
||||
termsEnum = iterator(null);
|
||||
threadEnums.set(termsEnum);
|
||||
}
|
||||
return termsEnum;
|
||||
|
|
|
@ -29,7 +29,7 @@ import org.apache.lucene.util.BytesRef;
|
|||
* field. It stores these tokens in a hash table, and
|
||||
* allocates separate byte streams per token. Consumers of
|
||||
* this class, eg {@link FreqProxTermsWriter} and {@link
|
||||
* TermVectorsTermsWriter}, write their own byte streams
|
||||
* TermVectorsConsumer}, write their own byte streams
|
||||
* under each term.
|
||||
*/
|
||||
final class TermsHash extends InvertedDocConsumer {
|
||||
|
|
|
@ -338,7 +338,7 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
for(int idx = tooBigCount; idx<infosSorted.size(); idx++) {
|
||||
final SegmentInfo info = infosSorted.get(idx);
|
||||
if (merging.contains(info)) {
|
||||
mergingBytes += info.sizeInBytes(true);
|
||||
mergingBytes += info.sizeInBytes();
|
||||
} else if (!toBeMerged.contains(info)) {
|
||||
eligible.add(info);
|
||||
}
|
||||
|
@ -430,7 +430,7 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
final long segBytes = size(info);
|
||||
totAfterMergeBytes += segBytes;
|
||||
totAfterMergeBytesFloored += floorSize(segBytes);
|
||||
totBeforeMergeBytes += info.sizeInBytes(true);
|
||||
totBeforeMergeBytes += info.sizeInBytes();
|
||||
}
|
||||
|
||||
// Measure "skew" of the merge, which can range
|
||||
|
@ -632,7 +632,7 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
|
||||
// Segment size in bytes, pro-rated by % deleted
|
||||
private long size(SegmentInfo info) throws IOException {
|
||||
final long byteSize = info.sizeInBytes(true);
|
||||
final long byteSize = info.sizeInBytes();
|
||||
final int delCount = writer.get().numDeletedDocs(info);
|
||||
final double delRatio = (info.docCount <= 0 ? 0.0f : ((double)delCount / (double)info.docCount));
|
||||
assert delRatio <= 1.0;
|
||||
|
|
|
@ -130,7 +130,6 @@ public class BlockTermsReader extends FieldsProducer {
|
|||
seekDir(in, dirOffset);
|
||||
|
||||
final int numFields = in.readVInt();
|
||||
|
||||
for(int i=0;i<numFields;i++) {
|
||||
final int field = in.readVInt();
|
||||
final long numTerms = in.readVLong();
|
||||
|
@ -208,6 +207,11 @@ public class BlockTermsReader extends FieldsProducer {
|
|||
return fields.get(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getUniqueFieldCount() {
|
||||
return fields.size();
|
||||
}
|
||||
|
||||
// Iterates through all fields
|
||||
private class TermFieldsEnum extends FieldsEnum {
|
||||
final Iterator<FieldReader> it;
|
||||
|
@ -229,8 +233,8 @@ public class BlockTermsReader extends FieldsProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public TermsEnum terms() throws IOException {
|
||||
return current.iterator();
|
||||
public Terms terms() throws IOException {
|
||||
return current;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -263,7 +267,7 @@ public class BlockTermsReader extends FieldsProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public TermsEnum iterator() throws IOException {
|
||||
public TermsEnum iterator(TermsEnum reuse) throws IOException {
|
||||
return new SegmentTermsEnum();
|
||||
}
|
||||
|
||||
|
|
|
@ -19,11 +19,8 @@ package org.apache.lucene.index.codecs;
|
|||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.Closeable;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStreamWriter;
|
||||
import java.io.PrintStream;
|
||||
import java.io.Writer;
|
||||
import java.util.Collection;
|
||||
import java.util.Comparator;
|
||||
import java.util.Iterator;
|
||||
|
@ -221,6 +218,11 @@ public class BlockTreeTermsReader extends FieldsProducer {
|
|||
return fields.get(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getUniqueFieldCount() {
|
||||
return fields.size();
|
||||
}
|
||||
|
||||
// Iterates through all fields
|
||||
private class TermFieldsEnum extends FieldsEnum {
|
||||
final Iterator<FieldReader> it;
|
||||
|
@ -242,8 +244,8 @@ public class BlockTreeTermsReader extends FieldsProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public TermsEnum terms() throws IOException {
|
||||
return current.iterator();
|
||||
public Terms terms() throws IOException {
|
||||
return current;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -454,7 +456,7 @@ public class BlockTreeTermsReader extends FieldsProducer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public TermsEnum iterator() throws IOException {
|
||||
public TermsEnum iterator(TermsEnum reuse) throws IOException {
|
||||
return new SegmentTermsEnum();
|
||||
}
|
||||
|
||||
|
@ -914,8 +916,6 @@ public class BlockTreeTermsReader extends FieldsProducer {
|
|||
|
||||
for(int idx=0;idx<=target.length;idx++) {
|
||||
|
||||
boolean lastIsSubBlock = false;
|
||||
|
||||
while (true) {
|
||||
final int savePos = currentFrame.suffixesReader.getPosition();
|
||||
final int saveStartBytePos = currentFrame.startBytePos;
|
||||
|
@ -950,7 +950,6 @@ public class BlockTreeTermsReader extends FieldsProducer {
|
|||
return;
|
||||
}
|
||||
}
|
||||
lastIsSubBlock = isSubBlock;
|
||||
continue;
|
||||
} else if (cmp == 0) {
|
||||
//if (DEBUG) System.out.println(" return term=" + brToString(term));
|
||||
|
|
|
@ -46,6 +46,10 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
|
|||
public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
postingsFormat().files(dir, info, "", files);
|
||||
storedFieldsFormat().files(dir, info, files);
|
||||
termVectorsFormat().files(dir, info, files);
|
||||
fieldInfosFormat().files(dir, info, files);
|
||||
// TODO: segmentInfosFormat should be allowed to declare additional files
|
||||
// if it wants, in addition to segments_N
|
||||
docValuesFormat().files(dir, info, files);
|
||||
}
|
||||
|
||||
|
@ -58,6 +62,12 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
|
|||
/** Encodes/decodes stored fields */
|
||||
public abstract StoredFieldsFormat storedFieldsFormat();
|
||||
|
||||
/** Encodes/decodes term vectors */
|
||||
public abstract TermVectorsFormat termVectorsFormat();
|
||||
|
||||
/** Encodes/decodes field infos file */
|
||||
public abstract FieldInfosFormat fieldInfosFormat();
|
||||
|
||||
/** Encodes/decodes segments file */
|
||||
public abstract SegmentInfosFormat segmentInfosFormat();
|
||||
|
||||
|
|
|
@ -0,0 +1,47 @@
|
|||
package org.apache.lucene.index.codecs;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
/**
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class DefaultFieldInfosFormat extends FieldInfosFormat {
|
||||
private final FieldInfosReader reader = new DefaultFieldInfosReader();
|
||||
private final FieldInfosWriter writer = new DefaultFieldInfosWriter();
|
||||
|
||||
@Override
|
||||
public FieldInfosReader getFieldInfosReader() throws IOException {
|
||||
return reader;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldInfosWriter getFieldInfosWriter() throws IOException {
|
||||
return writer;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
DefaultFieldInfosReader.files(dir, info, files);
|
||||
}
|
||||
}
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue