mirror of https://github.com/apache/lucene.git
Merge branch 'master' into new_rangetypes_or
This commit is contained in:
commit
1cdcd0cdaf
|
@ -38,4 +38,4 @@ Please review the following and check all that apply:
|
|||
- [ ] I have developed this patch against the `master` branch.
|
||||
- [ ] I have run `ant precommit` and the appropriate test suite.
|
||||
- [ ] I have added tests for my changes.
|
||||
- [ ] I have added documentation for the Ref Guide (for Solr changes only).
|
||||
- [ ] I have added documentation for the [Ref Guide](https://github.com/apache/lucene-solr/tree/master/solr/solr-ref-guide) (for Solr changes only).
|
||||
|
|
|
@ -46,6 +46,9 @@ Improvements
|
|||
|
||||
* LUCENE-8769: Introduce Range Query For Multiple Connected Ranges (Atri Sharma)
|
||||
|
||||
* LUCENE-8937: Avoid agressive stemming on numbers in the FrenchMinimalStemmer.
|
||||
(Adrien Gallou via Tomoko Uchida)
|
||||
|
||||
Bug fixes
|
||||
|
||||
* LUCENE-8663: NRTCachingDirectory.slowFileExists may open a file while
|
||||
|
@ -62,6 +65,12 @@ API Changes
|
|||
* LUCENE-8909: IndexWriter#getFieldNames() method is used to get fields present in index. After LUCENE-8316, this
|
||||
method is no longer required. Hence, deprecate IndexWriter#getFieldNames() method. (Adrien Grand, Munendra S N)
|
||||
|
||||
New Features
|
||||
|
||||
* LUCENE-8936: Add SpanishMinimalStemFilter (vinod kumar via Tomoko Uchida)
|
||||
|
||||
* LUCENE-8764: Add "export all terms" feature to Luke. (Leonardo Menezes via Tomoko Uchida)
|
||||
|
||||
Improvements
|
||||
|
||||
* LUCENE-8874: Show SPI names instead of class names in Luke Analysis tab. (Tomoko Uchida)
|
||||
|
@ -77,11 +86,17 @@ Improvements
|
|||
* LUCENE-8916: GraphTokenStreamFiniteStrings preserves all Token attributes
|
||||
through its finite strings TokenStreams (Alan Woodward)
|
||||
|
||||
* LUCENE-8906: Expose Lucene50PostingsFormat.IntBlockTermState as public so that other postings formats can re-use it.
|
||||
(Bruno Roustant)
|
||||
|
||||
Optimizations
|
||||
|
||||
* LUCENE-8922: DisjunctionMaxQuery more efficiently leverages impacts to skip
|
||||
non-competitive hits. (Adrien Grand)
|
||||
|
||||
* LUCENE-8935: BooleanQuery with no scoring clause can now early terminate the query when
|
||||
the total hits is not requested.
|
||||
|
||||
Other
|
||||
|
||||
* LUCENE-8778 LUCENE-8911: Define analyzer SPI names as static final fields and document the names in Javadocs.
|
||||
|
|
|
@ -1,5 +1,20 @@
|
|||
# Apache Lucene Migration Guide
|
||||
|
||||
## Analysis factories now have customizable symbolic names (LUCENE-8778) ##
|
||||
|
||||
The SPI names for concrete subclasses of TokenizerFactory, TokenFilterFactory, and CharfilterFactory are no longer
|
||||
derived from their class name. Instead, each factory must have a static "NAME" field like this:
|
||||
|
||||
/** o.a.l.a.standard.StandardTokenizerFactory's SPI name */
|
||||
public static final String NAME = "standard";
|
||||
|
||||
A factory can be resolved/instantiated with its NAME by using methods such as TokenizerFactory#lookupClass(String)
|
||||
or TokenizerFactory#forName(String, Map<String,String>).
|
||||
|
||||
If there are any user-defined factory classes that don't have proper NAME field, an exception will be thrown
|
||||
when (re)loading factories. e.g., when calling TokenizerFactory#reloadTokenizers(ClassLoader).
|
||||
|
||||
|
||||
## TermsEnum is now fully abstract (LUCENE-8292) ##
|
||||
|
||||
TermsEnum has been changed to be fully abstract, so non-abstract subclass must implement all it's methods.
|
||||
|
|
|
@ -106,17 +106,17 @@ public class MappingCharFilter extends BaseCharFilter {
|
|||
// Fast pass for single character match:
|
||||
assert arc.isFinal();
|
||||
lastMatchLen = 1;
|
||||
lastMatch = arc.output;
|
||||
lastMatch = arc.output();
|
||||
} else {
|
||||
int lookahead = 0;
|
||||
CharsRef output = arc.output;
|
||||
CharsRef output = arc.output();
|
||||
while (true) {
|
||||
lookahead++;
|
||||
|
||||
if (arc.isFinal()) {
|
||||
// Match! (to node is final)
|
||||
lastMatchLen = lookahead;
|
||||
lastMatch = outputs.add(output, arc.nextFinalOutput);
|
||||
lastMatch = outputs.add(output, arc.nextFinalOutput());
|
||||
// Greedy: keep searching to see if there's a
|
||||
// longer match...
|
||||
}
|
||||
|
@ -133,7 +133,7 @@ public class MappingCharFilter extends BaseCharFilter {
|
|||
// Dead end
|
||||
break;
|
||||
}
|
||||
output = outputs.add(output, arc.output);
|
||||
output = outputs.add(output, arc.output());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -50,10 +50,10 @@ public class NormalizeCharMap {
|
|||
final FST.BytesReader fstReader = map.getBytesReader();
|
||||
map.getFirstArc(scratchArc);
|
||||
if (FST.targetHasArcs(scratchArc)) {
|
||||
map.readFirstRealTargetArc(scratchArc.target, scratchArc, fstReader);
|
||||
map.readFirstRealTargetArc(scratchArc.target(), scratchArc, fstReader);
|
||||
while(true) {
|
||||
assert scratchArc.label != FST.END_LABEL;
|
||||
cachedRootArcs.put(Character.valueOf((char) scratchArc.label), new FST.Arc<CharsRef>().copyFrom(scratchArc));
|
||||
assert scratchArc.label() != FST.END_LABEL;
|
||||
cachedRootArcs.put(Character.valueOf((char) scratchArc.label()), new FST.Arc<CharsRef>().copyFrom(scratchArc));
|
||||
if (scratchArc.isLast()) {
|
||||
break;
|
||||
}
|
||||
|
|
|
@ -0,0 +1,58 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.lucene.analysis.es;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.analysis.TokenFilter;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.miscellaneous.SetKeywordMarkerFilter;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.KeywordAttribute;
|
||||
|
||||
/**
|
||||
* A {@link TokenFilter} that applies {@link SpanishMinimalStemmer} to stem Spanish
|
||||
* words.
|
||||
* <p>
|
||||
* To prevent terms from being stemmed use an instance of
|
||||
* {@link SetKeywordMarkerFilter} or a custom {@link TokenFilter} that sets
|
||||
* the {@link KeywordAttribute} before this {@link TokenStream}.
|
||||
* </p>
|
||||
*/
|
||||
public final class SpanishMinimalStemFilter extends TokenFilter {
|
||||
private final SpanishMinimalStemmer stemmer = new SpanishMinimalStemmer();
|
||||
private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
|
||||
private final KeywordAttribute keywordAttr = addAttribute(KeywordAttribute.class);
|
||||
|
||||
public SpanishMinimalStemFilter(TokenStream input) {
|
||||
super(input);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean incrementToken() throws IOException {
|
||||
if (input.incrementToken()) {
|
||||
if (!keywordAttr.isKeyword()) {
|
||||
final int newlen = stemmer.stem(termAtt.buffer(), termAtt.length());
|
||||
termAtt.setLength(newlen);
|
||||
}
|
||||
return true;
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,52 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.lucene.analysis.es;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.util.TokenFilterFactory;
|
||||
|
||||
/**
|
||||
* Factory for {@link SpanishMinimalStemFilter}.
|
||||
* <pre class="prettyprint">
|
||||
* <fieldType name="text_eslgtstem" class="solr.TextField" positionIncrementGap="100">
|
||||
* <analyzer>
|
||||
* <tokenizer class="solr.StandardTokenizerFactory"/>
|
||||
* <filter class="solr.LowerCaseFilterFactory"/>
|
||||
* <filter class="solr.SpanishMinimalStemFilterFactory"/>
|
||||
* </analyzer>
|
||||
* </fieldType></pre>
|
||||
* @lucene.spi {@value #NAME}
|
||||
*/
|
||||
public class SpanishMinimalStemFilterFactory extends TokenFilterFactory {
|
||||
|
||||
/** SPI name */
|
||||
public static final String NAME = "spanishMinimalStem";
|
||||
|
||||
/** Creates a new SpanishMinimalStemFilterFactory */
|
||||
public SpanishMinimalStemFilterFactory(Map<String,String> args) {
|
||||
super(args);
|
||||
if (!args.isEmpty()) {
|
||||
throw new IllegalArgumentException("Unknown parameters: " + args);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public TokenStream create(TokenStream input) { return new SpanishMinimalStemFilter(input); }
|
||||
}
|
|
@ -0,0 +1,80 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.lucene.analysis.es;
|
||||
|
||||
/**
|
||||
* Minimal plural stemmer for Spanish.
|
||||
* <p>
|
||||
* This stemmer implements the "plurals" stemmer for
|
||||
* spanish lanugage.
|
||||
*
|
||||
*/
|
||||
public class SpanishMinimalStemmer {
|
||||
|
||||
public int stem(char s[], int len) {
|
||||
if (len < 4 || s[len-1] != 's')
|
||||
return len;
|
||||
|
||||
for (int i = 0; i < len; i++)
|
||||
switch(s[i]) {
|
||||
case 'à':
|
||||
case 'á':
|
||||
case 'â':
|
||||
case 'ä': s[i] = 'a'; break;
|
||||
case 'ò':
|
||||
case 'ó':
|
||||
case 'ô':
|
||||
case 'ö': s[i] = 'o'; break;
|
||||
case 'è':
|
||||
case 'é':
|
||||
case 'ê':
|
||||
case 'ë': s[i] = 'e'; break;
|
||||
case 'ù':
|
||||
case 'ú':
|
||||
case 'û':
|
||||
case 'ü': s[i] = 'u'; break;
|
||||
case 'ì':
|
||||
case 'í':
|
||||
case 'î':
|
||||
case 'ï': s[i] = 'i'; break;
|
||||
case 'ñ': s[i] = 'n'; break;
|
||||
}
|
||||
|
||||
switch(s[len-1]) {
|
||||
case 's':
|
||||
if (s[len-2] == 'a' || s[len-2] == 'o') {
|
||||
return len-1;
|
||||
}
|
||||
if (s[len-2] == 'e') {
|
||||
if (s[len-3] == 's' && s[len-4] == 'e') {
|
||||
return len-2;
|
||||
}
|
||||
if (s[len-3] == 'c') {
|
||||
s[len-3] = 'z';
|
||||
return len-2;
|
||||
} else {
|
||||
return len-2;
|
||||
}
|
||||
} else {
|
||||
return len-1;
|
||||
}
|
||||
}
|
||||
|
||||
return len;
|
||||
}
|
||||
}
|
|
@ -74,7 +74,7 @@ public class FrenchMinimalStemmer {
|
|||
if (s[len-1] == 'r') len--;
|
||||
if (s[len-1] == 'e') len--;
|
||||
if (s[len-1] == 'é') len--;
|
||||
if (s[len-1] == s[len-2]) len--;
|
||||
if (s[len-1] == s[len-2] && Character.isLetter(s[len-1])) len--;
|
||||
return len;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -280,14 +280,14 @@ public class Dictionary {
|
|||
cp = Character.codePointAt(word, i, l);
|
||||
if (fst.findTargetArc(cp, arc, arc, bytesReader) == null) {
|
||||
return null;
|
||||
} else if (arc.output != NO_OUTPUT) {
|
||||
output = fst.outputs.add(output, arc.output);
|
||||
} else if (arc.output() != NO_OUTPUT) {
|
||||
output = fst.outputs.add(output, arc.output());
|
||||
}
|
||||
}
|
||||
if (fst.findTargetArc(FST.END_LABEL, arc, arc, bytesReader) == null) {
|
||||
return null;
|
||||
} else if (arc.output != NO_OUTPUT) {
|
||||
return fst.outputs.add(output, arc.output);
|
||||
} else if (arc.output() != NO_OUTPUT) {
|
||||
return fst.outputs.add(output, arc.output());
|
||||
} else {
|
||||
return output;
|
||||
}
|
||||
|
@ -1228,10 +1228,10 @@ public class Dictionary {
|
|||
if (fst.findTargetArc(ch, arc, arc, bytesReader) == null) {
|
||||
break;
|
||||
} else {
|
||||
output = fst.outputs.add(output, arc.output);
|
||||
output = fst.outputs.add(output, arc.output());
|
||||
}
|
||||
if (arc.isFinal()) {
|
||||
longestOutput = fst.outputs.add(output, arc.nextFinalOutput);
|
||||
longestOutput = fst.outputs.add(output, arc.nextFinalOutput());
|
||||
longestMatch = j;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -311,15 +311,15 @@ final class Stemmer {
|
|||
int ch = word[i-1];
|
||||
if (fst.findTargetArc(ch, arc, arc, bytesReader) == null) {
|
||||
break;
|
||||
} else if (arc.output != NO_OUTPUT) {
|
||||
output = fst.outputs.add(output, arc.output);
|
||||
} else if (arc.output() != NO_OUTPUT) {
|
||||
output = fst.outputs.add(output, arc.output());
|
||||
}
|
||||
}
|
||||
IntsRef prefixes = null;
|
||||
if (!arc.isFinal()) {
|
||||
continue;
|
||||
} else {
|
||||
prefixes = fst.outputs.add(output, arc.nextFinalOutput);
|
||||
prefixes = fst.outputs.add(output, arc.nextFinalOutput());
|
||||
}
|
||||
|
||||
for (int j = 0; j < prefixes.length; j++) {
|
||||
|
@ -395,15 +395,15 @@ final class Stemmer {
|
|||
int ch = word[i];
|
||||
if (fst.findTargetArc(ch, arc, arc, bytesReader) == null) {
|
||||
break;
|
||||
} else if (arc.output != NO_OUTPUT) {
|
||||
output = fst.outputs.add(output, arc.output);
|
||||
} else if (arc.output() != NO_OUTPUT) {
|
||||
output = fst.outputs.add(output, arc.output());
|
||||
}
|
||||
}
|
||||
IntsRef suffixes = null;
|
||||
if (!arc.isFinal()) {
|
||||
continue;
|
||||
} else {
|
||||
suffixes = fst.outputs.add(output, arc.nextFinalOutput);
|
||||
suffixes = fst.outputs.add(output, arc.nextFinalOutput());
|
||||
}
|
||||
|
||||
for (int j = 0; j < suffixes.length; j++) {
|
||||
|
|
|
@ -132,11 +132,11 @@ public final class StemmerOverrideFilter extends TokenFilter {
|
|||
if (fst.findTargetArc(ignoreCase ? Character.toLowerCase(codePoint) : codePoint, scratchArc, scratchArc, fstReader) == null) {
|
||||
return null;
|
||||
}
|
||||
pendingOutput = fst.outputs.add(pendingOutput, scratchArc.output);
|
||||
pendingOutput = fst.outputs.add(pendingOutput, scratchArc.output());
|
||||
bufUpto += Character.charCount(codePoint);
|
||||
}
|
||||
if (scratchArc.isFinal()) {
|
||||
matchOutput = fst.outputs.add(pendingOutput, scratchArc.nextFinalOutput);
|
||||
matchOutput = fst.outputs.add(pendingOutput, scratchArc.nextFinalOutput());
|
||||
}
|
||||
return matchOutput;
|
||||
}
|
||||
|
|
|
@ -330,7 +330,7 @@ public final class SynonymFilter extends TokenFilter {
|
|||
BytesRef pendingOutput = fst.outputs.getNoOutput();
|
||||
fst.getFirstArc(scratchArc);
|
||||
|
||||
assert scratchArc.output == fst.outputs.getNoOutput();
|
||||
assert scratchArc.output() == fst.outputs.getNoOutput();
|
||||
|
||||
int tokenCount = 0;
|
||||
|
||||
|
@ -399,7 +399,7 @@ public final class SynonymFilter extends TokenFilter {
|
|||
}
|
||||
|
||||
// Accum the output
|
||||
pendingOutput = fst.outputs.add(pendingOutput, scratchArc.output);
|
||||
pendingOutput = fst.outputs.add(pendingOutput, scratchArc.output());
|
||||
//System.out.println(" char=" + buffer[bufUpto] + " output=" + pendingOutput + " arc.output=" + scratchArc.output);
|
||||
bufUpto += Character.charCount(codePoint);
|
||||
}
|
||||
|
@ -407,7 +407,7 @@ public final class SynonymFilter extends TokenFilter {
|
|||
// OK, entire token matched; now see if this is a final
|
||||
// state:
|
||||
if (scratchArc.isFinal()) {
|
||||
matchOutput = fst.outputs.add(pendingOutput, scratchArc.nextFinalOutput);
|
||||
matchOutput = fst.outputs.add(pendingOutput, scratchArc.nextFinalOutput());
|
||||
matchInputLength = tokenCount;
|
||||
matchEndOffset = inputEndOffset;
|
||||
//System.out.println(" found matchLength=" + matchInputLength + " output=" + matchOutput);
|
||||
|
@ -423,7 +423,7 @@ public final class SynonymFilter extends TokenFilter {
|
|||
} else {
|
||||
// More matching is possible -- accum the output (if
|
||||
// any) of the WORD_SEP arc:
|
||||
pendingOutput = fst.outputs.add(pendingOutput, scratchArc.output);
|
||||
pendingOutput = fst.outputs.add(pendingOutput, scratchArc.output());
|
||||
if (nextRead == nextWrite) {
|
||||
capture();
|
||||
}
|
||||
|
|
|
@ -290,7 +290,7 @@ public final class SynonymGraphFilter extends TokenFilter {
|
|||
BytesRef pendingOutput = fst.outputs.getNoOutput();
|
||||
fst.getFirstArc(scratchArc);
|
||||
|
||||
assert scratchArc.output == fst.outputs.getNoOutput();
|
||||
assert scratchArc.output() == fst.outputs.getNoOutput();
|
||||
|
||||
// How many tokens in the current match
|
||||
int matchLength = 0;
|
||||
|
@ -360,7 +360,7 @@ public final class SynonymGraphFilter extends TokenFilter {
|
|||
}
|
||||
|
||||
// Accum the output
|
||||
pendingOutput = fst.outputs.add(pendingOutput, scratchArc.output);
|
||||
pendingOutput = fst.outputs.add(pendingOutput, scratchArc.output());
|
||||
bufUpto += Character.charCount(codePoint);
|
||||
}
|
||||
|
||||
|
@ -369,7 +369,7 @@ public final class SynonymGraphFilter extends TokenFilter {
|
|||
// OK, entire token matched; now see if this is a final
|
||||
// state in the FST (a match):
|
||||
if (scratchArc.isFinal()) {
|
||||
matchOutput = fst.outputs.add(pendingOutput, scratchArc.nextFinalOutput);
|
||||
matchOutput = fst.outputs.add(pendingOutput, scratchArc.nextFinalOutput());
|
||||
matchInputLength = matchLength;
|
||||
matchEndOffset = inputEndOffset;
|
||||
//System.out.println(" ** match");
|
||||
|
@ -385,7 +385,7 @@ public final class SynonymGraphFilter extends TokenFilter {
|
|||
} else {
|
||||
// More matching is possible -- accum the output (if
|
||||
// any) of the WORD_SEP arc:
|
||||
pendingOutput = fst.outputs.add(pendingOutput, scratchArc.output);
|
||||
pendingOutput = fst.outputs.add(pendingOutput, scratchArc.output());
|
||||
doFinalCapture = true;
|
||||
if (liveToken) {
|
||||
capture();
|
||||
|
|
|
@ -45,6 +45,7 @@ org.apache.lucene.analysis.en.EnglishPossessiveFilterFactory
|
|||
org.apache.lucene.analysis.en.KStemFilterFactory
|
||||
org.apache.lucene.analysis.en.PorterStemFilterFactory
|
||||
org.apache.lucene.analysis.es.SpanishLightStemFilterFactory
|
||||
org.apache.lucene.analysis.es.SpanishMinimalStemFilterFactory
|
||||
org.apache.lucene.analysis.fa.PersianNormalizationFilterFactory
|
||||
org.apache.lucene.analysis.fi.FinnishLightStemFilterFactory
|
||||
org.apache.lucene.analysis.fr.FrenchLightStemFilterFactory
|
||||
|
|
|
@ -0,0 +1,80 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.lucene.analysis.es;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.BaseTokenStreamTestCase;
|
||||
import org.apache.lucene.analysis.MockTokenizer;
|
||||
import org.apache.lucene.analysis.Tokenizer;
|
||||
import org.apache.lucene.analysis.core.KeywordTokenizer;
|
||||
import org.apache.lucene.analysis.en.EnglishMinimalStemFilter;
|
||||
|
||||
/**
|
||||
* Simple tests for {@link SpanishMinimalStemFilter}
|
||||
*/
|
||||
public class TestSpanishMinimalStemFilter extends BaseTokenStreamTestCase {
|
||||
private Analyzer analyzer;
|
||||
|
||||
@Override
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
analyzer = new Analyzer() {
|
||||
@Override
|
||||
protected TokenStreamComponents createComponents(String fieldName) {
|
||||
Tokenizer source = new MockTokenizer(MockTokenizer.WHITESPACE, false);
|
||||
return new TokenStreamComponents(source, new SpanishMinimalStemFilter(source));
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public void tearDown() throws Exception {
|
||||
analyzer.close();
|
||||
super.tearDown();
|
||||
}
|
||||
|
||||
/** Test some examples */
|
||||
public void testExamples() throws IOException {
|
||||
checkOneTerm(analyzer, "actrices", "actriz");
|
||||
checkOneTerm(analyzer, "niños", "nino");
|
||||
checkOneTerm(analyzer, "países", "pais");
|
||||
checkOneTerm(analyzer, "caragodor", "caragodor");
|
||||
checkOneTerm(analyzer, "móviles", "movil");
|
||||
checkOneTerm(analyzer, "chicas", "chica");
|
||||
}
|
||||
|
||||
/** blast some random strings through the analyzer */
|
||||
public void testRandomStrings() throws Exception {
|
||||
checkRandomData(random(), analyzer, 50*RANDOM_MULTIPLIER);
|
||||
}
|
||||
|
||||
public void testEmptyTerm() throws IOException {
|
||||
Analyzer a = new Analyzer() {
|
||||
@Override
|
||||
protected TokenStreamComponents createComponents(String fieldName) {
|
||||
Tokenizer tokenizer = new KeywordTokenizer();
|
||||
return new TokenStreamComponents(tokenizer, new EnglishMinimalStemFilter(tokenizer));
|
||||
}
|
||||
};
|
||||
checkOneTerm(a, "", "");
|
||||
a.close();
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,47 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.lucene.analysis.es;
|
||||
|
||||
import java.io.Reader;
|
||||
import java.io.StringReader;
|
||||
|
||||
import org.apache.lucene.analysis.MockTokenizer;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.Tokenizer;
|
||||
import org.apache.lucene.analysis.util.BaseTokenStreamFactoryTestCase;
|
||||
|
||||
/**
|
||||
* Simple tests to ensure the spanish minimal stem factory is working.
|
||||
*/
|
||||
public class TestSpanishMinimalStemFilterFactory extends BaseTokenStreamFactoryTestCase {
|
||||
public void testStemming() throws Exception {
|
||||
Reader reader = new StringReader("camisetas");
|
||||
TokenStream stream = new MockTokenizer(MockTokenizer.WHITESPACE, false);
|
||||
((Tokenizer)stream).setReader(reader);
|
||||
stream = tokenFilterFactory("spanishMinimalStem").create(stream);
|
||||
assertTokenStreamContents(stream, new String[] { "camiseta" });
|
||||
}
|
||||
|
||||
/** Test that bogus arguments result in exception */
|
||||
public void testBogusArguments() throws Exception {
|
||||
IllegalArgumentException expected = expectThrows(IllegalArgumentException.class, () -> {
|
||||
tokenFilterFactory("spanishMinimalStem", "bogusArg", "bogusValue");
|
||||
});
|
||||
assertTrue(expected.getMessage().contains("Unknown parameters"));
|
||||
}
|
||||
}
|
|
@ -67,7 +67,24 @@ public class TestFrenchMinimalStemFilter extends BaseTokenStreamTestCase {
|
|||
checkOneTerm(analyzer, "barons", "baron");
|
||||
checkOneTerm(analyzer, "baron", "baron");
|
||||
}
|
||||
|
||||
|
||||
public void testIntergerWithLastCharactersEqual() throws IOException {
|
||||
// Trailing repeated char elision :
|
||||
checkOneTerm(analyzer, "1234555", "1234555");
|
||||
// Repeated char within numbers with more than 6 characters :
|
||||
checkOneTerm(analyzer, "12333345", "12333345");
|
||||
// Short numbers weren't affected already:
|
||||
checkOneTerm(analyzer, "1234", "1234");
|
||||
// Ensure behaviour is preserved for words!
|
||||
// Trailing repeated char elision :
|
||||
checkOneTerm(analyzer, "abcdeff", "abcdef");
|
||||
// Repeated char within words with more than 6 characters :
|
||||
checkOneTerm(analyzer, "abcccddeef", "abcccddeef");
|
||||
checkOneTerm(analyzer, "créées", "cré");
|
||||
// Combined letter and digit repetition
|
||||
checkOneTerm(analyzer, "22hh00", "22hh00"); // 10:00pm
|
||||
}
|
||||
|
||||
public void testKeyword() throws IOException {
|
||||
final CharArraySet exclusionSet = new CharArraySet( asSet("chevaux"), false);
|
||||
Analyzer a = new Analyzer() {
|
||||
|
|
|
@ -175,12 +175,7 @@ public class TestSynonymGraphFilter extends BaseTokenStreamTestCase {
|
|||
String testFile = "a => 1";
|
||||
Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, false);
|
||||
SolrSynonymParser parser = new SolrSynonymParser(true, true, analyzer);
|
||||
try {
|
||||
parser.parse(new StringReader(testFile));
|
||||
fail("didn't get expected exception");
|
||||
} catch (ParseException expected) {
|
||||
// expected exc
|
||||
}
|
||||
expectThrows(ParseException.class, () -> parser.parse(new StringReader(testFile)));
|
||||
analyzer.close();
|
||||
}
|
||||
|
||||
|
@ -191,12 +186,7 @@ public class TestSynonymGraphFilter extends BaseTokenStreamTestCase {
|
|||
String testFile = "a => b => c";
|
||||
Analyzer analyzer = new MockAnalyzer(random());
|
||||
SolrSynonymParser parser = new SolrSynonymParser(true, true, analyzer);
|
||||
try {
|
||||
parser.parse(new StringReader(testFile));
|
||||
fail("didn't get expected exception");
|
||||
} catch (ParseException expected) {
|
||||
// expected exc
|
||||
}
|
||||
expectThrows(ParseException.class, () -> parser.parse(new StringReader(testFile)));
|
||||
analyzer.close();
|
||||
}
|
||||
|
||||
|
@ -561,13 +551,10 @@ public class TestSynonymGraphFilter extends BaseTokenStreamTestCase {
|
|||
public void testZeroSyns() throws Exception {
|
||||
Tokenizer tokenizer = new MockTokenizer();
|
||||
tokenizer.setReader(new StringReader("aa bb"));
|
||||
try {
|
||||
new SynonymGraphFilter(tokenizer, new SynonymMap.Builder(true).build(), true);
|
||||
fail("did not hit expected exception");
|
||||
} catch (IllegalArgumentException iae) {
|
||||
// expected
|
||||
assertEquals("fst must be non-null", iae.getMessage());
|
||||
}
|
||||
|
||||
IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, () ->
|
||||
new SynonymGraphFilter(tokenizer, new SynonymMap.Builder(true).build(), true));
|
||||
assertEquals("fst must be non-null", ex.getMessage());
|
||||
}
|
||||
|
||||
public void testOutputHangsOffEnd() throws Exception {
|
||||
|
|
|
@ -772,12 +772,12 @@ public final class JapaneseTokenizer extends Tokenizer {
|
|||
if (userFST.findTargetArc(ch, arc, arc, posAhead == posData.pos, userFSTReader) == null) {
|
||||
break;
|
||||
}
|
||||
output += arc.output.intValue();
|
||||
output += arc.output().intValue();
|
||||
if (arc.isFinal()) {
|
||||
if (VERBOSE) {
|
||||
System.out.println(" USER word " + new String(buffer.get(pos, posAhead - pos + 1)) + " toPos=" + (posAhead + 1));
|
||||
}
|
||||
add(userDictionary, posData, posAhead+1, output + arc.nextFinalOutput.intValue(), Type.USER, false);
|
||||
add(userDictionary, posData, posAhead+1, output + arc.nextFinalOutput().intValue(), Type.USER, false);
|
||||
anyMatches = true;
|
||||
}
|
||||
}
|
||||
|
@ -803,7 +803,7 @@ public final class JapaneseTokenizer extends Tokenizer {
|
|||
break;
|
||||
}
|
||||
|
||||
output += arc.output.intValue();
|
||||
output += arc.output().intValue();
|
||||
|
||||
// Optimization: for known words that are too-long
|
||||
// (compound), we should pre-compute the 2nd
|
||||
|
@ -812,7 +812,7 @@ public final class JapaneseTokenizer extends Tokenizer {
|
|||
// match is found.
|
||||
|
||||
if (arc.isFinal()) {
|
||||
dictionary.lookupWordIds(output + arc.nextFinalOutput.intValue(), wordIdRef);
|
||||
dictionary.lookupWordIds(output + arc.nextFinalOutput().intValue(), wordIdRef);
|
||||
if (VERBOSE) {
|
||||
System.out.println(" KNOWN word " + new String(buffer.get(pos, posAhead - pos + 1)) + " toPos=" + (posAhead + 1) + " " + wordIdRef.length + " wordIDs");
|
||||
}
|
||||
|
|
|
@ -162,9 +162,9 @@ public final class UserDictionary implements Dictionary {
|
|||
if (fst.findTargetArc(ch, arc, arc, i == 0, fstReader) == null) {
|
||||
break; // continue to next position
|
||||
}
|
||||
output += arc.output.intValue();
|
||||
output += arc.output().intValue();
|
||||
if (arc.isFinal()) {
|
||||
final int finalOutput = output + arc.nextFinalOutput.intValue();
|
||||
final int finalOutput = output + arc.nextFinalOutput().intValue();
|
||||
result.put(startOffset-off, segmentations[finalOutput]);
|
||||
found = true;
|
||||
}
|
||||
|
|
|
@ -26,7 +26,6 @@
|
|||
<!-- currently whether rat detects this as binary or not
|
||||
is platform dependent?! -->
|
||||
<property name="rat.excludes" value="**/*.txt,**/bocchan.utf-8"/>
|
||||
<property name="rat.additional-includes" value="src/tools/**"/>
|
||||
|
||||
<!-- we don't want to pull in ipadic/naist etc -->
|
||||
<property name="ivy.default.configuration" value="default"/>
|
||||
|
@ -45,6 +44,9 @@
|
|||
<available type="dir" file="${build.dir}/${dict.version}" property="mecab-ko.dict.available"/>
|
||||
|
||||
<path id="classpath">
|
||||
<dirset dir="${build.dir}">
|
||||
<include name="classes/java"/>
|
||||
</dirset>
|
||||
<pathelement path="${analyzers-common.jar}"/>
|
||||
<path refid="base.classpath"/>
|
||||
</path>
|
||||
|
@ -57,28 +59,14 @@
|
|||
<untar src="${build.dir}/${dict.version}.tar" dest="${build.dir}"/>
|
||||
</target>
|
||||
|
||||
<path id="tools.classpath">
|
||||
<path refid="classpath"/>
|
||||
<pathelement location="${build.dir}/classes/java"/>
|
||||
<pathelement location="${build.dir}/classes/tools"/>
|
||||
</path>
|
||||
|
||||
<path id="tools.test.classpath">
|
||||
<path refid="tools.classpath"/>
|
||||
<path refid="test.base.classpath"/>
|
||||
<pathelement location="${build.dir}/classes/tools-test"/>
|
||||
</path>
|
||||
|
||||
<target name="build-dict" depends="compile-tools, download-dict">
|
||||
<target name="build-dict" depends="compile, download-dict">
|
||||
<sequential>
|
||||
<delete verbose="true">
|
||||
<fileset dir="${resources.dir}/org/apache/lucene/analysis/ko/dict" includes="**/*"/>
|
||||
</delete>
|
||||
<!-- TODO: optimize the dictionary construction a bit so that you don't need 1G -->
|
||||
<java fork="true" failonerror="true" maxmemory="1g" classname="org.apache.lucene.analysis.ko.util.DictionaryBuilder">
|
||||
<classpath>
|
||||
<path refid="tools.classpath"/>
|
||||
</classpath>
|
||||
<classpath refid="classpath"/>
|
||||
<assertions>
|
||||
<enable package="org.apache.lucene"/>
|
||||
</assertions>
|
||||
|
@ -90,34 +78,7 @@
|
|||
</sequential>
|
||||
</target>
|
||||
|
||||
<target name="compile-tools" depends="compile-core, common.compile-tools">
|
||||
<compile
|
||||
srcdir="src/tools/java"
|
||||
destdir="${build.dir}/classes/tools">
|
||||
<classpath>
|
||||
<path refid="tools.classpath"/>
|
||||
</classpath>
|
||||
</compile>
|
||||
</target>
|
||||
|
||||
<target name="compile-tools-tests" depends="compile-tools">
|
||||
<compile
|
||||
srcdir="src/tools/test"
|
||||
destdir="${build.dir}/classes/tools-test">
|
||||
<classpath>
|
||||
<path refid="tools.test.classpath"/>
|
||||
<pathelement path="src/tools/test"/>
|
||||
</classpath>
|
||||
</compile>
|
||||
</target>
|
||||
|
||||
<target name="test-tools" depends="install-junit4-taskdef, compile-tools-tests">
|
||||
<test-macro testsDir="${build.dir}/classes/tools-test" workDir="src/tools/test" junit.classpath="tools.test.classpath"/>
|
||||
</target>
|
||||
|
||||
<target name="compile-test" depends="module-build.compile-test, compile-tools-tests"/>
|
||||
<!-- TODO: not until we properly make 'test-tools' work with clover etc
|
||||
<target name="test" depends="module-build.test, test-tools"/> -->
|
||||
<target name="compile-test" depends="module-build.compile-test"/>
|
||||
|
||||
<target name="regenerate" depends="build-dict"/>
|
||||
</project>
|
||||
|
|
|
@ -681,11 +681,11 @@ public final class KoreanTokenizer extends Tokenizer {
|
|||
if (userFST.findTargetArc(ch, arc, arc, posAhead == pos, userFSTReader) == null) {
|
||||
break;
|
||||
}
|
||||
output += arc.output.intValue();
|
||||
output += arc.output().intValue();
|
||||
if (arc.isFinal()) {
|
||||
maxPosAhead = posAhead;
|
||||
outputMaxPosAhead = output;
|
||||
arcFinalOutMaxPosAhead = arc.nextFinalOutput.intValue();
|
||||
arcFinalOutMaxPosAhead = arc.nextFinalOutput().intValue();
|
||||
anyMatches = true;
|
||||
}
|
||||
}
|
||||
|
@ -720,7 +720,7 @@ public final class KoreanTokenizer extends Tokenizer {
|
|||
break;
|
||||
}
|
||||
|
||||
output += arc.output.intValue();
|
||||
output += arc.output().intValue();
|
||||
|
||||
// Optimization: for known words that are too-long
|
||||
// (compound), we should pre-compute the 2nd
|
||||
|
@ -729,7 +729,7 @@ public final class KoreanTokenizer extends Tokenizer {
|
|||
// match is found.
|
||||
|
||||
if (arc.isFinal()) {
|
||||
dictionary.lookupWordIds(output + arc.nextFinalOutput.intValue(), wordIdRef);
|
||||
dictionary.lookupWordIds(output + arc.nextFinalOutput().intValue(), wordIdRef);
|
||||
if (VERBOSE) {
|
||||
System.out.println(" KNOWN word " + new String(buffer.get(pos, posAhead - pos + 1)) + " toPos=" + (posAhead + 1) + " " + wordIdRef.length + " wordIDs");
|
||||
}
|
||||
|
|
|
@ -21,7 +21,6 @@ import java.io.InputStream;
|
|||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.store.InputStreamDataInput;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.fst.FST;
|
||||
import org.apache.lucene.util.fst.PositiveIntOutputs;
|
||||
|
||||
|
@ -46,20 +45,9 @@ public final class TokenInfoDictionary extends BinaryDictionary {
|
|||
*/
|
||||
TokenInfoDictionary(ResourceScheme resourceScheme, String resourcePath) throws IOException {
|
||||
super(resourceScheme, resourcePath);
|
||||
InputStream is = null;
|
||||
FST<Long> fst;
|
||||
boolean success = false;
|
||||
try {
|
||||
is = getResource(FST_FILENAME_SUFFIX);
|
||||
is = new BufferedInputStream(is);
|
||||
try (InputStream is = new BufferedInputStream(getResource(FST_FILENAME_SUFFIX))) {
|
||||
fst = new FST<>(new InputStreamDataInput(is), PositiveIntOutputs.getSingleton());
|
||||
success = true;
|
||||
} finally {
|
||||
if (success) {
|
||||
IOUtils.close(is);
|
||||
} else {
|
||||
IOUtils.closeWhileHandlingException(is);
|
||||
}
|
||||
}
|
||||
this.fst = new TokenInfoFST(fst);
|
||||
}
|
||||
|
|
|
@ -221,9 +221,9 @@ public final class UserDictionary implements Dictionary {
|
|||
if (fst.findTargetArc(ch, arc, arc, i == 0, fstReader) == null) {
|
||||
break; // continue to next position
|
||||
}
|
||||
output += arc.output.intValue();
|
||||
output += arc.output().intValue();
|
||||
if (arc.isFinal()) {
|
||||
final int finalOutput = output + arc.nextFinalOutput.intValue();
|
||||
final int finalOutput = output + arc.nextFinalOutput().intValue();
|
||||
result.add(finalOutput);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,13 +17,13 @@
|
|||
package org.apache.lucene.analysis.ko.util;
|
||||
|
||||
import java.io.BufferedOutputStream;
|
||||
import java.io.File;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.channels.Channels;
|
||||
import java.nio.channels.WritableByteChannel;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
@ -37,17 +37,17 @@ import org.apache.lucene.util.ArrayUtil;
|
|||
|
||||
import org.apache.lucene.analysis.ko.dict.BinaryDictionary;
|
||||
|
||||
public abstract class BinaryDictionaryWriter {
|
||||
abstract class BinaryDictionaryWriter {
|
||||
private final static int ID_LIMIT = 8192;
|
||||
|
||||
protected final Class<? extends BinaryDictionary> implClazz;
|
||||
private final Class<? extends BinaryDictionary> implClazz;
|
||||
protected ByteBuffer buffer;
|
||||
private int targetMapEndOffset = 0, lastWordId = -1, lastSourceId = -1;
|
||||
private int[] targetMap = new int[8192];
|
||||
private int[] targetMapOffsets = new int[8192];
|
||||
private final ArrayList<String> posDict = new ArrayList<>();
|
||||
|
||||
public BinaryDictionaryWriter(Class<? extends BinaryDictionary> implClazz, int size) {
|
||||
BinaryDictionaryWriter(Class<? extends BinaryDictionary> implClazz, int size) {
|
||||
this.implClazz = implClazz;
|
||||
buffer = ByteBuffer.allocate(size);
|
||||
}
|
||||
|
@ -183,7 +183,7 @@ public abstract class BinaryDictionaryWriter {
|
|||
}
|
||||
}
|
||||
|
||||
public void addMapping(int sourceId, int wordId) {
|
||||
void addMapping(int sourceId, int wordId) {
|
||||
if (wordId <= lastWordId) {
|
||||
throw new IllegalStateException("words out of order: " + wordId + " vs lastID: " + lastWordId);
|
||||
}
|
||||
|
@ -205,27 +205,26 @@ public abstract class BinaryDictionaryWriter {
|
|||
lastWordId = wordId;
|
||||
}
|
||||
|
||||
protected final String getBaseFileName(String baseDir) {
|
||||
return baseDir + File.separator + implClazz.getName().replace('.', File.separatorChar);
|
||||
final String getBaseFileName() {
|
||||
return implClazz.getName().replace('.', '/');
|
||||
}
|
||||
|
||||
/**
|
||||
* Write dictionary in file
|
||||
* @throws IOException if an I/O error occurs writing the dictionary files
|
||||
*/
|
||||
public void write(String baseDir) throws IOException {
|
||||
final String baseName = getBaseFileName(baseDir);
|
||||
writeDictionary(baseName + BinaryDictionary.DICT_FILENAME_SUFFIX);
|
||||
writeTargetMap(baseName + BinaryDictionary.TARGETMAP_FILENAME_SUFFIX);
|
||||
writePosDict(baseName + BinaryDictionary.POSDICT_FILENAME_SUFFIX);
|
||||
public void write(Path baseDir) throws IOException {
|
||||
final String baseName = getBaseFileName();
|
||||
writeDictionary(baseDir.resolve(baseName + BinaryDictionary.DICT_FILENAME_SUFFIX));
|
||||
writeTargetMap(baseDir.resolve(baseName + BinaryDictionary.TARGETMAP_FILENAME_SUFFIX));
|
||||
writePosDict(baseDir.resolve(baseName + BinaryDictionary.POSDICT_FILENAME_SUFFIX));
|
||||
}
|
||||
|
||||
protected void writeTargetMap(String filename) throws IOException {
|
||||
new File(filename).getParentFile().mkdirs();
|
||||
OutputStream os = new FileOutputStream(filename);
|
||||
try {
|
||||
os = new BufferedOutputStream(os);
|
||||
final DataOutput out = new OutputStreamDataOutput(os);
|
||||
private void writeTargetMap(Path path) throws IOException {
|
||||
Files.createDirectories(path.getParent());
|
||||
try (OutputStream os = Files.newOutputStream(path);
|
||||
OutputStream bos = new BufferedOutputStream(os)) {
|
||||
final DataOutput out = new OutputStreamDataOutput(bos);
|
||||
CodecUtil.writeHeader(out, BinaryDictionary.TARGETMAP_HEADER, BinaryDictionary.VERSION);
|
||||
|
||||
final int numSourceIds = lastSourceId + 1;
|
||||
|
@ -246,17 +245,14 @@ public abstract class BinaryDictionaryWriter {
|
|||
if (sourceId != numSourceIds) {
|
||||
throw new IllegalStateException("sourceId:" + sourceId + " != numSourceIds:" + numSourceIds);
|
||||
}
|
||||
} finally {
|
||||
os.close();
|
||||
}
|
||||
}
|
||||
|
||||
protected void writePosDict(String filename) throws IOException {
|
||||
new File(filename).getParentFile().mkdirs();
|
||||
OutputStream os = new FileOutputStream(filename);
|
||||
try {
|
||||
os = new BufferedOutputStream(os);
|
||||
final DataOutput out = new OutputStreamDataOutput(os);
|
||||
private void writePosDict(Path path) throws IOException {
|
||||
Files.createDirectories(path.getParent());
|
||||
try (OutputStream os = Files.newOutputStream(path);
|
||||
OutputStream bos = new BufferedOutputStream(os)) {
|
||||
final DataOutput out = new OutputStreamDataOutput(bos);
|
||||
CodecUtil.writeHeader(out, BinaryDictionary.POSDICT_HEADER, BinaryDictionary.VERSION);
|
||||
out.writeVInt(posDict.size());
|
||||
for (String s : posDict) {
|
||||
|
@ -270,25 +266,21 @@ public abstract class BinaryDictionaryWriter {
|
|||
out.writeByte((byte) POS.Tag.valueOf(data[0]).ordinal());
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
os.close();
|
||||
}
|
||||
}
|
||||
|
||||
protected void writeDictionary(String filename) throws IOException {
|
||||
new File(filename).getParentFile().mkdirs();
|
||||
final FileOutputStream os = new FileOutputStream(filename);
|
||||
try {
|
||||
final DataOutput out = new OutputStreamDataOutput(os);
|
||||
private void writeDictionary(Path path) throws IOException {
|
||||
Files.createDirectories(path.getParent());
|
||||
try (OutputStream os = Files.newOutputStream(path);
|
||||
OutputStream bos = new BufferedOutputStream(os)) {
|
||||
final DataOutput out = new OutputStreamDataOutput(bos);
|
||||
CodecUtil.writeHeader(out, BinaryDictionary.DICT_HEADER, BinaryDictionary.VERSION);
|
||||
out.writeVInt(buffer.position());
|
||||
final WritableByteChannel channel = Channels.newChannel(os);
|
||||
final WritableByteChannel channel = Channels.newChannel(bos);
|
||||
// Write Buffer
|
||||
buffer.flip(); // set position to 0, set limit to current position
|
||||
channel.write(buffer);
|
||||
assert buffer.remaining() == 0L;
|
||||
} finally {
|
||||
os.close();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -17,10 +17,10 @@
|
|||
package org.apache.lucene.analysis.ko.util;
|
||||
|
||||
import java.io.BufferedOutputStream;
|
||||
import java.io.File;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.util.Arrays;
|
||||
|
||||
import org.apache.lucene.analysis.ko.dict.CharacterDefinition;
|
||||
|
@ -29,7 +29,7 @@ import org.apache.lucene.codecs.CodecUtil;
|
|||
import org.apache.lucene.store.DataOutput;
|
||||
import org.apache.lucene.store.OutputStreamDataOutput;
|
||||
|
||||
public final class CharacterDefinitionWriter {
|
||||
final class CharacterDefinitionWriter {
|
||||
|
||||
private final byte[] characterCategoryMap = new byte[0x10000];
|
||||
|
||||
|
@ -39,7 +39,7 @@ public final class CharacterDefinitionWriter {
|
|||
/**
|
||||
* Constructor for building. TODO: remove write access
|
||||
*/
|
||||
public CharacterDefinitionWriter() {
|
||||
CharacterDefinitionWriter() {
|
||||
Arrays.fill(characterCategoryMap, CharacterDefinition.DEFAULT);
|
||||
}
|
||||
|
||||
|
@ -50,7 +50,7 @@ public final class CharacterDefinitionWriter {
|
|||
* code point
|
||||
* @param characterClassName character class name
|
||||
*/
|
||||
public void putCharacterCategory(int codePoint, String characterClassName) {
|
||||
void putCharacterCategory(int codePoint, String characterClassName) {
|
||||
characterClassName = characterClassName.split(" ")[0]; // use first
|
||||
// category
|
||||
// class
|
||||
|
@ -62,20 +62,17 @@ public final class CharacterDefinitionWriter {
|
|||
characterCategoryMap[codePoint] = CharacterDefinition.lookupCharacterClass(characterClassName);
|
||||
}
|
||||
|
||||
public void putInvokeDefinition(String characterClassName, int invoke, int group, int length) {
|
||||
void putInvokeDefinition(String characterClassName, int invoke, int group, int length) {
|
||||
final byte characterClass = CharacterDefinition.lookupCharacterClass(characterClassName);
|
||||
invokeMap[characterClass] = invoke == 1;
|
||||
groupMap[characterClass] = group == 1;
|
||||
// TODO: length def ignored
|
||||
}
|
||||
|
||||
public void write(String baseDir) throws IOException {
|
||||
String filename = baseDir + File.separator +
|
||||
CharacterDefinition.class.getName().replace('.', File.separatorChar) + CharacterDefinition.FILENAME_SUFFIX;
|
||||
new File(filename).getParentFile().mkdirs();
|
||||
OutputStream os = new FileOutputStream(filename);
|
||||
try {
|
||||
os = new BufferedOutputStream(os);
|
||||
public void write(Path baseDir) throws IOException {
|
||||
Path path = baseDir.resolve(CharacterDefinition.class.getName().replace('.', '/') + CharacterDefinition.FILENAME_SUFFIX);
|
||||
Files.createDirectories(path.getParent());
|
||||
try (OutputStream os = new BufferedOutputStream(Files.newOutputStream(path))){
|
||||
final DataOutput out = new OutputStreamDataOutput(os);
|
||||
CodecUtil.writeHeader(out, CharacterDefinition.HEADER, CharacterDefinition.VERSION);
|
||||
out.writeBytes(characterCategoryMap, 0, characterCategoryMap.length);
|
||||
|
@ -86,8 +83,6 @@ public final class CharacterDefinitionWriter {
|
|||
);
|
||||
out.writeByte(b);
|
||||
}
|
||||
} finally {
|
||||
os.close();
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,61 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.lucene.analysis.ko.util;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.LineNumberReader;
|
||||
import java.io.Reader;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
|
||||
class ConnectionCostsBuilder {
|
||||
|
||||
private ConnectionCostsBuilder() {
|
||||
}
|
||||
|
||||
public static ConnectionCostsWriter build(Path path) throws IOException {
|
||||
try (Reader reader = Files.newBufferedReader(path, StandardCharsets.US_ASCII);
|
||||
LineNumberReader lineReader = new LineNumberReader(reader)) {
|
||||
|
||||
String line = lineReader.readLine();
|
||||
String[] dimensions = line.split("\\s+");
|
||||
|
||||
assert dimensions.length == 2;
|
||||
|
||||
int forwardSize = Integer.parseInt(dimensions[0]);
|
||||
int backwardSize = Integer.parseInt(dimensions[1]);
|
||||
|
||||
assert forwardSize > 0 && backwardSize > 0;
|
||||
|
||||
ConnectionCostsWriter costs = new ConnectionCostsWriter(forwardSize, backwardSize);
|
||||
|
||||
while ((line = lineReader.readLine()) != null) {
|
||||
String[] fields = line.split("\\s+");
|
||||
|
||||
assert fields.length == 3;
|
||||
|
||||
int forwardId = Integer.parseInt(fields[0]);
|
||||
int backwardId = Integer.parseInt(fields[1]);
|
||||
int cost = Integer.parseInt(fields[2]);
|
||||
|
||||
costs.add(forwardId, backwardId, cost);
|
||||
}
|
||||
return costs;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -17,10 +17,10 @@
|
|||
package org.apache.lucene.analysis.ko.util;
|
||||
|
||||
import java.io.BufferedOutputStream;
|
||||
import java.io.File;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
|
||||
import org.apache.lucene.analysis.ko.dict.ConnectionCosts;
|
||||
|
||||
|
@ -28,7 +28,7 @@ import org.apache.lucene.codecs.CodecUtil;
|
|||
import org.apache.lucene.store.DataOutput;
|
||||
import org.apache.lucene.store.OutputStreamDataOutput;
|
||||
|
||||
public final class ConnectionCostsWriter {
|
||||
final class ConnectionCostsWriter {
|
||||
|
||||
private final short[][] costs; // array is backward IDs first since get is called using the same backward ID consecutively. maybe doesn't matter.
|
||||
private final int forwardSize;
|
||||
|
@ -36,7 +36,7 @@ public final class ConnectionCostsWriter {
|
|||
/**
|
||||
* Constructor for building. TODO: remove write access
|
||||
*/
|
||||
public ConnectionCostsWriter(int forwardSize, int backwardSize) {
|
||||
ConnectionCostsWriter(int forwardSize, int backwardSize) {
|
||||
this.forwardSize = forwardSize;
|
||||
this.backwardSize = backwardSize;
|
||||
this.costs = new short[backwardSize][forwardSize];
|
||||
|
@ -46,14 +46,12 @@ public final class ConnectionCostsWriter {
|
|||
this.costs[backwardId][forwardId] = (short)cost;
|
||||
}
|
||||
|
||||
public void write(String baseDir) throws IOException {
|
||||
String filename = baseDir + File.separator +
|
||||
ConnectionCosts.class.getName().replace('.', File.separatorChar) + ConnectionCosts.FILENAME_SUFFIX;
|
||||
new File(filename).getParentFile().mkdirs();
|
||||
OutputStream os = new FileOutputStream(filename);
|
||||
try {
|
||||
os = new BufferedOutputStream(os);
|
||||
final DataOutput out = new OutputStreamDataOutput(os);
|
||||
public void write(Path baseDir) throws IOException {
|
||||
Files.createDirectories(baseDir);
|
||||
String fileName = ConnectionCosts.class.getName().replace('.', '/') + ConnectionCosts.FILENAME_SUFFIX;
|
||||
try (OutputStream os = Files.newOutputStream(baseDir.resolve(fileName));
|
||||
OutputStream bos = new BufferedOutputStream(os)) {
|
||||
final DataOutput out = new OutputStreamDataOutput(bos);
|
||||
CodecUtil.writeHeader(out, ConnectionCosts.HEADER, ConnectionCosts.VERSION);
|
||||
out.writeVInt(forwardSize);
|
||||
out.writeVInt(backwardSize);
|
||||
|
@ -61,14 +59,12 @@ public final class ConnectionCostsWriter {
|
|||
assert costs.length == backwardSize;
|
||||
for (short[] a : costs) {
|
||||
assert a.length == forwardSize;
|
||||
for (int i = 0; i < a.length; i++) {
|
||||
int delta = (int)a[i] - last;
|
||||
for (short cost : a) {
|
||||
int delta = (int) cost - last;
|
||||
out.writeZInt(delta);
|
||||
last = a[i];
|
||||
last = cost;
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
os.close();
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,54 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.lucene.analysis.ko.util;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.file.Path;
|
||||
import java.nio.file.Paths;
|
||||
|
||||
/**
|
||||
* Tool to build dictionaries.
|
||||
*/
|
||||
public class DictionaryBuilder {
|
||||
|
||||
private DictionaryBuilder() {
|
||||
}
|
||||
|
||||
public static void build(Path inputDir, Path outputDir, String encoding, boolean normalizeEntry) throws IOException {
|
||||
// Build TokenInfo Dictionary
|
||||
new TokenInfoDictionaryBuilder(encoding, normalizeEntry)
|
||||
.build(inputDir)
|
||||
.write(outputDir);
|
||||
|
||||
// Build Unknown Word Dictionary
|
||||
new UnknownDictionaryBuilder(encoding)
|
||||
.build(inputDir)
|
||||
.write(outputDir);
|
||||
|
||||
// Build Connection Cost
|
||||
ConnectionCostsBuilder.build(inputDir.resolve("matrix.def"))
|
||||
.write(outputDir);
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws IOException {
|
||||
String inputDirname = args[0];
|
||||
String outputDirname = args[1];
|
||||
String inputEncoding = args[2];
|
||||
boolean normalizeEntries = Boolean.parseBoolean(args[3]);
|
||||
DictionaryBuilder.build(Paths.get(inputDirname), Paths.get(outputDirname), inputEncoding, normalizeEntries);
|
||||
}
|
||||
}
|
|
@ -17,20 +17,17 @@
|
|||
package org.apache.lucene.analysis.ko.util;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.FilenameFilter;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStreamReader;
|
||||
import java.nio.charset.Charset;
|
||||
import java.nio.charset.CharsetDecoder;
|
||||
import java.nio.charset.CodingErrorAction;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.text.Normalizer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import org.apache.lucene.util.IntsRefBuilder;
|
||||
import org.apache.lucene.util.fst.Builder;
|
||||
|
@ -38,72 +35,59 @@ import org.apache.lucene.util.fst.FST;
|
|||
|
||||
import org.apache.lucene.util.fst.PositiveIntOutputs;
|
||||
|
||||
public class TokenInfoDictionaryBuilder {
|
||||
class TokenInfoDictionaryBuilder {
|
||||
|
||||
/** Internal word id - incrementally assigned as entries are read and added. This will be byte offset of dictionary file */
|
||||
private int offset = 0;
|
||||
|
||||
private String encoding = "utf-8";
|
||||
|
||||
private String encoding;
|
||||
private Normalizer.Form normalForm;
|
||||
|
||||
public TokenInfoDictionaryBuilder(String encoding, boolean normalizeEntries) {
|
||||
TokenInfoDictionaryBuilder(String encoding, boolean normalizeEntries) {
|
||||
this.encoding = encoding;
|
||||
this.normalForm = normalizeEntries ? Normalizer.Form.NFKC : null;
|
||||
normalForm = normalizeEntries ? Normalizer.Form.NFKC : null;
|
||||
}
|
||||
|
||||
public TokenInfoDictionaryWriter build(String dirname) throws IOException {
|
||||
FilenameFilter filter = (dir, name) -> name.endsWith(".csv");
|
||||
ArrayList<File> csvFiles = new ArrayList<>();
|
||||
for (File file : new File(dirname).listFiles(filter)) {
|
||||
csvFiles.add(file);
|
||||
public TokenInfoDictionaryWriter build(Path dir) throws IOException {
|
||||
try (Stream<Path> files = Files.list(dir)) {
|
||||
List<Path> csvFiles = files
|
||||
.filter(path -> path.getFileName().toString().endsWith(".csv"))
|
||||
.sorted()
|
||||
.collect(Collectors.toList());
|
||||
return buildDictionary(csvFiles);
|
||||
}
|
||||
Collections.sort(csvFiles);
|
||||
return buildDictionary(csvFiles);
|
||||
}
|
||||
|
||||
public TokenInfoDictionaryWriter buildDictionary(List<File> csvFiles) throws IOException {
|
||||
private TokenInfoDictionaryWriter buildDictionary(List<Path> csvFiles) throws IOException {
|
||||
TokenInfoDictionaryWriter dictionary = new TokenInfoDictionaryWriter(10 * 1024 * 1024);
|
||||
|
||||
// all lines in the file
|
||||
System.out.println(" parse...");
|
||||
List<String[]> lines = new ArrayList<>(400000);
|
||||
for (File file : csvFiles){
|
||||
FileInputStream inputStream = new FileInputStream(file);
|
||||
Charset cs = Charset.forName(encoding);
|
||||
CharsetDecoder decoder = cs.newDecoder()
|
||||
.onMalformedInput(CodingErrorAction.REPORT)
|
||||
.onUnmappableCharacter(CodingErrorAction.REPORT);
|
||||
InputStreamReader streamReader = new InputStreamReader(inputStream, decoder);
|
||||
BufferedReader reader = new BufferedReader(streamReader);
|
||||
|
||||
String line = null;
|
||||
while ((line = reader.readLine()) != null) {
|
||||
String[] entry = CSVUtil.parse(line);
|
||||
for (Path path : csvFiles) {
|
||||
try (BufferedReader reader = Files.newBufferedReader(path, Charset.forName(encoding))) {
|
||||
String line;
|
||||
while ((line = reader.readLine()) != null) {
|
||||
String[] entry = CSVUtil.parse(line);
|
||||
|
||||
if(entry.length < 12) {
|
||||
throw new IllegalArgumentException("Entry in CSV is not valid (12 field values expected): " + line);
|
||||
}
|
||||
|
||||
// NFKC normalize dictionary entry
|
||||
if (normalForm != null) {
|
||||
String[] normalizedEntry = new String[entry.length];
|
||||
for (int i = 0; i < entry.length; i++) {
|
||||
normalizedEntry[i] = Normalizer.normalize(entry[i], normalForm);
|
||||
if (entry.length < 12) {
|
||||
throw new IllegalArgumentException("Entry in CSV is not valid (12 field values expected): " + line);
|
||||
}
|
||||
|
||||
// NFKC normalize dictionary entry
|
||||
if (normalForm != null) {
|
||||
String[] normalizedEntry = new String[entry.length];
|
||||
for (int i = 0; i < entry.length; i++) {
|
||||
normalizedEntry[i] = Normalizer.normalize(entry[i], normalForm);
|
||||
}
|
||||
lines.add(normalizedEntry);
|
||||
} else {
|
||||
lines.add(entry);
|
||||
}
|
||||
lines.add(normalizedEntry);
|
||||
} else {
|
||||
lines.add(entry);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
System.out.println(" sort...");
|
||||
|
||||
// sort by term: we sorted the files already and use a stable sort.
|
||||
Collections.sort(lines, Comparator.comparing(left -> left[0]));
|
||||
|
||||
System.out.println(" encode...");
|
||||
lines.sort(Comparator.comparing(left -> left[0]));
|
||||
|
||||
PositiveIntOutputs fstOutput = PositiveIntOutputs.getSingleton();
|
||||
Builder<Long> fstBuilder = new Builder<>(FST.INPUT_TYPE.BYTE2, 0, 0, true, true, Integer.MAX_VALUE, fstOutput, true, 15);
|
||||
|
@ -111,7 +95,7 @@ public class TokenInfoDictionaryBuilder {
|
|||
long ord = -1; // first ord will be 0
|
||||
String lastValue = null;
|
||||
|
||||
// build tokeninfo dictionary
|
||||
// build token info dictionary
|
||||
for (String[] entry : lines) {
|
||||
String surfaceForm = entry[0].trim();
|
||||
if (surfaceForm.isEmpty()) {
|
||||
|
@ -119,9 +103,8 @@ public class TokenInfoDictionaryBuilder {
|
|||
}
|
||||
int next = dictionary.put(entry);
|
||||
|
||||
if(next == offset){
|
||||
System.out.println("Failed to process line: " + Arrays.toString(entry));
|
||||
continue;
|
||||
if(next == offset) {
|
||||
throw new IllegalStateException("Failed to process line: " + Arrays.toString(entry));
|
||||
}
|
||||
|
||||
if (!surfaceForm.equals(lastValue)) {
|
||||
|
@ -135,16 +118,10 @@ public class TokenInfoDictionaryBuilder {
|
|||
}
|
||||
fstBuilder.add(scratch.get(), ord);
|
||||
}
|
||||
dictionary.addMapping((int)ord, offset);
|
||||
dictionary.addMapping((int) ord, offset);
|
||||
offset = next;
|
||||
}
|
||||
|
||||
final FST<Long> fst = fstBuilder.finish();
|
||||
|
||||
System.out.print(" " + fstBuilder.getNodeCount() + " nodes, " + fstBuilder.getArcCount() + " arcs, " + fst.ramBytesUsed() + " bytes... ");
|
||||
dictionary.setFST(fst);
|
||||
System.out.println(" done");
|
||||
|
||||
dictionary.setFST(fstBuilder.finish());
|
||||
return dictionary;
|
||||
}
|
||||
}
|
|
@ -19,31 +19,31 @@ package org.apache.lucene.analysis.ko.util;
|
|||
import java.io.IOException;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.Objects;
|
||||
|
||||
import org.apache.lucene.analysis.ko.dict.TokenInfoDictionary;
|
||||
import org.apache.lucene.util.fst.FST;
|
||||
|
||||
public class TokenInfoDictionaryWriter extends BinaryDictionaryWriter {
|
||||
class TokenInfoDictionaryWriter extends BinaryDictionaryWriter {
|
||||
private FST<Long> fst;
|
||||
|
||||
public TokenInfoDictionaryWriter(int size) {
|
||||
TokenInfoDictionaryWriter(int size) {
|
||||
super(TokenInfoDictionary.class, size);
|
||||
}
|
||||
|
||||
public void setFST(FST<Long> fst) {
|
||||
Objects.requireNonNull(fst, "dictionary must not be empty");
|
||||
this.fst = fst;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(String baseDir) throws IOException {
|
||||
public void write(Path baseDir) throws IOException {
|
||||
super.write(baseDir);
|
||||
writeFST(getBaseFileName(baseDir) + TokenInfoDictionary.FST_FILENAME_SUFFIX);
|
||||
writeFST(baseDir.resolve(getBaseFileName() + TokenInfoDictionary.FST_FILENAME_SUFFIX));
|
||||
}
|
||||
|
||||
protected void writeFST(String filename) throws IOException {
|
||||
Path p = Paths.get(filename);
|
||||
Files.createDirectories(p.getParent());
|
||||
fst.save(p);
|
||||
private void writeFST(Path path) throws IOException {
|
||||
Files.createDirectories(path.getParent());
|
||||
fst.save(path);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,118 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.lucene.analysis.ko.util;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.LineNumberReader;
|
||||
import java.io.Reader;
|
||||
import java.nio.charset.Charset;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.analysis.ko.dict.CharacterDefinition;
|
||||
|
||||
class UnknownDictionaryBuilder {
|
||||
private static final String NGRAM_DICTIONARY_ENTRY = "NGRAM,1798,3559,3677,SY,*,*,*,*,*,*,*";
|
||||
|
||||
private String encoding;
|
||||
|
||||
UnknownDictionaryBuilder(String encoding) {
|
||||
this.encoding = encoding;
|
||||
}
|
||||
|
||||
public UnknownDictionaryWriter build(Path dir) throws IOException {
|
||||
UnknownDictionaryWriter unkDictionary = readDictionaryFile(dir.resolve("unk.def")); //Should be only one file
|
||||
readCharacterDefinition(dir.resolve("char.def"), unkDictionary);
|
||||
return unkDictionary;
|
||||
}
|
||||
|
||||
private UnknownDictionaryWriter readDictionaryFile(Path path) throws IOException {
|
||||
return readDictionaryFile(path, encoding);
|
||||
}
|
||||
|
||||
private UnknownDictionaryWriter readDictionaryFile(Path path, String encoding) throws IOException {
|
||||
UnknownDictionaryWriter dictionary = new UnknownDictionaryWriter(5 * 1024 * 1024);
|
||||
|
||||
List<String[]> lines = new ArrayList<>();
|
||||
try (Reader reader = Files.newBufferedReader(path, Charset.forName(encoding));
|
||||
LineNumberReader lineReader = new LineNumberReader(reader)) {
|
||||
|
||||
dictionary.put(CSVUtil.parse(NGRAM_DICTIONARY_ENTRY));
|
||||
|
||||
String line;
|
||||
while ((line = lineReader.readLine()) != null) {
|
||||
// note: unk.def only has 10 fields, it simplifies the writer to just append empty reading and pronunciation,
|
||||
// even though the unknown dictionary returns hardcoded null here.
|
||||
final String[] parsed = CSVUtil.parse(line + ",*,*"); // Probably we don't need to validate entry
|
||||
lines.add(parsed);
|
||||
}
|
||||
}
|
||||
|
||||
lines.sort(Comparator.comparingInt(entry -> CharacterDefinition.lookupCharacterClass(entry[0])));
|
||||
|
||||
for (String[] entry : lines) {
|
||||
dictionary.put(entry);
|
||||
}
|
||||
|
||||
return dictionary;
|
||||
}
|
||||
|
||||
private void readCharacterDefinition(Path path, UnknownDictionaryWriter dictionary) throws IOException {
|
||||
try (Reader reader = Files.newBufferedReader(path, Charset.forName(encoding));
|
||||
LineNumberReader lineReader = new LineNumberReader(reader)) {
|
||||
|
||||
String line;
|
||||
while ((line = lineReader.readLine()) != null) {
|
||||
line = line.replaceAll("^\\s", "");
|
||||
line = line.replaceAll("\\s*#.*", "");
|
||||
line = line.replaceAll("\\s+", " ");
|
||||
|
||||
// Skip empty line or comment line
|
||||
if (line.length() == 0) {
|
||||
continue;
|
||||
}
|
||||
|
||||
if (line.startsWith("0x")) { // Category mapping
|
||||
String[] values = line.split(" ", 2); // Split only first space
|
||||
|
||||
if (!values[0].contains("..")) {
|
||||
int cp = Integer.decode(values[0]);
|
||||
dictionary.putCharacterCategory(cp, values[1]);
|
||||
} else {
|
||||
String[] codePoints = values[0].split("\\.\\.");
|
||||
int cpFrom = Integer.decode(codePoints[0]);
|
||||
int cpTo = Integer.decode(codePoints[1]);
|
||||
|
||||
for (int i = cpFrom; i <= cpTo; i++) {
|
||||
dictionary.putCharacterCategory(i, values[1]);
|
||||
}
|
||||
}
|
||||
} else { // Invoke definition
|
||||
String[] values = line.split(" "); // Consecutive space is merged above
|
||||
String characterClassName = values[0];
|
||||
int invoke = Integer.parseInt(values[1]);
|
||||
int group = Integer.parseInt(values[2]);
|
||||
int length = Integer.parseInt(values[3]);
|
||||
dictionary.putInvokeDefinition(characterClassName, invoke, group, length);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -17,11 +17,12 @@
|
|||
package org.apache.lucene.analysis.ko.util;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.file.Path;
|
||||
|
||||
import org.apache.lucene.analysis.ko.dict.CharacterDefinition;
|
||||
import org.apache.lucene.analysis.ko.dict.UnknownDictionary;
|
||||
|
||||
public class UnknownDictionaryWriter extends BinaryDictionaryWriter {
|
||||
class UnknownDictionaryWriter extends BinaryDictionaryWriter {
|
||||
|
||||
private final CharacterDefinitionWriter characterDefinition = new CharacterDefinitionWriter();
|
||||
|
||||
|
@ -58,7 +59,7 @@ public class UnknownDictionaryWriter extends BinaryDictionaryWriter {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void write(String baseDir) throws IOException {
|
||||
public void write(Path baseDir) throws IOException {
|
||||
super.write(baseDir);
|
||||
characterDefinition.write(baseDir);
|
||||
}
|
Binary file not shown.
|
@ -16,15 +16,74 @@
|
|||
*/
|
||||
package org.apache.lucene.analysis.ko.dict;
|
||||
|
||||
import java.io.OutputStream;
|
||||
import java.io.OutputStreamWriter;
|
||||
import java.io.PrintWriter;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
|
||||
import org.apache.lucene.analysis.ko.POS;
|
||||
import org.apache.lucene.analysis.ko.util.DictionaryBuilder;
|
||||
import org.apache.lucene.util.IntsRef;
|
||||
import org.apache.lucene.util.IntsRefBuilder;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.UnicodeUtil;
|
||||
import org.apache.lucene.util.fst.FST;
|
||||
import org.apache.lucene.util.fst.IntsRefFSTEnum;
|
||||
import org.apache.lucene.util.fst.IntsRefFSTEnum.InputOutput;
|
||||
|
||||
public class TestTokenInfoDictionary extends LuceneTestCase {
|
||||
import static org.apache.lucene.analysis.ko.dict.BinaryDictionary.ResourceScheme;
|
||||
|
||||
/**
|
||||
* Tests of TokenInfoDictionary build tools; run using ant test-tools
|
||||
*/
|
||||
public class TokenInfoDictionaryTest extends LuceneTestCase {
|
||||
|
||||
public void testPut() throws Exception {
|
||||
TokenInfoDictionary dict = newDictionary("명사,1,1,2,NNG,*,*,*,*,*,*,*",
|
||||
// "large" id
|
||||
"일반,5000,5000,3,NNG,*,*,*,*,*,*,*");
|
||||
IntsRef wordIdRef = new IntsRefBuilder().get();
|
||||
|
||||
dict.lookupWordIds(0, wordIdRef);
|
||||
int wordId = wordIdRef.ints[wordIdRef.offset];
|
||||
assertEquals(1, dict.getLeftId(wordId));
|
||||
assertEquals(1, dict.getRightId(wordId));
|
||||
assertEquals(2, dict.getWordCost(wordId));
|
||||
|
||||
dict.lookupWordIds(1, wordIdRef);
|
||||
wordId = wordIdRef.ints[wordIdRef.offset];
|
||||
assertEquals(5000, dict.getLeftId(wordId));
|
||||
assertEquals(5000, dict.getRightId(wordId));
|
||||
assertEquals(3, dict.getWordCost(wordId));
|
||||
}
|
||||
|
||||
private TokenInfoDictionary newDictionary(String... entries) throws Exception {
|
||||
Path dir = createTempDir();
|
||||
try (OutputStream out = Files.newOutputStream(dir.resolve("test.csv"));
|
||||
PrintWriter printer = new PrintWriter(new OutputStreamWriter(out, StandardCharsets.UTF_8))) {
|
||||
for (String entry : entries) {
|
||||
printer.println(entry);
|
||||
}
|
||||
}
|
||||
Files.createFile(dir.resolve("unk.def"));
|
||||
Files.createFile(dir.resolve("char.def"));
|
||||
try (OutputStream out = Files.newOutputStream(dir.resolve("matrix.def"));
|
||||
PrintWriter printer = new PrintWriter(new OutputStreamWriter(out, StandardCharsets.UTF_8))) {
|
||||
printer.println("1 1");
|
||||
}
|
||||
DictionaryBuilder.build(dir, dir, "utf-8", true);
|
||||
String dictionaryPath = TokenInfoDictionary.class.getName().replace('.', '/');
|
||||
// We must also load the other files (in BinaryDictionary) from the correct path
|
||||
return new TokenInfoDictionary(ResourceScheme.FILE, dir.resolve(dictionaryPath).toString());
|
||||
}
|
||||
|
||||
public void testPutException() {
|
||||
//too few columns
|
||||
expectThrows(IllegalArgumentException.class, () -> newDictionary("HANGUL,1,1,1,NNG,*,*,*,*,*"));
|
||||
// id too large
|
||||
expectThrows(IllegalArgumentException.class, () -> newDictionary("HANGUL,8192,8192,1,NNG,*,*,*,*,*,*,*"));
|
||||
}
|
||||
|
||||
/** enumerates the entire FST/lookup data and just does basic sanity checks */
|
||||
public void testEnumerateAll() throws Exception {
|
||||
|
@ -38,12 +97,12 @@ public class TestTokenInfoDictionary extends LuceneTestCase {
|
|||
ConnectionCosts matrix = ConnectionCosts.getInstance();
|
||||
FST<Long> fst = tid.getFST().getInternalFST();
|
||||
IntsRefFSTEnum<Long> fstEnum = new IntsRefFSTEnum<>(fst);
|
||||
InputOutput<Long> mapping;
|
||||
IntsRefFSTEnum.InputOutput<Long> mapping;
|
||||
IntsRef scratch = new IntsRef();
|
||||
while ((mapping = fstEnum.next()) != null) {
|
||||
numTerms++;
|
||||
IntsRef input = mapping.input;
|
||||
char chars[] = new char[input.length];
|
||||
char[] chars = new char[input.length];
|
||||
for (int i = 0; i < chars.length; i++) {
|
||||
chars[i] = (char)input.ints[input.offset+i];
|
||||
}
|
||||
|
@ -51,7 +110,7 @@ public class TestTokenInfoDictionary extends LuceneTestCase {
|
|||
assertFalse(surfaceForm.isEmpty());
|
||||
assertEquals(surfaceForm.trim(), surfaceForm);
|
||||
assertTrue(UnicodeUtil.validUTF16String(surfaceForm));
|
||||
|
||||
|
||||
Long output = mapping.output;
|
||||
int sourceId = output.intValue();
|
||||
// we walk in order, terms, sourceIds, and wordIds should always be increasing
|
|
@ -14,11 +14,8 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.lucene.analysis.ko.dict;
|
||||
package org.apache.lucene.analysis.ko.util;
|
||||
|
||||
|
||||
import org.apache.lucene.analysis.ko.util.CSVUtil;
|
||||
import org.apache.lucene.analysis.ko.util.UnknownDictionaryWriter;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.junit.Test;
|
||||
|
|
@ -1,67 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.lucene.analysis.ko.util;
|
||||
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStreamReader;
|
||||
import java.io.LineNumberReader;
|
||||
import java.nio.charset.Charset;
|
||||
import java.nio.charset.CharsetDecoder;
|
||||
import java.nio.charset.CodingErrorAction;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
|
||||
public class ConnectionCostsBuilder {
|
||||
|
||||
private ConnectionCostsBuilder() {
|
||||
}
|
||||
|
||||
public static ConnectionCostsWriter build(String filename) throws IOException {
|
||||
FileInputStream inputStream = new FileInputStream(filename);
|
||||
Charset cs = StandardCharsets.US_ASCII;
|
||||
CharsetDecoder decoder = cs.newDecoder()
|
||||
.onMalformedInput(CodingErrorAction.REPORT)
|
||||
.onUnmappableCharacter(CodingErrorAction.REPORT);
|
||||
InputStreamReader streamReader = new InputStreamReader(inputStream, decoder);
|
||||
LineNumberReader lineReader = new LineNumberReader(streamReader);
|
||||
|
||||
String line = lineReader.readLine();
|
||||
String[] dimensions = line.split("\\s+");
|
||||
|
||||
assert dimensions.length == 2;
|
||||
|
||||
int forwardSize = Integer.parseInt(dimensions[0]);
|
||||
int backwardSize = Integer.parseInt(dimensions[1]);
|
||||
|
||||
assert forwardSize > 0 && backwardSize > 0;
|
||||
|
||||
ConnectionCostsWriter costs = new ConnectionCostsWriter(forwardSize, backwardSize);
|
||||
|
||||
while ((line = lineReader.readLine()) != null) {
|
||||
String[] fields = line.split("\\s+");
|
||||
|
||||
assert fields.length == 3;
|
||||
|
||||
int forwardId = Integer.parseInt(fields[0]);
|
||||
int backwardId = Integer.parseInt(fields[1]);
|
||||
int cost = Integer.parseInt(fields[2]);
|
||||
|
||||
costs.add(forwardId, backwardId, cost);
|
||||
}
|
||||
return costs;
|
||||
}
|
||||
}
|
|
@ -1,67 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.lucene.analysis.ko.util;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
|
||||
public class DictionaryBuilder {
|
||||
|
||||
private DictionaryBuilder() {
|
||||
}
|
||||
|
||||
public static void build(String inputDirname, String outputDirname, String encoding, boolean normalizeEntry) throws IOException {
|
||||
System.out.println("building tokeninfo dict...");
|
||||
TokenInfoDictionaryBuilder tokenInfoBuilder = new TokenInfoDictionaryBuilder(encoding, normalizeEntry);
|
||||
TokenInfoDictionaryWriter tokenInfoDictionary = tokenInfoBuilder.build(inputDirname);
|
||||
tokenInfoDictionary.write(outputDirname);
|
||||
tokenInfoDictionary = null;
|
||||
tokenInfoBuilder = null;
|
||||
System.out.println("done");
|
||||
|
||||
System.out.print("building unknown word dict...");
|
||||
UnknownDictionaryBuilder unkBuilder = new UnknownDictionaryBuilder(encoding);
|
||||
UnknownDictionaryWriter unkDictionary = unkBuilder.build(inputDirname);
|
||||
unkDictionary.write(outputDirname);
|
||||
unkDictionary = null;
|
||||
unkBuilder = null;
|
||||
System.out.println("done");
|
||||
|
||||
System.out.print("building connection costs...");
|
||||
ConnectionCostsWriter connectionCosts
|
||||
= ConnectionCostsBuilder.build(inputDirname + File.separator + "matrix.def");
|
||||
connectionCosts.write(outputDirname);
|
||||
System.out.println("done");
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws IOException {
|
||||
String inputDirname = args[0];
|
||||
String outputDirname = args[1];
|
||||
String inputEncoding = args[2];
|
||||
boolean normalizeEntries = Boolean.parseBoolean(args[3]);
|
||||
|
||||
System.out.println("dictionary builder");
|
||||
System.out.println("");
|
||||
System.out.println("input directory: " + inputDirname);
|
||||
System.out.println("output directory: " + outputDirname);
|
||||
System.out.println("input encoding: " + inputEncoding);
|
||||
System.out.println("normalize entries: " + normalizeEntries);
|
||||
System.out.println("");
|
||||
DictionaryBuilder.build(inputDirname, outputDirname, inputEncoding, normalizeEntries);
|
||||
}
|
||||
|
||||
}
|
|
@ -1,134 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.lucene.analysis.ko.util;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStreamReader;
|
||||
import java.io.LineNumberReader;
|
||||
import java.nio.charset.Charset;
|
||||
import java.nio.charset.CharsetDecoder;
|
||||
import java.nio.charset.CodingErrorAction;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.analysis.ko.dict.CharacterDefinition;
|
||||
|
||||
public class UnknownDictionaryBuilder {
|
||||
private static final String NGRAM_DICTIONARY_ENTRY = "NGRAM,1798,3559,3677,SY,*,*,*,*,*,*,*";
|
||||
|
||||
private String encoding = "utf-8";
|
||||
|
||||
public UnknownDictionaryBuilder(String encoding) {
|
||||
this.encoding = encoding;
|
||||
}
|
||||
|
||||
public UnknownDictionaryWriter build(String dirname) throws IOException {
|
||||
UnknownDictionaryWriter unkDictionary = readDictionaryFile(dirname + File.separator + "unk.def"); //Should be only one file
|
||||
readCharacterDefinition(dirname + File.separator + "char.def", unkDictionary);
|
||||
return unkDictionary;
|
||||
}
|
||||
|
||||
public UnknownDictionaryWriter readDictionaryFile(String filename)
|
||||
throws IOException {
|
||||
return readDictionaryFile(filename, encoding);
|
||||
}
|
||||
|
||||
public UnknownDictionaryWriter readDictionaryFile(String filename, String encoding)
|
||||
throws IOException {
|
||||
UnknownDictionaryWriter dictionary = new UnknownDictionaryWriter(5 * 1024 * 1024);
|
||||
|
||||
FileInputStream inputStream = new FileInputStream(filename);
|
||||
Charset cs = Charset.forName(encoding);
|
||||
CharsetDecoder decoder = cs.newDecoder()
|
||||
.onMalformedInput(CodingErrorAction.REPORT)
|
||||
.onUnmappableCharacter(CodingErrorAction.REPORT);
|
||||
InputStreamReader streamReader = new InputStreamReader(inputStream, decoder);
|
||||
LineNumberReader lineReader = new LineNumberReader(streamReader);
|
||||
|
||||
dictionary.put(CSVUtil.parse(NGRAM_DICTIONARY_ENTRY));
|
||||
|
||||
List<String[]> lines = new ArrayList<>();
|
||||
String line = null;
|
||||
while ((line = lineReader.readLine()) != null) {
|
||||
// note: unk.def only has 10 fields, it simplifies the writer to just append empty reading and pronunciation,
|
||||
// even though the unknown dictionary returns hardcoded null here.
|
||||
final String[] parsed = CSVUtil.parse(line + ",*,*"); // Probably we don't need to validate entry
|
||||
lines.add(parsed);
|
||||
}
|
||||
|
||||
Collections.sort(lines, new Comparator<String[]>() {
|
||||
public int compare(String[] left, String[] right) {
|
||||
int leftId = CharacterDefinition.lookupCharacterClass(left[0]);
|
||||
int rightId = CharacterDefinition.lookupCharacterClass(right[0]);
|
||||
return leftId - rightId;
|
||||
}
|
||||
});
|
||||
|
||||
for (String[] entry : lines) {
|
||||
dictionary.put(entry);
|
||||
}
|
||||
|
||||
return dictionary;
|
||||
}
|
||||
|
||||
public void readCharacterDefinition(String filename, UnknownDictionaryWriter dictionary) throws IOException {
|
||||
FileInputStream inputStream = new FileInputStream(filename);
|
||||
InputStreamReader streamReader = new InputStreamReader(inputStream, encoding);
|
||||
LineNumberReader lineReader = new LineNumberReader(streamReader);
|
||||
|
||||
String line = null;
|
||||
|
||||
while ((line = lineReader.readLine()) != null) {
|
||||
line = line.replaceAll("^\\s", "");
|
||||
line = line.replaceAll("\\s*#.*", "");
|
||||
line = line.replaceAll("\\s+", " ");
|
||||
|
||||
// Skip empty line or comment line
|
||||
if(line.length() == 0) {
|
||||
continue;
|
||||
}
|
||||
|
||||
if(line.startsWith("0x")) { // Category mapping
|
||||
String[] values = line.split(" ", 2); // Split only first space
|
||||
|
||||
if(!values[0].contains("..")) {
|
||||
int cp = Integer.decode(values[0]).intValue();
|
||||
dictionary.putCharacterCategory(cp, values[1]);
|
||||
} else {
|
||||
String[] codePoints = values[0].split("\\.\\.");
|
||||
int cpFrom = Integer.decode(codePoints[0]).intValue();
|
||||
int cpTo = Integer.decode(codePoints[1]).intValue();
|
||||
|
||||
for(int i = cpFrom; i <= cpTo; i++){
|
||||
dictionary.putCharacterCategory(i, values[1]);
|
||||
}
|
||||
}
|
||||
} else { // Invoke definition
|
||||
String[] values = line.split(" "); // Consecutive space is merged above
|
||||
String characterClassName = values[0];
|
||||
int invoke = Integer.parseInt(values[1]);
|
||||
int group = Integer.parseInt(values[2]);
|
||||
int length = Integer.parseInt(values[3]);
|
||||
dictionary.putInvokeDefinition(characterClassName, invoke, group, length);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,80 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.lucene.analysis.ko.dict;
|
||||
|
||||
import java.io.OutputStream;
|
||||
import java.io.OutputStreamWriter;
|
||||
import java.io.PrintWriter;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
|
||||
import org.apache.lucene.analysis.ko.util.TokenInfoDictionaryBuilder;
|
||||
import org.apache.lucene.analysis.ko.util.TokenInfoDictionaryWriter;
|
||||
import org.apache.lucene.util.IntsRef;
|
||||
import org.apache.lucene.util.IntsRefBuilder;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
||||
import static java.io.File.separatorChar;
|
||||
import static org.apache.lucene.analysis.ko.dict.BinaryDictionary.ResourceScheme;
|
||||
|
||||
/**
|
||||
* Tests of TokenInfoDictionary build tools; run using ant test-tools
|
||||
*/
|
||||
public class TokenInfoDictionaryTest extends LuceneTestCase {
|
||||
|
||||
public void testPut() throws Exception {
|
||||
TokenInfoDictionary dict = newDictionary("명사,1,1,2,NNG,*,*,*,*,*,*,*",
|
||||
// "large" id
|
||||
"일반,5000,5000,3,NNG,*,*,*,*,*,*,*");
|
||||
IntsRef wordIdRef = new IntsRefBuilder().get();
|
||||
|
||||
dict.lookupWordIds(0, wordIdRef);
|
||||
int wordId = wordIdRef.ints[wordIdRef.offset];
|
||||
assertEquals(1, dict.getLeftId(wordId));
|
||||
assertEquals(1, dict.getRightId(wordId));
|
||||
assertEquals(2, dict.getWordCost(wordId));
|
||||
|
||||
dict.lookupWordIds(1, wordIdRef);
|
||||
wordId = wordIdRef.ints[wordIdRef.offset];
|
||||
assertEquals(5000, dict.getLeftId(wordId));
|
||||
assertEquals(5000, dict.getRightId(wordId));
|
||||
assertEquals(3, dict.getWordCost(wordId));
|
||||
}
|
||||
|
||||
private TokenInfoDictionary newDictionary(String... entries) throws Exception {
|
||||
Path dir = createTempDir();
|
||||
try (OutputStream out = Files.newOutputStream(dir.resolve("test.csv"));
|
||||
PrintWriter printer = new PrintWriter(new OutputStreamWriter(out, "utf-8"))) {
|
||||
for (String entry : entries) {
|
||||
printer.println(entry);
|
||||
}
|
||||
}
|
||||
TokenInfoDictionaryBuilder builder = new TokenInfoDictionaryBuilder("utf-8", true);
|
||||
TokenInfoDictionaryWriter writer = builder.build(dir.toString());
|
||||
writer.write(dir.toString());
|
||||
String dictionaryPath = TokenInfoDictionary.class.getName().replace('.', separatorChar);
|
||||
// We must also load the other files (in BinaryDictionary) from the correct path
|
||||
return new TokenInfoDictionary(ResourceScheme.FILE, dir.resolve(dictionaryPath).toString());
|
||||
}
|
||||
|
||||
public void testPutException() throws Exception {
|
||||
// too few columns
|
||||
expectThrows(IllegalArgumentException.class, () -> newDictionary("HANGUL,1,1,1,NNG,*,*,*,*,*"));
|
||||
// id too large
|
||||
expectThrows(IllegalArgumentException.class, () -> newDictionary("HANGUL,8192,8192,1,NNG,*,*,*,*,*,*,*"));
|
||||
}
|
||||
}
|
|
@ -95,7 +95,7 @@ final class OrdsIntersectTermsEnum extends BaseTermsEnum {
|
|||
f.prefix = 0;
|
||||
f.setState(0);
|
||||
f.arc = arc;
|
||||
f.outputPrefix = arc.output;
|
||||
f.outputPrefix = arc.output();
|
||||
f.load(fr.rootCode);
|
||||
|
||||
// for assert:
|
||||
|
@ -168,14 +168,14 @@ final class OrdsIntersectTermsEnum extends BaseTermsEnum {
|
|||
// passed to findTargetArc
|
||||
arc = fr.index.findTargetArc(target, arc, getArc(1+idx), fstReader);
|
||||
assert arc != null;
|
||||
output = OrdsBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.output);
|
||||
output = OrdsBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.output());
|
||||
idx++;
|
||||
}
|
||||
|
||||
f.arc = arc;
|
||||
f.outputPrefix = output;
|
||||
assert arc.isFinal();
|
||||
f.load(OrdsBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.nextFinalOutput));
|
||||
f.load(OrdsBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.nextFinalOutput()));
|
||||
return f;
|
||||
}
|
||||
|
||||
|
|
|
@ -271,7 +271,7 @@ public final class OrdsSegmentTermsEnum extends BaseTermsEnum {
|
|||
|
||||
arc = arcs[0];
|
||||
assert arc.isFinal();
|
||||
output = arc.output;
|
||||
output = arc.output();
|
||||
targetUpto = 0;
|
||||
|
||||
OrdsSegmentTermsEnumFrame lastFrame = stack[0];
|
||||
|
@ -294,9 +294,9 @@ public final class OrdsSegmentTermsEnum extends BaseTermsEnum {
|
|||
break;
|
||||
}
|
||||
arc = arcs[1+targetUpto];
|
||||
assert arc.label == (target.bytes[target.offset + targetUpto] & 0xFF): "arc.label=" + (char) arc.label + " targetLabel=" + (char) (target.bytes[target.offset + targetUpto] & 0xFF);
|
||||
if (arc.output != OrdsBlockTreeTermsWriter.NO_OUTPUT) {
|
||||
output = OrdsBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.output);
|
||||
assert arc.label() == (target.bytes[target.offset + targetUpto] & 0xFF): "arc.label=" + (char) arc.label() + " targetLabel=" + (char) (target.bytes[target.offset + targetUpto] & 0xFF);
|
||||
if (arc.output() != OrdsBlockTreeTermsWriter.NO_OUTPUT) {
|
||||
output = OrdsBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.output());
|
||||
}
|
||||
if (arc.isFinal()) {
|
||||
lastFrame = stack[1+lastFrame.ord];
|
||||
|
@ -374,19 +374,19 @@ public final class OrdsSegmentTermsEnum extends BaseTermsEnum {
|
|||
|
||||
// Empty string prefix must have an output (block) in the index!
|
||||
assert arc.isFinal();
|
||||
assert arc.output != null;
|
||||
assert arc.output() != null;
|
||||
|
||||
// if (DEBUG) {
|
||||
// System.out.println(" no seek state; push root frame");
|
||||
// }
|
||||
|
||||
output = arc.output;
|
||||
output = arc.output();
|
||||
|
||||
currentFrame = staticFrame;
|
||||
|
||||
//term.length = 0;
|
||||
targetUpto = 0;
|
||||
currentFrame = pushFrame(arc, OrdsBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.nextFinalOutput), 0);
|
||||
currentFrame = pushFrame(arc, OrdsBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.nextFinalOutput()), 0);
|
||||
}
|
||||
|
||||
positioned = true;
|
||||
|
@ -443,9 +443,9 @@ public final class OrdsSegmentTermsEnum extends BaseTermsEnum {
|
|||
arc = nextArc;
|
||||
term.setByteAt(targetUpto, (byte) targetLabel);
|
||||
// Aggregate output as we go:
|
||||
assert arc.output != null;
|
||||
if (arc.output != OrdsBlockTreeTermsWriter.NO_OUTPUT) {
|
||||
output = OrdsBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.output);
|
||||
assert arc.output() != null;
|
||||
if (arc.output() != OrdsBlockTreeTermsWriter.NO_OUTPUT) {
|
||||
output = OrdsBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.output());
|
||||
}
|
||||
|
||||
// if (DEBUG) {
|
||||
|
@ -455,7 +455,7 @@ public final class OrdsSegmentTermsEnum extends BaseTermsEnum {
|
|||
|
||||
if (arc.isFinal()) {
|
||||
//if (DEBUG) System.out.println(" arc is final!");
|
||||
currentFrame = pushFrame(arc, OrdsBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.nextFinalOutput), targetUpto);
|
||||
currentFrame = pushFrame(arc, OrdsBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.nextFinalOutput()), targetUpto);
|
||||
//if (DEBUG) System.out.println(" curFrame.ord=" + currentFrame.ord + " hasTerms=" + currentFrame.hasTerms);
|
||||
}
|
||||
}
|
||||
|
@ -529,7 +529,7 @@ public final class OrdsSegmentTermsEnum extends BaseTermsEnum {
|
|||
|
||||
arc = arcs[0];
|
||||
assert arc.isFinal();
|
||||
output = arc.output;
|
||||
output = arc.output();
|
||||
targetUpto = 0;
|
||||
|
||||
OrdsSegmentTermsEnumFrame lastFrame = stack[0];
|
||||
|
@ -552,14 +552,14 @@ public final class OrdsSegmentTermsEnum extends BaseTermsEnum {
|
|||
break;
|
||||
}
|
||||
arc = arcs[1+targetUpto];
|
||||
assert arc.label == (target.bytes[target.offset + targetUpto] & 0xFF): "arc.label=" + (char) arc.label + " targetLabel=" + (char) (target.bytes[target.offset + targetUpto] & 0xFF);
|
||||
assert arc.label() == (target.bytes[target.offset + targetUpto] & 0xFF): "arc.label=" + (char) arc.label() + " targetLabel=" + (char) (target.bytes[target.offset + targetUpto] & 0xFF);
|
||||
// TODO: we could save the outputs in local
|
||||
// byte[][] instead of making new objs ever
|
||||
// seek; but, often the FST doesn't have any
|
||||
// shared bytes (but this could change if we
|
||||
// reverse vLong byte order)
|
||||
if (arc.output != OrdsBlockTreeTermsWriter.NO_OUTPUT) {
|
||||
output = OrdsBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.output);
|
||||
if (arc.output() != OrdsBlockTreeTermsWriter.NO_OUTPUT) {
|
||||
output = OrdsBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.output());
|
||||
}
|
||||
if (arc.isFinal()) {
|
||||
lastFrame = stack[1+lastFrame.ord];
|
||||
|
@ -632,19 +632,19 @@ public final class OrdsSegmentTermsEnum extends BaseTermsEnum {
|
|||
|
||||
// Empty string prefix must have an output (block) in the index!
|
||||
assert arc.isFinal();
|
||||
assert arc.output != null;
|
||||
assert arc.output() != null;
|
||||
|
||||
//if (DEBUG) {
|
||||
//System.out.println(" no seek state; push root frame");
|
||||
//}
|
||||
|
||||
output = arc.output;
|
||||
output = arc.output();
|
||||
|
||||
currentFrame = staticFrame;
|
||||
|
||||
//term.length = 0;
|
||||
targetUpto = 0;
|
||||
currentFrame = pushFrame(arc, OrdsBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.nextFinalOutput), 0);
|
||||
currentFrame = pushFrame(arc, OrdsBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.nextFinalOutput()), 0);
|
||||
}
|
||||
|
||||
positioned = true;
|
||||
|
@ -701,9 +701,9 @@ public final class OrdsSegmentTermsEnum extends BaseTermsEnum {
|
|||
term.setByteAt(targetUpto, (byte) targetLabel);
|
||||
arc = nextArc;
|
||||
// Aggregate output as we go:
|
||||
assert arc.output != null;
|
||||
if (arc.output != OrdsBlockTreeTermsWriter.NO_OUTPUT) {
|
||||
output = OrdsBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.output);
|
||||
assert arc.output() != null;
|
||||
if (arc.output() != OrdsBlockTreeTermsWriter.NO_OUTPUT) {
|
||||
output = OrdsBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.output());
|
||||
}
|
||||
|
||||
//if (DEBUG) {
|
||||
|
@ -713,7 +713,7 @@ public final class OrdsSegmentTermsEnum extends BaseTermsEnum {
|
|||
|
||||
if (arc.isFinal()) {
|
||||
//if (DEBUG) System.out.println(" arc is final!");
|
||||
currentFrame = pushFrame(arc, OrdsBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.nextFinalOutput), targetUpto);
|
||||
currentFrame = pushFrame(arc, OrdsBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.nextFinalOutput()), targetUpto);
|
||||
//if (DEBUG) System.out.println(" curFrame.ord=" + currentFrame.ord + " hasTerms=" + currentFrame.hasTerms);
|
||||
}
|
||||
}
|
||||
|
@ -766,8 +766,8 @@ public final class OrdsSegmentTermsEnum extends BaseTermsEnum {
|
|||
}
|
||||
if (fr.index != null) {
|
||||
assert !isSeekFrame || f.arc != null: "isSeekFrame=" + isSeekFrame + " f.arc=" + f.arc;
|
||||
if (f.prefix > 0 && isSeekFrame && f.arc.label != (term.byteAt(f.prefix-1)&0xFF)) {
|
||||
out.println(" broken seek state: arc.label=" + (char) f.arc.label + " vs term byte=" + (char) (term.byteAt(f.prefix-1)&0xFF));
|
||||
if (f.prefix > 0 && isSeekFrame && f.arc.label() != (term.byteAt(f.prefix-1)&0xFF)) {
|
||||
out.println(" broken seek state: arc.label=" + (char) f.arc.label() + " vs term byte=" + (char) (term.byteAt(f.prefix-1)&0xFF));
|
||||
throw new RuntimeException("seek state is broken");
|
||||
}
|
||||
Output output = Util.get(fr.index, prefix);
|
||||
|
@ -1052,7 +1052,7 @@ public final class OrdsSegmentTermsEnum extends BaseTermsEnum {
|
|||
final IntsRefBuilder result = new IntsRefBuilder();
|
||||
|
||||
fr.index.getFirstArc(arc);
|
||||
Output output = arc.output;
|
||||
Output output = arc.output();
|
||||
int upto = 0;
|
||||
|
||||
int bestUpto = 0;
|
||||
|
@ -1069,7 +1069,7 @@ public final class OrdsSegmentTermsEnum extends BaseTermsEnum {
|
|||
while (true) {
|
||||
// System.out.println(" loop: output=" + output.startOrd + "-" + (Long.MAX_VALUE-output.endOrd) + " upto=" + upto + " arc=" + arc + " final?=" + arc.isFinal());
|
||||
if (arc.isFinal()) {
|
||||
final Output finalOutput = OrdsBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.nextFinalOutput);
|
||||
final Output finalOutput = OrdsBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.nextFinalOutput());
|
||||
// System.out.println(" isFinal: " + finalOutput.startOrd + "-" + (Long.MAX_VALUE-finalOutput.endOrd));
|
||||
if (targetOrd >= finalOutput.startOrd && targetOrd <= Long.MAX_VALUE-finalOutput.endOrd) {
|
||||
// Only one range should match across all arc leaving this node
|
||||
|
@ -1082,19 +1082,19 @@ public final class OrdsSegmentTermsEnum extends BaseTermsEnum {
|
|||
if (FST.targetHasArcs(arc)) {
|
||||
// System.out.println(" targetHasArcs");
|
||||
result.grow(1+upto);
|
||||
fr.index.readFirstRealTargetArc(arc.target, arc, fstReader);
|
||||
fr.index.readFirstRealTargetArc(arc.target(), arc, fstReader);
|
||||
|
||||
if (arc.bytesPerArc != 0 && arc.arcIdx > Integer.MIN_VALUE) {
|
||||
if (arc.bytesPerArc() != 0 && arc.arcIdx() > Integer.MIN_VALUE) {
|
||||
// System.out.println(" array arcs");
|
||||
int low = 0;
|
||||
int high = arc.numArcs-1;
|
||||
int high = arc.numArcs() -1;
|
||||
int mid = 0;
|
||||
//System.out.println("bsearch: numArcs=" + arc.numArcs + " target=" + targetOutput + " output=" + output);
|
||||
boolean found = false;
|
||||
while (low <= high) {
|
||||
mid = (low + high) >>> 1;
|
||||
fstReader.setPosition(arc.posArcsStart);
|
||||
fstReader.skipBytes(arc.bytesPerArc*mid);
|
||||
fstReader.setPosition(arc.posArcsStart());
|
||||
fstReader.skipBytes(arc.bytesPerArc() *mid);
|
||||
final byte flags = fstReader.readByte();
|
||||
fr.index.readLabel(fstReader);
|
||||
final Output minArcOutput;
|
||||
|
@ -1115,10 +1115,7 @@ public final class OrdsSegmentTermsEnum extends BaseTermsEnum {
|
|||
}
|
||||
}
|
||||
|
||||
if (found) {
|
||||
// Keep recursing
|
||||
arc.arcIdx = mid-1;
|
||||
} else {
|
||||
if (found == false) {
|
||||
result.setLength(bestUpto);
|
||||
InputOutput io = new InputOutput();
|
||||
io.input = result.get();
|
||||
|
@ -1127,11 +1124,10 @@ public final class OrdsSegmentTermsEnum extends BaseTermsEnum {
|
|||
return io;
|
||||
}
|
||||
|
||||
fr.index.readNextRealArc(arc, fstReader);
|
||||
|
||||
// Recurse on this arc:
|
||||
result.setIntAt(upto++, arc.label);
|
||||
output = OrdsBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.output);
|
||||
fr.index.readArcByIndex(arc, fstReader, mid);
|
||||
result.setIntAt(upto++, arc.label());
|
||||
output = OrdsBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.output());
|
||||
|
||||
} else {
|
||||
// System.out.println(" non-array arc");
|
||||
|
@ -1141,14 +1137,14 @@ public final class OrdsSegmentTermsEnum extends BaseTermsEnum {
|
|||
|
||||
// This is the min output we'd hit if we follow
|
||||
// this arc:
|
||||
final Output minArcOutput = OrdsBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.output);
|
||||
final Output minArcOutput = OrdsBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.output());
|
||||
long endOrd = Long.MAX_VALUE - minArcOutput.endOrd;
|
||||
// System.out.println(" endOrd=" + endOrd + " targetOrd=" + targetOrd);
|
||||
|
||||
if (targetOrd >= minArcOutput.startOrd && targetOrd <= endOrd) {
|
||||
// Recurse on this arc:
|
||||
output = minArcOutput;
|
||||
result.setIntAt(upto++, arc.label);
|
||||
result.setIntAt(upto++, arc.label());
|
||||
break;
|
||||
} else if (targetOrd < endOrd || arc.isLast()) {
|
||||
result.setLength(bestUpto);
|
||||
|
|
|
@ -65,7 +65,7 @@ import org.apache.lucene.util.fst.Util;
|
|||
* FST-based terms dictionary reader.
|
||||
*
|
||||
* The FST index maps each term and its ord, and during seek
|
||||
* the ord is used fetch metadata from a single block.
|
||||
* the ord is used to fetch metadata from a single block.
|
||||
* The term dictionary is fully memory resident.
|
||||
*
|
||||
* @lucene.experimental
|
||||
|
@ -563,6 +563,8 @@ public class FSTOrdTermsReader extends FieldsProducer {
|
|||
/* fst stats */
|
||||
FST.Arc<Long> arc;
|
||||
|
||||
Long output;
|
||||
|
||||
/* automaton stats */
|
||||
int state;
|
||||
|
||||
|
@ -620,9 +622,7 @@ public class FSTOrdTermsReader extends FieldsProducer {
|
|||
|
||||
@Override
|
||||
void decodeStats() throws IOException {
|
||||
final FST.Arc<Long> arc = topFrame().arc;
|
||||
assert arc.nextFinalOutput == fstOutputs.getNoOutput();
|
||||
ord = arc.output;
|
||||
ord = topFrame().output;
|
||||
super.decodeStats();
|
||||
}
|
||||
|
||||
|
@ -675,7 +675,7 @@ public class FSTOrdTermsReader extends FieldsProducer {
|
|||
frame = newFrame();
|
||||
label = target.bytes[upto] & 0xff;
|
||||
frame = loadCeilFrame(label, topFrame(), frame);
|
||||
if (frame == null || frame.arc.label != label) {
|
||||
if (frame == null || frame.arc.label() != label) {
|
||||
break;
|
||||
}
|
||||
assert isValid(frame); // target must be fetched from automaton
|
||||
|
@ -703,16 +703,16 @@ public class FSTOrdTermsReader extends FieldsProducer {
|
|||
}
|
||||
|
||||
/** Virtual frame, never pop */
|
||||
Frame loadVirtualFrame(Frame frame) throws IOException {
|
||||
frame.arc.output = fstOutputs.getNoOutput();
|
||||
frame.arc.nextFinalOutput = fstOutputs.getNoOutput();
|
||||
Frame loadVirtualFrame(Frame frame) {
|
||||
frame.output = fstOutputs.getNoOutput();
|
||||
frame.state = -1;
|
||||
return frame;
|
||||
}
|
||||
|
||||
/** Load frame for start arc(node) on fst */
|
||||
Frame loadFirstFrame(Frame frame) throws IOException {
|
||||
Frame loadFirstFrame(Frame frame) {
|
||||
frame.arc = fst.getFirstArc(frame.arc);
|
||||
frame.output = frame.arc.output();
|
||||
frame.state = 0;
|
||||
return frame;
|
||||
}
|
||||
|
@ -722,8 +722,9 @@ public class FSTOrdTermsReader extends FieldsProducer {
|
|||
if (!canGrow(top)) {
|
||||
return null;
|
||||
}
|
||||
frame.arc = fst.readFirstRealTargetArc(top.arc.target, frame.arc, fstReader);
|
||||
frame.state = fsa.step(top.state, frame.arc.label);
|
||||
frame.arc = fst.readFirstRealTargetArc(top.arc.target(), frame.arc, fstReader);
|
||||
frame.state = fsa.step(top.state, frame.arc.label());
|
||||
frame.output = frame.arc.output();
|
||||
//if (TEST) System.out.println(" loadExpand frame="+frame);
|
||||
if (frame.state == -1) {
|
||||
return loadNextFrame(top, frame);
|
||||
|
@ -738,7 +739,8 @@ public class FSTOrdTermsReader extends FieldsProducer {
|
|||
}
|
||||
while (!frame.arc.isLast()) {
|
||||
frame.arc = fst.readNextRealArc(frame.arc, fstReader);
|
||||
frame.state = fsa.step(top.state, frame.arc.label);
|
||||
frame.output = frame.arc.output();
|
||||
frame.state = fsa.step(top.state, frame.arc.label());
|
||||
if (frame.state != -1) {
|
||||
break;
|
||||
}
|
||||
|
@ -758,11 +760,12 @@ public class FSTOrdTermsReader extends FieldsProducer {
|
|||
if (arc == null) {
|
||||
return null;
|
||||
}
|
||||
frame.state = fsa.step(top.state, arc.label);
|
||||
frame.state = fsa.step(top.state, arc.label());
|
||||
//if (TEST) System.out.println(" loadCeil frame="+frame);
|
||||
if (frame.state == -1) {
|
||||
return loadNextFrame(top, frame);
|
||||
}
|
||||
frame.output = arc.output();
|
||||
return frame;
|
||||
}
|
||||
|
||||
|
@ -781,8 +784,8 @@ public class FSTOrdTermsReader extends FieldsProducer {
|
|||
|
||||
void pushFrame(Frame frame) {
|
||||
final FST.Arc<Long> arc = frame.arc;
|
||||
arc.output = fstOutputs.add(topFrame().arc.output, arc.output);
|
||||
term = grow(arc.label);
|
||||
frame.output = fstOutputs.add(topFrame().output, frame.output);
|
||||
term = grow(arc.label());
|
||||
level++;
|
||||
assert frame == stack[level];
|
||||
}
|
||||
|
@ -836,7 +839,7 @@ public class FSTOrdTermsReader extends FieldsProducer {
|
|||
queue.add(startArc);
|
||||
while (!queue.isEmpty()) {
|
||||
final FST.Arc<T> arc = queue.remove(0);
|
||||
final long node = arc.target;
|
||||
final long node = arc.target();
|
||||
//System.out.println(arc);
|
||||
if (FST.targetHasArcs(arc) && !seen.get((int) node)) {
|
||||
seen.set((int) node);
|
||||
|
|
|
@ -415,7 +415,7 @@ public class FSTTermsReader extends FieldsProducer {
|
|||
/* True when there is pending term when calling next() */
|
||||
boolean pending;
|
||||
|
||||
/* stack to record how current term is constructed,
|
||||
/* stack to record how current term is constructed,
|
||||
* used to accumulate metadata or rewind term:
|
||||
* level == term.length + 1,
|
||||
* == 0 when term is null */
|
||||
|
@ -438,6 +438,8 @@ public class FSTTermsReader extends FieldsProducer {
|
|||
/* fst stats */
|
||||
FST.Arc<FSTTermOutputs.TermData> fstArc;
|
||||
|
||||
FSTTermOutputs.TermData output;
|
||||
|
||||
/* automaton stats */
|
||||
int fsaState;
|
||||
|
||||
|
@ -464,11 +466,9 @@ public class FSTTermsReader extends FieldsProducer {
|
|||
this.stack[i] = new Frame();
|
||||
}
|
||||
|
||||
Frame frame;
|
||||
frame = loadVirtualFrame(newFrame());
|
||||
loadVirtualFrame(newFrame());
|
||||
this.level++;
|
||||
frame = loadFirstFrame(newFrame());
|
||||
pushFrame(frame);
|
||||
pushFrame(loadFirstFrame(newFrame()));
|
||||
|
||||
this.meta = null;
|
||||
this.metaUpto = 1;
|
||||
|
@ -501,17 +501,17 @@ public class FSTTermsReader extends FieldsProducer {
|
|||
}
|
||||
|
||||
/** Lazily accumulate meta data, when we got a accepted term */
|
||||
void loadMetaData() throws IOException {
|
||||
FST.Arc<FSTTermOutputs.TermData> last, next;
|
||||
last = stack[metaUpto].fstArc;
|
||||
void loadMetaData() {
|
||||
Frame last, next;
|
||||
last = stack[metaUpto];
|
||||
while (metaUpto != level) {
|
||||
metaUpto++;
|
||||
next = stack[metaUpto].fstArc;
|
||||
next = stack[metaUpto];
|
||||
next.output = fstOutputs.add(next.output, last.output);
|
||||
last = next;
|
||||
}
|
||||
if (last.isFinal()) {
|
||||
meta = fstOutputs.add(last.output, last.nextFinalOutput);
|
||||
if (last.fstArc.isFinal()) {
|
||||
meta = fstOutputs.add(last.output, last.fstArc.nextFinalOutput());
|
||||
} else {
|
||||
meta = last.output;
|
||||
}
|
||||
|
@ -575,7 +575,7 @@ public class FSTTermsReader extends FieldsProducer {
|
|||
frame = newFrame();
|
||||
label = target.bytes[upto] & 0xff;
|
||||
frame = loadCeilFrame(label, topFrame(), frame);
|
||||
if (frame == null || frame.fstArc.label != label) {
|
||||
if (frame == null || frame.fstArc.label() != label) {
|
||||
break;
|
||||
}
|
||||
assert isValid(frame); // target must be fetched from automaton
|
||||
|
@ -603,9 +603,8 @@ public class FSTTermsReader extends FieldsProducer {
|
|||
}
|
||||
|
||||
/** Virtual frame, never pop */
|
||||
Frame loadVirtualFrame(Frame frame) throws IOException {
|
||||
frame.fstArc.output = fstOutputs.getNoOutput();
|
||||
frame.fstArc.nextFinalOutput = fstOutputs.getNoOutput();
|
||||
Frame loadVirtualFrame(Frame frame) {
|
||||
frame.output = fstOutputs.getNoOutput();
|
||||
frame.fsaState = -1;
|
||||
return frame;
|
||||
}
|
||||
|
@ -613,6 +612,7 @@ public class FSTTermsReader extends FieldsProducer {
|
|||
/** Load frame for start arc(node) on fst */
|
||||
Frame loadFirstFrame(Frame frame) throws IOException {
|
||||
frame.fstArc = fst.getFirstArc(frame.fstArc);
|
||||
frame.output = frame.fstArc.output();
|
||||
frame.fsaState = 0;
|
||||
return frame;
|
||||
}
|
||||
|
@ -622,12 +622,13 @@ public class FSTTermsReader extends FieldsProducer {
|
|||
if (!canGrow(top)) {
|
||||
return null;
|
||||
}
|
||||
frame.fstArc = fst.readFirstRealTargetArc(top.fstArc.target, frame.fstArc, fstReader);
|
||||
frame.fsaState = fsa.step(top.fsaState, frame.fstArc.label);
|
||||
frame.fstArc = fst.readFirstRealTargetArc(top.fstArc.target(), frame.fstArc, fstReader);
|
||||
frame.fsaState = fsa.step(top.fsaState, frame.fstArc.label());
|
||||
//if (TEST) System.out.println(" loadExpand frame="+frame);
|
||||
if (frame.fsaState == -1) {
|
||||
return loadNextFrame(top, frame);
|
||||
}
|
||||
frame.output = frame.fstArc.output();
|
||||
return frame;
|
||||
}
|
||||
|
||||
|
@ -638,7 +639,7 @@ public class FSTTermsReader extends FieldsProducer {
|
|||
}
|
||||
while (!frame.fstArc.isLast()) {
|
||||
frame.fstArc = fst.readNextRealArc(frame.fstArc, fstReader);
|
||||
frame.fsaState = fsa.step(top.fsaState, frame.fstArc.label);
|
||||
frame.fsaState = fsa.step(top.fsaState, frame.fstArc.label());
|
||||
if (frame.fsaState != -1) {
|
||||
break;
|
||||
}
|
||||
|
@ -647,6 +648,7 @@ public class FSTTermsReader extends FieldsProducer {
|
|||
if (frame.fsaState == -1) {
|
||||
return null;
|
||||
}
|
||||
frame.output = frame.fstArc.output();
|
||||
return frame;
|
||||
}
|
||||
|
||||
|
@ -658,11 +660,12 @@ public class FSTTermsReader extends FieldsProducer {
|
|||
if (arc == null) {
|
||||
return null;
|
||||
}
|
||||
frame.fsaState = fsa.step(top.fsaState, arc.label);
|
||||
frame.fsaState = fsa.step(top.fsaState, arc.label());
|
||||
//if (TEST) System.out.println(" loadCeil frame="+frame);
|
||||
if (frame.fsaState == -1) {
|
||||
return loadNextFrame(top, frame);
|
||||
}
|
||||
frame.output = frame.fstArc.output();
|
||||
return frame;
|
||||
}
|
||||
|
||||
|
@ -680,7 +683,7 @@ public class FSTTermsReader extends FieldsProducer {
|
|||
}
|
||||
|
||||
void pushFrame(Frame frame) {
|
||||
term = grow(frame.fstArc.label);
|
||||
term = grow(frame.fstArc.label());
|
||||
level++;
|
||||
//if (TEST) System.out.println(" term=" + term + " level=" + level);
|
||||
}
|
||||
|
@ -737,7 +740,7 @@ public class FSTTermsReader extends FieldsProducer {
|
|||
queue.add(startArc);
|
||||
while (!queue.isEmpty()) {
|
||||
final FST.Arc<T> arc = queue.remove(0);
|
||||
final long node = arc.target;
|
||||
final long node = arc.target();
|
||||
//System.out.println(arc);
|
||||
if (FST.targetHasArcs(arc) && !seen.get((int) node)) {
|
||||
seen.set((int) node);
|
||||
|
|
|
@ -111,7 +111,7 @@ final class IntersectTermsEnum extends BaseTermsEnum {
|
|||
f.prefix = 0;
|
||||
f.setState(0);
|
||||
f.arc = arc;
|
||||
f.outputPrefix = arc.output;
|
||||
f.outputPrefix = arc.output();
|
||||
f.load(fr.rootCode);
|
||||
|
||||
// for assert:
|
||||
|
@ -186,14 +186,14 @@ final class IntersectTermsEnum extends BaseTermsEnum {
|
|||
// passed to findTargetArc
|
||||
arc = fr.index.findTargetArc(target, arc, getArc(1+idx), fstReader);
|
||||
assert arc != null;
|
||||
output = fstOutputs.add(output, arc.output);
|
||||
output = fstOutputs.add(output, arc.output());
|
||||
idx++;
|
||||
}
|
||||
|
||||
f.arc = arc;
|
||||
f.outputPrefix = output;
|
||||
assert arc.isFinal();
|
||||
f.load(fstOutputs.add(output, arc.nextFinalOutput));
|
||||
f.load(fstOutputs.add(output, arc.nextFinalOutput()));
|
||||
return f;
|
||||
}
|
||||
|
||||
|
|
|
@ -351,7 +351,7 @@ final class SegmentTermsEnum extends BaseTermsEnum {
|
|||
|
||||
arc = arcs[0];
|
||||
assert arc.isFinal();
|
||||
output = arc.output;
|
||||
output = arc.output();
|
||||
targetUpto = 0;
|
||||
|
||||
SegmentTermsEnumFrame lastFrame = stack[0];
|
||||
|
@ -374,9 +374,9 @@ final class SegmentTermsEnum extends BaseTermsEnum {
|
|||
break;
|
||||
}
|
||||
arc = arcs[1+targetUpto];
|
||||
assert arc.label == (target.bytes[target.offset + targetUpto] & 0xFF): "arc.label=" + (char) arc.label + " targetLabel=" + (char) (target.bytes[target.offset + targetUpto] & 0xFF);
|
||||
if (arc.output != BlockTreeTermsReader.NO_OUTPUT) {
|
||||
output = BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.output);
|
||||
assert arc.label() == (target.bytes[target.offset + targetUpto] & 0xFF): "arc.label=" + (char) arc.label() + " targetLabel=" + (char) (target.bytes[target.offset + targetUpto] & 0xFF);
|
||||
if (arc.output() != BlockTreeTermsReader.NO_OUTPUT) {
|
||||
output = BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.output());
|
||||
}
|
||||
if (arc.isFinal()) {
|
||||
lastFrame = stack[1+lastFrame.ord];
|
||||
|
@ -454,19 +454,19 @@ final class SegmentTermsEnum extends BaseTermsEnum {
|
|||
|
||||
// Empty string prefix must have an output (block) in the index!
|
||||
assert arc.isFinal();
|
||||
assert arc.output != null;
|
||||
assert arc.output() != null;
|
||||
|
||||
// if (DEBUG) {
|
||||
// System.out.println(" no seek state; push root frame");
|
||||
// }
|
||||
|
||||
output = arc.output;
|
||||
output = arc.output();
|
||||
|
||||
currentFrame = staticFrame;
|
||||
|
||||
//term.length = 0;
|
||||
targetUpto = 0;
|
||||
currentFrame = pushFrame(arc, BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.nextFinalOutput), 0);
|
||||
currentFrame = pushFrame(arc, BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.nextFinalOutput()), 0);
|
||||
}
|
||||
|
||||
// if (DEBUG) {
|
||||
|
@ -521,9 +521,9 @@ final class SegmentTermsEnum extends BaseTermsEnum {
|
|||
arc = nextArc;
|
||||
term.setByteAt(targetUpto, (byte) targetLabel);
|
||||
// Aggregate output as we go:
|
||||
assert arc.output != null;
|
||||
if (arc.output != BlockTreeTermsReader.NO_OUTPUT) {
|
||||
output = BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.output);
|
||||
assert arc.output() != null;
|
||||
if (arc.output() != BlockTreeTermsReader.NO_OUTPUT) {
|
||||
output = BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.output());
|
||||
}
|
||||
|
||||
// if (DEBUG) {
|
||||
|
@ -533,7 +533,7 @@ final class SegmentTermsEnum extends BaseTermsEnum {
|
|||
|
||||
if (arc.isFinal()) {
|
||||
//if (DEBUG) System.out.println(" arc is final!");
|
||||
currentFrame = pushFrame(arc, BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.nextFinalOutput), targetUpto);
|
||||
currentFrame = pushFrame(arc, BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.nextFinalOutput()), targetUpto);
|
||||
//if (DEBUG) System.out.println(" curFrame.ord=" + currentFrame.ord + " hasTerms=" + currentFrame.hasTerms);
|
||||
}
|
||||
}
|
||||
|
@ -608,7 +608,7 @@ final class SegmentTermsEnum extends BaseTermsEnum {
|
|||
|
||||
arc = arcs[0];
|
||||
assert arc.isFinal();
|
||||
output = arc.output;
|
||||
output = arc.output();
|
||||
targetUpto = 0;
|
||||
|
||||
SegmentTermsEnumFrame lastFrame = stack[0];
|
||||
|
@ -631,14 +631,14 @@ final class SegmentTermsEnum extends BaseTermsEnum {
|
|||
break;
|
||||
}
|
||||
arc = arcs[1+targetUpto];
|
||||
assert arc.label == (target.bytes[target.offset + targetUpto] & 0xFF): "arc.label=" + (char) arc.label + " targetLabel=" + (char) (target.bytes[target.offset + targetUpto] & 0xFF);
|
||||
assert arc.label() == (target.bytes[target.offset + targetUpto] & 0xFF): "arc.label=" + (char) arc.label() + " targetLabel=" + (char) (target.bytes[target.offset + targetUpto] & 0xFF);
|
||||
// TODO: we could save the outputs in local
|
||||
// byte[][] instead of making new objs ever
|
||||
// seek; but, often the FST doesn't have any
|
||||
// shared bytes (but this could change if we
|
||||
// reverse vLong byte order)
|
||||
if (arc.output != BlockTreeTermsReader.NO_OUTPUT) {
|
||||
output = BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.output);
|
||||
if (arc.output() != BlockTreeTermsReader.NO_OUTPUT) {
|
||||
output = BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.output());
|
||||
}
|
||||
if (arc.isFinal()) {
|
||||
lastFrame = stack[1+lastFrame.ord];
|
||||
|
@ -711,19 +711,19 @@ final class SegmentTermsEnum extends BaseTermsEnum {
|
|||
|
||||
// Empty string prefix must have an output (block) in the index!
|
||||
assert arc.isFinal();
|
||||
assert arc.output != null;
|
||||
assert arc.output() != null;
|
||||
|
||||
//if (DEBUG) {
|
||||
//System.out.println(" no seek state; push root frame");
|
||||
//}
|
||||
|
||||
output = arc.output;
|
||||
output = arc.output();
|
||||
|
||||
currentFrame = staticFrame;
|
||||
|
||||
//term.length = 0;
|
||||
targetUpto = 0;
|
||||
currentFrame = pushFrame(arc, BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.nextFinalOutput), 0);
|
||||
currentFrame = pushFrame(arc, BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.nextFinalOutput()), 0);
|
||||
}
|
||||
|
||||
//if (DEBUG) {
|
||||
|
@ -779,9 +779,9 @@ final class SegmentTermsEnum extends BaseTermsEnum {
|
|||
term.setByteAt(targetUpto, (byte) targetLabel);
|
||||
arc = nextArc;
|
||||
// Aggregate output as we go:
|
||||
assert arc.output != null;
|
||||
if (arc.output != BlockTreeTermsReader.NO_OUTPUT) {
|
||||
output = BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.output);
|
||||
assert arc.output() != null;
|
||||
if (arc.output() != BlockTreeTermsReader.NO_OUTPUT) {
|
||||
output = BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.output());
|
||||
}
|
||||
|
||||
//if (DEBUG) {
|
||||
|
@ -791,7 +791,7 @@ final class SegmentTermsEnum extends BaseTermsEnum {
|
|||
|
||||
if (arc.isFinal()) {
|
||||
//if (DEBUG) System.out.println(" arc is final!");
|
||||
currentFrame = pushFrame(arc, BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.nextFinalOutput), targetUpto);
|
||||
currentFrame = pushFrame(arc, BlockTreeTermsReader.FST_OUTPUTS.add(output, arc.nextFinalOutput()), targetUpto);
|
||||
//if (DEBUG) System.out.println(" curFrame.ord=" + currentFrame.ord + " hasTerms=" + currentFrame.hasTerms);
|
||||
}
|
||||
}
|
||||
|
@ -844,8 +844,8 @@ final class SegmentTermsEnum extends BaseTermsEnum {
|
|||
}
|
||||
if (fr.index != null) {
|
||||
assert !isSeekFrame || f.arc != null: "isSeekFrame=" + isSeekFrame + " f.arc=" + f.arc;
|
||||
if (f.prefix > 0 && isSeekFrame && f.arc.label != (term.byteAt(f.prefix-1)&0xFF)) {
|
||||
out.println(" broken seek state: arc.label=" + (char) f.arc.label + " vs term byte=" + (char) (term.byteAt(f.prefix-1)&0xFF));
|
||||
if (f.prefix > 0 && isSeekFrame && f.arc.label() != (term.byteAt(f.prefix-1)&0xFF)) {
|
||||
out.println(" broken seek state: arc.label=" + (char) f.arc.label() + " vs term byte=" + (char) (term.byteAt(f.prefix-1)&0xFF));
|
||||
throw new RuntimeException("seek state is broken");
|
||||
}
|
||||
BytesRef output = Util.get(fr.index, prefix);
|
||||
|
|
|
@ -452,16 +452,36 @@ public final class Lucene50PostingsFormat extends PostingsFormat {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
final static class IntBlockTermState extends BlockTermState {
|
||||
long docStartFP = 0;
|
||||
long posStartFP = 0;
|
||||
long payStartFP = 0;
|
||||
long skipOffset = -1;
|
||||
long lastPosBlockOffset = -1;
|
||||
// docid when there is a single pulsed posting, otherwise -1
|
||||
// freq is always implicitly totalTermFreq in this case.
|
||||
int singletonDocID = -1;
|
||||
|
||||
/**
|
||||
* Holds all state required for {@link Lucene50PostingsReader} to produce a
|
||||
* {@link org.apache.lucene.index.PostingsEnum} without re-seeking the terms dict.
|
||||
*
|
||||
* @lucene.internal
|
||||
*/
|
||||
public static final class IntBlockTermState extends BlockTermState {
|
||||
/** file pointer to the start of the doc ids enumeration, in {@link #DOC_EXTENSION} file */
|
||||
public long docStartFP;
|
||||
/** file pointer to the start of the positions enumeration, in {@link #POS_EXTENSION} file */
|
||||
public long posStartFP;
|
||||
/** file pointer to the start of the payloads enumeration, in {@link #PAY_EXTENSION} file */
|
||||
public long payStartFP;
|
||||
/** file offset for the start of the skip list, relative to docStartFP, if there are more
|
||||
* than {@link #BLOCK_SIZE} docs; otherwise -1 */
|
||||
public long skipOffset;
|
||||
/** file offset for the last position in the last block, if there are more than
|
||||
* {@link #BLOCK_SIZE} positions; otherwise -1 */
|
||||
public long lastPosBlockOffset;
|
||||
/** docid when there is a single pulsed posting, otherwise -1.
|
||||
* freq is always implicitly totalTermFreq in this case. */
|
||||
public int singletonDocID;
|
||||
|
||||
/** Sole constructor. */
|
||||
public IntBlockTermState() {
|
||||
skipOffset = -1;
|
||||
lastPosBlockOffset = -1;
|
||||
singletonDocID = -1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public IntBlockTermState clone() {
|
||||
|
|
|
@ -85,6 +85,19 @@ final class Boolean2ScorerSupplier extends ScorerSupplier {
|
|||
|
||||
@Override
|
||||
public Scorer get(long leadCost) throws IOException {
|
||||
Scorer scorer = getInternal(leadCost);
|
||||
if (scoreMode == ScoreMode.TOP_SCORES &&
|
||||
subs.get(Occur.SHOULD).isEmpty() && subs.get(Occur.MUST).isEmpty()) {
|
||||
// no scoring clauses but scores are needed so we wrap the scorer in
|
||||
// a constant score in order to allow early termination
|
||||
return scorer.twoPhaseIterator() != null ?
|
||||
new ConstantScoreScorer(weight, 0f, scoreMode, scorer.twoPhaseIterator()) :
|
||||
new ConstantScoreScorer(weight, 0f, scoreMode, scorer.iterator());
|
||||
}
|
||||
return scorer;
|
||||
}
|
||||
|
||||
private Scorer getInternal(long leadCost) throws IOException {
|
||||
// three cases: conjunction, disjunction, or mix
|
||||
leadCost = Math.min(leadCost, cost());
|
||||
|
||||
|
|
|
@ -81,7 +81,7 @@ public final class BytesRefFSTEnum<T> extends FSTEnum<T> {
|
|||
public InputOutput<T> seekExact(BytesRef target) throws IOException {
|
||||
this.target = target;
|
||||
targetLength = target.length;
|
||||
if (super.doSeekExact()) {
|
||||
if (doSeekExact()) {
|
||||
assert upto == 1+target.length;
|
||||
return setResult();
|
||||
} else {
|
||||
|
|
|
@ -62,24 +62,24 @@ import org.apache.lucene.util.RamUsageEstimator;
|
|||
*/
|
||||
public final class FST<T> implements Accountable {
|
||||
|
||||
/** Specifies allowed range of each int input label for
|
||||
* this FST. */
|
||||
public enum INPUT_TYPE {BYTE1, BYTE2, BYTE4}
|
||||
|
||||
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(FST.class);
|
||||
private static final long ARC_SHALLOW_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(Arc.class);
|
||||
|
||||
/** Specifies allowed range of each int input label for
|
||||
* this FST. */
|
||||
public static enum INPUT_TYPE {BYTE1, BYTE2, BYTE4};
|
||||
|
||||
static final int BIT_FINAL_ARC = 1 << 0;
|
||||
private static final int BIT_FINAL_ARC = 1 << 0;
|
||||
static final int BIT_LAST_ARC = 1 << 1;
|
||||
static final int BIT_TARGET_NEXT = 1 << 2;
|
||||
|
||||
// TODO: we can free up a bit if we can nuke this:
|
||||
static final int BIT_STOP_NODE = 1 << 3;
|
||||
private static final int BIT_STOP_NODE = 1 << 3;
|
||||
|
||||
/** This flag is set if the arc has an output. */
|
||||
public static final int BIT_ARC_HAS_OUTPUT = 1 << 4;
|
||||
|
||||
static final int BIT_ARC_HAS_FINAL_OUTPUT = 1 << 5;
|
||||
private static final int BIT_ARC_HAS_FINAL_OUTPUT = 1 << 5;
|
||||
|
||||
// We use this as a marker (because this one flag is
|
||||
// illegal by itself ...):
|
||||
|
@ -119,10 +119,13 @@ public final class FST<T> implements Accountable {
|
|||
// non-final node w/ no arcs:
|
||||
private static final long NON_FINAL_END_NODE = 0;
|
||||
|
||||
/* Used for memory accounting */
|
||||
private int cachedArcsBytesUsed;
|
||||
|
||||
/** If arc has this label then that arc is final/accepted */
|
||||
public static final int END_LABEL = -1;
|
||||
|
||||
public final INPUT_TYPE inputType;
|
||||
final INPUT_TYPE inputType;
|
||||
|
||||
// if non-null, this FST accepts the empty string and
|
||||
// produces this output
|
||||
|
@ -139,56 +142,45 @@ public final class FST<T> implements Accountable {
|
|||
|
||||
public final Outputs<T> outputs;
|
||||
|
||||
private Arc<T> cachedRootArcs[];
|
||||
private Arc<T>[] cachedRootArcs;
|
||||
|
||||
/** Represents a single arc. */
|
||||
public static final class Arc<T> {
|
||||
public int label;
|
||||
public T output;
|
||||
|
||||
/** To node (ord or address) */
|
||||
public long target;
|
||||
private int label;
|
||||
|
||||
byte flags;
|
||||
public T nextFinalOutput;
|
||||
private T output;
|
||||
|
||||
private long target;
|
||||
|
||||
private byte flags;
|
||||
|
||||
private T nextFinalOutput;
|
||||
|
||||
// address (into the byte[]), or ord/address if label == END_LABEL
|
||||
long nextArc;
|
||||
private long nextArc;
|
||||
|
||||
/** Where the first arc in the array starts; only valid if
|
||||
* bytesPerArc != 0 */
|
||||
public long posArcsStart;
|
||||
|
||||
/** Non-zero if this arc is part of an array, which means all
|
||||
* arcs for the node are encoded with a fixed number of bytes so
|
||||
* that we can random access by index. We do when there are enough
|
||||
* arcs leaving one node. It wastes some bytes but gives faster
|
||||
* lookups. */
|
||||
public int bytesPerArc;
|
||||
private long posArcsStart;
|
||||
|
||||
/** Where we are in the array; only valid if bytesPerArc != 0, and the array has no holes.
|
||||
* arcIdx = Integer.MIN_VALUE indicates that the arc is part of a direct array, addressed by
|
||||
* label.
|
||||
*/
|
||||
public int arcIdx;
|
||||
private int bytesPerArc;
|
||||
|
||||
/** How many arc, if bytesPerArc == 0. Otherwise, the size of the arc array. If the array is
|
||||
* direct, this may include holes. Otherwise it is also how many arcs are in the array */
|
||||
public int numArcs;
|
||||
private int arcIdx;
|
||||
|
||||
private int numArcs;
|
||||
|
||||
/** Returns this */
|
||||
public Arc<T> copyFrom(Arc<T> other) {
|
||||
label = other.label;
|
||||
target = other.target;
|
||||
flags = other.flags;
|
||||
output = other.output;
|
||||
nextFinalOutput = other.nextFinalOutput;
|
||||
nextArc = other.nextArc;
|
||||
bytesPerArc = other.bytesPerArc;
|
||||
if (bytesPerArc != 0) {
|
||||
posArcsStart = other.posArcsStart;
|
||||
arcIdx = other.arcIdx;
|
||||
numArcs = other.numArcs;
|
||||
label = other.label();
|
||||
target = other.target();
|
||||
flags = other.flags();
|
||||
output = other.output();
|
||||
nextFinalOutput = other.nextFinalOutput();
|
||||
nextArc = other.nextArc();
|
||||
bytesPerArc = other.bytesPerArc();
|
||||
if (bytesPerArc() != 0) {
|
||||
posArcsStart = other.posArcsStart();
|
||||
arcIdx = other.arcIdx();
|
||||
numArcs = other.numArcs();
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
@ -205,11 +197,15 @@ public final class FST<T> implements Accountable {
|
|||
return flag(BIT_FINAL_ARC);
|
||||
}
|
||||
|
||||
public boolean isPackedArray() {
|
||||
return bytesPerArc != 0 && arcIdx > Integer.MIN_VALUE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder b = new StringBuilder();
|
||||
b.append(" target=").append(target);
|
||||
b.append(" label=0x").append(Integer.toHexString(label));
|
||||
b.append(" target=").append(target());
|
||||
b.append(" label=0x").append(Integer.toHexString(label()));
|
||||
if (flag(BIT_FINAL_ARC)) {
|
||||
b.append(" final");
|
||||
}
|
||||
|
@ -223,40 +219,89 @@ public final class FST<T> implements Accountable {
|
|||
b.append(" stop");
|
||||
}
|
||||
if (flag(BIT_ARC_HAS_OUTPUT)) {
|
||||
b.append(" output=").append(output);
|
||||
b.append(" output=").append(output());
|
||||
}
|
||||
if (flag(BIT_ARC_HAS_FINAL_OUTPUT)) {
|
||||
b.append(" nextFinalOutput=").append(nextFinalOutput);
|
||||
b.append(" nextFinalOutput=").append(nextFinalOutput());
|
||||
}
|
||||
if (bytesPerArc != 0) {
|
||||
b.append(" arcArray(idx=").append(arcIdx).append(" of ").append(numArcs).append(")");
|
||||
if (bytesPerArc() != 0) {
|
||||
b.append(" arcArray(idx=").append(arcIdx()).append(" of ").append(numArcs()).append(")");
|
||||
}
|
||||
return b.toString();
|
||||
}
|
||||
};
|
||||
|
||||
public int label() {
|
||||
return label;
|
||||
}
|
||||
|
||||
public T output() {
|
||||
return output;
|
||||
}
|
||||
|
||||
/** To node (ord or address) */
|
||||
public long target() {
|
||||
return target;
|
||||
}
|
||||
|
||||
public byte flags() {
|
||||
return flags;
|
||||
}
|
||||
|
||||
public T nextFinalOutput() {
|
||||
return nextFinalOutput;
|
||||
}
|
||||
|
||||
long nextArc() {
|
||||
return nextArc;
|
||||
}
|
||||
|
||||
/** Where the first arc in the array starts; only valid if
|
||||
* bytesPerArc != 0 */
|
||||
public long posArcsStart() {
|
||||
return posArcsStart;
|
||||
}
|
||||
|
||||
/** Non-zero if this arc is part of an array, which means all
|
||||
* arcs for the node are encoded with a fixed number of bytes so
|
||||
* that we can random access by index. We do when there are enough
|
||||
* arcs leaving one node. It wastes some bytes but gives faster
|
||||
* lookups. */
|
||||
public int bytesPerArc() {
|
||||
return bytesPerArc;
|
||||
}
|
||||
|
||||
/** Where we are in the array; only valid if bytesPerArc != 0, and the array has no holes.
|
||||
* arcIdx = Integer.MIN_VALUE indicates that the arc is part of a direct array, addressed by
|
||||
* label.
|
||||
*/
|
||||
public int arcIdx() {
|
||||
return arcIdx;
|
||||
}
|
||||
|
||||
/** How many arc, if bytesPerArc == 0. Otherwise, the size of the arc array. If the array is
|
||||
* direct, this may include holes. Otherwise it is also how many arcs are in the array */
|
||||
public int numArcs() {
|
||||
return numArcs;
|
||||
}
|
||||
}
|
||||
|
||||
private static boolean flag(int flags, int bit) {
|
||||
return (flags & bit) != 0;
|
||||
}
|
||||
|
||||
private final int version;
|
||||
|
||||
// make a new empty FST, for building; Builder invokes
|
||||
// this ctor
|
||||
// make a new empty FST, for building; Builder invokes this
|
||||
FST(INPUT_TYPE inputType, Outputs<T> outputs, int bytesPageBits) {
|
||||
this.inputType = inputType;
|
||||
this.outputs = outputs;
|
||||
version = VERSION_CURRENT;
|
||||
fstStore = null;
|
||||
bytes = new BytesStore(bytesPageBits);
|
||||
// pad: ensure no node gets address 0 which is reserved to mean
|
||||
// the stop state w/ no arcs
|
||||
bytes.writeByte((byte) 0);
|
||||
|
||||
emptyOutput = null;
|
||||
}
|
||||
|
||||
public static final int DEFAULT_MAX_BLOCK_BITS = Constants.JRE_IS_64BIT ? 30 : 28;
|
||||
private static final int DEFAULT_MAX_BLOCK_BITS = Constants.JRE_IS_64BIT ? 30 : 28;
|
||||
|
||||
/** Load a previously saved FST. */
|
||||
public FST(DataInput in, Outputs<T> outputs) throws IOException {
|
||||
|
@ -270,9 +315,9 @@ public final class FST<T> implements Accountable {
|
|||
this.fstStore = fstStore;
|
||||
this.outputs = outputs;
|
||||
|
||||
// NOTE: only reads most recent format; we don't have
|
||||
// back-compat promise for FSTs (they are experimental):
|
||||
version = CodecUtil.checkHeader(in, FILE_FORMAT_NAME, VERSION_START, VERSION_CURRENT);
|
||||
// NOTE: only reads formats VERSION_START up to VERSION_CURRENT; we don't have
|
||||
// back-compat promise for FSTs (they are experimental), but we are sometimes able to offer it
|
||||
CodecUtil.checkHeader(in, FILE_FORMAT_NAME, VERSION_START, VERSION_CURRENT);
|
||||
if (in.readByte() == 1) {
|
||||
// accepts empty string
|
||||
// 1 KB blocks:
|
||||
|
@ -313,10 +358,6 @@ public final class FST<T> implements Accountable {
|
|||
cacheRootArcs();
|
||||
}
|
||||
|
||||
public INPUT_TYPE getInputType() {
|
||||
return inputType;
|
||||
}
|
||||
|
||||
private long ramBytesUsed(Arc<T>[] arcs) {
|
||||
long size = 0;
|
||||
if (arcs != null) {
|
||||
|
@ -324,11 +365,11 @@ public final class FST<T> implements Accountable {
|
|||
for (Arc<T> arc : arcs) {
|
||||
if (arc != null) {
|
||||
size += ARC_SHALLOW_RAM_BYTES_USED;
|
||||
if (arc.output != null && arc.output != outputs.getNoOutput()) {
|
||||
size += outputs.ramBytesUsed(arc.output);
|
||||
if (arc.output() != null && arc.output() != outputs.getNoOutput()) {
|
||||
size += outputs.ramBytesUsed(arc.output());
|
||||
}
|
||||
if (arc.nextFinalOutput != null && arc.nextFinalOutput != outputs.getNoOutput()) {
|
||||
size += outputs.ramBytesUsed(arc.nextFinalOutput);
|
||||
if (arc.nextFinalOutput() != null && arc.nextFinalOutput() != outputs.getNoOutput()) {
|
||||
size += outputs.ramBytesUsed(arc.nextFinalOutput());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -336,8 +377,6 @@ public final class FST<T> implements Accountable {
|
|||
return size;
|
||||
}
|
||||
|
||||
private int cachedArcsBytesUsed;
|
||||
|
||||
@Override
|
||||
public long ramBytesUsed() {
|
||||
long size = BASE_RAM_BYTES_USED;
|
||||
|
@ -380,12 +419,12 @@ public final class FST<T> implements Accountable {
|
|||
if (targetHasArcs(arc)) {
|
||||
final BytesReader in = getBytesReader();
|
||||
Arc<T>[] arcs = (Arc<T>[]) new Arc[0x80];
|
||||
readFirstRealTargetArc(arc.target, arc, in);
|
||||
readFirstRealTargetArc(arc.target(), arc, in);
|
||||
int count = 0;
|
||||
while(true) {
|
||||
assert arc.label != END_LABEL;
|
||||
if (arc.label < arcs.length) {
|
||||
arcs[arc.label] = new Arc<T>().copyFrom(arc);
|
||||
assert arc.label() != END_LABEL;
|
||||
if (arc.label() < arcs.length) {
|
||||
arcs[arc.label()] = new Arc<T>().copyFrom(arc);
|
||||
} else {
|
||||
break;
|
||||
}
|
||||
|
@ -410,7 +449,7 @@ public final class FST<T> implements Accountable {
|
|||
return emptyOutput;
|
||||
}
|
||||
|
||||
void setEmptyOutput(T v) throws IOException {
|
||||
void setEmptyOutput(T v) {
|
||||
if (emptyOutput != null) {
|
||||
emptyOutput = outputs.merge(emptyOutput, v);
|
||||
} else {
|
||||
|
@ -433,18 +472,19 @@ public final class FST<T> implements Accountable {
|
|||
ByteBuffersDataOutput ros = new ByteBuffersDataOutput();
|
||||
outputs.writeFinalOutput(emptyOutput, ros);
|
||||
byte[] emptyOutputBytes = ros.toArrayCopy();
|
||||
int emptyLen = emptyOutputBytes.length;
|
||||
|
||||
// reverse
|
||||
final int stopAt = emptyOutputBytes.length/2;
|
||||
final int stopAt = emptyLen / 2;
|
||||
int upto = 0;
|
||||
while (upto < stopAt) {
|
||||
final byte b = emptyOutputBytes[upto];
|
||||
emptyOutputBytes[upto] = emptyOutputBytes[emptyOutputBytes.length-upto-1];
|
||||
emptyOutputBytes[emptyOutputBytes.length-upto-1] = b;
|
||||
emptyOutputBytes[upto] = emptyOutputBytes[emptyLen - upto - 1];
|
||||
emptyOutputBytes[emptyLen - upto - 1] = b;
|
||||
upto++;
|
||||
}
|
||||
out.writeVInt(emptyOutputBytes.length);
|
||||
out.writeBytes(emptyOutputBytes, 0, emptyOutputBytes.length);
|
||||
out.writeVInt(emptyLen);
|
||||
out.writeBytes(emptyOutputBytes, 0, emptyLen);
|
||||
} else {
|
||||
out.writeByte((byte) 0);
|
||||
}
|
||||
|
@ -517,7 +557,7 @@ public final class FST<T> implements Accountable {
|
|||
/** returns true if the node at this address has any
|
||||
* outgoing arcs */
|
||||
public static<T> boolean targetHasArcs(Arc<T> arc) {
|
||||
return arc.target > 0;
|
||||
return arc.target() > 0;
|
||||
}
|
||||
|
||||
// serializes new node by appending its bytes to the end
|
||||
|
@ -533,7 +573,6 @@ public final class FST<T> implements Accountable {
|
|||
return NON_FINAL_END_NODE;
|
||||
}
|
||||
}
|
||||
|
||||
final long startAddress = builder.bytes.getPosition();
|
||||
//System.out.println(" startAddr=" + startAddress);
|
||||
|
||||
|
@ -649,10 +688,9 @@ public final class FST<T> implements Accountable {
|
|||
int labelRange = nodeIn.arcs[nodeIn.numArcs - 1].label - nodeIn.arcs[0].label + 1;
|
||||
boolean writeDirectly = labelRange > 0 && labelRange < Builder.DIRECT_ARC_LOAD_FACTOR * nodeIn.numArcs;
|
||||
|
||||
//System.out.println("write int @pos=" + (fixedArrayStart-4) + " numArcs=" + nodeIn.numArcs);
|
||||
// create the header
|
||||
// TODO: clean this up: or just rewind+reuse and deal with it
|
||||
byte header[] = new byte[MAX_HEADER_SIZE];
|
||||
byte[] header = new byte[MAX_HEADER_SIZE];
|
||||
ByteArrayDataOutput bad = new ByteArrayDataOutput(header);
|
||||
// write a "false" first arc:
|
||||
if (writeDirectly) {
|
||||
|
@ -742,8 +780,7 @@ public final class FST<T> implements Accountable {
|
|||
}
|
||||
}
|
||||
|
||||
/** Fills virtual 'start' arc, ie, an empty incoming arc to
|
||||
* the FST's start node */
|
||||
/** Fills virtual 'start' arc, ie, an empty incoming arc to the FST's start node */
|
||||
public Arc<T> getFirstArc(Arc<T> arc) {
|
||||
T NO_OUTPUT = outputs.getNoOutput();
|
||||
|
||||
|
@ -751,7 +788,7 @@ public final class FST<T> implements Accountable {
|
|||
arc.flags = BIT_FINAL_ARC | BIT_LAST_ARC;
|
||||
arc.nextFinalOutput = emptyOutput;
|
||||
if (emptyOutput != NO_OUTPUT) {
|
||||
arc.flags |= BIT_ARC_HAS_FINAL_OUTPUT;
|
||||
arc.flags = (byte) (arc.flags() | BIT_ARC_HAS_FINAL_OUTPUT);
|
||||
}
|
||||
} else {
|
||||
arc.flags = BIT_LAST_ARC;
|
||||
|
@ -771,18 +808,18 @@ public final class FST<T> implements Accountable {
|
|||
*
|
||||
* @return Returns the second argument
|
||||
* (<code>arc</code>). */
|
||||
public Arc<T> readLastTargetArc(Arc<T> follow, Arc<T> arc, BytesReader in) throws IOException {
|
||||
Arc<T> readLastTargetArc(Arc<T> follow, Arc<T> arc, BytesReader in) throws IOException {
|
||||
//System.out.println("readLast");
|
||||
if (!targetHasArcs(follow)) {
|
||||
//System.out.println(" end node");
|
||||
assert follow.isFinal();
|
||||
arc.label = END_LABEL;
|
||||
arc.target = FINAL_END_NODE;
|
||||
arc.output = follow.nextFinalOutput;
|
||||
arc.output = follow.nextFinalOutput();
|
||||
arc.flags = BIT_LAST_ARC;
|
||||
return arc;
|
||||
} else {
|
||||
in.setPosition(follow.target);
|
||||
in.setPosition(follow.target());
|
||||
final byte b = in.readByte();
|
||||
if (b == ARCS_AS_ARRAY_PACKED || b == ARCS_AS_ARRAY_WITH_GAPS) {
|
||||
// array: jump straight to end
|
||||
|
@ -792,9 +829,9 @@ public final class FST<T> implements Accountable {
|
|||
arc.posArcsStart = in.getPosition();
|
||||
if (b == ARCS_AS_ARRAY_WITH_GAPS) {
|
||||
arc.arcIdx = Integer.MIN_VALUE;
|
||||
arc.nextArc = arc.posArcsStart - (arc.numArcs - 1) * arc.bytesPerArc;
|
||||
arc.nextArc = arc.posArcsStart() - (arc.numArcs() - 1) * arc.bytesPerArc();
|
||||
} else {
|
||||
arc.arcIdx = arc.numArcs - 2;
|
||||
arc.arcIdx = arc.numArcs() - 2;
|
||||
}
|
||||
} else {
|
||||
arc.flags = b;
|
||||
|
@ -844,25 +881,24 @@ public final class FST<T> implements Accountable {
|
|||
if (follow.isFinal()) {
|
||||
// Insert "fake" final first arc:
|
||||
arc.label = END_LABEL;
|
||||
arc.output = follow.nextFinalOutput;
|
||||
arc.output = follow.nextFinalOutput();
|
||||
arc.flags = BIT_FINAL_ARC;
|
||||
if (follow.target <= 0) {
|
||||
if (follow.target() <= 0) {
|
||||
arc.flags |= BIT_LAST_ARC;
|
||||
} else {
|
||||
// NOTE: nextArc is a node (not an address!) in this case:
|
||||
arc.nextArc = follow.target;
|
||||
arc.nextArc = follow.target();
|
||||
}
|
||||
arc.target = FINAL_END_NODE;
|
||||
//System.out.println(" insert isFinal; nextArc=" + follow.target + " isLast=" + arc.isLast() + " output=" + outputs.outputToString(arc.output));
|
||||
return arc;
|
||||
} else {
|
||||
return readFirstRealTargetArc(follow.target, arc, in);
|
||||
return readFirstRealTargetArc(follow.target(), arc, in);
|
||||
}
|
||||
}
|
||||
|
||||
public Arc<T> readFirstRealTargetArc(long node, Arc<T> arc, final BytesReader in) throws IOException {
|
||||
final long address = node;
|
||||
in.setPosition(address);
|
||||
public Arc<T> readFirstRealTargetArc(long nodeAddress, Arc<T> arc, final BytesReader in) throws IOException {
|
||||
in.setPosition(nodeAddress);
|
||||
//System.out.println(" flags=" + arc.flags);
|
||||
|
||||
byte flags = in.readByte();
|
||||
|
@ -880,7 +916,7 @@ public final class FST<T> implements Accountable {
|
|||
//System.out.println(" bytesPer=" + arc.bytesPerArc + " numArcs=" + arc.numArcs + " arcsStart=" + pos);
|
||||
} else {
|
||||
//arc.flags = b;
|
||||
arc.nextArc = address;
|
||||
arc.nextArc = nodeAddress;
|
||||
arc.bytesPerArc = 0;
|
||||
}
|
||||
|
||||
|
@ -897,7 +933,7 @@ public final class FST<T> implements Accountable {
|
|||
if (!targetHasArcs(follow)) {
|
||||
return false;
|
||||
} else {
|
||||
in.setPosition(follow.target);
|
||||
in.setPosition(follow.target());
|
||||
byte flags = in.readByte();
|
||||
return flags == ARCS_AS_ARRAY_PACKED || flags == ARCS_AS_ARRAY_WITH_GAPS;
|
||||
}
|
||||
|
@ -905,12 +941,12 @@ public final class FST<T> implements Accountable {
|
|||
|
||||
/** In-place read; returns the arc. */
|
||||
public Arc<T> readNextArc(Arc<T> arc, BytesReader in) throws IOException {
|
||||
if (arc.label == END_LABEL) {
|
||||
if (arc.label() == END_LABEL) {
|
||||
// This was a fake inserted "final" arc
|
||||
if (arc.nextArc <= 0) {
|
||||
if (arc.nextArc() <= 0) {
|
||||
throw new IllegalArgumentException("cannot readNextArc when arc.isLast()=true");
|
||||
}
|
||||
return readFirstRealTargetArc(arc.nextArc, arc, in);
|
||||
return readFirstRealTargetArc(arc.nextArc(), arc, in);
|
||||
} else {
|
||||
return readNextRealArc(arc, in);
|
||||
}
|
||||
|
@ -918,14 +954,14 @@ public final class FST<T> implements Accountable {
|
|||
|
||||
/** Peeks at next arc's label; does not alter arc. Do
|
||||
* not call this if arc.isLast()! */
|
||||
public int readNextArcLabel(Arc<T> arc, BytesReader in) throws IOException {
|
||||
int readNextArcLabel(Arc<T> arc, BytesReader in) throws IOException {
|
||||
assert !arc.isLast();
|
||||
|
||||
if (arc.label == END_LABEL) {
|
||||
if (arc.label() == END_LABEL) {
|
||||
//System.out.println(" nextArc fake " +
|
||||
//arc.nextArc);
|
||||
|
||||
long pos = arc.nextArc;
|
||||
long pos = arc.nextArc();
|
||||
in.setPosition(pos);
|
||||
|
||||
final byte flags = in.readByte();
|
||||
|
@ -941,19 +977,19 @@ public final class FST<T> implements Accountable {
|
|||
// skip flags
|
||||
in.readByte();
|
||||
} else {
|
||||
if (arc.bytesPerArc != 0) {
|
||||
if (arc.bytesPerArc() != 0) {
|
||||
//System.out.println(" nextArc real array");
|
||||
// arcs are in an array
|
||||
if (arc.arcIdx >= 0) {
|
||||
in.setPosition(arc.posArcsStart);
|
||||
if (arc.arcIdx() >= 0) {
|
||||
in.setPosition(arc.posArcsStart());
|
||||
// point at next arc, -1 to skip flags
|
||||
in.skipBytes((1 + arc.arcIdx) * arc.bytesPerArc + 1);
|
||||
in.skipBytes((1 + arc.arcIdx()) * arc.bytesPerArc() + 1);
|
||||
} else {
|
||||
in.setPosition(arc.nextArc);
|
||||
in.setPosition(arc.nextArc());
|
||||
byte flags = in.readByte();
|
||||
// skip missing arcs
|
||||
while (flag(flags, BIT_MISSING_ARC)) {
|
||||
in.skipBytes(arc.bytesPerArc - 1);
|
||||
in.skipBytes(arc.bytesPerArc() - 1);
|
||||
flags = in.readByte();
|
||||
}
|
||||
}
|
||||
|
@ -961,12 +997,33 @@ public final class FST<T> implements Accountable {
|
|||
// arcs are packed
|
||||
//System.out.println(" nextArc real packed");
|
||||
// -1 to skip flags
|
||||
in.setPosition(arc.nextArc - 1);
|
||||
in.setPosition(arc.nextArc() - 1);
|
||||
}
|
||||
}
|
||||
return readLabel(in);
|
||||
}
|
||||
|
||||
public Arc<T> readArcAtPosition(Arc<T> arc, final BytesReader in, long pos) throws IOException {
|
||||
in.setPosition(pos);
|
||||
arc.flags = in.readByte();
|
||||
arc.nextArc = pos;
|
||||
while (flag(arc.flags(), BIT_MISSING_ARC)) {
|
||||
// skip empty arcs
|
||||
arc.nextArc -= arc.bytesPerArc();
|
||||
in.skipBytes(arc.bytesPerArc() - 1);
|
||||
arc.flags = in.readByte();
|
||||
}
|
||||
return readArc(arc, in);
|
||||
}
|
||||
|
||||
public Arc<T> readArcByIndex(Arc<T> arc, final BytesReader in, int idx) throws IOException {
|
||||
arc.arcIdx = idx;
|
||||
assert arc.arcIdx() < arc.numArcs();
|
||||
in.setPosition(arc.posArcsStart() - arc.arcIdx() * arc.bytesPerArc());
|
||||
arc.flags = in.readByte();
|
||||
return readArc(arc, in);
|
||||
}
|
||||
|
||||
/** Never returns null, but you should never call this if
|
||||
* arc.isLast() is true. */
|
||||
public Arc<T> readNextRealArc(Arc<T> arc, final BytesReader in) throws IOException {
|
||||
|
@ -975,29 +1032,33 @@ public final class FST<T> implements Accountable {
|
|||
// assert !flag(arc.flags, BIT_LAST_ARC);
|
||||
|
||||
// this is a continuing arc in a fixed array
|
||||
if (arc.bytesPerArc != 0) {
|
||||
if (arc.bytesPerArc() != 0) {
|
||||
// arcs are in an array
|
||||
if (arc.arcIdx > Integer.MIN_VALUE) {
|
||||
if (arc.arcIdx() > Integer.MIN_VALUE) {
|
||||
arc.arcIdx++;
|
||||
assert arc.arcIdx < arc.numArcs;
|
||||
in.setPosition(arc.posArcsStart - arc.arcIdx * arc.bytesPerArc);
|
||||
in.setPosition(arc.posArcsStart() - arc.arcIdx() * arc.bytesPerArc());
|
||||
arc.flags = in.readByte();
|
||||
} else {
|
||||
assert arc.nextArc <= arc.posArcsStart && arc.nextArc > arc.posArcsStart - arc.numArcs * arc.bytesPerArc;
|
||||
in.setPosition(arc.nextArc);
|
||||
assert arc.nextArc() <= arc.posArcsStart() && arc.nextArc() > arc.posArcsStart() - arc.numArcs() * arc.bytesPerArc();
|
||||
in.setPosition(arc.nextArc());
|
||||
arc.flags = in.readByte();
|
||||
while (flag(arc.flags, BIT_MISSING_ARC)) {
|
||||
while (flag(arc.flags(), BIT_MISSING_ARC)) {
|
||||
// skip empty arcs
|
||||
arc.nextArc -= arc.bytesPerArc;
|
||||
in.skipBytes(arc.bytesPerArc - 1);
|
||||
arc.nextArc = arc.nextArc() - arc.bytesPerArc();
|
||||
in.skipBytes(arc.bytesPerArc() - 1);
|
||||
arc.flags = in.readByte();
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// arcs are packed
|
||||
in.setPosition(arc.nextArc);
|
||||
in.setPosition(arc.nextArc());
|
||||
arc.flags = in.readByte();
|
||||
}
|
||||
return readArc(arc, in);
|
||||
}
|
||||
|
||||
private Arc<T> readArc(Arc<T> arc, BytesReader in) throws IOException {
|
||||
arc.label = readLabel(in);
|
||||
|
||||
if (arc.flag(BIT_ARC_HAS_OUTPUT)) {
|
||||
|
@ -1018,31 +1079,31 @@ public final class FST<T> implements Accountable {
|
|||
} else {
|
||||
arc.target = NON_FINAL_END_NODE;
|
||||
}
|
||||
if (arc.bytesPerArc == 0) {
|
||||
if (arc.bytesPerArc() == 0) {
|
||||
arc.nextArc = in.getPosition();
|
||||
} else {
|
||||
arc.nextArc -= arc.bytesPerArc;
|
||||
arc.nextArc -= arc.bytesPerArc();
|
||||
}
|
||||
} else if (arc.flag(BIT_TARGET_NEXT)) {
|
||||
arc.nextArc = in.getPosition();
|
||||
// TODO: would be nice to make this lazy -- maybe
|
||||
// caller doesn't need the target and is scanning arcs...
|
||||
if (!arc.flag(BIT_LAST_ARC)) {
|
||||
if (arc.bytesPerArc == 0) {
|
||||
if (arc.bytesPerArc() == 0) {
|
||||
// must scan
|
||||
seekToNextNode(in);
|
||||
} else {
|
||||
in.setPosition(arc.posArcsStart);
|
||||
in.skipBytes(arc.bytesPerArc * arc.numArcs);
|
||||
in.setPosition(arc.posArcsStart());
|
||||
in.skipBytes(arc.bytesPerArc() * arc.numArcs());
|
||||
}
|
||||
}
|
||||
arc.target = in.getPosition();
|
||||
} else {
|
||||
arc.target = readUnpackedNodeTarget(in);
|
||||
if (arc.bytesPerArc > 0 && arc.arcIdx == Integer.MIN_VALUE) {
|
||||
if (arc.bytesPerArc() > 0 && arc.arcIdx() == Integer.MIN_VALUE) {
|
||||
// nextArc was pointing to *this* arc when we entered; advance to the next
|
||||
// if it is a missing arc, we will skip it later
|
||||
arc.nextArc -= arc.bytesPerArc;
|
||||
arc.nextArc = arc.nextArc() - arc.bytesPerArc();
|
||||
} else {
|
||||
// in list and fixed table encodings, the next arc always follows this one
|
||||
arc.nextArc = in.getPosition();
|
||||
|
@ -1051,6 +1112,23 @@ public final class FST<T> implements Accountable {
|
|||
return arc;
|
||||
}
|
||||
|
||||
static <T> Arc<T> readEndArc(Arc<T> follow, Arc<T> arc) {
|
||||
if (follow.isFinal()) {
|
||||
if (follow.target() <= 0) {
|
||||
arc.flags = FST.BIT_LAST_ARC;
|
||||
} else {
|
||||
arc.flags = 0;
|
||||
// NOTE: nextArc is a node (not an address!) in this case:
|
||||
arc.nextArc = follow.target();
|
||||
}
|
||||
arc.output = follow.nextFinalOutput();
|
||||
arc.label = FST.END_LABEL;
|
||||
return arc;
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
// LUCENE-5152: called only from asserts, to validate that the
|
||||
// non-cached arc lookup would produce the same result, to
|
||||
// catch callers that illegally modify shared structures with
|
||||
|
@ -1065,19 +1143,16 @@ public final class FST<T> implements Accountable {
|
|||
assert cachedArc == null;
|
||||
} else {
|
||||
assert cachedArc != null;
|
||||
assert cachedArc.arcIdx == result.arcIdx;
|
||||
assert cachedArc.bytesPerArc == result.bytesPerArc;
|
||||
assert cachedArc.flags == result.flags;
|
||||
assert cachedArc.label == result.label;
|
||||
if (cachedArc.bytesPerArc == 0 || cachedArc.arcIdx == Integer.MIN_VALUE) {
|
||||
// in the sparse array case, this value is not valid, so don't assert it
|
||||
assert cachedArc.nextArc == result.nextArc;
|
||||
}
|
||||
assert cachedArc.nextFinalOutput.equals(result.nextFinalOutput);
|
||||
assert cachedArc.numArcs == result.numArcs;
|
||||
assert cachedArc.output.equals(result.output);
|
||||
assert cachedArc.posArcsStart == result.posArcsStart;
|
||||
assert cachedArc.target == result.target;
|
||||
assert cachedArc.arcIdx() == result.arcIdx();
|
||||
assert cachedArc.bytesPerArc() == result.bytesPerArc();
|
||||
assert cachedArc.flags() == result.flags();
|
||||
assert cachedArc.label() == result.label();
|
||||
assert (cachedArc.bytesPerArc() != 0 && cachedArc.arcIdx() != Integer.MIN_VALUE) || cachedArc.nextArc() == result.nextArc();
|
||||
assert cachedArc.nextFinalOutput().equals(result.nextFinalOutput());
|
||||
assert cachedArc.numArcs() == result.numArcs();
|
||||
assert cachedArc.output().equals(result.output());
|
||||
assert cachedArc.posArcsStart() == result.posArcsStart();
|
||||
assert cachedArc.target() == result.target();
|
||||
}
|
||||
|
||||
return true;
|
||||
|
@ -1098,14 +1173,14 @@ public final class FST<T> implements Accountable {
|
|||
|
||||
if (labelToMatch == END_LABEL) {
|
||||
if (follow.isFinal()) {
|
||||
if (follow.target <= 0) {
|
||||
if (follow.target() <= 0) {
|
||||
arc.flags = BIT_LAST_ARC;
|
||||
} else {
|
||||
arc.flags = 0;
|
||||
// NOTE: nextArc is a node (not an address!) in this case:
|
||||
arc.nextArc = follow.target;
|
||||
arc.nextArc = follow.target();
|
||||
}
|
||||
arc.output = follow.nextFinalOutput;
|
||||
arc.output = follow.nextFinalOutput();
|
||||
arc.label = END_LABEL;
|
||||
return arc;
|
||||
} else {
|
||||
|
@ -1114,7 +1189,7 @@ public final class FST<T> implements Accountable {
|
|||
}
|
||||
|
||||
// Short-circuit if this arc is in the root arc cache:
|
||||
if (useRootArcCache && cachedRootArcs != null && follow.target == startNode && labelToMatch < cachedRootArcs.length) {
|
||||
if (useRootArcCache && cachedRootArcs != null && follow.target() == startNode && labelToMatch < cachedRootArcs.length) {
|
||||
final Arc<T> result = cachedRootArcs[labelToMatch];
|
||||
|
||||
// LUCENE-5152: detect tricky cases where caller
|
||||
|
@ -1133,7 +1208,7 @@ public final class FST<T> implements Accountable {
|
|||
return null;
|
||||
}
|
||||
|
||||
in.setPosition(follow.target);
|
||||
in.setPosition(follow.target());
|
||||
|
||||
// System.out.println("fta label=" + (char) labelToMatch);
|
||||
|
||||
|
@ -1148,12 +1223,12 @@ public final class FST<T> implements Accountable {
|
|||
int firstLabel = readLabel(in);
|
||||
int arcPos = labelToMatch - firstLabel;
|
||||
if (arcPos == 0) {
|
||||
arc.nextArc = arc.posArcsStart;
|
||||
arc.nextArc = arc.posArcsStart();
|
||||
} else if (arcPos > 0) {
|
||||
if (arcPos >= arc.numArcs) {
|
||||
if (arcPos >= arc.numArcs()) {
|
||||
return null;
|
||||
}
|
||||
in.setPosition(arc.posArcsStart - arc.bytesPerArc * arcPos);
|
||||
in.setPosition(arc.posArcsStart() - arc.bytesPerArc() * arcPos);
|
||||
flags = in.readByte();
|
||||
if (flag(flags, BIT_MISSING_ARC)) {
|
||||
return null;
|
||||
|
@ -1172,12 +1247,12 @@ public final class FST<T> implements Accountable {
|
|||
|
||||
// Array is sparse; do binary search:
|
||||
int low = 0;
|
||||
int high = arc.numArcs - 1;
|
||||
int high = arc.numArcs() - 1;
|
||||
while (low <= high) {
|
||||
//System.out.println(" cycle");
|
||||
int mid = (low + high) >>> 1;
|
||||
// +1 to skip over flags
|
||||
in.setPosition(arc.posArcsStart - (arc.bytesPerArc * mid + 1));
|
||||
in.setPosition(arc.posArcsStart() - (arc.bytesPerArc() * mid + 1));
|
||||
int midLabel = readLabel(in);
|
||||
final int cmp = midLabel - labelToMatch;
|
||||
if (cmp < 0) {
|
||||
|
@ -1194,17 +1269,17 @@ public final class FST<T> implements Accountable {
|
|||
}
|
||||
|
||||
// Linear scan
|
||||
readFirstRealTargetArc(follow.target, arc, in);
|
||||
readFirstRealTargetArc(follow.target(), arc, in);
|
||||
|
||||
while(true) {
|
||||
//System.out.println(" non-bs cycle");
|
||||
// TODO: we should fix this code to not have to create
|
||||
// object for the output of every arc we scan... only
|
||||
// for the matching arc, if found
|
||||
if (arc.label == labelToMatch) {
|
||||
if (arc.label() == labelToMatch) {
|
||||
//System.out.println(" found!");
|
||||
return arc;
|
||||
} else if (arc.label > labelToMatch) {
|
||||
} else if (arc.label() > labelToMatch) {
|
||||
return null;
|
||||
} else if (arc.isLast()) {
|
||||
return null;
|
||||
|
|
|
@ -24,7 +24,7 @@ import org.apache.lucene.util.RamUsageEstimator;
|
|||
|
||||
/** Can next() and advance() through the terms in an FST
|
||||
*
|
||||
* @lucene.experimental
|
||||
* @lucene.experimental
|
||||
*/
|
||||
|
||||
abstract class FSTEnum<T> {
|
||||
|
@ -39,12 +39,12 @@ abstract class FSTEnum<T> {
|
|||
protected final FST.Arc<T> scratchArc = new FST.Arc<>();
|
||||
|
||||
protected int upto;
|
||||
protected int targetLength;
|
||||
int targetLength;
|
||||
|
||||
/** doFloor controls the behavior of advance: if it's true
|
||||
* doFloor is true, advance positions to the biggest
|
||||
* term before target. */
|
||||
protected FSTEnum(FST<T> fst) {
|
||||
FSTEnum(FST<T> fst) {
|
||||
this.fst = fst;
|
||||
fstReader = fst.getBytesReader();
|
||||
NO_OUTPUT = fst.outputs.getNoOutput();
|
||||
|
@ -60,7 +60,7 @@ abstract class FSTEnum<T> {
|
|||
|
||||
/** Rewinds enum state to match the shared prefix between
|
||||
* current term and target term */
|
||||
protected final void rewindPrefix() throws IOException {
|
||||
private void rewindPrefix() throws IOException {
|
||||
if (upto == 0) {
|
||||
//System.out.println(" init");
|
||||
upto = 1;
|
||||
|
@ -138,10 +138,10 @@ abstract class FSTEnum<T> {
|
|||
while(arc != null) {
|
||||
int targetLabel = getTargetLabel();
|
||||
//System.out.println(" cycle upto=" + upto + " arc.label=" + arc.label + " (" + (char) arc.label + ") vs targetLabel=" + targetLabel);
|
||||
if (arc.bytesPerArc != 0 && arc.label != -1) {
|
||||
if (arc.bytesPerArc() != 0 && arc.label() != -1) {
|
||||
// Arcs are in an array
|
||||
final FST.BytesReader in = fst.getBytesReader();
|
||||
if (arc.arcIdx == Integer.MIN_VALUE) {
|
||||
if (arc.arcIdx() == Integer.MIN_VALUE) {
|
||||
arc = doSeekCeilArrayWithGaps(arc, targetLabel, in);
|
||||
} else {
|
||||
arc = doSeekCeilArrayPacked(arc, targetLabel, in);
|
||||
|
@ -155,14 +155,13 @@ abstract class FSTEnum<T> {
|
|||
private FST.Arc<T> doSeekCeilArrayWithGaps(final FST.Arc<T> arc, final int targetLabel, final FST.BytesReader in) throws IOException {
|
||||
// The array is addressed directly by label and may contain holes.
|
||||
|
||||
in.setPosition(arc.posArcsStart);
|
||||
in.setPosition(arc.posArcsStart());
|
||||
in.skipBytes(1);
|
||||
int firstLabel = fst.readLabel(in);
|
||||
int arcOffset = targetLabel - firstLabel;
|
||||
if (arcOffset >= arc.numArcs) {
|
||||
if (arcOffset >= arc.numArcs()) {
|
||||
// target is beyond the last arc
|
||||
arc.nextArc = arc.posArcsStart - (arc.numArcs - 1) * arc.bytesPerArc;
|
||||
fst.readNextRealArc(arc, in);
|
||||
fst.readArcAtPosition(arc, in, arc.posArcsStart() - (arc.numArcs() - 1) * arc.bytesPerArc());
|
||||
assert arc.isLast();
|
||||
// Dead end (target is after the last arc);
|
||||
// rollback to last fork then push
|
||||
|
@ -182,24 +181,25 @@ abstract class FSTEnum<T> {
|
|||
}
|
||||
} else {
|
||||
// TODO: if firstLabel == targetLabel
|
||||
long pos;
|
||||
if (arcOffset >= 0) {
|
||||
arc.nextArc = arc.posArcsStart - (arc.bytesPerArc * arcOffset);
|
||||
pos = arc.posArcsStart() - (arc.bytesPerArc() * arcOffset);
|
||||
} else {
|
||||
arc.nextArc = arc.posArcsStart;
|
||||
pos = arc.posArcsStart();
|
||||
}
|
||||
fst.readNextRealArc(arc, in);
|
||||
if (arc.label == targetLabel) {
|
||||
fst.readArcAtPosition(arc, in, pos);
|
||||
if (arc.label() == targetLabel) {
|
||||
// found -- copy pasta from below
|
||||
output[upto] = fst.outputs.add(output[upto-1], arc.output);
|
||||
output[upto] = fst.outputs.add(output[upto-1], arc.output());
|
||||
if (targetLabel == FST.END_LABEL) {
|
||||
return null;
|
||||
}
|
||||
setCurrentLabel(arc.label);
|
||||
setCurrentLabel(arc.label());
|
||||
incr();
|
||||
return fst.readFirstTargetArc(arc, getArc(upto), fstReader);
|
||||
}
|
||||
// not found, return the next highest
|
||||
assert arc.label > targetLabel;
|
||||
assert arc.label() > targetLabel;
|
||||
pushFirst();
|
||||
return null;
|
||||
}
|
||||
|
@ -207,48 +207,24 @@ abstract class FSTEnum<T> {
|
|||
|
||||
private FST.Arc<T> doSeekCeilArrayPacked(final FST.Arc<T> arc, final int targetLabel, final FST.BytesReader in) throws IOException {
|
||||
// The array is packed -- use binary search to find the target.
|
||||
|
||||
int low = arc.arcIdx;
|
||||
int high = arc.numArcs-1;
|
||||
int mid = 0;
|
||||
//System.out.println("do arc array low=" + low + " high=" + high + " targetLabel=" + targetLabel);
|
||||
boolean found = false;
|
||||
while (low <= high) {
|
||||
mid = (low + high) >>> 1;
|
||||
in.setPosition(arc.posArcsStart);
|
||||
in.skipBytes(arc.bytesPerArc * mid + 1);
|
||||
final int midLabel = fst.readLabel(in);
|
||||
final int cmp = midLabel - targetLabel;
|
||||
//System.out.println(" cycle low=" + low + " high=" + high + " mid=" + mid + " midLabel=" + midLabel + " cmp=" + cmp);
|
||||
if (cmp < 0)
|
||||
low = mid + 1;
|
||||
else if (cmp > 0)
|
||||
high = mid - 1;
|
||||
else {
|
||||
found = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
// NOTE: this code is dup'd w/ the code below (in
|
||||
// the outer else clause):
|
||||
if (found) {
|
||||
int idx = Util.binarySearch(fst, arc, targetLabel);
|
||||
if (idx >= 0) {
|
||||
// Match
|
||||
arc.arcIdx = mid-1;
|
||||
fst.readNextRealArc(arc, in);
|
||||
assert arc.arcIdx == mid;
|
||||
assert arc.label == targetLabel: "arc.label=" + arc.label + " vs targetLabel=" + targetLabel + " mid=" + mid;
|
||||
output[upto] = fst.outputs.add(output[upto-1], arc.output);
|
||||
fst.readArcByIndex(arc, in, idx);
|
||||
assert arc.arcIdx() == idx;
|
||||
assert arc.label() == targetLabel: "arc.label=" + arc.label() + " vs targetLabel=" + targetLabel + " mid=" + idx;
|
||||
output[upto] = fst.outputs.add(output[upto-1], arc.output());
|
||||
if (targetLabel == FST.END_LABEL) {
|
||||
return null;
|
||||
}
|
||||
setCurrentLabel(arc.label);
|
||||
setCurrentLabel(arc.label());
|
||||
incr();
|
||||
return fst.readFirstTargetArc(arc, getArc(upto), fstReader);
|
||||
} else if (low == arc.numArcs) {
|
||||
}
|
||||
idx = -1 - idx;
|
||||
if (idx == arc.numArcs()) {
|
||||
// Dead end
|
||||
arc.arcIdx = arc.numArcs-2;
|
||||
fst.readNextRealArc(arc, in);
|
||||
fst.readArcByIndex(arc, in, idx - 1);
|
||||
assert arc.isLast();
|
||||
// Dead end (target is after the last arc);
|
||||
// rollback to last fork then push
|
||||
|
@ -267,9 +243,9 @@ abstract class FSTEnum<T> {
|
|||
upto--;
|
||||
}
|
||||
} else {
|
||||
arc.arcIdx = (low > high ? low : high)-1;
|
||||
fst.readNextRealArc(arc, in);
|
||||
assert arc.label > targetLabel;
|
||||
// Ceiling - arc with least higher label
|
||||
fst.readArcByIndex(arc, in, idx);
|
||||
assert arc.label() > targetLabel;
|
||||
pushFirst();
|
||||
return null;
|
||||
}
|
||||
|
@ -277,16 +253,16 @@ abstract class FSTEnum<T> {
|
|||
|
||||
private FST.Arc<T> doSeekCeilList(final FST.Arc<T> arc, final int targetLabel) throws IOException {
|
||||
// Arcs are not array'd -- must do linear scan:
|
||||
if (arc.label == targetLabel) {
|
||||
if (arc.label() == targetLabel) {
|
||||
// recurse
|
||||
output[upto] = fst.outputs.add(output[upto-1], arc.output);
|
||||
output[upto] = fst.outputs.add(output[upto-1], arc.output());
|
||||
if (targetLabel == FST.END_LABEL) {
|
||||
return null;
|
||||
}
|
||||
setCurrentLabel(arc.label);
|
||||
setCurrentLabel(arc.label());
|
||||
incr();
|
||||
return fst.readFirstTargetArc(arc, getArc(upto), fstReader);
|
||||
} else if (arc.label > targetLabel) {
|
||||
} else if (arc.label() > targetLabel) {
|
||||
pushFirst();
|
||||
return null;
|
||||
} else if (arc.isLast()) {
|
||||
|
@ -317,7 +293,7 @@ abstract class FSTEnum<T> {
|
|||
// Todo: should we return a status here (SEEK_FOUND / SEEK_NOT_FOUND /
|
||||
// SEEK_END)? saves the eq check above?
|
||||
/** Seeks to largest term that's <= target. */
|
||||
protected void doSeekFloor() throws IOException {
|
||||
void doSeekFloor() throws IOException {
|
||||
|
||||
// TODO: possibly caller could/should provide common
|
||||
// prefix length? ie this work may be redundant if
|
||||
|
@ -340,10 +316,10 @@ abstract class FSTEnum<T> {
|
|||
//System.out.println(" cycle upto=" + upto + " arc.label=" + arc.label + " (" + (char) arc.label + ") targetLabel=" + targetLabel + " isLast?=" + arc.isLast() + " bba=" + arc.bytesPerArc);
|
||||
int targetLabel = getTargetLabel();
|
||||
|
||||
if (arc.bytesPerArc != 0 && arc.label != FST.END_LABEL) {
|
||||
if (arc.bytesPerArc() != 0 && arc.label() != FST.END_LABEL) {
|
||||
// Arcs are in an array
|
||||
final FST.BytesReader in = fst.getBytesReader();
|
||||
if (arc.arcIdx == Integer.MIN_VALUE) {
|
||||
if (arc.arcIdx() == Integer.MIN_VALUE) {
|
||||
arc = doSeekFloorArrayWithGaps(arc, targetLabel, in);
|
||||
} else {
|
||||
arc = doSeekFloorArrayPacked(arc, targetLabel, in);
|
||||
|
@ -356,7 +332,7 @@ abstract class FSTEnum<T> {
|
|||
|
||||
private FST.Arc<T> doSeekFloorArrayWithGaps(FST.Arc<T> arc, int targetLabel, final FST.BytesReader in) throws IOException {
|
||||
// The array is addressed directly by label and may contain holes.
|
||||
in.setPosition(arc.posArcsStart);
|
||||
in.setPosition(arc.posArcsStart());
|
||||
in.skipBytes(1);
|
||||
int firstLabel = fst.readLabel(in);
|
||||
int targetOffset = targetLabel - firstLabel;
|
||||
|
@ -368,7 +344,7 @@ abstract class FSTEnum<T> {
|
|||
// First, walk backwards until we find a first arc
|
||||
// that's before our target label:
|
||||
fst.readFirstTargetArc(getArc(upto-1), arc, fstReader);
|
||||
if (arc.label < targetLabel) {
|
||||
if (arc.label() < targetLabel) {
|
||||
// Then, scan forwards to the arc just before
|
||||
// the targetLabel:
|
||||
while(!arc.isLast() && fst.readNextArcLabel(arc, in) < targetLabel) {
|
||||
|
@ -385,85 +361,57 @@ abstract class FSTEnum<T> {
|
|||
arc = getArc(upto);
|
||||
}
|
||||
} else {
|
||||
if (targetOffset >= arc.numArcs) {
|
||||
arc.nextArc = arc.posArcsStart - arc.bytesPerArc * (arc.numArcs - 1);
|
||||
fst.readNextRealArc(arc, in);
|
||||
if (targetOffset >= arc.numArcs()) {
|
||||
fst.readArcAtPosition(arc, in, arc.posArcsStart() - arc.bytesPerArc() * (arc.numArcs() - 1));
|
||||
assert arc.isLast();
|
||||
assert arc.label < targetLabel: "arc.label=" + arc.label + " vs targetLabel=" + targetLabel;
|
||||
assert arc.label() < targetLabel: "arc.label=" + arc.label() + " vs targetLabel=" + targetLabel;
|
||||
pushLast();
|
||||
return null;
|
||||
}
|
||||
arc.nextArc = arc.posArcsStart - arc.bytesPerArc * targetOffset;
|
||||
fst.readNextRealArc(arc, in);
|
||||
if (arc.label == targetLabel) {
|
||||
fst.readArcAtPosition(arc, in, arc.posArcsStart() - arc.bytesPerArc() * targetOffset);
|
||||
if (arc.label() == targetLabel) {
|
||||
// found -- copy pasta from below
|
||||
output[upto] = fst.outputs.add(output[upto-1], arc.output);
|
||||
output[upto] = fst.outputs.add(output[upto-1], arc.output());
|
||||
if (targetLabel == FST.END_LABEL) {
|
||||
return null;
|
||||
}
|
||||
setCurrentLabel(arc.label);
|
||||
setCurrentLabel(arc.label());
|
||||
incr();
|
||||
return fst.readFirstTargetArc(arc, getArc(upto), fstReader);
|
||||
}
|
||||
// Scan backwards to find a floor arc that is not missing
|
||||
for (long arcOffset = arc.posArcsStart - targetOffset * arc.bytesPerArc; arcOffset <= arc.posArcsStart; arcOffset += arc.bytesPerArc) {
|
||||
for (long arcOffset = arc.posArcsStart() - targetOffset * arc.bytesPerArc(); arcOffset <= arc.posArcsStart(); arcOffset += arc.bytesPerArc()) {
|
||||
// TODO: we can do better here by skipping missing arcs
|
||||
arc.nextArc = arcOffset;
|
||||
//System.out.println(" hasFloor arcIdx=" + (arc.arcIdx+1));
|
||||
fst.readNextRealArc(arc, in);
|
||||
if (arc.label < targetLabel) {
|
||||
fst.readArcAtPosition(arc, in, arcOffset);
|
||||
if (arc.label() < targetLabel) {
|
||||
assert arc.isLast() || fst.readNextArcLabel(arc, in) > targetLabel;
|
||||
pushLast();
|
||||
return null;
|
||||
}
|
||||
}
|
||||
assert false: "arc.label=" + arc.label + " vs targetLabel=" + targetLabel;
|
||||
assert false: "arc.label=" + arc.label() + " vs targetLabel=" + targetLabel;
|
||||
return arc; // unreachable
|
||||
}
|
||||
}
|
||||
|
||||
private FST.Arc<T> doSeekFloorArrayPacked(FST.Arc<T> arc, int targetLabel, final FST.BytesReader in) throws IOException {
|
||||
// Arcs are fixed array -- use binary search to find the target.
|
||||
int idx = Util.binarySearch(fst, arc, targetLabel);
|
||||
|
||||
int low = arc.arcIdx;
|
||||
int high = arc.numArcs-1;
|
||||
int mid = 0;
|
||||
//System.out.println("do arc array low=" + low + " high=" + high + " targetLabel=" + targetLabel);
|
||||
boolean found = false;
|
||||
while (low <= high) {
|
||||
mid = (low + high) >>> 1;
|
||||
in.setPosition(arc.posArcsStart);
|
||||
in.skipBytes(arc.bytesPerArc*mid+1);
|
||||
final int midLabel = fst.readLabel(in);
|
||||
final int cmp = midLabel - targetLabel;
|
||||
//System.out.println(" cycle low=" + low + " high=" + high + " mid=" + mid + " midLabel=" + midLabel + " cmp=" + cmp);
|
||||
if (cmp < 0) {
|
||||
low = mid + 1;
|
||||
} else if (cmp > 0) {
|
||||
high = mid - 1;
|
||||
} else {
|
||||
found = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
// NOTE: this code is dup'd w/ the code below (in
|
||||
// the outer else clause):
|
||||
if (found) {
|
||||
if (idx >= 0) {
|
||||
// Match -- recurse
|
||||
//System.out.println(" match! arcIdx=" + mid);
|
||||
arc.arcIdx = mid-1;
|
||||
fst.readNextRealArc(arc, in);
|
||||
assert arc.arcIdx == mid;
|
||||
assert arc.label == targetLabel: "arc.label=" + arc.label + " vs targetLabel=" + targetLabel + " mid=" + mid;
|
||||
output[upto] = fst.outputs.add(output[upto-1], arc.output);
|
||||
//System.out.println(" match! arcIdx=" + idx);
|
||||
fst.readArcByIndex(arc, in, idx);
|
||||
assert arc.arcIdx() == idx;
|
||||
assert arc.label() == targetLabel: "arc.label=" + arc.label() + " vs targetLabel=" + targetLabel + " mid=" + idx;
|
||||
output[upto] = fst.outputs.add(output[upto-1], arc.output());
|
||||
if (targetLabel == FST.END_LABEL) {
|
||||
return null;
|
||||
}
|
||||
setCurrentLabel(arc.label);
|
||||
setCurrentLabel(arc.label());
|
||||
incr();
|
||||
return fst.readFirstTargetArc(arc, getArc(upto), fstReader);
|
||||
} else if (high == -1) {
|
||||
} else if (idx == -1) {
|
||||
//System.out.println(" before first");
|
||||
// Very first arc is after our target
|
||||
// TODO: if each arc could somehow read the arc just
|
||||
|
@ -474,7 +422,7 @@ abstract class FSTEnum<T> {
|
|||
// First, walk backwards until we find a first arc
|
||||
// that's before our target label:
|
||||
fst.readFirstTargetArc(getArc(upto-1), arc, fstReader);
|
||||
if (arc.label < targetLabel) {
|
||||
if (arc.label() < targetLabel) {
|
||||
// Then, scan forwards to the arc just before
|
||||
// the targetLabel:
|
||||
while(!arc.isLast() && fst.readNextArcLabel(arc, in) < targetLabel) {
|
||||
|
@ -491,28 +439,26 @@ abstract class FSTEnum<T> {
|
|||
arc = getArc(upto);
|
||||
}
|
||||
} else {
|
||||
// There is a floor arc:
|
||||
arc.arcIdx = (low > high ? high : low)-1;
|
||||
//System.out.println(" hasFloor arcIdx=" + (arc.arcIdx+1));
|
||||
fst.readNextRealArc(arc, in);
|
||||
// There is a floor arc; idx will be {@code -1 - (floor + 1)}.
|
||||
fst.readArcByIndex(arc, in, -2 - idx);
|
||||
assert arc.isLast() || fst.readNextArcLabel(arc, in) > targetLabel;
|
||||
assert arc.label < targetLabel: "arc.label=" + arc.label + " vs targetLabel=" + targetLabel;
|
||||
assert arc.label() < targetLabel: "arc.label=" + arc.label() + " vs targetLabel=" + targetLabel;
|
||||
pushLast();
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
private FST.Arc<T> doSeekFloorList(FST.Arc<T> arc, int targetLabel) throws IOException {
|
||||
if (arc.label == targetLabel) {
|
||||
if (arc.label() == targetLabel) {
|
||||
// Match -- recurse
|
||||
output[upto] = fst.outputs.add(output[upto-1], arc.output);
|
||||
output[upto] = fst.outputs.add(output[upto-1], arc.output());
|
||||
if (targetLabel == FST.END_LABEL) {
|
||||
return null;
|
||||
}
|
||||
setCurrentLabel(arc.label);
|
||||
setCurrentLabel(arc.label());
|
||||
incr();
|
||||
return fst.readFirstTargetArc(arc, getArc(upto), fstReader);
|
||||
} else if (arc.label > targetLabel) {
|
||||
} else if (arc.label() > targetLabel) {
|
||||
// TODO: if each arc could somehow read the arc just
|
||||
// before, we can save this re-scan. The ceil case
|
||||
// doesn't need this because it reads the next arc
|
||||
|
@ -521,7 +467,7 @@ abstract class FSTEnum<T> {
|
|||
// First, walk backwards until we find a first arc
|
||||
// that's before our target label:
|
||||
fst.readFirstTargetArc(getArc(upto-1), arc, fstReader);
|
||||
if (arc.label < targetLabel) {
|
||||
if (arc.label() < targetLabel) {
|
||||
// Then, scan forwards to the arc just before
|
||||
// the targetLabel:
|
||||
while(!arc.isLast() && fst.readNextArcLabel(arc, fstReader) < targetLabel) {
|
||||
|
@ -553,7 +499,7 @@ abstract class FSTEnum<T> {
|
|||
}
|
||||
|
||||
/** Seeks to exactly target term. */
|
||||
protected boolean doSeekExact() throws IOException {
|
||||
boolean doSeekExact() throws IOException {
|
||||
|
||||
// TODO: possibly caller could/should provide common
|
||||
// prefix length? ie this work may be redundant if
|
||||
|
@ -584,7 +530,7 @@ abstract class FSTEnum<T> {
|
|||
return false;
|
||||
}
|
||||
// Match -- recurse:
|
||||
output[upto] = fst.outputs.add(output[upto-1], nextArc.output);
|
||||
output[upto] = fst.outputs.add(output[upto-1], nextArc.output());
|
||||
if (targetLabel == FST.END_LABEL) {
|
||||
//System.out.println(" return found; upto=" + upto + " output=" + output[upto] + " nextArc=" + nextArc.isLast());
|
||||
return true;
|
||||
|
@ -621,13 +567,13 @@ abstract class FSTEnum<T> {
|
|||
assert arc != null;
|
||||
|
||||
while (true) {
|
||||
output[upto] = fst.outputs.add(output[upto-1], arc.output);
|
||||
if (arc.label == FST.END_LABEL) {
|
||||
output[upto] = fst.outputs.add(output[upto-1], arc.output());
|
||||
if (arc.label() == FST.END_LABEL) {
|
||||
// Final node
|
||||
break;
|
||||
}
|
||||
//System.out.println(" pushFirst label=" + (char) arc.label + " upto=" + upto + " output=" + fst.outputs.outputToString(output[upto]));
|
||||
setCurrentLabel(arc.label);
|
||||
setCurrentLabel(arc.label());
|
||||
incr();
|
||||
|
||||
final FST.Arc<T> nextArc = getArc(upto);
|
||||
|
@ -644,9 +590,9 @@ abstract class FSTEnum<T> {
|
|||
assert arc != null;
|
||||
|
||||
while (true) {
|
||||
setCurrentLabel(arc.label);
|
||||
output[upto] = fst.outputs.add(output[upto-1], arc.output);
|
||||
if (arc.label == FST.END_LABEL) {
|
||||
setCurrentLabel(arc.label());
|
||||
output[upto] = fst.outputs.add(output[upto-1], arc.output());
|
||||
if (arc.label() == FST.END_LABEL) {
|
||||
// Final node
|
||||
break;
|
||||
}
|
||||
|
@ -662,4 +608,5 @@ abstract class FSTEnum<T> {
|
|||
}
|
||||
return arcs[idx];
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -81,7 +81,7 @@ public final class IntsRefFSTEnum<T> extends FSTEnum<T> {
|
|||
public InputOutput<T> seekExact(IntsRef target) throws IOException {
|
||||
this.target = target;
|
||||
targetLength = target.length;
|
||||
if (super.doSeekExact()) {
|
||||
if (doSeekExact()) {
|
||||
assert upto == 1+target.length;
|
||||
return setResult();
|
||||
} else {
|
||||
|
|
|
@ -41,15 +41,15 @@ final class NodeHash<T> {
|
|||
|
||||
private boolean nodesEqual(Builder.UnCompiledNode<T> node, long address) throws IOException {
|
||||
fst.readFirstRealTargetArc(address, scratchArc, in);
|
||||
if (scratchArc.bytesPerArc != 0 && node.numArcs != scratchArc.numArcs) {
|
||||
if (scratchArc.isPackedArray() && node.numArcs != scratchArc.numArcs()) {
|
||||
return false;
|
||||
}
|
||||
for(int arcUpto=0;arcUpto<node.numArcs;arcUpto++) {
|
||||
for(int arcUpto=0; arcUpto < node.numArcs; arcUpto++) {
|
||||
final Builder.Arc<T> arc = node.arcs[arcUpto];
|
||||
if (arc.label != scratchArc.label ||
|
||||
!arc.output.equals(scratchArc.output) ||
|
||||
((Builder.CompiledNode) arc.target).node != scratchArc.target ||
|
||||
!arc.nextFinalOutput.equals(scratchArc.nextFinalOutput) ||
|
||||
if (arc.label != scratchArc.label() ||
|
||||
!arc.output.equals(scratchArc.output()) ||
|
||||
((Builder.CompiledNode) arc.target).node != scratchArc.target() ||
|
||||
!arc.nextFinalOutput.equals(scratchArc.nextFinalOutput()) ||
|
||||
arc.isFinal != scratchArc.isFinal()) {
|
||||
return false;
|
||||
}
|
||||
|
@ -98,10 +98,10 @@ final class NodeHash<T> {
|
|||
fst.readFirstRealTargetArc(node, scratchArc, in);
|
||||
while(true) {
|
||||
// System.out.println(" label=" + scratchArc.label + " target=" + scratchArc.target + " h=" + h + " output=" + fst.outputs.outputToString(scratchArc.output) + " next?=" + scratchArc.flag(4) + " final?=" + scratchArc.isFinal() + " pos=" + in.getPosition());
|
||||
h = PRIME * h + scratchArc.label;
|
||||
h = PRIME * h + (int) (scratchArc.target^(scratchArc.target>>32));
|
||||
h = PRIME * h + scratchArc.output.hashCode();
|
||||
h = PRIME * h + scratchArc.nextFinalOutput.hashCode();
|
||||
h = PRIME * h + scratchArc.label();
|
||||
h = PRIME * h + (int) (scratchArc.target() ^(scratchArc.target() >>32));
|
||||
h = PRIME * h + scratchArc.output().hashCode();
|
||||
h = PRIME * h + scratchArc.nextFinalOutput().hashCode();
|
||||
if (scratchArc.isFinal()) {
|
||||
h += 17;
|
||||
}
|
||||
|
@ -170,4 +170,5 @@ final class NodeHash<T> {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -55,11 +55,11 @@ public final class Util {
|
|||
if (fst.findTargetArc(input.ints[input.offset + i], arc, arc, fstReader) == null) {
|
||||
return null;
|
||||
}
|
||||
output = fst.outputs.add(output, arc.output);
|
||||
output = fst.outputs.add(output, arc.output());
|
||||
}
|
||||
|
||||
if (arc.isFinal()) {
|
||||
return fst.outputs.add(output, arc.nextFinalOutput);
|
||||
return fst.outputs.add(output, arc.nextFinalOutput());
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
|
@ -75,7 +75,7 @@ public final class Util {
|
|||
final BytesReader fstReader = fst.getBytesReader();
|
||||
|
||||
// TODO: would be nice not to alloc this on every lookup
|
||||
final FST.Arc<T> arc = fst.getFirstArc(new FST.Arc<T>());
|
||||
final FST.Arc<T> arc = fst.getFirstArc(new FST.Arc<>());
|
||||
|
||||
// Accumulate output as we go
|
||||
T output = fst.outputs.getNoOutput();
|
||||
|
@ -83,11 +83,11 @@ public final class Util {
|
|||
if (fst.findTargetArc(input.bytes[i+input.offset] & 0xFF, arc, arc, fstReader) == null) {
|
||||
return null;
|
||||
}
|
||||
output = fst.outputs.add(output, arc.output);
|
||||
output = fst.outputs.add(output, arc.output());
|
||||
}
|
||||
|
||||
if (arc.isFinal()) {
|
||||
return fst.outputs.add(output, arc.nextFinalOutput);
|
||||
return fst.outputs.add(output, arc.nextFinalOutput());
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
|
@ -125,7 +125,7 @@ public final class Util {
|
|||
*/
|
||||
@Deprecated
|
||||
public static IntsRef getByOutput(FST<Long> fst, long targetOutput, BytesReader in, Arc<Long> arc, Arc<Long> scratchArc, IntsRefBuilder result) throws IOException {
|
||||
long output = arc.output;
|
||||
long output = arc.output();
|
||||
int upto = 0;
|
||||
|
||||
//System.out.println("reverseLookup output=" + targetOutput);
|
||||
|
@ -133,7 +133,7 @@ public final class Util {
|
|||
while(true) {
|
||||
//System.out.println("loop: output=" + output + " upto=" + upto + " arc=" + arc);
|
||||
if (arc.isFinal()) {
|
||||
final long finalOutput = output + arc.nextFinalOutput;
|
||||
final long finalOutput = output + arc.nextFinalOutput();
|
||||
//System.out.println(" isFinal finalOutput=" + finalOutput);
|
||||
if (finalOutput == targetOutput) {
|
||||
result.setLength(upto);
|
||||
|
@ -149,19 +149,19 @@ public final class Util {
|
|||
//System.out.println(" targetHasArcs");
|
||||
result.grow(1+upto);
|
||||
|
||||
fst.readFirstRealTargetArc(arc.target, arc, in);
|
||||
fst.readFirstRealTargetArc(arc.target(), arc, in);
|
||||
|
||||
if (arc.bytesPerArc != 0 && arc.arcIdx > Integer.MIN_VALUE) {
|
||||
if (arc.bytesPerArc() != 0 && arc.arcIdx() > Integer.MIN_VALUE) {
|
||||
|
||||
int low = 0;
|
||||
int high = arc.numArcs-1;
|
||||
int high = arc.numArcs() -1;
|
||||
int mid = 0;
|
||||
//System.out.println("bsearch: numArcs=" + arc.numArcs + " target=" + targetOutput + " output=" + output);
|
||||
boolean exact = false;
|
||||
while (low <= high) {
|
||||
mid = (low + high) >>> 1;
|
||||
in.setPosition(arc.posArcsStart);
|
||||
in.skipBytes(arc.bytesPerArc*mid);
|
||||
in.setPosition(arc.posArcsStart());
|
||||
in.skipBytes(arc.bytesPerArc() *mid);
|
||||
final byte flags = in.readByte();
|
||||
fst.readLabel(in);
|
||||
final long minArcOutput;
|
||||
|
@ -182,17 +182,18 @@ public final class Util {
|
|||
}
|
||||
}
|
||||
|
||||
int idx;
|
||||
if (high == -1) {
|
||||
return null;
|
||||
} else if (exact) {
|
||||
arc.arcIdx = mid-1;
|
||||
idx = mid;
|
||||
} else {
|
||||
arc.arcIdx = low-2;
|
||||
idx = low - 1;
|
||||
}
|
||||
|
||||
fst.readNextRealArc(arc, in);
|
||||
result.setIntAt(upto++, arc.label);
|
||||
output += arc.output;
|
||||
fst.readArcByIndex(arc, in, idx);
|
||||
result.setIntAt(upto++, arc.label());
|
||||
output += arc.output();
|
||||
|
||||
} else {
|
||||
|
||||
|
@ -203,13 +204,13 @@ public final class Util {
|
|||
|
||||
// This is the min output we'd hit if we follow
|
||||
// this arc:
|
||||
final long minArcOutput = output + arc.output;
|
||||
final long minArcOutput = output + arc.output();
|
||||
|
||||
if (minArcOutput == targetOutput) {
|
||||
// Recurse on this arc:
|
||||
//System.out.println(" match! break");
|
||||
output = minArcOutput;
|
||||
result.setIntAt(upto++, arc.label);
|
||||
result.setIntAt(upto++, arc.label());
|
||||
break;
|
||||
} else if (minArcOutput > targetOutput) {
|
||||
if (prevArc == null) {
|
||||
|
@ -218,8 +219,8 @@ public final class Util {
|
|||
} else {
|
||||
// Recurse on previous arc:
|
||||
arc.copyFrom(prevArc);
|
||||
result.setIntAt(upto++, arc.label);
|
||||
output += arc.output;
|
||||
result.setIntAt(upto++, arc.label());
|
||||
output += arc.output();
|
||||
//System.out.println(" recurse prev label=" + (char) arc.label + " output=" + output);
|
||||
break;
|
||||
}
|
||||
|
@ -227,7 +228,7 @@ public final class Util {
|
|||
// Recurse on this arc:
|
||||
output = minArcOutput;
|
||||
//System.out.println(" recurse last label=" + (char) arc.label + " output=" + output);
|
||||
result.setIntAt(upto++, arc.label);
|
||||
result.setIntAt(upto++, arc.label());
|
||||
break;
|
||||
} else {
|
||||
// Read next arc in this node:
|
||||
|
@ -261,12 +262,7 @@ public final class Util {
|
|||
// Custom int payload for consumers; the NRT suggester uses this to record if this path has already enumerated a surface form
|
||||
public int payload;
|
||||
|
||||
/** Sole constructor */
|
||||
public FSTPath(T output, FST.Arc<T> arc, IntsRefBuilder input) {
|
||||
this(output, arc, input, 0, null, -1);
|
||||
}
|
||||
|
||||
public FSTPath(T output, FST.Arc<T> arc, IntsRefBuilder input, float boost, CharSequence context, int payload) {
|
||||
FSTPath(T output, FST.Arc<T> arc, IntsRefBuilder input, float boost, CharSequence context, int payload) {
|
||||
this.arc = new FST.Arc<T>().copyFrom(arc);
|
||||
this.output = output;
|
||||
this.input = input;
|
||||
|
@ -275,7 +271,7 @@ public final class Util {
|
|||
this.payload = payload;
|
||||
}
|
||||
|
||||
public FSTPath<T> newPath(T output, IntsRefBuilder input) {
|
||||
FSTPath<T> newPath(T output, IntsRefBuilder input) {
|
||||
return new FSTPath<>(output, this.arc, input, this.boost, this.context, this.payload);
|
||||
}
|
||||
|
||||
|
@ -289,7 +285,8 @@ public final class Util {
|
|||
* tie breaks by path.input. */
|
||||
private static class TieBreakByInputComparator<T> implements Comparator<FSTPath<T>> {
|
||||
private final Comparator<T> comparator;
|
||||
public TieBreakByInputComparator(Comparator<T> comparator) {
|
||||
|
||||
TieBreakByInputComparator(Comparator<T> comparator) {
|
||||
this.comparator = comparator;
|
||||
}
|
||||
|
||||
|
@ -318,7 +315,7 @@ public final class Util {
|
|||
private final Comparator<T> comparator;
|
||||
private final Comparator<FSTPath<T>> pathComparator;
|
||||
|
||||
TreeSet<FSTPath<T>> queue = null;
|
||||
TreeSet<FSTPath<T>> queue;
|
||||
|
||||
/**
|
||||
* Creates an unbounded TopNSearcher
|
||||
|
@ -347,7 +344,7 @@ public final class Util {
|
|||
|
||||
assert queue != null;
|
||||
|
||||
T output = fst.outputs.add(path.output, path.arc.output);
|
||||
T output = fst.outputs.add(path.output, path.arc.output());
|
||||
|
||||
if (queue.size() == maxQueueDepth) {
|
||||
FSTPath<T> bottom = queue.last();
|
||||
|
@ -357,7 +354,7 @@ public final class Util {
|
|||
return;
|
||||
} else if (comp == 0) {
|
||||
// Tie break by alpha sort on the input:
|
||||
path.input.append(path.arc.label);
|
||||
path.input.append(path.arc.label());
|
||||
final int cmp = bottom.input.get().compareTo(path.input.get());
|
||||
path.input.setLength(path.input.length() - 1);
|
||||
|
||||
|
@ -370,15 +367,14 @@ public final class Util {
|
|||
}
|
||||
}
|
||||
// Competes
|
||||
} else {
|
||||
// Queue isn't full yet, so any path we hit competes:
|
||||
}
|
||||
// else ... Queue isn't full yet, so any path we hit competes:
|
||||
|
||||
// copy over the current input to the new input
|
||||
// and add the arc.label to the end
|
||||
IntsRefBuilder newInput = new IntsRefBuilder();
|
||||
newInput.copyInts(path.input.get());
|
||||
newInput.append(path.arc.label);
|
||||
newInput.append(path.arc.label());
|
||||
|
||||
FSTPath<T> newPath = path.newPath(output, newInput);
|
||||
if (acceptPartialPath(newPath)) {
|
||||
|
@ -408,7 +404,7 @@ public final class Util {
|
|||
|
||||
// Bootstrap: find the min starting arc
|
||||
while (true) {
|
||||
if (allowEmptyString || path.arc.label != FST.END_LABEL) {
|
||||
if (allowEmptyString || path.arc.label() != FST.END_LABEL) {
|
||||
addIfCompetitive(path);
|
||||
}
|
||||
if (path.arc.isLast()) {
|
||||
|
@ -457,7 +453,7 @@ public final class Util {
|
|||
continue;
|
||||
}
|
||||
|
||||
if (path.arc.label == FST.END_LABEL) {
|
||||
if (path.arc.label() == FST.END_LABEL) {
|
||||
// Empty string!
|
||||
path.input.setLength(path.input.length() - 1);
|
||||
results.add(new Result<>(path.input.get(), path.output));
|
||||
|
@ -485,7 +481,7 @@ public final class Util {
|
|||
while(true) {
|
||||
// tricky: instead of comparing output == 0, we must
|
||||
// express it via the comparator compare(output, 0) == 0
|
||||
if (comparator.compare(NO_OUTPUT, path.arc.output) == 0) {
|
||||
if (comparator.compare(NO_OUTPUT, path.arc.output()) == 0) {
|
||||
if (queue == null) {
|
||||
foundZero = true;
|
||||
break;
|
||||
|
@ -514,9 +510,9 @@ public final class Util {
|
|||
path.arc.copyFrom(scratchArc);
|
||||
}
|
||||
|
||||
if (path.arc.label == FST.END_LABEL) {
|
||||
if (path.arc.label() == FST.END_LABEL) {
|
||||
// Add final output:
|
||||
path.output = fst.outputs.add(path.output, path.arc.output);
|
||||
path.output = fst.outputs.add(path.output, path.arc.output());
|
||||
if (acceptResult(path)) {
|
||||
results.add(new Result<>(path.input.get(), path.output));
|
||||
} else {
|
||||
|
@ -524,8 +520,8 @@ public final class Util {
|
|||
}
|
||||
break;
|
||||
} else {
|
||||
path.input.append(path.arc.label);
|
||||
path.output = fst.outputs.add(path.output, path.arc.output);
|
||||
path.input.append(path.arc.label());
|
||||
path.output = fst.outputs.add(path.output, path.arc.output());
|
||||
if (acceptPartialPath(path) == false) {
|
||||
break;
|
||||
}
|
||||
|
@ -641,7 +637,7 @@ public final class Util {
|
|||
|
||||
// This is the start arc in the automaton (from the epsilon state to the first state
|
||||
// with outgoing transitions.
|
||||
final FST.Arc<T> startArc = fst.getFirstArc(new FST.Arc<T>());
|
||||
final FST.Arc<T> startArc = fst.getFirstArc(new FST.Arc<>());
|
||||
|
||||
// A queue of transitions to consider for the next level.
|
||||
final List<FST.Arc<T>> thisLevelQueue = new ArrayList<>();
|
||||
|
@ -656,7 +652,7 @@ public final class Util {
|
|||
|
||||
// A bitset of already seen states (target offset).
|
||||
final BitSet seen = new BitSet();
|
||||
seen.set((int) startArc.target);
|
||||
seen.set((int) startArc.target());
|
||||
|
||||
// Shape for states.
|
||||
final String stateShape = "circle";
|
||||
|
@ -689,16 +685,16 @@ public final class Util {
|
|||
final T finalOutput;
|
||||
if (startArc.isFinal()) {
|
||||
isFinal = true;
|
||||
finalOutput = startArc.nextFinalOutput == NO_OUTPUT ? null : startArc.nextFinalOutput;
|
||||
finalOutput = startArc.nextFinalOutput() == NO_OUTPUT ? null : startArc.nextFinalOutput();
|
||||
} else {
|
||||
isFinal = false;
|
||||
finalOutput = null;
|
||||
}
|
||||
|
||||
emitDotState(out, Long.toString(startArc.target), isFinal ? finalStateShape : stateShape, stateColor, finalOutput == null ? "" : fst.outputs.outputToString(finalOutput));
|
||||
emitDotState(out, Long.toString(startArc.target()), isFinal ? finalStateShape : stateShape, stateColor, finalOutput == null ? "" : fst.outputs.outputToString(finalOutput));
|
||||
}
|
||||
|
||||
out.write(" initial -> " + startArc.target + "\n");
|
||||
out.write(" initial -> " + startArc.target() + "\n");
|
||||
|
||||
int level = 0;
|
||||
|
||||
|
@ -717,9 +713,9 @@ public final class Util {
|
|||
// scan all target arcs
|
||||
//System.out.println(" readFirstTarget...");
|
||||
|
||||
final long node = arc.target;
|
||||
final long node = arc.target();
|
||||
|
||||
fst.readFirstRealTargetArc(arc.target, arc, r);
|
||||
fst.readFirstRealTargetArc(arc.target(), arc, r);
|
||||
|
||||
//System.out.println(" firstTarget: " + arc);
|
||||
|
||||
|
@ -727,7 +723,7 @@ public final class Util {
|
|||
|
||||
//System.out.println(" cycle arc=" + arc);
|
||||
// Emit the unseen state and add it to the queue for the next level.
|
||||
if (arc.target >= 0 && !seen.get((int) arc.target)) {
|
||||
if (arc.target() >= 0 && !seen.get((int) arc.target())) {
|
||||
|
||||
/*
|
||||
boolean isFinal = false;
|
||||
|
@ -748,35 +744,35 @@ public final class Util {
|
|||
}
|
||||
|
||||
final String finalOutput;
|
||||
if (arc.nextFinalOutput != null && arc.nextFinalOutput != NO_OUTPUT) {
|
||||
finalOutput = fst.outputs.outputToString(arc.nextFinalOutput);
|
||||
if (arc.nextFinalOutput() != null && arc.nextFinalOutput() != NO_OUTPUT) {
|
||||
finalOutput = fst.outputs.outputToString(arc.nextFinalOutput());
|
||||
} else {
|
||||
finalOutput = "";
|
||||
}
|
||||
|
||||
emitDotState(out, Long.toString(arc.target), stateShape, stateColor, finalOutput);
|
||||
emitDotState(out, Long.toString(arc.target()), stateShape, stateColor, finalOutput);
|
||||
// To see the node address, use this instead:
|
||||
//emitDotState(out, Integer.toString(arc.target), stateShape, stateColor, String.valueOf(arc.target));
|
||||
seen.set((int) arc.target);
|
||||
seen.set((int) arc.target());
|
||||
nextLevelQueue.add(new FST.Arc<T>().copyFrom(arc));
|
||||
sameLevelStates.add((int) arc.target);
|
||||
sameLevelStates.add((int) arc.target());
|
||||
}
|
||||
|
||||
String outs;
|
||||
if (arc.output != NO_OUTPUT) {
|
||||
outs = "/" + fst.outputs.outputToString(arc.output);
|
||||
if (arc.output() != NO_OUTPUT) {
|
||||
outs = "/" + fst.outputs.outputToString(arc.output());
|
||||
} else {
|
||||
outs = "";
|
||||
}
|
||||
|
||||
if (!FST.targetHasArcs(arc) && arc.isFinal() && arc.nextFinalOutput != NO_OUTPUT) {
|
||||
if (!FST.targetHasArcs(arc) && arc.isFinal() && arc.nextFinalOutput() != NO_OUTPUT) {
|
||||
// Tricky special case: sometimes, due to
|
||||
// pruning, the builder can [sillily] produce
|
||||
// an FST with an arc into the final end state
|
||||
// (-1) but also with a next final output; in
|
||||
// this case we pull that output up onto this
|
||||
// arc
|
||||
outs = outs + "/[" + fst.outputs.outputToString(arc.nextFinalOutput) + "]";
|
||||
outs = outs + "/[" + fst.outputs.outputToString(arc.nextFinalOutput()) + "]";
|
||||
}
|
||||
|
||||
final String arcColor;
|
||||
|
@ -786,8 +782,8 @@ public final class Util {
|
|||
arcColor = "black";
|
||||
}
|
||||
|
||||
assert arc.label != FST.END_LABEL;
|
||||
out.write(" " + node + " -> " + arc.target + " [label=\"" + printableLabel(arc.label) + outs + "\"" + (arc.isFinal() ? " style=\"bold\"" : "" ) + " color=\"" + arcColor + "\"]\n");
|
||||
assert arc.label() != FST.END_LABEL;
|
||||
out.write(" " + node + " -> " + arc.target() + " [label=\"" + printableLabel(arc.label()) + outs + "\"" + (arc.isFinal() ? " style=\"bold\"" : "" ) + " color=\"" + arcColor + "\"]\n");
|
||||
|
||||
// Break the loop if we're on the last arc of this state.
|
||||
if (arc.isLast()) {
|
||||
|
@ -935,86 +931,48 @@ public final class Util {
|
|||
* @param arc the arc to read into in place
|
||||
* @param in the fst's {@link BytesReader}
|
||||
*/
|
||||
public static <T> Arc<T> readCeilArc(int label, FST<T> fst, Arc<T> follow,
|
||||
Arc<T> arc, BytesReader in) throws IOException {
|
||||
// TODO maybe this is a useful in the FST class - we could simplify some other code like FSTEnum?
|
||||
public static <T> Arc<T> readCeilArc(int label, FST<T> fst, Arc<T> follow, Arc<T> arc, BytesReader in) throws IOException {
|
||||
if (label == FST.END_LABEL) {
|
||||
if (follow.isFinal()) {
|
||||
if (follow.target <= 0) {
|
||||
arc.flags = FST.BIT_LAST_ARC;
|
||||
} else {
|
||||
arc.flags = 0;
|
||||
// NOTE: nextArc is a node (not an address!) in this case:
|
||||
arc.nextArc = follow.target;
|
||||
}
|
||||
arc.output = follow.nextFinalOutput;
|
||||
arc.label = FST.END_LABEL;
|
||||
return arc;
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
return FST.readEndArc(follow, arc);
|
||||
}
|
||||
|
||||
if (!FST.targetHasArcs(follow)) {
|
||||
return null;
|
||||
}
|
||||
fst.readFirstTargetArc(follow, arc, in);
|
||||
if (arc.bytesPerArc != 0 && arc.label != FST.END_LABEL) {
|
||||
if (arc.arcIdx == Integer.MIN_VALUE) {
|
||||
if (arc.bytesPerArc() != 0 && arc.label() != FST.END_LABEL) {
|
||||
if (arc.arcIdx() == Integer.MIN_VALUE) {
|
||||
// Arcs are in an array-with-gaps
|
||||
int offset = label - arc.label;
|
||||
if (offset >= arc.numArcs) {
|
||||
int offset = label - arc.label();
|
||||
if (offset >= arc.numArcs()) {
|
||||
return null;
|
||||
} else if (offset < 0) {
|
||||
return arc;
|
||||
} else {
|
||||
arc.nextArc = arc.posArcsStart - offset * arc.bytesPerArc;
|
||||
return fst.readNextRealArc(arc, in);
|
||||
return fst.readArcAtPosition(arc, in, arc.posArcsStart() - offset * arc.bytesPerArc());
|
||||
}
|
||||
}
|
||||
// Arcs are packed array -- use binary search to find
|
||||
// the target.
|
||||
|
||||
int low = arc.arcIdx;
|
||||
int high = arc.numArcs - 1;
|
||||
int mid = 0;
|
||||
// System.out.println("do arc array low=" + low + " high=" + high +
|
||||
// " targetLabel=" + targetLabel);
|
||||
while (low <= high) {
|
||||
mid = (low + high) >>> 1;
|
||||
in.setPosition(arc.posArcsStart);
|
||||
in.skipBytes(arc.bytesPerArc * mid + 1);
|
||||
final int midLabel = fst.readLabel(in);
|
||||
final int cmp = midLabel - label;
|
||||
// System.out.println(" cycle low=" + low + " high=" + high + " mid=" +
|
||||
// mid + " midLabel=" + midLabel + " cmp=" + cmp);
|
||||
if (cmp < 0) {
|
||||
low = mid + 1;
|
||||
} else if (cmp > 0) {
|
||||
high = mid - 1;
|
||||
} else {
|
||||
arc.arcIdx = mid-1;
|
||||
return fst.readNextRealArc(arc, in);
|
||||
}
|
||||
// Arcs are packed array -- use binary search to find the target.
|
||||
int idx = binarySearch(fst, arc, label);
|
||||
if (idx >= 0) {
|
||||
return fst.readArcByIndex(arc, in, idx);
|
||||
}
|
||||
if (low == arc.numArcs) {
|
||||
idx = -1 - idx;
|
||||
if (idx == arc.numArcs()) {
|
||||
// DEAD END!
|
||||
return null;
|
||||
}
|
||||
|
||||
arc.arcIdx = (low > high ? high : low);
|
||||
return fst.readNextRealArc(arc, in);
|
||||
return fst.readArcByIndex(arc, in , idx);
|
||||
}
|
||||
|
||||
// Linear scan
|
||||
fst.readFirstRealTargetArc(follow.target, arc, in);
|
||||
fst.readFirstRealTargetArc(follow.target(), arc, in);
|
||||
|
||||
while (true) {
|
||||
// System.out.println(" non-bs cycle");
|
||||
// TODO: we should fix this code to not have to create
|
||||
// object for the output of every arc we scan... only
|
||||
// for the matching arc, if found
|
||||
if (arc.label >= label) {
|
||||
if (arc.label() >= label) {
|
||||
// System.out.println(" found!");
|
||||
return arc;
|
||||
} else if (arc.isLast()) {
|
||||
|
@ -1024,4 +982,37 @@ public final class Util {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform a binary search of Arcs encoded as a packed array
|
||||
* @param fst the FST from which to read
|
||||
* @param arc the starting arc; sibling arcs greater than this will be searched. Usually the first arc in the array.
|
||||
* @param targetLabel the label to search for
|
||||
* @param <T> the output type of the FST
|
||||
* @return the index of the Arc having the target label, or if no Arc has the matching label, {@code -1 - idx)},
|
||||
* where {@code idx} is the index of the Arc with the next highest label, or the total number of arcs
|
||||
* if the target label exceeds the maximum.
|
||||
* @throws IOException when the FST reader does
|
||||
*/
|
||||
static <T> int binarySearch(FST<T> fst, FST.Arc<T> arc, int targetLabel) throws IOException {
|
||||
BytesReader in = fst.getBytesReader();
|
||||
int low = arc.arcIdx();
|
||||
int mid = 0;
|
||||
int high = arc.numArcs() -1;
|
||||
while (low <= high) {
|
||||
mid = (low + high) >>> 1;
|
||||
in.setPosition(arc.posArcsStart());
|
||||
in.skipBytes(arc.bytesPerArc() * mid + 1);
|
||||
final int midLabel = fst.readLabel(in);
|
||||
final int cmp = midLabel - targetLabel;
|
||||
if (cmp < 0) {
|
||||
low = mid + 1;
|
||||
} else if (cmp > 0) {
|
||||
high = mid - 1;
|
||||
} else {
|
||||
return mid;
|
||||
}
|
||||
}
|
||||
return -1 - low;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -135,95 +135,55 @@ public class TestCharArrayMap extends LuceneTestCase {
|
|||
map.put("bar",2);
|
||||
final int size = map.size();
|
||||
assertEquals(2, size);
|
||||
assertTrue(map.containsKey("foo"));
|
||||
assertEquals(1, map.get("foo").intValue());
|
||||
assertTrue(map.containsKey("bar"));
|
||||
assertEquals(2, map.get("bar").intValue());
|
||||
assertTrue(map.containsKey("foo"));
|
||||
assertEquals(1, map.get("foo").intValue());
|
||||
assertTrue(map.containsKey("bar"));
|
||||
assertEquals(2, map.get("bar").intValue());
|
||||
|
||||
map = CharArrayMap.unmodifiableMap(map);
|
||||
assertEquals("Map size changed due to unmodifiableMap call" , size, map.size());
|
||||
CharArrayMap<Integer> unmodifiableMap = CharArrayMap.unmodifiableMap(map);
|
||||
assertEquals("Map size changed due to unmodifiableMap call" , size, unmodifiableMap.size());
|
||||
String NOT_IN_MAP = "SirGallahad";
|
||||
assertFalse("Test String already exists in map", map.containsKey(NOT_IN_MAP));
|
||||
assertNull("Test String already exists in map", map.get(NOT_IN_MAP));
|
||||
|
||||
try{
|
||||
map.put(NOT_IN_MAP.toCharArray(), 3);
|
||||
fail("Modified unmodifiable map");
|
||||
}catch (UnsupportedOperationException e) {
|
||||
// expected
|
||||
assertFalse("Test String has been added to unmodifiable map", map.containsKey(NOT_IN_MAP));
|
||||
assertNull("Test String has been added to unmodifiable map", map.get(NOT_IN_MAP));
|
||||
assertEquals("Size of unmodifiable map has changed", size, map.size());
|
||||
}
|
||||
|
||||
try{
|
||||
map.put(NOT_IN_MAP, 3);
|
||||
fail("Modified unmodifiable map");
|
||||
}catch (UnsupportedOperationException e) {
|
||||
// expected
|
||||
assertFalse("Test String has been added to unmodifiable map", map.containsKey(NOT_IN_MAP));
|
||||
assertNull("Test String has been added to unmodifiable map", map.get(NOT_IN_MAP));
|
||||
assertEquals("Size of unmodifiable map has changed", size, map.size());
|
||||
}
|
||||
|
||||
try{
|
||||
map.put(new StringBuilder(NOT_IN_MAP), 3);
|
||||
fail("Modified unmodifiable map");
|
||||
}catch (UnsupportedOperationException e) {
|
||||
// expected
|
||||
assertFalse("Test String has been added to unmodifiable map", map.containsKey(NOT_IN_MAP));
|
||||
assertNull("Test String has been added to unmodifiable map", map.get(NOT_IN_MAP));
|
||||
assertEquals("Size of unmodifiable map has changed", size, map.size());
|
||||
}
|
||||
|
||||
try{
|
||||
map.clear();
|
||||
fail("Modified unmodifiable map");
|
||||
}catch (UnsupportedOperationException e) {
|
||||
// expected
|
||||
assertEquals("Size of unmodifiable map has changed", size, map.size());
|
||||
}
|
||||
|
||||
try{
|
||||
map.entrySet().clear();
|
||||
fail("Modified unmodifiable map");
|
||||
}catch (UnsupportedOperationException e) {
|
||||
// expected
|
||||
assertEquals("Size of unmodifiable map has changed", size, map.size());
|
||||
}
|
||||
|
||||
try{
|
||||
map.keySet().clear();
|
||||
fail("Modified unmodifiable map");
|
||||
}catch (UnsupportedOperationException e) {
|
||||
// expected
|
||||
assertEquals("Size of unmodifiable map has changed", size, map.size());
|
||||
}
|
||||
|
||||
try{
|
||||
map.put((Object) NOT_IN_MAP, 3);
|
||||
fail("Modified unmodifiable map");
|
||||
}catch (UnsupportedOperationException e) {
|
||||
// expected
|
||||
assertFalse("Test String has been added to unmodifiable map", map.containsKey(NOT_IN_MAP));
|
||||
assertNull("Test String has been added to unmodifiable map", map.get(NOT_IN_MAP));
|
||||
assertEquals("Size of unmodifiable map has changed", size, map.size());
|
||||
}
|
||||
|
||||
try{
|
||||
map.putAll(Collections.singletonMap(NOT_IN_MAP, 3));
|
||||
fail("Modified unmodifiable map");
|
||||
}catch (UnsupportedOperationException e) {
|
||||
// expected
|
||||
assertFalse("Test String has been added to unmodifiable map", map.containsKey(NOT_IN_MAP));
|
||||
assertNull("Test String has been added to unmodifiable map", map.get(NOT_IN_MAP));
|
||||
assertEquals("Size of unmodifiable map has changed", size, map.size());
|
||||
}
|
||||
|
||||
assertTrue(map.containsKey("foo"));
|
||||
assertEquals(1, map.get("foo").intValue());
|
||||
assertTrue(map.containsKey("bar"));
|
||||
assertEquals(2, map.get("bar").intValue());
|
||||
assertFalse("Test String already exists in map", unmodifiableMap.containsKey(NOT_IN_MAP));
|
||||
assertNull("Test String already exists in map", unmodifiableMap.get(NOT_IN_MAP));
|
||||
|
||||
expectThrows(UnsupportedOperationException.class, () -> unmodifiableMap.put(NOT_IN_MAP.toCharArray(), 3));
|
||||
assertFalse("Test String has been added to unmodifiable map", unmodifiableMap.containsKey(NOT_IN_MAP));
|
||||
assertNull("Test String has been added to unmodifiable map", unmodifiableMap.get(NOT_IN_MAP));
|
||||
assertEquals("Size of unmodifiable map has changed", size, unmodifiableMap.size());
|
||||
|
||||
expectThrows(UnsupportedOperationException.class, () -> unmodifiableMap.put(NOT_IN_MAP, 3));
|
||||
assertFalse("Test String has been added to unmodifiable map", unmodifiableMap.containsKey(NOT_IN_MAP));
|
||||
assertNull("Test String has been added to unmodifiable map", unmodifiableMap.get(NOT_IN_MAP));
|
||||
assertEquals("Size of unmodifiable map has changed", size, unmodifiableMap.size());
|
||||
|
||||
expectThrows(UnsupportedOperationException.class, () -> unmodifiableMap.put(new StringBuilder(NOT_IN_MAP), 3));
|
||||
assertFalse("Test String has been added to unmodifiable map", unmodifiableMap.containsKey(NOT_IN_MAP));
|
||||
assertNull("Test String has been added to unmodifiable map", unmodifiableMap.get(NOT_IN_MAP));
|
||||
assertEquals("Size of unmodifiable map has changed", size, unmodifiableMap.size());
|
||||
|
||||
expectThrows(UnsupportedOperationException.class, unmodifiableMap::clear);
|
||||
assertEquals("Size of unmodifiable map has changed", size, unmodifiableMap.size());
|
||||
|
||||
expectThrows(UnsupportedOperationException.class, () -> unmodifiableMap.entrySet().clear());
|
||||
assertEquals("Size of unmodifiable map has changed", size, unmodifiableMap.size());
|
||||
|
||||
expectThrows(UnsupportedOperationException.class, () -> unmodifiableMap.keySet().clear());
|
||||
assertEquals("Size of unmodifiable map has changed", size, unmodifiableMap.size());
|
||||
|
||||
expectThrows(UnsupportedOperationException.class, () -> unmodifiableMap.put((Object) NOT_IN_MAP, 3));
|
||||
assertFalse("Test String has been added to unmodifiable map", unmodifiableMap.containsKey(NOT_IN_MAP));
|
||||
assertNull("Test String has been added to unmodifiable map", unmodifiableMap.get(NOT_IN_MAP));
|
||||
assertEquals("Size of unmodifiable map has changed", size, unmodifiableMap.size());
|
||||
|
||||
expectThrows(UnsupportedOperationException.class, () -> unmodifiableMap.putAll(Collections.singletonMap(NOT_IN_MAP, 3)));
|
||||
assertFalse("Test String has been added to unmodifiable map", unmodifiableMap.containsKey(NOT_IN_MAP));
|
||||
assertNull("Test String has been added to unmodifiable map", unmodifiableMap.get(NOT_IN_MAP));
|
||||
assertEquals("Size of unmodifiable map has changed", size, unmodifiableMap.size());
|
||||
|
||||
assertTrue(unmodifiableMap.containsKey("foo"));
|
||||
assertEquals(1, unmodifiableMap.get("foo").intValue());
|
||||
assertTrue(unmodifiableMap.containsKey("bar"));
|
||||
assertEquals(2, unmodifiableMap.get("bar").intValue());
|
||||
}
|
||||
|
||||
public void testToString() {
|
||||
|
|
|
@ -90,87 +90,49 @@ public class TestCharArraySet extends LuceneTestCase {
|
|||
|
||||
// TODO: break this up into simpler test methods, vs "telling a story"
|
||||
public void testModifyOnUnmodifiable(){
|
||||
CharArraySet set=new CharArraySet(10, true);
|
||||
CharArraySet set = new CharArraySet(10, true);
|
||||
set.addAll(Arrays.asList(TEST_STOP_WORDS));
|
||||
final int size = set.size();
|
||||
set = CharArraySet.unmodifiableSet(set);
|
||||
assertEquals("Set size changed due to unmodifiableSet call" , size, set.size());
|
||||
CharArraySet unmodifiableSet = CharArraySet.unmodifiableSet(set);
|
||||
assertEquals("Set size changed due to unmodifiableSet call" , size, unmodifiableSet.size());
|
||||
String NOT_IN_SET = "SirGallahad";
|
||||
assertFalse("Test String already exists in set", set.contains(NOT_IN_SET));
|
||||
|
||||
try{
|
||||
set.add(NOT_IN_SET.toCharArray());
|
||||
fail("Modified unmodifiable set");
|
||||
}catch (UnsupportedOperationException e) {
|
||||
// expected
|
||||
assertFalse("Test String has been added to unmodifiable set", set.contains(NOT_IN_SET));
|
||||
assertEquals("Size of unmodifiable set has changed", size, set.size());
|
||||
}
|
||||
|
||||
try{
|
||||
set.add(NOT_IN_SET);
|
||||
fail("Modified unmodifiable set");
|
||||
}catch (UnsupportedOperationException e) {
|
||||
// expected
|
||||
assertFalse("Test String has been added to unmodifiable set", set.contains(NOT_IN_SET));
|
||||
assertEquals("Size of unmodifiable set has changed", size, set.size());
|
||||
}
|
||||
|
||||
try{
|
||||
set.add(new StringBuilder(NOT_IN_SET));
|
||||
fail("Modified unmodifiable set");
|
||||
}catch (UnsupportedOperationException e) {
|
||||
// expected
|
||||
assertFalse("Test String has been added to unmodifiable set", set.contains(NOT_IN_SET));
|
||||
assertEquals("Size of unmodifiable set has changed", size, set.size());
|
||||
}
|
||||
|
||||
try{
|
||||
set.clear();
|
||||
fail("Modified unmodifiable set");
|
||||
}catch (UnsupportedOperationException e) {
|
||||
// expected
|
||||
assertFalse("Changed unmodifiable set", set.contains(NOT_IN_SET));
|
||||
assertEquals("Size of unmodifiable set has changed", size, set.size());
|
||||
}
|
||||
try{
|
||||
set.add((Object) NOT_IN_SET);
|
||||
fail("Modified unmodifiable set");
|
||||
}catch (UnsupportedOperationException e) {
|
||||
// expected
|
||||
assertFalse("Test String has been added to unmodifiable set", set.contains(NOT_IN_SET));
|
||||
assertEquals("Size of unmodifiable set has changed", size, set.size());
|
||||
}
|
||||
|
||||
assertFalse("Test String already exists in set", unmodifiableSet.contains(NOT_IN_SET));
|
||||
|
||||
expectThrows(UnsupportedOperationException.class, () -> unmodifiableSet.add(NOT_IN_SET.toCharArray()));
|
||||
assertFalse("Test String has been added to unmodifiable set", unmodifiableSet.contains(NOT_IN_SET));
|
||||
assertEquals("Size of unmodifiable set has changed", size, unmodifiableSet.size());
|
||||
|
||||
expectThrows(UnsupportedOperationException.class, () -> unmodifiableSet.add(NOT_IN_SET));
|
||||
assertFalse("Test String has been added to unmodifiable set", unmodifiableSet.contains(NOT_IN_SET));
|
||||
assertEquals("Size of unmodifiable set has changed", size, unmodifiableSet.size());
|
||||
|
||||
expectThrows(UnsupportedOperationException.class, () -> unmodifiableSet.add(new StringBuilder(NOT_IN_SET)));
|
||||
assertFalse("Test String has been added to unmodifiable set", unmodifiableSet.contains(NOT_IN_SET));
|
||||
assertEquals("Size of unmodifiable set has changed", size, unmodifiableSet.size());
|
||||
|
||||
expectThrows(UnsupportedOperationException.class, () -> unmodifiableSet.clear());
|
||||
assertFalse("Changed unmodifiable set", unmodifiableSet.contains(NOT_IN_SET));
|
||||
assertEquals("Size of unmodifiable set has changed", size, unmodifiableSet.size());
|
||||
|
||||
expectThrows(UnsupportedOperationException.class, () -> unmodifiableSet.add((Object) NOT_IN_SET));
|
||||
assertFalse("Test String has been added to unmodifiable set", unmodifiableSet.contains(NOT_IN_SET));
|
||||
assertEquals("Size of unmodifiable set has changed", size, unmodifiableSet.size());
|
||||
|
||||
// This test was changed in 3.1, as a contains() call on the given Collection using the "correct" iterator's
|
||||
// current key (now a char[]) on a Set<String> would not hit any element of the CAS and therefor never call
|
||||
// remove() on the iterator
|
||||
try{
|
||||
set.removeAll(new CharArraySet(Arrays.asList(TEST_STOP_WORDS), true));
|
||||
fail("Modified unmodifiable set");
|
||||
}catch (UnsupportedOperationException e) {
|
||||
// expected
|
||||
assertEquals("Size of unmodifiable set has changed", size, set.size());
|
||||
}
|
||||
|
||||
try{
|
||||
set.retainAll(new CharArraySet(Arrays.asList(NOT_IN_SET), true));
|
||||
fail("Modified unmodifiable set");
|
||||
}catch (UnsupportedOperationException e) {
|
||||
// expected
|
||||
assertEquals("Size of unmodifiable set has changed", size, set.size());
|
||||
}
|
||||
|
||||
try{
|
||||
set.addAll(Arrays.asList(NOT_IN_SET));
|
||||
fail("Modified unmodifiable set");
|
||||
}catch (UnsupportedOperationException e) {
|
||||
// expected
|
||||
assertFalse("Test String has been added to unmodifiable set", set.contains(NOT_IN_SET));
|
||||
}
|
||||
|
||||
expectThrows(UnsupportedOperationException.class, () -> unmodifiableSet.removeAll(new CharArraySet(Arrays.asList(TEST_STOP_WORDS), true)));
|
||||
assertEquals("Size of unmodifiable set has changed", size, unmodifiableSet.size());
|
||||
|
||||
expectThrows(UnsupportedOperationException.class, () -> unmodifiableSet.retainAll(new CharArraySet(Arrays.asList(NOT_IN_SET), true)));
|
||||
assertEquals("Size of unmodifiable set has changed", size, unmodifiableSet.size());
|
||||
|
||||
expectThrows(UnsupportedOperationException.class, () -> unmodifiableSet.addAll(Arrays.asList(NOT_IN_SET)));
|
||||
assertFalse("Test String has been added to unmodifiable set", unmodifiableSet.contains(NOT_IN_SET));
|
||||
|
||||
for (int i = 0; i < TEST_STOP_WORDS.length; i++) {
|
||||
assertTrue(set.contains(TEST_STOP_WORDS[i]));
|
||||
assertTrue(set.contains(TEST_STOP_WORDS[i]));
|
||||
assertTrue(unmodifiableSet.contains(TEST_STOP_WORDS[i]));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -127,21 +127,15 @@ public class TestAllFilesCheckIndexHeader extends LuceneTestCase {
|
|||
dirCopy.sync(Collections.singleton(name));
|
||||
}
|
||||
|
||||
try {
|
||||
// NOTE: we .close so that if the test fails (truncation not detected) we don't also get all these confusing errors about open files:
|
||||
DirectoryReader.open(dirCopy).close();
|
||||
fail("wrong bytes not detected after randomizing first " + wrongBytes + " bytes out of " + victimLength + " for file " + victim);
|
||||
} catch (CorruptIndexException | EOFException | IndexFormatTooOldException e) {
|
||||
// expected
|
||||
}
|
||||
// NOTE: we .close so that if the test fails (truncation not detected) we don't also get all these confusing errors about open files:
|
||||
expectThrowsAnyOf(Arrays.asList(CorruptIndexException.class, EOFException.class, IndexFormatTooOldException.class),
|
||||
() -> DirectoryReader.open(dirCopy).close()
|
||||
);
|
||||
|
||||
// CheckIndex should also fail:
|
||||
try {
|
||||
TestUtil.checkIndex(dirCopy, true, true, null);
|
||||
fail("wrong bytes not detected after randomizing first " + wrongBytes + " bytes out of " + victimLength + " for file " + victim);
|
||||
} catch (CorruptIndexException | EOFException | IndexFormatTooOldException e) {
|
||||
// expected
|
||||
}
|
||||
expectThrowsAnyOf(Arrays.asList(CorruptIndexException.class, EOFException.class, IndexFormatTooOldException.class),
|
||||
() -> DirectoryReader.open(dirCopy).close()
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.io.EOFException;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
|
@ -104,21 +105,15 @@ public class TestAllFilesDetectTruncation extends LuceneTestCase {
|
|||
dirCopy.sync(Collections.singleton(name));
|
||||
}
|
||||
|
||||
try {
|
||||
// NOTE: we .close so that if the test fails (truncation not detected) we don't also get all these confusing errors about open files:
|
||||
DirectoryReader.open(dirCopy).close();
|
||||
fail("truncation not detected after removing " + lostBytes + " bytes out of " + victimLength + " for file " + victim);
|
||||
} catch (CorruptIndexException | EOFException e) {
|
||||
// expected
|
||||
}
|
||||
// NOTE: we .close so that if the test fails (truncation not detected) we don't also get all these confusing errors about open files:
|
||||
expectThrowsAnyOf(Arrays.asList(CorruptIndexException.class, EOFException.class),
|
||||
() -> DirectoryReader.open(dirCopy).close()
|
||||
);
|
||||
|
||||
// CheckIndex should also fail:
|
||||
try {
|
||||
TestUtil.checkIndex(dirCopy, true, true, null);
|
||||
fail("truncation not detected after removing " + lostBytes + " bytes out of " + victimLength + " for file " + victim);
|
||||
} catch (CorruptIndexException | EOFException e) {
|
||||
// expected
|
||||
}
|
||||
expectThrowsAnyOf(Arrays.asList(CorruptIndexException.class, EOFException.class),
|
||||
() -> TestUtil.checkIndex(dirCopy, true, true, null)
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,6 +21,7 @@ import java.io.FileNotFoundException;
|
|||
import java.io.IOException;
|
||||
import java.nio.file.NoSuchFileException;
|
||||
import java.nio.file.Path;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
|
@ -475,22 +476,16 @@ public class TestDirectoryReader extends LuceneTestCase {
|
|||
if (dir instanceof BaseDirectoryWrapper) {
|
||||
((BaseDirectoryWrapper)dir).setCheckIndexOnClose(false); // we will hit NoSuchFileException in MDW since we nuked it!
|
||||
}
|
||||
try {
|
||||
DirectoryReader.open(dir);
|
||||
fail("expected FileNotFoundException/NoSuchFileException");
|
||||
} catch (FileNotFoundException | NoSuchFileException e) {
|
||||
// expected
|
||||
}
|
||||
expectThrowsAnyOf(Arrays.asList(FileNotFoundException.class, NoSuchFileException.class),
|
||||
() -> DirectoryReader.open(dir)
|
||||
);
|
||||
|
||||
IOUtils.rm(dirFile);
|
||||
|
||||
// Make sure we still get a CorruptIndexException (not NPE):
|
||||
try {
|
||||
DirectoryReader.open(dir);
|
||||
fail("expected FileNotFoundException/NoSuchFileException");
|
||||
} catch (FileNotFoundException | NoSuchFileException e) {
|
||||
// expected
|
||||
}
|
||||
expectThrowsAnyOf(Arrays.asList(FileNotFoundException.class, NoSuchFileException.class),
|
||||
() -> DirectoryReader.open(dir)
|
||||
);
|
||||
|
||||
dir.close();
|
||||
}
|
||||
|
|
|
@ -372,12 +372,7 @@ public class TestIndexableField extends LuceneTestCase {
|
|||
public void testNotIndexedTermVectors() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir);
|
||||
try {
|
||||
w.addDocument(Collections.<IndexableField>singletonList(new CustomField()));
|
||||
fail("didn't hit exception");
|
||||
} catch (IllegalArgumentException iae) {
|
||||
// expected
|
||||
}
|
||||
expectThrows(IllegalArgumentException.class, () -> w.addDocument(Collections.singletonList(new CustomField())));
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
|
|
@ -38,6 +38,7 @@ import org.apache.lucene.util.TestUtil;
|
|||
*/
|
||||
@SuppressFileSystems("ExtrasFS")
|
||||
public class TestSwappedIndexFiles extends LuceneTestCase {
|
||||
|
||||
public void test() throws Exception {
|
||||
Directory dir1 = newDirectory();
|
||||
Directory dir2 = newDirectory();
|
||||
|
@ -107,21 +108,15 @@ public class TestSwappedIndexFiles extends LuceneTestCase {
|
|||
dirCopy.sync(Collections.singleton(name));
|
||||
}
|
||||
|
||||
try {
|
||||
// NOTE: we .close so that if the test fails (truncation not detected) we don't also get all these confusing errors about open files:
|
||||
DirectoryReader.open(dirCopy).close();
|
||||
fail("wrong file " + victim + " not detected");
|
||||
} catch (CorruptIndexException | EOFException | IndexFormatTooOldException e) {
|
||||
// expected
|
||||
}
|
||||
// NOTE: we .close so that if the test fails (truncation not detected) we don't also get all these confusing errors about open files:
|
||||
expectThrowsAnyOf(Arrays.asList(CorruptIndexException.class, EOFException.class, IndexFormatTooOldException.class),
|
||||
() -> DirectoryReader.open(dirCopy).close()
|
||||
);
|
||||
|
||||
// CheckIndex should also fail:
|
||||
try {
|
||||
TestUtil.checkIndex(dirCopy, true, true, null);
|
||||
fail("wrong file " + victim + " not detected");
|
||||
} catch (CorruptIndexException | EOFException | IndexFormatTooOldException e) {
|
||||
// expected
|
||||
}
|
||||
expectThrowsAnyOf(Arrays.asList(CorruptIndexException.class, EOFException.class, IndexFormatTooOldException.class),
|
||||
() -> TestUtil.checkIndex(dirCopy, true, true, null)
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -283,4 +283,88 @@ public class TestBooleanScorer extends LuceneTestCase {
|
|||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testFilterConstantScore() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), dir);
|
||||
Document doc = new Document();
|
||||
doc.add(new StringField("foo", "bar", Store.NO));
|
||||
doc.add(new StringField("foo", "bat", Store.NO));
|
||||
doc.add(new StringField("foo", "baz", Store.NO));
|
||||
w.addDocument(doc);
|
||||
IndexReader reader = w.getReader();
|
||||
IndexSearcher searcher = new IndexSearcher(reader);
|
||||
searcher.setQueryCache(null);
|
||||
|
||||
{
|
||||
// single filter rewrites to a constant score query
|
||||
Query query = new BooleanQuery.Builder().add(new TermQuery(new Term("foo", "bar")), Occur.FILTER).build();
|
||||
Query rewrite = searcher.rewrite(query);
|
||||
assertTrue(rewrite instanceof BoostQuery);
|
||||
assertTrue(((BoostQuery) rewrite).getQuery() instanceof ConstantScoreQuery);
|
||||
}
|
||||
|
||||
Query[] queries = new Query[] {
|
||||
new BooleanQuery.Builder()
|
||||
.add(new TermQuery(new Term("foo", "bar")), Occur.FILTER)
|
||||
.add(new TermQuery(new Term("foo", "baz")), Occur.FILTER)
|
||||
.build(),
|
||||
new BooleanQuery.Builder()
|
||||
.add(new TermQuery(new Term("foo", "baz")), Occur.FILTER)
|
||||
// non-existing term
|
||||
.add(new TermQuery(new Term("foo", "arf")), Occur.SHOULD)
|
||||
.build(),
|
||||
new BooleanQuery.Builder()
|
||||
.add(new TermQuery(new Term("foo", "baz")), Occur.FILTER)
|
||||
.add(new TermQuery(new Term("foo", "baz")), Occur.FILTER)
|
||||
// non-existing term
|
||||
.add(new TermQuery(new Term("foo", "arf")), Occur.SHOULD)
|
||||
.add(new TermQuery(new Term("foo", "arw")), Occur.SHOULD)
|
||||
.build()
|
||||
};
|
||||
for (Query query : queries) {
|
||||
Query rewrite = searcher.rewrite(query);
|
||||
for (ScoreMode scoreMode : ScoreMode.values()) {
|
||||
Weight weight = searcher.createWeight(rewrite, scoreMode, 1f);
|
||||
Scorer scorer = weight.scorer(reader.leaves().get(0));
|
||||
if (scoreMode == ScoreMode.TOP_SCORES) {
|
||||
assertTrue(scorer instanceof ConstantScoreScorer);
|
||||
} else {
|
||||
assertFalse(scorer instanceof ConstantScoreScorer);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
queries = new Query[]{
|
||||
new BooleanQuery.Builder()
|
||||
.add(new TermQuery(new Term("foo", "bar")), Occur.FILTER)
|
||||
.add(new TermQuery(new Term("foo", "baz")), Occur.SHOULD)
|
||||
.build(),
|
||||
new BooleanQuery.Builder()
|
||||
.add(new TermQuery(new Term("foo", "bar")), Occur.FILTER)
|
||||
.add(new TermQuery(new Term("foo", "baz")), Occur.MUST)
|
||||
// non-existing term
|
||||
.add(new TermQuery(new Term("foo", "arf")), Occur.SHOULD)
|
||||
.build(),
|
||||
new BooleanQuery.Builder()
|
||||
// non-existing term
|
||||
.add(new TermQuery(new Term("foo", "bar")), Occur.FILTER)
|
||||
.add(new TermQuery(new Term("foo", "baz")), Occur.SHOULD)
|
||||
// non-existing term
|
||||
.add(new TermQuery(new Term("foo", "arf")), Occur.MUST)
|
||||
.build()
|
||||
};
|
||||
for (Query query : queries) {
|
||||
Query rewrite = searcher.rewrite(query);
|
||||
for (ScoreMode scoreMode : ScoreMode.values()) {
|
||||
Weight weight = searcher.createWeight(rewrite, scoreMode, 1f);
|
||||
Scorer scorer = weight.scorer(reader.leaves().get(0));
|
||||
assertFalse(scorer instanceof ConstantScoreScorer);
|
||||
}
|
||||
}
|
||||
|
||||
reader.close();
|
||||
w.close();
|
||||
dir.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -35,12 +35,9 @@ public final class TestRateLimiter extends LuceneTestCase {
|
|||
Thread t = new Thread() {
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
expectThrows(ThreadInterruptedException.class, () -> {
|
||||
new SimpleRateLimiter(1).pause((long) (1.5*Integer.MAX_VALUE*1024*1024/1000));
|
||||
fail("should have been interrupted");
|
||||
} catch (ThreadInterruptedException tie) {
|
||||
// expected
|
||||
}
|
||||
});
|
||||
}
|
||||
};
|
||||
t.start();
|
||||
|
|
|
@ -1201,19 +1201,19 @@ public class TestFSTs extends LuceneTestCase {
|
|||
private void checkStopNodes(FST<Long> fst, PositiveIntOutputs outputs) throws Exception {
|
||||
final Long nothing = outputs.getNoOutput();
|
||||
FST.Arc<Long> startArc = fst.getFirstArc(new FST.Arc<Long>());
|
||||
assertEquals(nothing, startArc.output);
|
||||
assertEquals(nothing, startArc.nextFinalOutput);
|
||||
assertEquals(nothing, startArc.output());
|
||||
assertEquals(nothing, startArc.nextFinalOutput());
|
||||
|
||||
FST.Arc<Long> arc = fst.readFirstTargetArc(startArc, new FST.Arc<Long>(),
|
||||
fst.getBytesReader());
|
||||
assertEquals('a', arc.label);
|
||||
assertEquals(17, arc.nextFinalOutput.longValue());
|
||||
assertEquals('a', arc.label());
|
||||
assertEquals(17, arc.nextFinalOutput().longValue());
|
||||
assertTrue(arc.isFinal());
|
||||
|
||||
arc = fst.readNextArc(arc, fst.getBytesReader());
|
||||
assertEquals('b', arc.label);
|
||||
assertEquals('b', arc.label());
|
||||
assertFalse(arc.isFinal());
|
||||
assertEquals(42, arc.output.longValue());
|
||||
assertEquals(42, arc.output().longValue());
|
||||
}
|
||||
|
||||
static final Comparator<Long> minLongComparator = new Comparator<Long> () {
|
||||
|
@ -1404,7 +1404,7 @@ public class TestFSTs extends LuceneTestCase {
|
|||
if (fst.findTargetArc((int) prefix.charAt(idx), arc, arc, reader) == null) {
|
||||
fail();
|
||||
}
|
||||
prefixOutput += arc.output;
|
||||
prefixOutput += arc.output();
|
||||
}
|
||||
|
||||
final int topN = TestUtil.nextInt(random, 1, 10);
|
||||
|
@ -1526,7 +1526,7 @@ public class TestFSTs extends LuceneTestCase {
|
|||
if (fst.findTargetArc((int) prefix.charAt(idx), arc, arc, reader) == null) {
|
||||
fail();
|
||||
}
|
||||
prefixOutput = outputs.add(prefixOutput, arc.output);
|
||||
prefixOutput = outputs.add(prefixOutput, arc.output());
|
||||
}
|
||||
|
||||
final int topN = TestUtil.nextInt(random, 1, 10);
|
||||
|
@ -1623,10 +1623,10 @@ public class TestFSTs extends LuceneTestCase {
|
|||
FST.BytesReader reader = fst.getBytesReader();
|
||||
arc = fst.findTargetArc((int) 'm', arc, arc, reader);
|
||||
assertNotNull(arc);
|
||||
assertEquals(new BytesRef("m"), arc.output);
|
||||
assertEquals(new BytesRef("m"), arc.output());
|
||||
|
||||
// NOTE: illegal:
|
||||
arc.output.length = 0;
|
||||
arc.output().length = 0;
|
||||
|
||||
fst.getFirstArc(arc);
|
||||
try {
|
||||
|
|
|
@ -29,38 +29,50 @@ import org.apache.lucene.util.BytesRef;
|
|||
import org.apache.lucene.util.IntsRefBuilder;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
||||
import org.junit.Before;
|
||||
|
||||
public class TestFstDirect extends LuceneTestCase {
|
||||
|
||||
private List<String> words;
|
||||
|
||||
@Before
|
||||
public void before() {
|
||||
words = new ArrayList<>();
|
||||
}
|
||||
|
||||
public void testDenseWithGap() throws Exception {
|
||||
//words.addAll(Arrays.asList("apple", "berry", "cherry", "damson", "fig", "grape"));
|
||||
words.addAll(Arrays.asList("ah", "bi", "cj", "dk", "fl", "gm"));
|
||||
final BytesRefFSTEnum<Object> fstEnum = new BytesRefFSTEnum<>(buildFST(words));
|
||||
List<String> words = Arrays.asList("ah", "bi", "cj", "dk", "fl", "gm");
|
||||
List<BytesRef> entries = new ArrayList<>();
|
||||
for (String word : words) {
|
||||
assertNotNull(word + " not found", fstEnum.seekExact(new BytesRef(word)));
|
||||
entries.add(new BytesRef(word.getBytes("ascii")));
|
||||
}
|
||||
final BytesRefFSTEnum<Object> fstEnum = new BytesRefFSTEnum<>(buildFST(entries));
|
||||
for (BytesRef entry : entries) {
|
||||
assertNotNull(entry.utf8ToString() + " not found", fstEnum.seekExact(entry));
|
||||
}
|
||||
}
|
||||
|
||||
public void testDeDupTails() throws Exception {
|
||||
List<BytesRef> entries = new ArrayList<>();
|
||||
for (int i = 0; i < 1000000; i += 4) {
|
||||
byte[] b = new byte[3];
|
||||
int val = i;
|
||||
for (int j = b.length - 1; j >= 0; --j) {
|
||||
b[j] = (byte) (val & 0xff);
|
||||
val >>= 8;
|
||||
}
|
||||
entries.add(new BytesRef(b));
|
||||
}
|
||||
long size = buildFST(entries).ramBytesUsed();
|
||||
// Size is 1664 when we use only list-encoding. We were previously failing to ever de-dup
|
||||
// arrays-with-gaps, which led this case to blow up.
|
||||
assertTrue(size < 3000);
|
||||
//printf("fst size = %d bytes", size);
|
||||
}
|
||||
|
||||
private FST<Object> buildFST(List<String> words) throws Exception {
|
||||
long start = System.nanoTime();
|
||||
private FST<Object> buildFST(List<BytesRef> entries) throws Exception {
|
||||
final Outputs<Object> outputs = NoOutputs.getSingleton();
|
||||
final Builder<Object> b = new Builder<>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, true, 15);
|
||||
|
||||
for (String word : words) {
|
||||
b.add(Util.toIntsRef(new BytesRef(word), new IntsRefBuilder()), outputs.getNoOutput());
|
||||
BytesRef last = null;
|
||||
for (BytesRef entry : entries) {
|
||||
if (entry.equals(last) == false) {
|
||||
b.add(Util.toIntsRef(entry, new IntsRefBuilder()), outputs.getNoOutput());
|
||||
}
|
||||
last = entry;
|
||||
}
|
||||
FST<Object> fst = b.finish();
|
||||
long t = System.nanoTime();
|
||||
printf("Built FST of %d bytes in %d ms", fst.ramBytesUsed(), nsToMs(t - start));
|
||||
return fst;
|
||||
}
|
||||
|
||||
|
@ -79,9 +91,9 @@ public class TestFstDirect extends LuceneTestCase {
|
|||
BytesRefFSTEnum<BytesRef> fstEnum = new BytesRefFSTEnum<>(fst);
|
||||
int sparseArrayArcCount = 0, directArrayArcCount = 0, listArcCount = 0;
|
||||
while(fstEnum.next() != null) {
|
||||
if (fstEnum.arcs[fstEnum.upto].bytesPerArc == 0) {
|
||||
if (fstEnum.arcs[fstEnum.upto].bytesPerArc() == 0) {
|
||||
listArcCount ++;
|
||||
} else if (fstEnum.arcs[fstEnum.upto].arcIdx == Integer.MIN_VALUE) {
|
||||
} else if (fstEnum.arcs[fstEnum.upto].arcIdx() == Integer.MIN_VALUE) {
|
||||
directArrayArcCount ++;
|
||||
} else {
|
||||
sparseArrayArcCount ++;
|
||||
|
|
|
@ -0,0 +1,115 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.lucene.util.fst;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IntsRefBuilder;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
||||
public class TestUtil extends LuceneTestCase {
|
||||
|
||||
public void testBinarySearch() throws Exception {
|
||||
// Creates a node with 8 arcs spanning (z-A) = 57 chars that will be encoded as a sparse array (no gaps)
|
||||
// requiring binary search
|
||||
List<String> letters = Arrays.asList("A", "E", "J", "K", "L", "O", "T", "z");
|
||||
FST<Object> fst = buildFST(letters, true);
|
||||
FST.Arc<Object> arc = fst.getFirstArc(new FST.Arc<>());
|
||||
arc = fst.readFirstTargetArc(arc, arc, fst.getBytesReader());
|
||||
for (int i = 0; i < letters.size(); i++) {
|
||||
assertEquals(i, Util.binarySearch(fst, arc, letters.get(i).charAt(0)));
|
||||
}
|
||||
// before the first
|
||||
assertEquals(-1, Util.binarySearch(fst, arc, ' '));
|
||||
// after the last
|
||||
assertEquals(-1 - letters.size(), Util.binarySearch(fst, arc, '~'));
|
||||
assertEquals(-2, Util.binarySearch(fst, arc, 'B'));
|
||||
assertEquals(-2, Util.binarySearch(fst, arc, 'C'));
|
||||
assertEquals(-7, Util.binarySearch(fst, arc, 'P'));
|
||||
}
|
||||
|
||||
public void testReadCeilArcPackedArray() throws Exception {
|
||||
List<String> letters = Arrays.asList("A", "E", "J", "K", "L", "O", "T", "z");
|
||||
verifyReadCeilArc(letters, true);
|
||||
}
|
||||
|
||||
public void testReadCeilArcArrayWithGaps() throws Exception {
|
||||
List<String> letters = Arrays.asList("A", "E", "J", "K", "L", "O", "T");
|
||||
verifyReadCeilArc(letters, true);
|
||||
}
|
||||
|
||||
public void testReadCeilArcList() throws Exception {
|
||||
List<String> letters = Arrays.asList("A", "E", "J", "K", "L", "O", "T", "z");
|
||||
verifyReadCeilArc(letters, false);
|
||||
}
|
||||
|
||||
private void verifyReadCeilArc(List<String> letters, boolean allowArrayArcs) throws Exception {
|
||||
FST<Object> fst = buildFST(letters, allowArrayArcs);
|
||||
FST.Arc<Object> first = fst.getFirstArc(new FST.Arc<>());
|
||||
FST.Arc<Object> arc = new FST.Arc<>();
|
||||
FST.BytesReader in = fst.getBytesReader();
|
||||
for (String letter : letters) {
|
||||
char c = letter.charAt(0);
|
||||
arc = Util.readCeilArc(c, fst, first, arc, in);
|
||||
assertNotNull(arc);
|
||||
assertEquals(c, arc.label());
|
||||
}
|
||||
// before the first
|
||||
assertEquals('A', Util.readCeilArc(' ', fst, first, arc, in).label());
|
||||
// after the last
|
||||
assertNull(Util.readCeilArc('~', fst, first, arc, in));
|
||||
// in the middle
|
||||
assertEquals('J', Util.readCeilArc('F', fst, first, arc, in).label());
|
||||
// no following arcs
|
||||
assertNull(Util.readCeilArc('Z', fst, arc, arc, in));
|
||||
}
|
||||
|
||||
private FST<Object> buildFST(List<String> words, boolean allowArrayArcs) throws Exception {
|
||||
final Outputs<Object> outputs = NoOutputs.getSingleton();
|
||||
final Builder<Object> b = new Builder<>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, allowArrayArcs, 15);
|
||||
|
||||
for (String word : words) {
|
||||
b.add(Util.toIntsRef(new BytesRef(word), new IntsRefBuilder()), outputs.getNoOutput());
|
||||
}
|
||||
return b.finish();
|
||||
}
|
||||
|
||||
private List<String> createRandomDictionary(int width, int depth) {
|
||||
return createRandomDictionary(new ArrayList<>(), new StringBuilder(), width, depth);
|
||||
}
|
||||
|
||||
private List<String> createRandomDictionary(List<String> dict, StringBuilder buf, int width, int depth) {
|
||||
char c = (char) random().nextInt(128);
|
||||
assert width < Character.MIN_SURROGATE / 8 - 128; // avoid surrogate chars
|
||||
int len = buf.length();
|
||||
for (int i = 0; i < width; i++) {
|
||||
buf.append(c);
|
||||
if (depth > 0) {
|
||||
createRandomDictionary(dict, buf, width, depth - 1);
|
||||
} else {
|
||||
dict.add(buf.toString());
|
||||
}
|
||||
c += random().nextInt(8);
|
||||
buf.setLength(len);
|
||||
}
|
||||
return dict;
|
||||
}
|
||||
|
||||
}
|
|
@ -47,10 +47,7 @@ public class TestCheckJoinIndex extends LuceneTestCase {
|
|||
w.close();
|
||||
BitSetProducer parentsFilter = new QueryBitSetProducer(new MatchNoDocsQuery());
|
||||
try {
|
||||
CheckJoinIndex.check(reader, parentsFilter);
|
||||
fail("Invalid index");
|
||||
} catch (IllegalStateException e) {
|
||||
// expected
|
||||
expectThrows(IllegalStateException.class, () -> CheckJoinIndex.check(reader, parentsFilter));
|
||||
} finally {
|
||||
reader.close();
|
||||
dir.close();
|
||||
|
@ -88,10 +85,7 @@ public class TestCheckJoinIndex extends LuceneTestCase {
|
|||
w.close();
|
||||
BitSetProducer parentsFilter = new QueryBitSetProducer(new TermQuery(new Term("parent", "true")));
|
||||
try {
|
||||
CheckJoinIndex.check(reader, parentsFilter);
|
||||
fail("Invalid index");
|
||||
} catch (IllegalStateException e) {
|
||||
// expected
|
||||
expectThrows(IllegalStateException.class, () -> CheckJoinIndex.check(reader, parentsFilter));
|
||||
} finally {
|
||||
reader.close();
|
||||
dir.close();
|
||||
|
@ -128,10 +122,7 @@ public class TestCheckJoinIndex extends LuceneTestCase {
|
|||
|
||||
BitSetProducer parentsFilter = new QueryBitSetProducer(new TermQuery(new Term("parent", "true")));
|
||||
try {
|
||||
CheckJoinIndex.check(reader, parentsFilter);
|
||||
fail("Invalid index");
|
||||
} catch (IllegalStateException e) {
|
||||
// expected
|
||||
expectThrows(IllegalStateException.class, () -> CheckJoinIndex.check(reader, parentsFilter));
|
||||
} finally {
|
||||
reader.close();
|
||||
dir.close();
|
||||
|
|
|
@ -33,6 +33,7 @@ import org.apache.lucene.luke.app.desktop.PreferencesFactory;
|
|||
import org.apache.lucene.luke.app.desktop.components.dialog.menubar.AboutDialogFactory;
|
||||
import org.apache.lucene.luke.app.desktop.components.dialog.menubar.CheckIndexDialogFactory;
|
||||
import org.apache.lucene.luke.app.desktop.components.dialog.menubar.CreateIndexDialogFactory;
|
||||
import org.apache.lucene.luke.app.desktop.components.dialog.menubar.ExportTermsDialogFactory;
|
||||
import org.apache.lucene.luke.app.desktop.components.dialog.menubar.OpenIndexDialogFactory;
|
||||
import org.apache.lucene.luke.app.desktop.components.dialog.menubar.OptimizeIndexDialogFactory;
|
||||
import org.apache.lucene.luke.app.desktop.util.DialogOpener;
|
||||
|
@ -57,6 +58,8 @@ public final class MenuBarProvider {
|
|||
|
||||
private final OptimizeIndexDialogFactory optimizeIndexDialogFactory;
|
||||
|
||||
private final ExportTermsDialogFactory exportTermsDialogFactory;
|
||||
|
||||
private final CheckIndexDialogFactory checkIndexDialogFactory;
|
||||
|
||||
private final AboutDialogFactory aboutDialogFactory;
|
||||
|
@ -81,6 +84,8 @@ public final class MenuBarProvider {
|
|||
|
||||
private final JMenuItem optimizeIndexMItem = new JMenuItem();
|
||||
|
||||
private final JMenuItem exportTermsMItem = new JMenuItem();
|
||||
|
||||
private final JMenuItem checkIndexMItem = new JMenuItem();
|
||||
|
||||
private final JMenuItem aboutMItem = new JMenuItem();
|
||||
|
@ -95,6 +100,7 @@ public final class MenuBarProvider {
|
|||
this.openIndexDialogFactory = OpenIndexDialogFactory.getInstance();
|
||||
this.createIndexDialogFactory = CreateIndexDialogFactory.getInstance();
|
||||
this.optimizeIndexDialogFactory = OptimizeIndexDialogFactory.getInstance();
|
||||
this.exportTermsDialogFactory = ExportTermsDialogFactory.getInstance();
|
||||
this.checkIndexDialogFactory = CheckIndexDialogFactory.getInstance();
|
||||
this.aboutDialogFactory = AboutDialogFactory.getInstance();
|
||||
|
||||
|
@ -173,6 +179,10 @@ public final class MenuBarProvider {
|
|||
checkIndexMItem.setEnabled(false);
|
||||
checkIndexMItem.addActionListener(listeners::showCheckIndexDialog);
|
||||
toolsMenu.add(checkIndexMItem);
|
||||
exportTermsMItem.setText(MessageUtils.getLocalizedMessage("menu.item.export.terms"));
|
||||
exportTermsMItem.setEnabled(false);
|
||||
exportTermsMItem.addActionListener(listeners::showExportTermsDialog);
|
||||
toolsMenu.add(exportTermsMItem);
|
||||
return toolsMenu;
|
||||
}
|
||||
|
||||
|
@ -258,6 +268,12 @@ public final class MenuBarProvider {
|
|||
});
|
||||
}
|
||||
|
||||
void showExportTermsDialog(ActionEvent e) {
|
||||
new DialogOpener<>(exportTermsDialogFactory).open("Export terms", 600, 400,
|
||||
factory -> {
|
||||
});
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private class Observer implements IndexObserver, DirectoryObserver {
|
||||
|
@ -267,6 +283,7 @@ public final class MenuBarProvider {
|
|||
reopenIndexMItem.setEnabled(false);
|
||||
closeIndexMItem.setEnabled(false);
|
||||
optimizeIndexMItem.setEnabled(false);
|
||||
exportTermsMItem.setEnabled(false);
|
||||
checkIndexMItem.setEnabled(true);
|
||||
}
|
||||
|
||||
|
@ -279,6 +296,7 @@ public final class MenuBarProvider {
|
|||
public void openIndex(LukeState state) {
|
||||
reopenIndexMItem.setEnabled(true);
|
||||
closeIndexMItem.setEnabled(true);
|
||||
exportTermsMItem.setEnabled(true);
|
||||
if (!state.readOnly() && state.hasDirectoryReader()) {
|
||||
optimizeIndexMItem.setEnabled(true);
|
||||
}
|
||||
|
@ -297,6 +315,7 @@ public final class MenuBarProvider {
|
|||
closeIndexMItem.setEnabled(false);
|
||||
optimizeIndexMItem.setEnabled(false);
|
||||
checkIndexMItem.setEnabled(false);
|
||||
exportTermsMItem.setEnabled(false);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,275 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.lucene.luke.app.desktop.components.dialog.menubar;
|
||||
|
||||
import javax.swing.BorderFactory;
|
||||
import javax.swing.BoxLayout;
|
||||
import javax.swing.JButton;
|
||||
import javax.swing.JComboBox;
|
||||
import javax.swing.JDialog;
|
||||
import javax.swing.JFileChooser;
|
||||
import javax.swing.JLabel;
|
||||
import javax.swing.JPanel;
|
||||
import javax.swing.JTextField;
|
||||
import javax.swing.SwingWorker;
|
||||
import java.awt.Color;
|
||||
import java.awt.Dialog;
|
||||
import java.awt.Dimension;
|
||||
import java.awt.FlowLayout;
|
||||
import java.awt.GridLayout;
|
||||
import java.awt.Insets;
|
||||
import java.awt.Window;
|
||||
import java.awt.event.ActionEvent;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.lang.invoke.MethodHandles;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.apache.lucene.luke.app.IndexHandler;
|
||||
import org.apache.lucene.luke.app.IndexObserver;
|
||||
import org.apache.lucene.luke.app.LukeState;
|
||||
import org.apache.lucene.luke.app.desktop.Preferences;
|
||||
import org.apache.lucene.luke.app.desktop.PreferencesFactory;
|
||||
import org.apache.lucene.luke.app.desktop.util.DialogOpener;
|
||||
import org.apache.lucene.luke.app.desktop.util.ImageUtils;
|
||||
import org.apache.lucene.luke.app.desktop.util.MessageUtils;
|
||||
import org.apache.lucene.luke.app.desktop.util.StyleConstants;
|
||||
import org.apache.lucene.luke.models.LukeException;
|
||||
import org.apache.lucene.luke.models.tools.IndexTools;
|
||||
import org.apache.lucene.luke.models.tools.IndexToolsFactory;
|
||||
import org.apache.lucene.luke.models.util.IndexUtils;
|
||||
import org.apache.lucene.luke.util.LoggerFactory;
|
||||
import org.apache.lucene.util.NamedThreadFactory;
|
||||
import org.apache.lucene.util.SuppressForbidden;
|
||||
|
||||
/**
|
||||
* Factory of export terms dialog
|
||||
*/
|
||||
public final class ExportTermsDialogFactory implements DialogOpener.DialogFactory {
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
|
||||
|
||||
private static ExportTermsDialogFactory instance;
|
||||
|
||||
private final IndexToolsFactory indexToolsFactory = new IndexToolsFactory();
|
||||
|
||||
private final Preferences prefs;
|
||||
|
||||
private final IndexHandler indexHandler;
|
||||
|
||||
private final JComboBox<String> fieldCombo = new JComboBox<String>();
|
||||
|
||||
private final JTextField destDir = new JTextField();
|
||||
|
||||
private final JLabel statusLbl = new JLabel();
|
||||
|
||||
private final JLabel indicatorLbl = new JLabel();
|
||||
|
||||
private final ListenerFunctions listeners = new ListenerFunctions();
|
||||
|
||||
private JDialog dialog;
|
||||
|
||||
private IndexTools toolsModel;
|
||||
|
||||
public synchronized static ExportTermsDialogFactory getInstance() throws IOException {
|
||||
if (instance == null) {
|
||||
instance = new ExportTermsDialogFactory();
|
||||
}
|
||||
return instance;
|
||||
}
|
||||
|
||||
private ExportTermsDialogFactory() throws IOException {
|
||||
this.prefs = PreferencesFactory.getInstance();
|
||||
this.indexHandler = IndexHandler.getInstance();
|
||||
indexHandler.addObserver(new Observer());
|
||||
}
|
||||
|
||||
@Override
|
||||
public JDialog create(Window owner, String title, int width, int height) {
|
||||
dialog = new JDialog(owner, title, Dialog.ModalityType.APPLICATION_MODAL);
|
||||
dialog.add(content());
|
||||
dialog.setSize(new Dimension(width, height));
|
||||
dialog.setLocationRelativeTo(owner);
|
||||
dialog.getContentPane().setBackground(prefs.getColorTheme().getBackgroundColor());
|
||||
return dialog;
|
||||
}
|
||||
|
||||
private JPanel content() {
|
||||
JPanel panel = new JPanel(new GridLayout(5, 1));
|
||||
panel.setOpaque(false);
|
||||
panel.setLayout(new BoxLayout(panel, BoxLayout.PAGE_AXIS));
|
||||
panel.setBorder(BorderFactory.createEmptyBorder(15, 15, 15, 15));
|
||||
|
||||
panel.add(currentOpenIndexPanel());
|
||||
panel.add(fieldComboPanel());
|
||||
panel.add(destinationDirPanel());
|
||||
panel.add(statusPanel());
|
||||
panel.add(actionButtonsPanel());
|
||||
|
||||
return panel;
|
||||
}
|
||||
|
||||
private JPanel currentOpenIndexPanel() {
|
||||
JPanel panel = new JPanel(new FlowLayout(FlowLayout.LEADING));
|
||||
panel.setBorder(BorderFactory.createEmptyBorder());
|
||||
panel.setOpaque(false);
|
||||
JLabel label = new JLabel(MessageUtils.getLocalizedMessage("export.terms.label.index_path"));
|
||||
JLabel value = new JLabel(indexHandler.getState().getIndexPath());
|
||||
value.setToolTipText(indexHandler.getState().getIndexPath());
|
||||
panel.add(label);
|
||||
panel.add(value);
|
||||
return panel;
|
||||
}
|
||||
|
||||
private JPanel fieldComboPanel() {
|
||||
JPanel panel = new JPanel(new GridLayout(2, 1));
|
||||
panel.setOpaque(false);
|
||||
panel.add(new JLabel(MessageUtils.getLocalizedMessage("export.terms.field")));
|
||||
panel.add(fieldCombo);
|
||||
return panel;
|
||||
}
|
||||
|
||||
private JPanel destinationDirPanel() {
|
||||
JPanel panel = new JPanel(new GridLayout(2, 1));
|
||||
panel.setOpaque(false);
|
||||
|
||||
panel.add(new JLabel(MessageUtils.getLocalizedMessage("export.terms.label.output_path")));
|
||||
|
||||
JPanel inputPanel = new JPanel(new FlowLayout(FlowLayout.LEADING));
|
||||
inputPanel.setBorder(BorderFactory.createEmptyBorder());
|
||||
inputPanel.setOpaque(false);
|
||||
destDir.setText(System.getProperty("user.home"));
|
||||
destDir.setColumns(60);
|
||||
destDir.setPreferredSize(new Dimension(200, 30));
|
||||
destDir.setFont(StyleConstants.FONT_MONOSPACE_LARGE);
|
||||
destDir.setEditable(false);
|
||||
destDir.setBackground(Color.white);
|
||||
inputPanel.add(destDir);
|
||||
|
||||
JButton browseBtn = new JButton(MessageUtils.getLocalizedMessage("export.terms.button.browse"));
|
||||
browseBtn.setFont(StyleConstants.FONT_BUTTON_LARGE);
|
||||
browseBtn.setMargin(new Insets(3, 0, 3, 0));
|
||||
browseBtn.addActionListener(listeners::browseDirectory);
|
||||
inputPanel.add(browseBtn);
|
||||
|
||||
panel.add(inputPanel);
|
||||
return panel;
|
||||
}
|
||||
|
||||
private JPanel actionButtonsPanel() {
|
||||
// Buttons
|
||||
JPanel execButtons = new JPanel(new FlowLayout(FlowLayout.TRAILING));
|
||||
execButtons.setOpaque(false);
|
||||
JButton exportBtn = new JButton(MessageUtils.getLocalizedMessage("export.terms.button.export"));
|
||||
exportBtn.setMargin(new Insets(3, 0, 3, 0));
|
||||
exportBtn.addActionListener(listeners::export);
|
||||
execButtons.add(exportBtn);
|
||||
JButton closeBtn = new JButton(MessageUtils.getLocalizedMessage("button.close"));
|
||||
closeBtn.setMargin(new Insets(3, 0, 3, 0));
|
||||
closeBtn.addActionListener(e -> dialog.dispose());
|
||||
execButtons.add(closeBtn);
|
||||
return execButtons;
|
||||
}
|
||||
|
||||
private JPanel statusPanel() {
|
||||
JPanel status = new JPanel(new FlowLayout(FlowLayout.LEADING));
|
||||
status.setOpaque(false);
|
||||
indicatorLbl.setIcon(ImageUtils.createImageIcon("indicator.gif", 20, 20));
|
||||
indicatorLbl.setVisible(false);
|
||||
status.add(statusLbl);
|
||||
status.add(indicatorLbl);
|
||||
return status;
|
||||
}
|
||||
|
||||
private class ListenerFunctions {
|
||||
|
||||
@SuppressForbidden(reason = "JFilechooser#getSelectedFile() returns java.io.File")
|
||||
void browseDirectory(ActionEvent e) {
|
||||
JFileChooser fileChooser = new JFileChooser();
|
||||
fileChooser.setFileSelectionMode(JFileChooser.DIRECTORIES_ONLY);
|
||||
fileChooser.setFileHidingEnabled(false);
|
||||
int retVal = fileChooser.showOpenDialog(dialog);
|
||||
if (retVal == JFileChooser.APPROVE_OPTION) {
|
||||
File f = fileChooser.getSelectedFile();
|
||||
destDir.setText(f.getAbsolutePath());
|
||||
}
|
||||
}
|
||||
|
||||
void export(ActionEvent e) {
|
||||
ExecutorService executor = Executors.newSingleThreadExecutor(new NamedThreadFactory("export-terms-dialog"));
|
||||
|
||||
SwingWorker<Void, Void> task = new SwingWorker<Void, Void>() {
|
||||
|
||||
String filename;
|
||||
|
||||
@Override
|
||||
protected Void doInBackground() {
|
||||
setProgress(0);
|
||||
statusLbl.setText("Exporting...");
|
||||
indicatorLbl.setVisible(true);
|
||||
String field = (String) fieldCombo.getSelectedItem();
|
||||
String directory = destDir.getText();
|
||||
try {
|
||||
filename = toolsModel.exportTerms(directory, field);
|
||||
} catch (LukeException e) {
|
||||
log.error("Error while exporting terms from field " + field, e);
|
||||
statusLbl.setText(MessageUtils.getLocalizedMessage("export.terms.label.error", e.getMessage()));
|
||||
} catch (Exception e) {
|
||||
log.error("Error while exporting terms from field " + field, e);
|
||||
statusLbl.setText(MessageUtils.getLocalizedMessage("message.error.unknown"));
|
||||
throw e;
|
||||
} finally {
|
||||
setProgress(100);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void done() {
|
||||
indicatorLbl.setVisible(false);
|
||||
if (filename != null) {
|
||||
statusLbl.setText(MessageUtils.getLocalizedMessage("export.terms.label.success", filename, "[term],[doc frequency]"));
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
executor.submit(task);
|
||||
executor.shutdown();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private class Observer implements IndexObserver {
|
||||
|
||||
@Override
|
||||
public void openIndex(LukeState state) {
|
||||
toolsModel = indexToolsFactory.newInstance(state.getIndexReader(), state.useCompound(), state.keepAllCommits());
|
||||
IndexUtils.getFieldNames(state.getIndexReader()).stream().sorted().forEach(fieldCombo::addItem);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void closeIndex() {
|
||||
fieldCombo.removeAllItems();
|
||||
toolsModel = null;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -94,4 +94,13 @@ public interface IndexTools {
|
|||
* @param dataDir - the directory path which contains sample documents (20 Newsgroups).
|
||||
*/
|
||||
void createNewIndex(String dataDir);
|
||||
|
||||
|
||||
/**
|
||||
* Export terms from given field into a new file on the destination directory
|
||||
* @param destDir - destination directory
|
||||
* @param field - field name
|
||||
* @return The file containing the export
|
||||
*/
|
||||
String exportTerms(String destDir, String field);
|
||||
}
|
||||
|
|
|
@ -17,11 +17,15 @@
|
|||
|
||||
package org.apache.lucene.luke.models.tools;
|
||||
|
||||
import java.io.BufferedWriter;
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
import java.nio.charset.Charset;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Objects;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
|
@ -30,6 +34,9 @@ import org.apache.lucene.index.CheckIndex;
|
|||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.MultiTerms;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.luke.models.LukeException;
|
||||
import org.apache.lucene.luke.models.LukeModel;
|
||||
import org.apache.lucene.luke.models.util.IndexUtils;
|
||||
|
@ -37,6 +44,7 @@ import org.apache.lucene.luke.models.util.twentynewsgroups.Message;
|
|||
import org.apache.lucene.luke.models.util.twentynewsgroups.MessageFilesParser;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/** Default implementation of {@link IndexTools} */
|
||||
public final class IndexToolsImpl extends LukeModel implements IndexTools {
|
||||
|
@ -184,4 +192,25 @@ public final class IndexToolsImpl extends LukeModel implements IndexTools {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
public String exportTerms(String destDir, String field) {
|
||||
String filename = "terms_" + field + "_" + System.currentTimeMillis() + ".out";
|
||||
Path path = Paths.get(destDir, filename);
|
||||
try {
|
||||
Terms terms = MultiTerms.getTerms(reader, field);
|
||||
if (terms == null) {
|
||||
throw new LukeException(String.format(Locale.US, "Field %s does not contain any terms to be exported", field));
|
||||
}
|
||||
try (BufferedWriter writer = Files.newBufferedWriter(path, Charset.forName("UTF-8"))) {
|
||||
TermsEnum termsEnum = terms.iterator();
|
||||
BytesRef term;
|
||||
while (!Thread.currentThread().isInterrupted() && (term = termsEnum.next()) != null) {
|
||||
writer.write(String.format(Locale.US, "%s,%d\n", term.utf8ToString(), +termsEnum.docFreq()));
|
||||
}
|
||||
return path.toString();
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new LukeException("Terms file export for field [" + field + "] to file [" + filename + "] has failed.", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -51,6 +51,7 @@ menu.item.create_index=Create new index
|
|||
menu.item.close_index=Close index
|
||||
menu.item.exit=Exit
|
||||
menu.item.optimize=Optimize index
|
||||
menu.item.export.terms=Export terms
|
||||
menu.item.check_index=Check index
|
||||
menu.item.theme_gray=Gray
|
||||
menu.item.theme_classic=Classic
|
||||
|
@ -83,6 +84,15 @@ createindex.label.data_link=http://kdd.ics.uci.edu/databases/20newsgroups/20news
|
|||
createindex.label.datadir=Data directory:
|
||||
createindex.textarea.data_help1=You can index sample documents from 20 Newsgroups corpus that is available at here:
|
||||
createindex.textarea.data_help2=Download and extract the tgz file, then select the extracted directory path.\nCreating an index with the full size corpus takes some time... :)
|
||||
# Export terms
|
||||
export.terms.label.index_path=Index directory path:
|
||||
export.terms.label.output_path=Output directory path:
|
||||
export.terms.field=Field to export terms from:
|
||||
export.terms.button.export=Export
|
||||
export.terms.button.browse=Browse
|
||||
export.terms.label.success=<html>Terms successfully exported to: <br>{0}<br><br>Output format is: {1}</html>
|
||||
export.terms.label.error=<html>Failed to export: <br>{0}</html>
|
||||
|
||||
# Optimize index
|
||||
optimize.dialog.title=Optimize index
|
||||
optimize.label.index_path=Index directory path:
|
||||
|
|
|
@ -169,18 +169,8 @@ public class TestQueryParser extends QueryParserTestBase {
|
|||
// doesn't work for some reason.
|
||||
@SuppressWarnings("rawtype")
|
||||
public void testProtectedCtors() throws Exception {
|
||||
try {
|
||||
QueryParser.class.getConstructor(CharStream.class);
|
||||
fail("please switch public QueryParser(CharStream) to be protected");
|
||||
} catch (NoSuchMethodException nsme) {
|
||||
// expected
|
||||
}
|
||||
try {
|
||||
QueryParser.class.getConstructor(QueryParserTokenManager.class);
|
||||
fail("please switch public QueryParser(QueryParserTokenManager) to be protected");
|
||||
} catch (NoSuchMethodException nsme) {
|
||||
// expected
|
||||
}
|
||||
expectThrows(NoSuchMethodException.class, () -> QueryParser.class.getConstructor(CharStream.class));
|
||||
expectThrows(NoSuchMethodException.class, () -> QueryParser.class.getConstructor(QueryParserTokenManager.class));
|
||||
}
|
||||
|
||||
public void testFuzzySlopeExtendability() throws ParseException {
|
||||
|
|
|
@ -24,7 +24,15 @@ import java.util.GregorianCalendar;
|
|||
import java.util.Locale;
|
||||
import java.util.TimeZone;
|
||||
|
||||
import org.apache.lucene.analysis.*;
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.LowerCaseFilter;
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.analysis.MockSynonymFilter;
|
||||
import org.apache.lucene.analysis.MockTokenFilter;
|
||||
import org.apache.lucene.analysis.MockTokenizer;
|
||||
import org.apache.lucene.analysis.TokenFilter;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.Tokenizer;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.document.DateTools;
|
||||
|
@ -34,17 +42,27 @@ import org.apache.lucene.index.DirectoryReader;
|
|||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.Term;
|
||||
//import org.apache.lucene.queryparser.classic.CharStream;
|
||||
//import org.apache.lucene.queryparser.classic.ParseException;
|
||||
//import org.apache.lucene.queryparser.classic.QueryParser;
|
||||
//import org.apache.lucene.queryparser.classic.QueryParserBase;
|
||||
import org.apache.lucene.queryparser.classic.QueryParser;
|
||||
import org.apache.lucene.queryparser.classic.QueryParserBase;
|
||||
//import org.apache.lucene.queryparser.classic.QueryParserTokenManager;
|
||||
import org.apache.lucene.queryparser.classic.TestQueryParser;
|
||||
import org.apache.lucene.queryparser.flexible.standard.CommonQueryParserConfiguration;
|
||||
import org.apache.lucene.search.*;
|
||||
import org.apache.lucene.search.BooleanClause;
|
||||
import org.apache.lucene.search.BooleanClause.Occur;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
import org.apache.lucene.search.BoostQuery;
|
||||
import org.apache.lucene.search.FuzzyQuery;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.MatchAllDocsQuery;
|
||||
import org.apache.lucene.search.MatchNoDocsQuery;
|
||||
import org.apache.lucene.search.MultiTermQuery;
|
||||
import org.apache.lucene.search.PhraseQuery;
|
||||
import org.apache.lucene.search.PrefixQuery;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.RegexpQuery;
|
||||
import org.apache.lucene.search.ScoreDoc;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.search.TermRangeQuery;
|
||||
import org.apache.lucene.search.WildcardQuery;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.automaton.Automata;
|
||||
|
@ -484,23 +502,16 @@ public abstract class QueryParserTestBase extends LuceneTestCase {
|
|||
// Range queries:
|
||||
assertWildcardQueryEquals("[A TO C]", "[a TO c]");
|
||||
// Test suffix queries: first disallow
|
||||
try {
|
||||
Exception ex = expectThrows(Exception.class, () -> {
|
||||
assertWildcardQueryEquals("*Term", "*term", false);
|
||||
} catch(Exception pe) {
|
||||
// expected exception
|
||||
if(!isQueryParserException(pe)){
|
||||
fail();
|
||||
}
|
||||
}
|
||||
try {
|
||||
});
|
||||
assertTrue(isQueryParserException(ex));
|
||||
|
||||
ex = expectThrows(Exception.class, () -> {
|
||||
assertWildcardQueryEquals("?Term", "?term");
|
||||
fail();
|
||||
} catch(Exception pe) {
|
||||
// expected exception
|
||||
if(!isQueryParserException(pe)){
|
||||
fail();
|
||||
}
|
||||
}
|
||||
});
|
||||
assertTrue(isQueryParserException(ex));
|
||||
|
||||
// Test suffix queries: then allow
|
||||
assertWildcardQueryEquals("*Term", "*term", true);
|
||||
assertWildcardQueryEquals("?Term", "?term", true);
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.lucene.replicator;
|
|||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.nio.file.NoSuchFileException;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map.Entry;
|
||||
|
@ -141,12 +142,9 @@ public class LocalReplicatorTest extends ReplicatorTestCase {
|
|||
public void testObtainMissingFile() throws IOException {
|
||||
replicator.publish(createRevision(1));
|
||||
SessionToken res = replicator.checkForUpdate(null);
|
||||
try {
|
||||
expectThrowsAnyOf(Arrays.asList(FileNotFoundException.class, NoSuchFileException.class), () -> {
|
||||
replicator.obtainFile(res.id, res.sourceFiles.keySet().iterator().next(), "madeUpFile");
|
||||
fail("should have failed obtaining an unrecognized file");
|
||||
} catch (FileNotFoundException | NoSuchFileException e) {
|
||||
// expected
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -136,14 +136,9 @@ public class HttpReplicatorTest extends ReplicatorTestCase {
|
|||
|
||||
try {
|
||||
publishRevision(5);
|
||||
|
||||
try {
|
||||
replicationServlet.setRespondWithError(true);
|
||||
client.updateNow();
|
||||
fail("expected exception");
|
||||
} catch (Throwable t) {
|
||||
// expected
|
||||
}
|
||||
|
||||
replicationServlet.setRespondWithError(true);
|
||||
expectThrows(Exception.class, client::updateNow);
|
||||
|
||||
replicationServlet.setRespondWithError(false);
|
||||
client.updateNow(); // now it should work
|
||||
|
|
|
@ -277,7 +277,7 @@ public final class IDVersionSegmentTermsEnum extends BaseTermsEnum {
|
|||
|
||||
arc = arcs[0];
|
||||
assert arc.isFinal();
|
||||
output = arc.output;
|
||||
output = arc.output();
|
||||
targetUpto = 0;
|
||||
|
||||
IDVersionSegmentTermsEnumFrame lastFrame = stack[0];
|
||||
|
@ -303,9 +303,9 @@ public final class IDVersionSegmentTermsEnum extends BaseTermsEnum {
|
|||
//if (arc.label != (target.bytes[target.offset + targetUpto] & 0xFF)) {
|
||||
//System.out.println("FAIL: arc.label=" + (char) arc.label + " targetLabel=" + (char) (target.bytes[target.offset + targetUpto] & 0xFF));
|
||||
//}
|
||||
assert arc.label == (target.bytes[target.offset + targetUpto] & 0xFF): "arc.label=" + (char) arc.label + " targetLabel=" + (char) (target.bytes[target.offset + targetUpto] & 0xFF);
|
||||
if (arc.output != VersionBlockTreeTermsWriter.NO_OUTPUT) {
|
||||
output = VersionBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.output);
|
||||
assert arc.label() == (target.bytes[target.offset + targetUpto] & 0xFF): "arc.label=" + (char) arc.label() + " targetLabel=" + (char) (target.bytes[target.offset + targetUpto] & 0xFF);
|
||||
if (arc.output() != VersionBlockTreeTermsWriter.NO_OUTPUT) {
|
||||
output = VersionBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.output());
|
||||
}
|
||||
if (arc.isFinal()) {
|
||||
lastFrame = stack[1+lastFrame.ord];
|
||||
|
@ -404,19 +404,19 @@ public final class IDVersionSegmentTermsEnum extends BaseTermsEnum {
|
|||
|
||||
// Empty string prefix must have an output (block) in the index!
|
||||
assert arc.isFinal();
|
||||
assert arc.output != null;
|
||||
assert arc.output() != null;
|
||||
|
||||
// if (DEBUG) {
|
||||
// System.out.println(" no seek state; push root frame");
|
||||
// }
|
||||
|
||||
output = arc.output;
|
||||
output = arc.output();
|
||||
|
||||
currentFrame = staticFrame;
|
||||
|
||||
//term.length = 0;
|
||||
targetUpto = 0;
|
||||
currentFrame = pushFrame(arc, VersionBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.nextFinalOutput), 0);
|
||||
currentFrame = pushFrame(arc, VersionBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.nextFinalOutput()), 0);
|
||||
}
|
||||
|
||||
// if (DEBUG) {
|
||||
|
@ -517,9 +517,9 @@ public final class IDVersionSegmentTermsEnum extends BaseTermsEnum {
|
|||
termExists = false;
|
||||
}
|
||||
// Aggregate output as we go:
|
||||
assert arc.output != null;
|
||||
if (arc.output != VersionBlockTreeTermsWriter.NO_OUTPUT) {
|
||||
output = VersionBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.output);
|
||||
assert arc.output() != null;
|
||||
if (arc.output() != VersionBlockTreeTermsWriter.NO_OUTPUT) {
|
||||
output = VersionBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.output());
|
||||
}
|
||||
|
||||
// if (DEBUG) {
|
||||
|
@ -529,7 +529,7 @@ public final class IDVersionSegmentTermsEnum extends BaseTermsEnum {
|
|||
|
||||
if (arc.isFinal()) {
|
||||
// if (DEBUG) System.out.println(" arc is final!");
|
||||
currentFrame = pushFrame(arc, VersionBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.nextFinalOutput), targetUpto);
|
||||
currentFrame = pushFrame(arc, VersionBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.nextFinalOutput()), targetUpto);
|
||||
// if (DEBUG) System.out.println(" curFrame.ord=" + currentFrame.ord + " hasTerms=" + currentFrame.hasTerms);
|
||||
}
|
||||
}
|
||||
|
@ -619,7 +619,7 @@ public final class IDVersionSegmentTermsEnum extends BaseTermsEnum {
|
|||
|
||||
arc = arcs[0];
|
||||
assert arc.isFinal();
|
||||
output = arc.output;
|
||||
output = arc.output();
|
||||
targetUpto = 0;
|
||||
|
||||
IDVersionSegmentTermsEnumFrame lastFrame = stack[0];
|
||||
|
@ -642,14 +642,14 @@ public final class IDVersionSegmentTermsEnum extends BaseTermsEnum {
|
|||
break;
|
||||
}
|
||||
arc = arcs[1+targetUpto];
|
||||
assert arc.label == (target.bytes[target.offset + targetUpto] & 0xFF): "arc.label=" + (char) arc.label + " targetLabel=" + (char) (target.bytes[target.offset + targetUpto] & 0xFF);
|
||||
assert arc.label() == (target.bytes[target.offset + targetUpto] & 0xFF): "arc.label=" + (char) arc.label() + " targetLabel=" + (char) (target.bytes[target.offset + targetUpto] & 0xFF);
|
||||
// TODO: we could save the outputs in local
|
||||
// byte[][] instead of making new objs ever
|
||||
// seek; but, often the FST doesn't have any
|
||||
// shared bytes (but this could change if we
|
||||
// reverse vLong byte order)
|
||||
if (arc.output != VersionBlockTreeTermsWriter.NO_OUTPUT) {
|
||||
output = VersionBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.output);
|
||||
if (arc.output() != VersionBlockTreeTermsWriter.NO_OUTPUT) {
|
||||
output = VersionBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.output());
|
||||
}
|
||||
if (arc.isFinal()) {
|
||||
lastFrame = stack[1+lastFrame.ord];
|
||||
|
@ -722,19 +722,19 @@ public final class IDVersionSegmentTermsEnum extends BaseTermsEnum {
|
|||
|
||||
// Empty string prefix must have an output (block) in the index!
|
||||
assert arc.isFinal();
|
||||
assert arc.output != null;
|
||||
assert arc.output() != null;
|
||||
|
||||
//if (DEBUG) {
|
||||
//System.out.println(" no seek state; push root frame");
|
||||
//}
|
||||
|
||||
output = arc.output;
|
||||
output = arc.output();
|
||||
|
||||
currentFrame = staticFrame;
|
||||
|
||||
//term.length = 0;
|
||||
targetUpto = 0;
|
||||
currentFrame = pushFrame(arc, VersionBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.nextFinalOutput), 0);
|
||||
currentFrame = pushFrame(arc, VersionBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.nextFinalOutput()), 0);
|
||||
}
|
||||
|
||||
//if (DEBUG) {
|
||||
|
@ -789,9 +789,9 @@ public final class IDVersionSegmentTermsEnum extends BaseTermsEnum {
|
|||
term.setByteAt(targetUpto, (byte) targetLabel);
|
||||
arc = nextArc;
|
||||
// Aggregate output as we go:
|
||||
assert arc.output != null;
|
||||
if (arc.output != VersionBlockTreeTermsWriter.NO_OUTPUT) {
|
||||
output = VersionBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.output);
|
||||
assert arc.output() != null;
|
||||
if (arc.output() != VersionBlockTreeTermsWriter.NO_OUTPUT) {
|
||||
output = VersionBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.output());
|
||||
}
|
||||
|
||||
//if (DEBUG) {
|
||||
|
@ -801,7 +801,7 @@ public final class IDVersionSegmentTermsEnum extends BaseTermsEnum {
|
|||
|
||||
if (arc.isFinal()) {
|
||||
//if (DEBUG) System.out.println(" arc is final!");
|
||||
currentFrame = pushFrame(arc, VersionBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.nextFinalOutput), targetUpto);
|
||||
currentFrame = pushFrame(arc, VersionBlockTreeTermsWriter.FST_OUTPUTS.add(output, arc.nextFinalOutput()), targetUpto);
|
||||
//if (DEBUG) System.out.println(" curFrame.ord=" + currentFrame.ord + " hasTerms=" + currentFrame.hasTerms);
|
||||
}
|
||||
}
|
||||
|
@ -854,8 +854,8 @@ public final class IDVersionSegmentTermsEnum extends BaseTermsEnum {
|
|||
}
|
||||
if (fr.index != null) {
|
||||
assert !isSeekFrame || f.arc != null: "isSeekFrame=" + isSeekFrame + " f.arc=" + f.arc;
|
||||
if (f.prefix > 0 && isSeekFrame && f.arc.label != (term.byteAt(f.prefix-1)&0xFF)) {
|
||||
out.println(" broken seek state: arc.label=" + (char) f.arc.label + " vs term byte=" + (char) (term.byteAt(f.prefix-1)&0xFF));
|
||||
if (f.prefix > 0 && isSeekFrame && f.arc.label() != (term.byteAt(f.prefix-1)&0xFF)) {
|
||||
out.println(" broken seek state: arc.label=" + (char) f.arc.label() + " vs term byte=" + (char) (term.byteAt(f.prefix-1)&0xFF));
|
||||
throw new RuntimeException("seek state is broken");
|
||||
}
|
||||
Pair<BytesRef,Long> output = Util.get(fr.index, prefix);
|
||||
|
|
|
@ -747,12 +747,7 @@ public class TestTermAutomatonQuery extends LuceneTestCase {
|
|||
TermAutomatonQuery q = new TermAutomatonQuery("field");
|
||||
int initState = q.createState();
|
||||
q.setAccept(initState, true);
|
||||
try {
|
||||
q.finish();
|
||||
fail("did not hit exc");
|
||||
} catch (IllegalStateException ise) {
|
||||
// expected
|
||||
}
|
||||
expectThrows(IllegalStateException.class, q::finish);
|
||||
}
|
||||
|
||||
public void testRewriteNoMatch() throws Exception {
|
||||
|
|
|
@ -727,7 +727,7 @@ public class AnalyzingSuggester extends Lookup implements Accountable {
|
|||
if (fst.findTargetArc(END_BYTE, path.fstNode, scratchArc, bytesReader) != null) {
|
||||
// This node has END_BYTE arc leaving, meaning it's an
|
||||
// "exact" match:
|
||||
searcher.addStartPaths(scratchArc, fst.outputs.add(path.output, scratchArc.output), false, path.input);
|
||||
searcher.addStartPaths(scratchArc, fst.outputs.add(path.output, scratchArc.output()), false, path.input);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -107,7 +107,7 @@ public class FSTUtil {
|
|||
newInput.append(t.min);
|
||||
queue.add(new Path<>(t.dest, new FST.Arc<T>()
|
||||
.copyFrom(nextArc), fst.outputs
|
||||
.add(path.output, nextArc.output), newInput));
|
||||
.add(path.output, nextArc.output()), newInput));
|
||||
}
|
||||
} else {
|
||||
// TODO: if this transition's TO state is accepting, and
|
||||
|
@ -119,21 +119,21 @@ public class FSTUtil {
|
|||
// done in AnalyzingSuggester).
|
||||
FST.Arc<T> nextArc = Util.readCeilArc(min, fst, path.fstNode,
|
||||
scratchArc, fstReader);
|
||||
while (nextArc != null && nextArc.label <= max) {
|
||||
assert nextArc.label <= max;
|
||||
assert nextArc.label >= min : nextArc.label + " "
|
||||
while (nextArc != null && nextArc.label() <= max) {
|
||||
assert nextArc.label() <= max;
|
||||
assert nextArc.label() >= min : nextArc.label() + " "
|
||||
+ min;
|
||||
final IntsRefBuilder newInput = new IntsRefBuilder();
|
||||
newInput.copyInts(currentInput.get());
|
||||
newInput.append(nextArc.label);
|
||||
newInput.append(nextArc.label());
|
||||
queue.add(new Path<>(t.dest, new FST.Arc<T>()
|
||||
.copyFrom(nextArc), fst.outputs
|
||||
.add(path.output, nextArc.output), newInput));
|
||||
final int label = nextArc.label; // used in assert
|
||||
.add(path.output, nextArc.output()), newInput));
|
||||
final int label = nextArc.label(); // used in assert
|
||||
nextArc = nextArc.isLast() ? null : fst.readNextRealArc(nextArc,
|
||||
fstReader);
|
||||
assert nextArc == null || label < nextArc.label : "last: " + label
|
||||
+ " next: " + nextArc.label;
|
||||
assert nextArc == null || label < nextArc.label() : "last: " + label
|
||||
+ " next: " + nextArc.label();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -599,7 +599,7 @@ public class FreeTextSuggester extends Lookup implements Accountable {
|
|||
|
||||
@Override
|
||||
protected void addIfCompetitive(Util.FSTPath<Long> path) {
|
||||
if (path.arc.label != separator) {
|
||||
if (path.arc.label() != separator) {
|
||||
//System.out.println(" keep path: " + Util.toBytesRef(path.input, new BytesRef()).utf8ToString() + "; " + path + "; arc=" + path.arc);
|
||||
super.addIfCompetitive(path);
|
||||
} else {
|
||||
|
@ -718,7 +718,7 @@ public class FreeTextSuggester extends Lookup implements Accountable {
|
|||
if (fst.findTargetArc(bytes[pos++] & 0xff, arc, arc, bytesReader) == null) {
|
||||
return null;
|
||||
} else {
|
||||
output = fst.outputs.add(output, arc.output);
|
||||
output = fst.outputs.add(output, arc.output());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -159,7 +159,7 @@ public final class NRTSuggester implements Accountable {
|
|||
// We are removing dups
|
||||
if (path.payload == -1) {
|
||||
// This path didn't yet see the complete surface form; let's see if it just did with the arc output we just added:
|
||||
BytesRef arcOutput = path.arc.output.output2;
|
||||
BytesRef arcOutput = path.arc.output().output2;
|
||||
BytesRef output = path.output.output2;
|
||||
for(int i=0;i<arcOutput.length;i++) {
|
||||
if (arcOutput.bytes[arcOutput.offset + i] == payloadSep) {
|
||||
|
|
|
@ -180,9 +180,9 @@ public class FSTCompletion {
|
|||
// Descend into the automaton using the key as prefix.
|
||||
if (descendWithPrefix(arc, utf8)) {
|
||||
automaton.readFirstTargetArc(arc, arc, fstReader);
|
||||
if (arc.label == FST.END_LABEL) {
|
||||
if (arc.label() == FST.END_LABEL) {
|
||||
// Normalize prefix-encoded weight.
|
||||
return rootArc.label;
|
||||
return rootArc.label();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -273,7 +273,7 @@ public class FSTCompletion {
|
|||
// of the key prefix. The arc we're at is the last key's byte,
|
||||
// so we will collect it too.
|
||||
output.length = key.length - 1;
|
||||
if (collect(res, num, rootArc.label, output, arc) && !collectAll) {
|
||||
if (collect(res, num, rootArc.label(), output, arc) && !collectAll) {
|
||||
// We have enough suggestions to return immediately. Keep on looking
|
||||
// for an
|
||||
// exact match, if requested.
|
||||
|
@ -360,11 +360,11 @@ public class FSTCompletion {
|
|||
output.bytes = ArrayUtil.grow(output.bytes);
|
||||
}
|
||||
assert output.offset == 0;
|
||||
output.bytes[output.length++] = (byte) arc.label;
|
||||
output.bytes[output.length++] = (byte) arc.label();
|
||||
FST.BytesReader fstReader = automaton.getBytesReader();
|
||||
automaton.readFirstTargetArc(arc, arc, fstReader);
|
||||
while (true) {
|
||||
if (arc.label == FST.END_LABEL) {
|
||||
if (arc.label() == FST.END_LABEL) {
|
||||
res.add(new Completion(output, bucket));
|
||||
if (res.size() >= num) return true;
|
||||
} else {
|
||||
|
|
|
@ -186,7 +186,7 @@ public class WFSTCompletionLookup extends Lookup implements Accountable {
|
|||
CharsRefBuilder spare = new CharsRefBuilder();
|
||||
if (exactFirst && arc.isFinal()) {
|
||||
spare.copyUTF8Bytes(scratch.get());
|
||||
results.add(new LookupResult(spare.toString(), decodeWeight(prefixOutput + arc.nextFinalOutput)));
|
||||
results.add(new LookupResult(spare.toString(), decodeWeight(prefixOutput + arc.nextFinalOutput())));
|
||||
if (--num == 0) {
|
||||
return results; // that was quick
|
||||
}
|
||||
|
@ -227,7 +227,7 @@ public class WFSTCompletionLookup extends Lookup implements Accountable {
|
|||
if (fst.findTargetArc(bytes[pos++] & 0xff, arc, arc, bytesReader) == null) {
|
||||
return null;
|
||||
} else {
|
||||
output += arc.output.longValue();
|
||||
output += arc.output().longValue();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -250,7 +250,7 @@ public class WFSTCompletionLookup extends Lookup implements Accountable {
|
|||
if (result == null || !arc.isFinal()) {
|
||||
return null;
|
||||
} else {
|
||||
return Integer.valueOf(decodeWeight(result + arc.nextFinalOutput));
|
||||
return Integer.valueOf(decodeWeight(result + arc.nextFinalOutput()));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -1212,14 +1212,11 @@ public class AnalyzingSuggesterTest extends LuceneTestCase {
|
|||
Directory tempDir = getDirectory();
|
||||
AnalyzingSuggester suggester = new AnalyzingSuggester(tempDir, "suggest", a);
|
||||
String bigString = TestUtil.randomSimpleString(random(), 30000, 30000);
|
||||
try {
|
||||
IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, () -> {
|
||||
suggester.build(new InputArrayIterator(new Input[] {
|
||||
new Input(bigString, 7)}));
|
||||
fail("did not hit expected exception");
|
||||
} catch (IllegalArgumentException iae) {
|
||||
// expected
|
||||
assertTrue(iae.getMessage().contains("input automaton is too large"));
|
||||
}
|
||||
new Input(bigString, 7)}));
|
||||
});
|
||||
assertTrue(ex.getMessage().contains("input automaton is too large"));
|
||||
IOUtils.close(a, tempDir);
|
||||
}
|
||||
|
||||
|
|
|
@ -220,7 +220,7 @@ public class FSTTester<T> {
|
|||
return null;
|
||||
}
|
||||
}
|
||||
output = fst.outputs.add(output, arc.output);
|
||||
output = fst.outputs.add(output, arc.output());
|
||||
}
|
||||
|
||||
if (prefixLength != null) {
|
||||
|
@ -253,14 +253,14 @@ public class FSTTester<T> {
|
|||
arcs.clear();
|
||||
|
||||
// accumulate output
|
||||
output = fst.outputs.add(output, arc.output);
|
||||
output = fst.outputs.add(output, arc.output());
|
||||
|
||||
// append label
|
||||
if (arc.label == FST.END_LABEL) {
|
||||
if (arc.label() == FST.END_LABEL) {
|
||||
break;
|
||||
}
|
||||
|
||||
in.append(arc.label);
|
||||
in.append(arc.label());
|
||||
}
|
||||
|
||||
return output;
|
||||
|
|
|
@ -60,19 +60,10 @@ public class TestCompressingTermVectorsFormat extends BaseTermVectorsFormatTestC
|
|||
assertNotNull(terms);
|
||||
TermsEnum termsEnum = terms.iterator();
|
||||
assertEquals(SeekStatus.FOUND, termsEnum.seekCeil(new BytesRef("this")));
|
||||
try {
|
||||
termsEnum.ord();
|
||||
fail();
|
||||
} catch (UnsupportedOperationException expected) {
|
||||
// expected exception
|
||||
}
|
||||
|
||||
try {
|
||||
termsEnum.seekExact(0);
|
||||
fail();
|
||||
} catch (UnsupportedOperationException expected) {
|
||||
// expected exception
|
||||
}
|
||||
|
||||
expectThrows(UnsupportedOperationException.class, termsEnum::ord);
|
||||
expectThrows(UnsupportedOperationException.class, () -> termsEnum.seekExact(0));
|
||||
|
||||
ir.close();
|
||||
iw.close();
|
||||
dir.close();
|
||||
|
|
|
@ -54,12 +54,9 @@ public class TestHandleLimitFS extends MockFileSystemTestCase {
|
|||
}
|
||||
|
||||
// now exceed
|
||||
try {
|
||||
Files.newOutputStream(Files.createTempFile(dir, null, null));
|
||||
fail("didn't hit exception");
|
||||
} catch (IOException e) {
|
||||
assertTrue(e.getMessage().contains("Too many open files"));
|
||||
}
|
||||
IOException e = expectThrows(IOException.class, () ->
|
||||
Files.newOutputStream(Files.createTempFile(dir, null, null)));
|
||||
assertTrue(e.getMessage().contains("Too many open files"));
|
||||
|
||||
IOUtils.close(toClose);
|
||||
}
|
||||
|
|
|
@ -53,37 +53,17 @@ public class TestHandleTrackingFS extends MockFileSystemTestCase {
|
|||
|
||||
OutputStream file = Files.newOutputStream(dir.resolve("somefile"));
|
||||
file.write(5);
|
||||
try {
|
||||
file.close();
|
||||
fail("expected IOException");
|
||||
} catch (IOException ex) {
|
||||
// expected
|
||||
}
|
||||
expectThrows(IOException.class, file::close);
|
||||
|
||||
SeekableByteChannel channel = Files.newByteChannel(dir.resolve("somefile"));
|
||||
try {
|
||||
channel.close();
|
||||
fail("expected IOException");
|
||||
} catch (IOException ex) {
|
||||
// expected
|
||||
}
|
||||
expectThrows(IOException.class, channel::close);
|
||||
|
||||
InputStream stream = Files.newInputStream(dir.resolve("somefile"));
|
||||
try {
|
||||
stream.close();
|
||||
fail("expected IOException");
|
||||
} catch (IOException ex) {
|
||||
// expected
|
||||
}
|
||||
expectThrows(IOException.class, stream::close);
|
||||
fs.close();
|
||||
|
||||
DirectoryStream<Path> dirStream = Files.newDirectoryStream(dir);
|
||||
try {
|
||||
dirStream.close();
|
||||
fail("expected IOException");
|
||||
} catch (IOException ex) {
|
||||
// expected
|
||||
}
|
||||
expectThrows(IOException.class, dirStream::close);
|
||||
}
|
||||
|
||||
|
||||
|
@ -102,34 +82,14 @@ public class TestHandleTrackingFS extends MockFileSystemTestCase {
|
|||
}.getFileSystem(URI.create("file:///"));
|
||||
Path dir = new FilterPath(path, fs);
|
||||
|
||||
try {
|
||||
OutputStream file = Files.newOutputStream(dir.resolve("somefile"));
|
||||
fail("expected IOException");
|
||||
} catch (IOException ex) {
|
||||
// expected
|
||||
}
|
||||
expectThrows(IOException.class, () -> Files.newOutputStream(dir.resolve("somefile")));
|
||||
|
||||
try {
|
||||
SeekableByteChannel channel = Files.newByteChannel(dir.resolve("somefile"));
|
||||
fail("expected IOException");
|
||||
} catch (IOException ex) {
|
||||
// expected
|
||||
}
|
||||
expectThrows(IOException.class, () -> Files.newByteChannel(dir.resolve("somefile")));
|
||||
|
||||
try {
|
||||
InputStream stream = Files.newInputStream(dir.resolve("somefile"));
|
||||
fail("expected IOException");
|
||||
} catch (IOException ex) {
|
||||
// expected
|
||||
}
|
||||
expectThrows(IOException.class, () -> Files.newInputStream(dir.resolve("somefile")));
|
||||
fs.close();
|
||||
|
||||
try {
|
||||
DirectoryStream<Path> dirStream = Files.newDirectoryStream(dir);
|
||||
fail("expected IOException");
|
||||
} catch (IOException ex) {
|
||||
// expected
|
||||
}
|
||||
expectThrows(IOException.class, () -> Files.newDirectoryStream(dir));
|
||||
fs.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -44,12 +44,9 @@ public class TestLeakFS extends MockFileSystemTestCase {
|
|||
file.write(5);
|
||||
file.close();
|
||||
InputStream leak = Files.newInputStream(dir.resolve("stillopen"));
|
||||
try {
|
||||
dir.getFileSystem().close();
|
||||
fail("should have gotten exception");
|
||||
} catch (Exception e) {
|
||||
assertTrue(e.getMessage().contains("file handle leaks"));
|
||||
}
|
||||
|
||||
Exception e = expectThrows(Exception.class, () -> dir.getFileSystem().close());
|
||||
assertTrue(e.getMessage().contains("file handle leaks"));
|
||||
leak.close();
|
||||
}
|
||||
|
||||
|
@ -58,12 +55,8 @@ public class TestLeakFS extends MockFileSystemTestCase {
|
|||
Path dir = wrap(createTempDir());
|
||||
|
||||
OutputStream leak = Files.newOutputStream(dir.resolve("leaky"));
|
||||
try {
|
||||
dir.getFileSystem().close();
|
||||
fail("should have gotten exception");
|
||||
} catch (Exception e) {
|
||||
assertTrue(e.getMessage().contains("file handle leaks"));
|
||||
}
|
||||
Exception e = expectThrows(Exception.class, () -> dir.getFileSystem().close());
|
||||
assertTrue(e.getMessage().contains("file handle leaks"));
|
||||
leak.close();
|
||||
}
|
||||
|
||||
|
@ -75,12 +68,9 @@ public class TestLeakFS extends MockFileSystemTestCase {
|
|||
file.write(5);
|
||||
file.close();
|
||||
FileChannel leak = FileChannel.open(dir.resolve("stillopen"));
|
||||
try {
|
||||
dir.getFileSystem().close();
|
||||
fail("should have gotten exception");
|
||||
} catch (Exception e) {
|
||||
assertTrue(e.getMessage().contains("file handle leaks"));
|
||||
}
|
||||
|
||||
Exception e = expectThrows(Exception.class, () -> dir.getFileSystem().close());
|
||||
assertTrue(e.getMessage().contains("file handle leaks"));
|
||||
leak.close();
|
||||
}
|
||||
|
||||
|
@ -92,12 +82,8 @@ public class TestLeakFS extends MockFileSystemTestCase {
|
|||
file.write(5);
|
||||
file.close();
|
||||
AsynchronousFileChannel leak = AsynchronousFileChannel.open(dir.resolve("stillopen"));
|
||||
try {
|
||||
dir.getFileSystem().close();
|
||||
fail("should have gotten exception");
|
||||
} catch (Exception e) {
|
||||
assertTrue(e.getMessage().contains("file handle leaks"));
|
||||
}
|
||||
Exception e = expectThrows(Exception.class, () -> dir.getFileSystem().close());
|
||||
assertTrue(e.getMessage().contains("file handle leaks"));
|
||||
leak.close();
|
||||
}
|
||||
|
||||
|
@ -109,12 +95,9 @@ public class TestLeakFS extends MockFileSystemTestCase {
|
|||
file.write(5);
|
||||
file.close();
|
||||
SeekableByteChannel leak = Files.newByteChannel(dir.resolve("stillopen"));
|
||||
try {
|
||||
dir.getFileSystem().close();
|
||||
fail("should have gotten exception");
|
||||
} catch (Exception e) {
|
||||
assertTrue(e.getMessage().contains("file handle leaks"));
|
||||
}
|
||||
|
||||
Exception e = expectThrows(Exception.class, () -> dir.getFileSystem().close());
|
||||
assertTrue(e.getMessage().contains("file handle leaks"));
|
||||
leak.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -82,10 +82,7 @@ public class TestVerboseFS extends MockFileSystemTestCase {
|
|||
Files.createDirectory(dir.resolve("subdir"));
|
||||
assertTrue(stream.sawMessage());
|
||||
|
||||
try {
|
||||
Files.createDirectory(dir.resolve("subdir"));
|
||||
fail("didn't get expected exception");
|
||||
} catch (IOException expected) {}
|
||||
expectThrows(IOException.class, () -> Files.createDirectory(dir.resolve("subdir")));
|
||||
}
|
||||
|
||||
/** Test delete */
|
||||
|
@ -96,10 +93,7 @@ public class TestVerboseFS extends MockFileSystemTestCase {
|
|||
Files.delete(dir.resolve("foobar"));
|
||||
assertTrue(stream.sawMessage());
|
||||
|
||||
try {
|
||||
Files.delete(dir.resolve("foobar"));
|
||||
fail("didn't get expected exception");
|
||||
} catch (IOException expected) {}
|
||||
expectThrows(IOException.class, () -> Files.delete(dir.resolve("foobar")));
|
||||
}
|
||||
|
||||
/** Test deleteIfExists */
|
||||
|
@ -122,10 +116,7 @@ public class TestVerboseFS extends MockFileSystemTestCase {
|
|||
Files.copy(dir.resolve("foobar"), dir.resolve("baz"));
|
||||
assertTrue(stream.sawMessage());
|
||||
|
||||
try {
|
||||
Files.copy(dir.resolve("nonexistent"), dir.resolve("something"));
|
||||
fail("didn't get expected exception");
|
||||
} catch (IOException expected) {}
|
||||
expectThrows(IOException.class, () -> Files.copy(dir.resolve("nonexistent"), dir.resolve("something")));
|
||||
}
|
||||
|
||||
/** Test move */
|
||||
|
@ -136,10 +127,7 @@ public class TestVerboseFS extends MockFileSystemTestCase {
|
|||
Files.move(dir.resolve("foobar"), dir.resolve("baz"));
|
||||
assertTrue(stream.sawMessage());
|
||||
|
||||
try {
|
||||
Files.move(dir.resolve("nonexistent"), dir.resolve("something"));
|
||||
fail("didn't get expected exception");
|
||||
} catch (IOException expected) {}
|
||||
expectThrows(IOException.class, () -> Files.move(dir.resolve("nonexistent"), dir.resolve("something")));
|
||||
}
|
||||
|
||||
/** Test newOutputStream */
|
||||
|
@ -149,11 +137,8 @@ public class TestVerboseFS extends MockFileSystemTestCase {
|
|||
OutputStream file = Files.newOutputStream(dir.resolve("output"));
|
||||
assertTrue(stream.sawMessage());
|
||||
file.close();
|
||||
|
||||
try {
|
||||
Files.newOutputStream(dir.resolve("output"), StandardOpenOption.CREATE_NEW);
|
||||
fail("didn't get expected exception");
|
||||
} catch (IOException expected) {}
|
||||
|
||||
expectThrows(IOException.class, () -> Files.newOutputStream(dir.resolve("output"), StandardOpenOption.CREATE_NEW));
|
||||
}
|
||||
|
||||
/** Test FileChannel.open */
|
||||
|
@ -163,11 +148,9 @@ public class TestVerboseFS extends MockFileSystemTestCase {
|
|||
FileChannel channel = FileChannel.open(dir.resolve("foobar"), StandardOpenOption.CREATE_NEW, StandardOpenOption.READ, StandardOpenOption.WRITE);
|
||||
assertTrue(stream.sawMessage());
|
||||
channel.close();
|
||||
|
||||
try {
|
||||
FileChannel.open(dir.resolve("foobar"), StandardOpenOption.CREATE_NEW, StandardOpenOption.READ, StandardOpenOption.WRITE);
|
||||
fail("didn't get expected exception");
|
||||
} catch (IOException expected) {}
|
||||
|
||||
expectThrows(IOException.class, () -> FileChannel.open(dir.resolve("foobar"),
|
||||
StandardOpenOption.CREATE_NEW, StandardOpenOption.READ, StandardOpenOption.WRITE));
|
||||
}
|
||||
|
||||
/** Test AsynchronousFileChannel.open */
|
||||
|
@ -177,11 +160,9 @@ public class TestVerboseFS extends MockFileSystemTestCase {
|
|||
AsynchronousFileChannel channel = AsynchronousFileChannel.open(dir.resolve("foobar"), StandardOpenOption.CREATE_NEW, StandardOpenOption.READ, StandardOpenOption.WRITE);
|
||||
assertTrue(stream.sawMessage());
|
||||
channel.close();
|
||||
|
||||
try {
|
||||
AsynchronousFileChannel.open(dir.resolve("foobar"), StandardOpenOption.CREATE_NEW, StandardOpenOption.READ, StandardOpenOption.WRITE);
|
||||
fail("didn't get expected exception");
|
||||
} catch (IOException expected) {}
|
||||
|
||||
expectThrows(IOException.class, () -> AsynchronousFileChannel.open(dir.resolve("foobar"),
|
||||
StandardOpenOption.CREATE_NEW, StandardOpenOption.READ, StandardOpenOption.WRITE));
|
||||
}
|
||||
|
||||
/** Test newByteChannel */
|
||||
|
@ -191,33 +172,16 @@ public class TestVerboseFS extends MockFileSystemTestCase {
|
|||
SeekableByteChannel channel = Files.newByteChannel(dir.resolve("foobar"), StandardOpenOption.CREATE_NEW, StandardOpenOption.READ, StandardOpenOption.WRITE);
|
||||
assertTrue(stream.sawMessage());
|
||||
channel.close();
|
||||
|
||||
try {
|
||||
Files.newByteChannel(dir.resolve("foobar"), StandardOpenOption.CREATE_NEW, StandardOpenOption.READ, StandardOpenOption.WRITE);
|
||||
fail("didn't get expected exception");
|
||||
} catch (IOException expected) {}
|
||||
|
||||
expectThrows(IOException.class, () -> Files.newByteChannel(dir.resolve("foobar"),
|
||||
StandardOpenOption.CREATE_NEW, StandardOpenOption.READ, StandardOpenOption.WRITE));
|
||||
}
|
||||
|
||||
/** Test that verbose does not corrumpt file not found exceptions */
|
||||
public void testVerboseFSNoSuchFileException() throws IOException {
|
||||
/** Test that verbose does not corrupt file not found exceptions */
|
||||
public void testVerboseFSNoSuchFileException() {
|
||||
Path dir = wrap(createTempDir());
|
||||
try {
|
||||
AsynchronousFileChannel.open(dir.resolve("doesNotExist.rip"));
|
||||
fail("did not hit exception");
|
||||
} catch (NoSuchFileException nsfe) {
|
||||
// expected
|
||||
}
|
||||
try {
|
||||
FileChannel.open(dir.resolve("doesNotExist.rip"));
|
||||
fail("did not hit exception");
|
||||
} catch (NoSuchFileException nsfe) {
|
||||
// expected
|
||||
}
|
||||
try {
|
||||
Files.newByteChannel(dir.resolve("stillopen"));
|
||||
fail("did not hit exception");
|
||||
} catch (NoSuchFileException nsfe) {
|
||||
// expected
|
||||
}
|
||||
expectThrows(NoSuchFileException.class, () -> AsynchronousFileChannel.open(dir.resolve("doesNotExist.rip")));
|
||||
expectThrows(NoSuchFileException.class, () -> FileChannel.open(dir.resolve("doesNotExist.rip")));
|
||||
expectThrows(NoSuchFileException.class, () -> Files.newByteChannel(dir.resolve("stillopen")));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -55,12 +55,9 @@ public class TestWindowsFS extends MockFileSystemTestCase {
|
|||
file.write(5);
|
||||
file.close();
|
||||
InputStream is = Files.newInputStream(dir.resolve("stillopen"));
|
||||
try {
|
||||
Files.delete(dir.resolve("stillopen"));
|
||||
fail("should have gotten exception");
|
||||
} catch (IOException e) {
|
||||
assertTrue(e.getMessage().contains("access denied"));
|
||||
}
|
||||
|
||||
IOException e = expectThrows(IOException.class, () -> Files.delete(dir.resolve("stillopen")));
|
||||
assertTrue(e.getMessage().contains("access denied"));
|
||||
is.close();
|
||||
}
|
||||
|
||||
|
@ -72,12 +69,9 @@ public class TestWindowsFS extends MockFileSystemTestCase {
|
|||
file.write(5);
|
||||
file.close();
|
||||
InputStream is = Files.newInputStream(dir.resolve("stillopen"));
|
||||
try {
|
||||
Files.deleteIfExists(dir.resolve("stillopen"));
|
||||
fail("should have gotten exception");
|
||||
} catch (IOException e) {
|
||||
assertTrue(e.getMessage().contains("access denied"));
|
||||
}
|
||||
|
||||
IOException e = expectThrows(IOException.class, () -> Files.deleteIfExists(dir.resolve("stillopen")));
|
||||
assertTrue(e.getMessage().contains("access denied"));
|
||||
is.close();
|
||||
}
|
||||
|
||||
|
@ -90,12 +84,10 @@ public class TestWindowsFS extends MockFileSystemTestCase {
|
|||
file.write(5);
|
||||
file.close();
|
||||
InputStream is = Files.newInputStream(dir.resolve("stillopen"));
|
||||
try {
|
||||
Files.move(dir.resolve("stillopen"), dir.resolve("target"), StandardCopyOption.ATOMIC_MOVE);
|
||||
fail("should have gotten exception");
|
||||
} catch (IOException e) {
|
||||
assertTrue(e.getMessage().contains("access denied"));
|
||||
}
|
||||
|
||||
IOException e = expectThrows(IOException.class, () ->
|
||||
Files.move(dir.resolve("stillopen"), dir.resolve("target"), StandardCopyOption.ATOMIC_MOVE));
|
||||
assertTrue(e.getMessage().contains("access denied"));
|
||||
is.close();
|
||||
}
|
||||
|
||||
|
|
|
@ -56,14 +56,10 @@ public class TestMockDirectoryWrapper extends BaseDirectoryTestCase {
|
|||
// close() to ensure the written bytes are not buffered and counted
|
||||
// against the directory size
|
||||
out.close();
|
||||
out = dir.createOutput("bar", IOContext.DEFAULT);
|
||||
try {
|
||||
out.writeBytes(bytes, bytes.length);
|
||||
fail("should have failed on disk full");
|
||||
} catch (IOException e) {
|
||||
// expected
|
||||
}
|
||||
out.close();
|
||||
|
||||
IndexOutput out2 = dir.createOutput("bar", IOContext.DEFAULT);
|
||||
expectThrows(IOException.class, () -> out2.writeBytes(bytes, bytes.length));
|
||||
out2.close();
|
||||
dir.close();
|
||||
|
||||
// test copyBytes
|
||||
|
@ -74,14 +70,10 @@ public class TestMockDirectoryWrapper extends BaseDirectoryTestCase {
|
|||
// close() to ensure the written bytes are not buffered and counted
|
||||
// against the directory size
|
||||
out.close();
|
||||
out = dir.createOutput("bar", IOContext.DEFAULT);
|
||||
try {
|
||||
out.copyBytes(new ByteArrayDataInput(bytes), bytes.length);
|
||||
fail("should have failed on disk full");
|
||||
} catch (IOException e) {
|
||||
// expected
|
||||
}
|
||||
out.close();
|
||||
|
||||
IndexOutput out3 = dir.createOutput("bar", IOContext.DEFAULT);
|
||||
expectThrows(IOException.class, () -> out3.copyBytes(new ByteArrayDataInput(bytes), bytes.length));
|
||||
out3.close();
|
||||
dir.close();
|
||||
}
|
||||
|
||||
|
|
|
@ -28,30 +28,16 @@ public class TestRunWithRestrictedPermissions extends LuceneTestCase {
|
|||
}
|
||||
|
||||
public void testNormallyAllowedStuff() throws Exception {
|
||||
try {
|
||||
runWithRestrictedPermissions(this::doSomeForbiddenStuff);
|
||||
fail("this should not pass!");
|
||||
} catch (SecurityException se) {
|
||||
// pass
|
||||
}
|
||||
expectThrows(SecurityException.class, () -> runWithRestrictedPermissions(this::doSomeForbiddenStuff));
|
||||
}
|
||||
|
||||
public void testCompletelyForbidden1() throws Exception {
|
||||
try {
|
||||
runWithRestrictedPermissions(this::doSomeCompletelyForbiddenStuff);
|
||||
fail("this should not pass!");
|
||||
} catch (SecurityException se) {
|
||||
// pass
|
||||
}
|
||||
expectThrows(SecurityException.class, () -> runWithRestrictedPermissions(this::doSomeCompletelyForbiddenStuff));
|
||||
}
|
||||
|
||||
public void testCompletelyForbidden2() throws Exception {
|
||||
try {
|
||||
runWithRestrictedPermissions(this::doSomeCompletelyForbiddenStuff, new AllPermission());
|
||||
fail("this should not pass (not even with AllPermission)");
|
||||
} catch (SecurityException se) {
|
||||
// pass
|
||||
}
|
||||
expectThrows(SecurityException.class, () ->
|
||||
runWithRestrictedPermissions(this::doSomeCompletelyForbiddenStuff, new AllPermission()));
|
||||
}
|
||||
|
||||
private Void doSomeForbiddenStuff() throws IOException {
|
||||
|
|
|
@ -57,6 +57,9 @@ Upgrade Notes
|
|||
|
||||
* SOLR-13596: Deprecated GroupingSpecification methods are removed. (Munendra S N)
|
||||
|
||||
* SOLR-11266: default Content-Type override for JSONResponseWriter from _default configSet is removed. Example has been
|
||||
provided in sample_techproducts_configs to override content-type. (Ishan Chattopadhyaya, Munendra S N, Gus Heck)
|
||||
|
||||
Other Changes
|
||||
----------------------
|
||||
|
||||
|
@ -75,6 +78,16 @@ Velocity 2.0 and Velocity Tools 3.0
|
|||
Apache ZooKeeper 3.5.5
|
||||
Jetty 9.4.19.v20190610
|
||||
|
||||
Upgrade Notes
|
||||
----------------------
|
||||
|
||||
* Users who have written test cases that extend SolrTestCaseJ4 may see NullPointerExceptions if
|
||||
their tests directly reference both SolrTestCaseJ4.initCoreDataDir and SolrTestCaseJ4.deleteCore().
|
||||
This change in behavior is due to a bug fix in deleteCore() to ensure the dataDir is properly reset
|
||||
in tests that call initCore()/deleteCore() multiple times in a given test (class). initCoreDataDir
|
||||
is now deprecated, and users are encouraged to use SolrTestCaseJ4.initAndGetDataDir() in it's place.
|
||||
See SOLR-13664 for more details.
|
||||
|
||||
New Features
|
||||
----------------------
|
||||
|
||||
|
@ -99,6 +112,8 @@ Improvements
|
|||
|
||||
* SOLR-13558: Allow dynamic resizing of SolrCache-s. (ab)
|
||||
|
||||
* SOLR-6305: Ability to set the replication factor for index files created by HDFSDirectoryFactory (Boris Pasko via Kevin Risden)
|
||||
|
||||
Bug Fixes
|
||||
----------------------
|
||||
|
||||
|
@ -107,6 +122,15 @@ Bug Fixes
|
|||
* SOLR-11556: Backup and restore command really supports picking one of a few repositories by
|
||||
repository parameter (Timothy Potter via Mikhail Khludnev)
|
||||
|
||||
* SOLR-13660: Fixed AbstractFullDistribZkTestBase.waitForActiveReplicaCount() to ensure
|
||||
replicas are active. (hossman)
|
||||
|
||||
* SOLR-13664: Fixed SolrTestCaseJ4.deleteCore() to properly reset the dataDir used by initCore()
|
||||
(hossman)
|
||||
|
||||
* SOLR-13679: Default style of ExplainDocTransformer registered via solrconfig.xml should be same as default style
|
||||
of ExplainDocTransformer registered in defaultFactories (Munendra S N)
|
||||
|
||||
Other Changes
|
||||
----------------------
|
||||
|
||||
|
@ -116,6 +140,12 @@ Other Changes
|
|||
|
||||
* SOLR-10377: Add `debug.explain.structured` to Admin UI. (David Smiley, Munendra S N)
|
||||
|
||||
* SOLR-13629: Clean trailing whitespace from 'analytics' contrib (Neal Sidhwaney via Jason Gerlowski)
|
||||
|
||||
* SOLR-13643: Add Getters/Setters in ResponseBuilder for analytics response handling (Neal Sidhwaney via Munendra S N)
|
||||
|
||||
* SOLR-13659: Refactor CacheConfig to lazily load the the implementation class (noble)
|
||||
|
||||
================== 8.2.0 ==================
|
||||
|
||||
Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.
|
||||
|
|
|
@ -31,10 +31,10 @@ import org.apache.solr.search.Filter;
|
|||
import org.apache.solr.search.SolrIndexSearcher;
|
||||
|
||||
public class AnalyticsDriver {
|
||||
|
||||
|
||||
/**
|
||||
* Drive the collection of reduction data. This includes overall data as well as faceted data.
|
||||
*
|
||||
*
|
||||
* @param manager of the request to drive
|
||||
* @param searcher the results of the query
|
||||
* @param filter that represents the overall query
|
||||
|
@ -45,9 +45,9 @@ public class AnalyticsDriver {
|
|||
StreamingInfo streamingInfo = manager.getStreamingFacetInfo();
|
||||
Iterable<StreamingFacet> streamingFacets = streamingInfo.streamingFacets;
|
||||
ReductionCollectionManager collectionManager = streamingInfo.streamingCollectionManager;
|
||||
|
||||
|
||||
Iterable<FacetValueQueryExecuter> facetExecuters = manager.getFacetExecuters(filter, queryRequest);
|
||||
|
||||
|
||||
// Streaming phase (Overall results & Value/Pivot Facets)
|
||||
// Loop through all documents and collect reduction data for streaming facets and overall results
|
||||
if (collectionManager.needsCollection()) {
|
||||
|
@ -72,7 +72,7 @@ public class AnalyticsDriver {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
// Executing phase (Query/Range Facets)
|
||||
// Send additional Solr Queries to compute facet values
|
||||
for (FacetValueQueryExecuter executer : facetExecuters) {
|
||||
|
|
|
@ -27,35 +27,35 @@ import org.apache.solr.analytics.value.AnalyticsValue;
|
|||
public class AnalyticsExpression {
|
||||
private final AnalyticsValue expression;
|
||||
private final String name;
|
||||
|
||||
|
||||
public AnalyticsExpression(String name, AnalyticsValue expression) {
|
||||
this.name = name;
|
||||
this.expression = expression;
|
||||
}
|
||||
|
||||
|
||||
public String getName() {
|
||||
return name;
|
||||
}
|
||||
|
||||
|
||||
public AnalyticsValue getExpression() {
|
||||
return expression;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Get the current value of the expression.
|
||||
* This method can, and will, be called multiple times to return different values.
|
||||
* The value returned is based on the {@link ReductionDataCollection} given
|
||||
* to the {@link ReductionCollectionManager#setData} method.
|
||||
*
|
||||
*
|
||||
* @return the current value of the expression
|
||||
*/
|
||||
public Object toObject() {
|
||||
return expression.getObject();
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* NOTE: Must be called after {@link #toObject()} is called, otherwise the value is not guaranteed to be correct.
|
||||
*
|
||||
*
|
||||
* @return whether the current value of the expression exists.
|
||||
*/
|
||||
public boolean exists() {
|
||||
|
|
|
@ -43,7 +43,7 @@ import org.apache.solr.search.Filter;
|
|||
/**
|
||||
* The manager for faceted analytics. This class manages one grouping of facets and expressions to compute
|
||||
* over those facets.
|
||||
*
|
||||
*
|
||||
* <p>
|
||||
* This class will only manage generating faceted results, not overall results.
|
||||
*/
|
||||
|
@ -53,9 +53,9 @@ public class AnalyticsGroupingManager {
|
|||
|
||||
private final Collection<AnalyticsExpression> topLevelExpressions;
|
||||
private final ExpressionCalculator expressionCalculator;
|
||||
|
||||
|
||||
private final Map<String, AnalyticsFacet> facets;
|
||||
|
||||
|
||||
public AnalyticsGroupingManager(String name,
|
||||
ReductionCollectionManager reductionCollectionManager,
|
||||
Collection<AnalyticsExpression> topLevelExpressions) {
|
||||
|
@ -70,11 +70,11 @@ public class AnalyticsGroupingManager {
|
|||
|
||||
// This is outside of the method, since it is used in the lambda and cannot be a local non-final variable
|
||||
private boolean hasStreamingFacets;
|
||||
|
||||
|
||||
/**
|
||||
* Get the {@link StreamingFacet}s (e.g. {@link ValueFacet} and {@link PivotFacet}) contained within this grouping,
|
||||
* returning them through the given consumer.
|
||||
*
|
||||
*
|
||||
* @param cons where the streaming facets are passed to
|
||||
* @return whether the grouping contains streaming facets
|
||||
*/
|
||||
|
@ -93,12 +93,12 @@ public class AnalyticsGroupingManager {
|
|||
* Create the {@link FacetValueQueryExecuter}s for all {@link AbstractSolrQueryFacet}s
|
||||
* (e.g. {@link QueryFacet} and {@link RangeFacet}) contained within this grouping.
|
||||
* The executers are returned through the given consumer.
|
||||
*
|
||||
*
|
||||
* <p>
|
||||
* One {@link FacetValueQueryExecuter} is created for each facet value to be returned for a facet.
|
||||
* Since every {@link AbstractSolrQueryFacet} has discrete and user-defined facet values,
|
||||
* unlike {@link StreamingFacet}s, a discrete number of {@link FacetValueQueryExecuter}s are created and returned.
|
||||
*
|
||||
*
|
||||
* @param filter representing the overall Solr Query of the request,
|
||||
* will be combined with the facet value queries
|
||||
* @param queryRequest from the overall search request
|
||||
|
@ -111,10 +111,10 @@ public class AnalyticsGroupingManager {
|
|||
}
|
||||
});
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Add a facet to the grouping. All expressions in this grouping will be computed over the facet.
|
||||
*
|
||||
*
|
||||
* @param facet to compute expressions over
|
||||
*/
|
||||
public void addFacet(AnalyticsFacet facet) {
|
||||
|
@ -122,11 +122,11 @@ public class AnalyticsGroupingManager {
|
|||
facet.setReductionCollectionManager(reductionCollectionManager);
|
||||
facets.put(facet.getName(), facet);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Import the shard data for this grouping from a bit-stream,
|
||||
* exported by the {@link #exportShardData} method in the each of the collection's shards.
|
||||
*
|
||||
*
|
||||
* @param input The bit-stream to import the grouping data from
|
||||
* @throws IOException if an exception occurs while reading from the {@link DataInput}
|
||||
*/
|
||||
|
@ -134,17 +134,17 @@ public class AnalyticsGroupingManager {
|
|||
// This allows mergeData() to import from the same input everytime it is called
|
||||
// while the facets are importing.
|
||||
reductionCollectionManager.setShardInput(input);
|
||||
|
||||
|
||||
int sz = input.readInt();
|
||||
for (int i = 0; i < sz; ++i) {
|
||||
facets.get(input.readUTF()).importShardData(input);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Export the shard data for this grouping through a bit-stream,
|
||||
* to be imported by the {@link #importShardData} method in the originating shard.
|
||||
*
|
||||
*
|
||||
* @param output The bit-stream to output the grouping data through
|
||||
* @throws IOException if an exception occurs while writing to the {@link DataOutput}
|
||||
*/
|
||||
|
@ -152,18 +152,18 @@ public class AnalyticsGroupingManager {
|
|||
// This allows exportData() to export to the same output everytime it is called
|
||||
// while the facets are exporting.
|
||||
reductionCollectionManager.setShardOutput(output);
|
||||
|
||||
|
||||
output.writeInt(facets.size());
|
||||
for (Entry<String,AnalyticsFacet> facet : facets.entrySet()) {
|
||||
output.writeUTF(facet.getKey());
|
||||
facet.getValue().exportShardData(output);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Get the {@link ReductionCollectionManager} that manages the collection of reduction data for the expressions
|
||||
* contained within this grouping.
|
||||
*
|
||||
* contained within this grouping.
|
||||
*
|
||||
* @return the grouping's reduction manager
|
||||
*/
|
||||
public ReductionCollectionManager getReductionManager() {
|
||||
|
@ -172,18 +172,18 @@ public class AnalyticsGroupingManager {
|
|||
|
||||
/**
|
||||
* Create the response for this grouping, a mapping from each of it's facets' names to the facet's response.
|
||||
*
|
||||
*
|
||||
* @return the named list representation of the response
|
||||
*/
|
||||
public Map<String,Object> createResponse() {
|
||||
Map<String,Object> response = new HashMap<>();
|
||||
|
||||
|
||||
// Add the value facet buckets to the output
|
||||
facets.forEach( (name, facet) -> response.put(name, facet.createResponse()) );
|
||||
|
||||
return response;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Create the response for this grouping, but in the old style of response.
|
||||
* This response has a bucket for the following if they are contained in the grouping:
|
||||
|
@ -192,13 +192,13 @@ public class AnalyticsGroupingManager {
|
|||
* <p>
|
||||
* Since groupings in the old notation must also return overall results, the overall results are
|
||||
* passed in and the values are used to populate the grouping response.
|
||||
*
|
||||
*
|
||||
* @param overallResults of the expressions to add to the grouping response
|
||||
* @return the named list representation of the response
|
||||
*/
|
||||
public NamedList<Object> createOldResponse(Map<String,Object> overallResults) {
|
||||
NamedList<Object> response = new NamedList<>();
|
||||
|
||||
|
||||
topLevelExpressions.forEach( expression -> response.add(expression.getName(), overallResults.get(name + expression.getName())));
|
||||
|
||||
NamedList<Object> fieldFacetResults = new NamedList<>();
|
||||
|
@ -230,7 +230,7 @@ public class AnalyticsGroupingManager {
|
|||
|
||||
/**
|
||||
* Get the name of the grouping.
|
||||
*
|
||||
*
|
||||
* @return the grouping name
|
||||
*/
|
||||
public String getName() {
|
||||
|
|
|
@ -42,25 +42,25 @@ import org.apache.solr.search.Filter;
|
|||
public class AnalyticsRequestManager {
|
||||
private final ReductionCollectionManager ungroupedReductionManager;
|
||||
private ReductionDataCollection ungroupedData;
|
||||
|
||||
|
||||
private final Map<String, AnalyticsGroupingManager> groupingManagers;
|
||||
|
||||
|
||||
private final Collection<AnalyticsExpression> ungroupedExpressions;
|
||||
private final ExpressionCalculator ungroupedExpressionCalculator;
|
||||
|
||||
|
||||
/**
|
||||
* If the request is distributed, the manager for shard requests.
|
||||
*/
|
||||
public String analyticsRequest;
|
||||
public AnalyticsShardRequestManager shardStream;
|
||||
public boolean sendShards;
|
||||
|
||||
public boolean sendShards;
|
||||
|
||||
/**
|
||||
* Create an manager with the given ungrouped expressions. This is straightforward in the new
|
||||
* style of request, however in the old olap-style requests all groupings' expressions are expected
|
||||
* to be ungrouped as well.
|
||||
*
|
||||
*
|
||||
*
|
||||
*
|
||||
* @param ungroupedReductionManager to manage the reduction collection for all ungrouped expressions
|
||||
* @param ungroupedExpressions to compute overall results for
|
||||
*/
|
||||
|
@ -69,85 +69,85 @@ public class AnalyticsRequestManager {
|
|||
this.ungroupedReductionManager = ungroupedReductionManager;
|
||||
this.ungroupedData = ungroupedReductionManager.newDataCollection();
|
||||
this.ungroupedReductionManager.addLastingCollectTarget(ungroupedData);
|
||||
|
||||
|
||||
this.ungroupedExpressions = ungroupedExpressions;
|
||||
this.ungroupedExpressionCalculator = new ExpressionCalculator(ungroupedExpressions);
|
||||
this.groupingManagers = new HashMap<>();
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Get the collection manager for ungrouped expressions, including grouped expressions if
|
||||
* the old request notation is used.
|
||||
*
|
||||
*
|
||||
* @return the collection manager for the ungrouped expressions
|
||||
*/
|
||||
public ReductionCollectionManager getUngroupedCollectionManager() {
|
||||
return ungroupedReductionManager;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Get the collection manager for all ungrouped expressions, including grouped expressions if
|
||||
* the old request notation is used.
|
||||
*
|
||||
*
|
||||
* @return the collection manager for the ungrouped expressions
|
||||
*/
|
||||
public ReductionDataCollection getUngroupedData() {
|
||||
return ungroupedData;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Return all ungrouped expressions, including grouped expressions if
|
||||
* the old request notation is used.
|
||||
*
|
||||
*
|
||||
* @return an {@link Iterable} of the ungrouped expressions
|
||||
*/
|
||||
public Iterable<AnalyticsExpression> getUngroupedExpressions() {
|
||||
return ungroupedExpressions;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Generate the results of all ungrouped expressions, including grouped expressions if
|
||||
* the old request notation is used.
|
||||
*
|
||||
*
|
||||
* @param response the response to add the ungrouped results to.
|
||||
*/
|
||||
public void addUngroupedResults(Map<String,Object> response) {
|
||||
ungroupedReductionManager.setData(ungroupedData);
|
||||
ungroupedExpressionCalculator.addResults(response);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Generate the results of all ungrouped expressions, including grouped expressions if
|
||||
* the old request notation is used.
|
||||
*
|
||||
*
|
||||
* @return the map containing the ungrouped results
|
||||
*/
|
||||
public Map<String,Object> getUngroupedResults() {
|
||||
ungroupedReductionManager.setData(ungroupedData);
|
||||
return ungroupedExpressionCalculator.getResults();
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Add a grouping to the request.
|
||||
*
|
||||
*
|
||||
* @param groupingManager that manages the grouping
|
||||
*/
|
||||
public void addGrouping(AnalyticsGroupingManager groupingManager) {
|
||||
groupingManagers.put(groupingManager.getName(), groupingManager);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Import the shard data for this request from a bit-stream,
|
||||
* exported by the {@link #exportShardData} method in the each of the collection's shards.
|
||||
* <p>
|
||||
* First the overall data is imported, then the grouping data is imported.
|
||||
*
|
||||
*
|
||||
* @param input The bit-stream to import the shard data from
|
||||
* @throws IOException if an exception occurs while reading from the {@link DataInput}
|
||||
*/
|
||||
public synchronized void importShardData(DataInput input) throws IOException {
|
||||
ungroupedReductionManager.setShardInput(input);
|
||||
|
||||
|
||||
// The ungroupedData will not exist for the first shard imported
|
||||
if (ungroupedData == null) {
|
||||
ungroupedData = ungroupedReductionManager.newDataCollectionIO();
|
||||
|
@ -155,29 +155,29 @@ public class AnalyticsRequestManager {
|
|||
ungroupedReductionManager.prepareReductionDataIO(ungroupedData);
|
||||
}
|
||||
ungroupedReductionManager.mergeData();
|
||||
|
||||
|
||||
int size = input.readInt();
|
||||
while (--size >= 0) {
|
||||
String groupingName = input.readUTF();
|
||||
groupingManagers.get(groupingName).importShardData(input);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Export the shard data for this request through a bit-stream,
|
||||
* to be imported by the {@link #importShardData} method in the originating shard.
|
||||
* <p>
|
||||
* First the overall data is exported, then the grouping data is exported.
|
||||
*
|
||||
*
|
||||
* @param output The bit-stream to output the shard data through
|
||||
* @throws IOException if an exception occurs while writing to the {@link DataOutput}
|
||||
*/
|
||||
public void exportShardData(DataOutput output) throws IOException {
|
||||
ungroupedReductionManager.setShardOutput(output);
|
||||
|
||||
|
||||
ungroupedReductionManager.prepareReductionDataIO(ungroupedData);
|
||||
ungroupedReductionManager.exportData();
|
||||
|
||||
|
||||
output.writeInt(groupingManagers.size());
|
||||
for (String groupingName : groupingManagers.keySet()) {
|
||||
output.writeUTF(groupingName);
|
||||
|
@ -187,9 +187,9 @@ public class AnalyticsRequestManager {
|
|||
|
||||
/**
|
||||
* Consolidate the information of all {@link StreamingFacet}s contained within the request, since
|
||||
* they need to be collected along with the overall results during the streaming phase of the
|
||||
* they need to be collected along with the overall results during the streaming phase of the
|
||||
* {@link AnalyticsDriver}.
|
||||
*
|
||||
*
|
||||
* @return the info for all {@link StreamingFacet}s
|
||||
*/
|
||||
public StreamingInfo getStreamingFacetInfo() {
|
||||
|
@ -197,18 +197,18 @@ public class AnalyticsRequestManager {
|
|||
ArrayList<ReductionCollectionManager> groupingCollectors = new ArrayList<>();
|
||||
groupingManagers.values().forEach( grouping -> {
|
||||
// If a grouping has streaming facets, then that groupings expressions
|
||||
// must be collected during the streaming phase.
|
||||
// must be collected during the streaming phase.
|
||||
if (grouping.getStreamingFacets( facet -> streamingInfo.streamingFacets.add(facet) )) {
|
||||
groupingCollectors.add(grouping.getReductionManager());
|
||||
}
|
||||
});
|
||||
|
||||
|
||||
// Create an streaming collection manager to manage the collection of all ungrouped expressions and
|
||||
// grouped expressions that are calculated over streaming facets.
|
||||
streamingInfo.streamingCollectionManager = ungroupedReductionManager.merge(groupingCollectors);
|
||||
return streamingInfo;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Class to encapsulate all necessary data for collecting {@link StreamingFacet}s.
|
||||
*/
|
||||
|
@ -222,7 +222,7 @@ public class AnalyticsRequestManager {
|
|||
|
||||
/**
|
||||
* Create the {@link FacetValueQueryExecuter}s for all {@link AbstractSolrQueryFacet}s contained in the request.
|
||||
*
|
||||
*
|
||||
* @param filter representing the overall search query
|
||||
* @param queryRequest of the overall search query
|
||||
* @return an {@link Iterable} of executers
|
||||
|
@ -234,11 +234,11 @@ public class AnalyticsRequestManager {
|
|||
});
|
||||
return facetExecutors;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Create the response for a request given in the old olap-style format.
|
||||
* The old response returned overall expressions within groupings.
|
||||
*
|
||||
*
|
||||
* @return a {@link NamedList} representation of the response
|
||||
*/
|
||||
public NamedList<Object> createOldResponse() {
|
||||
|
@ -247,17 +247,17 @@ public class AnalyticsRequestManager {
|
|||
groupingManagers.forEach( (name, groupingManager) -> {
|
||||
analyticsResponse.add(name, groupingManager.createOldResponse(ungroupedResults));
|
||||
});
|
||||
|
||||
|
||||
return analyticsResponse;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Create the response for a request.
|
||||
*
|
||||
*
|
||||
* <p>
|
||||
* NOTE: Analytics requests specified in the old olap-style format
|
||||
* have their responses generated by {@link #createOldResponse()}.
|
||||
*
|
||||
*
|
||||
* @return a {@link Map} representation of the response
|
||||
*/
|
||||
public Map<String,Object> createResponse() {
|
||||
|
@ -270,7 +270,7 @@ public class AnalyticsRequestManager {
|
|||
groupingManagers.forEach( (name, groupingManager) -> {
|
||||
groupingsResponse.put(name, groupingManager.createResponse());
|
||||
});
|
||||
|
||||
|
||||
if (groupingsResponse.size() > 0) {
|
||||
analyticsResponse.put(AnalyticsResponseHeadings.GROUPINGS, groupingsResponse);
|
||||
}
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue