Index: lucene/src/test/org/apache/lucene/index/RandomIndexWriter.java =================================================================== --- lucene/src/test/org/apache/lucene/index/RandomIndexWriter.java (revision 1054574) +++ lucene/src/test/org/apache/lucene/index/RandomIndexWriter.java (working copy) @@ -87,6 +87,7 @@ if (LuceneTestCase.VERBOSE) { System.out.println("RIW config=" + w.getConfig()); System.out.println("codec default=" + w.getConfig().getCodecProvider().getDefaultFieldCodec()); + w.setInfoStream(System.out); } } Index: lucene/src/test/org/apache/lucene/index/TestCodecs.java =================================================================== --- lucene/src/test/org/apache/lucene/index/TestCodecs.java (revision 1054574) +++ lucene/src/test/org/apache/lucene/index/TestCodecs.java (working copy) @@ -23,14 +23,15 @@ import org.apache.lucene.analysis.MockAnalyzer; import org.apache.lucene.document.Document; -import org.apache.lucene.document.Field; import org.apache.lucene.document.Field.Store; +import org.apache.lucene.document.Field; import org.apache.lucene.index.codecs.CodecProvider; import org.apache.lucene.index.codecs.FieldsConsumer; import org.apache.lucene.index.codecs.FieldsProducer; import org.apache.lucene.index.codecs.PostingsConsumer; import org.apache.lucene.index.codecs.TermsConsumer; import org.apache.lucene.index.codecs.mocksep.MockSepCodec; +import org.apache.lucene.index.codecs.preflex.PreFlexCodec; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.PhraseQuery; @@ -40,6 +41,7 @@ import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.Version; +import org.apache.lucene.util._TestUtil; // TODO: test multiple codecs here? @@ -68,43 +70,6 @@ private final static int DOC_FREQ_RAND = 500; // must be > 16 to test skipping private final static int TERM_DOC_FREQ_RAND = 20; - // start is inclusive and end is exclusive - public int nextInt(final int start, final int end) { - return start + random.nextInt(end-start); - } - - private int nextInt(final int lim) { - return random.nextInt(lim); - } - - char[] getRandomText() { - - final int len = 1+this.nextInt(10); - final char[] buffer = new char[len+1]; - for(int i=0;i= 1) { - final int inc = 1+TestCodecs.this.nextInt(left-1); + if (TestCodecs.random.nextInt(3) == 1 && left >= 1) { + final int inc = 1+TestCodecs.random.nextInt(left-1); upto2 += inc; - if (TestCodecs.this.nextInt(2) == 1) { + if (TestCodecs.random.nextInt(2) == 1) { doc = docsEnum.advance(term.docs[upto2]); assertEquals(term.docs[upto2], doc); } else { @@ -586,7 +562,7 @@ assertEquals(term.docs[upto2], doc); if (!field.omitTF) { assertEquals(term.positions[upto2].length, docsEnum.freq()); - if (TestCodecs.this.nextInt(2) == 1) { + if (TestCodecs.random.nextInt(2) == 1) { this.verifyPositions(term.positions[upto2], postings); } } @@ -603,15 +579,19 @@ } } - private void write(final FieldInfos fieldInfos, final Directory dir, final FieldData[] fields) throws Throwable { + private void write(final FieldInfos fieldInfos, final Directory dir, final FieldData[] fields, boolean allowPreFlex) throws Throwable { - final int termIndexInterval = this.nextInt(13, 27); + final int termIndexInterval = _TestUtil.nextInt(random, 13, 27); final SegmentCodecs codecInfo = SegmentCodecs.build(fieldInfos, CodecProvider.getDefault()); final SegmentWriteState state = new SegmentWriteState(null, dir, SEGMENT, fieldInfos, 10000, termIndexInterval, codecInfo); final FieldsConsumer consumer = state.segmentCodecs.codec().fieldsConsumer(state); Arrays.sort(fields); for (final FieldData field : fields) { + if (!allowPreFlex && codecInfo.codecs[field.fieldInfo.codecId] instanceof PreFlexCodec) { + // code below expects unicode sort order + continue; + } field.write(consumer); } consumer.close(); Index: lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java =================================================================== --- lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java (revision 1054574) +++ lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockCodec.java (working copy) @@ -67,8 +67,14 @@ return name + "(baseBlockSize="+ baseBlockSize + ")"; } - private class MockIntFactory extends IntStreamFactory { + public static class MockIntFactory extends IntStreamFactory { + private final int baseBlockSize; + + public MockIntFactory(int baseBlockSize) { + this.baseBlockSize = baseBlockSize; + } + @Override public IntIndexInput openInput(Directory dir, String fileName, int readBufferSize) throws IOException { final IndexInput in = dir.openInput(fileName, readBufferSize); @@ -104,6 +110,7 @@ @Override protected int add(int value) throws IOException { + assert value >= 0; buffer[pendingCount++] = value; // silly variable block length int encoder: if // first value <= 3, we write N vints at once; @@ -128,7 +135,7 @@ @Override public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { - PostingsWriterBase postingsWriter = new SepPostingsWriterImpl(state, new MockIntFactory()); + PostingsWriterBase postingsWriter = new SepPostingsWriterImpl(state, new MockIntFactory(baseBlockSize)); boolean success = false; TermsIndexWriterBase indexWriter; @@ -162,7 +169,7 @@ PostingsReaderBase postingsReader = new SepPostingsReaderImpl(state.dir, state.segmentInfo, state.readBufferSize, - new MockIntFactory(), state.codecId); + new MockIntFactory(baseBlockSize), state.codecId); TermsIndexReaderBase indexReader; boolean success = false; Index: lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java =================================================================== --- lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java (revision 1054574) +++ lucene/src/test/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockCodec.java (working copy) @@ -67,11 +67,16 @@ // only for testing public IntStreamFactory getIntFactory() { - return new MockIntFactory(); + return new MockIntFactory(blockSize); } - private class MockIntFactory extends IntStreamFactory { + public static class MockIntFactory extends IntStreamFactory { + private final int blockSize; + public MockIntFactory(int blockSize) { + this.blockSize = blockSize; + } + @Override public IntIndexInput openInput(Directory dir, String fileName, int readBufferSize) throws IOException { return new FixedIntBlockIndexInput(dir.openInput(fileName, readBufferSize)) { @@ -96,6 +101,7 @@ @Override protected void flushBlock() throws IOException { for(int i=0;i= 0; out.writeVInt(buffer[i]); } } @@ -105,7 +111,7 @@ @Override public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { - PostingsWriterBase postingsWriter = new SepPostingsWriterImpl(state, new MockIntFactory()); + PostingsWriterBase postingsWriter = new SepPostingsWriterImpl(state, new MockIntFactory(blockSize)); boolean success = false; TermsIndexWriterBase indexWriter; @@ -139,7 +145,7 @@ PostingsReaderBase postingsReader = new SepPostingsReaderImpl(state.dir, state.segmentInfo, state.readBufferSize, - new MockIntFactory(), state.codecId); + new MockIntFactory(blockSize), state.codecId); TermsIndexReaderBase indexReader; boolean success = false; Index: lucene/src/test/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexOutput.java =================================================================== --- lucene/src/test/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexOutput.java (revision 1054574) +++ lucene/src/test/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexOutput.java (working copy) @@ -42,6 +42,7 @@ /** Write an int to the primary file */ @Override public void write(int v) throws IOException { + assert v >= 0; out.writeVInt(v); } Index: lucene/src/test/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java =================================================================== --- lucene/src/test/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java (revision 0) +++ lucene/src/test/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java (revision 0) @@ -0,0 +1,316 @@ +package org.apache.lucene.index.codecs.mockrandom; + +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import java.io.IOException; +import java.util.Iterator; +import java.util.Random; +import java.util.Set; + +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.SegmentInfo; +import org.apache.lucene.index.SegmentReadState; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.index.codecs.Codec; +import org.apache.lucene.index.codecs.FieldsConsumer; +import org.apache.lucene.index.codecs.FieldsProducer; +import org.apache.lucene.index.codecs.FixedGapTermsIndexReader; +import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter; +import org.apache.lucene.index.codecs.PostingsReaderBase; +import org.apache.lucene.index.codecs.PostingsWriterBase; +import org.apache.lucene.index.codecs.PrefixCodedTermsReader; +import org.apache.lucene.index.codecs.PrefixCodedTermsWriter; +import org.apache.lucene.index.codecs.TermsIndexReaderBase; +import org.apache.lucene.index.codecs.TermsIndexWriterBase; +import org.apache.lucene.index.codecs.VariableGapTermsIndexReader; +import org.apache.lucene.index.codecs.VariableGapTermsIndexWriter; +import org.apache.lucene.index.codecs.mockintblock.MockFixedIntBlockCodec; +import org.apache.lucene.index.codecs.mockintblock.MockVariableIntBlockCodec; +import org.apache.lucene.index.codecs.mocksep.MockSingleIntFactory; +import org.apache.lucene.index.codecs.pulsing.PulsingPostingsReaderImpl; +import org.apache.lucene.index.codecs.pulsing.PulsingPostingsWriterImpl; +import org.apache.lucene.index.codecs.sep.SepPostingsReaderImpl; +import org.apache.lucene.index.codecs.sep.SepPostingsWriterImpl; +import org.apache.lucene.index.codecs.standard.StandardPostingsReader; +import org.apache.lucene.index.codecs.standard.StandardPostingsWriter; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.LuceneTestCase; +import org.apache.lucene.util._TestUtil; + +/** + * Randomly combines terms index impl w/ postings impls. + */ + +public class MockRandomCodec extends Codec { + + private final Random seedRandom; + private final String SEED_EXT = "sd"; + + public MockRandomCodec(Random random) { + name = "MockRandom"; + this.seedRandom = random; + } + + @Override + public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException { + + final long seed = seedRandom.nextLong(); + + final String seedFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, SEED_EXT); + final IndexOutput out = state.directory.createOutput(seedFileName); + out.writeLong(seed); + out.close(); + state.flushedFiles.add(seedFileName); + + final Random random = new Random(seed); + PostingsWriterBase postingsWriter; + final int n = random.nextInt(4); + + if (n == 0) { + if (LuceneTestCase.VERBOSE) { + System.out.println("MockRandomCodec: writing MockSep postings"); + } + postingsWriter = new SepPostingsWriterImpl(state, new MockSingleIntFactory()); + } else if (n == 1) { + final int blockSize = _TestUtil.nextInt(random, 1, 2000); + if (LuceneTestCase.VERBOSE) { + System.out.println("MockRandomCodec: writing MockFixedIntBlock(" + blockSize + ") postings"); + } + postingsWriter = new SepPostingsWriterImpl(state, new MockFixedIntBlockCodec.MockIntFactory(blockSize)); + } else if (n == 2) { + final int baseBlockSize = _TestUtil.nextInt(random, 1, 127); + if (LuceneTestCase.VERBOSE) { + System.out.println("MockRandomCodec: writing MockVariableIntBlock(" + baseBlockSize + ") postings"); + } + postingsWriter = new SepPostingsWriterImpl(state, new MockVariableIntBlockCodec.MockIntFactory(baseBlockSize)); + } else { + if (LuceneTestCase.VERBOSE) { + System.out.println("MockRandomCodec: writing Standard postings"); + } + postingsWriter = new StandardPostingsWriter(state); + } + + if (random.nextBoolean()) { + final int freqCutoff = _TestUtil.nextInt(random, 1, 20); + if (LuceneTestCase.VERBOSE) { + System.out.println("MockRandomCodec: pulsing postings with freqCutoff=" + freqCutoff); + } + postingsWriter = new PulsingPostingsWriterImpl(freqCutoff, postingsWriter); + } + + final TermsIndexWriterBase indexWriter; + boolean success = false; + + try { + if (random.nextBoolean()) { + state.termIndexInterval = _TestUtil.nextInt(random, 1, 100); + if (LuceneTestCase.VERBOSE) { + System.out.println("MockRandomCodec: fixed-gap terms index (tii=" + state.termIndexInterval + ")"); + } + indexWriter = new FixedGapTermsIndexWriter(state); + } else { + final VariableGapTermsIndexWriter.IndexTermSelector selector; + if (random.nextBoolean()) { + final int tii = _TestUtil.nextInt(random, 1, 100); + selector = new VariableGapTermsIndexWriter.EveryNTermSelector(tii); + if (LuceneTestCase.VERBOSE) { + System.out.println("MockRandomCodec: variable-gap terms index (tii=" + tii + ")"); + } + } else { + final long seed2 = random.nextLong(); + final int gap = _TestUtil.nextInt(random, 2, 40); + if (LuceneTestCase.VERBOSE) { + System.out.println("MockRandomCodec: random-gap terms index (max gap=" + gap + ")"); + } + selector = new VariableGapTermsIndexWriter.IndexTermSelector() { + final Random rand = new Random(seed2); + + @Override + public boolean isIndexTerm(BytesRef term, int docFreq) { + return random.nextInt(gap) == 17; + } + }; + } + indexWriter = new VariableGapTermsIndexWriter(state, selector); + } + success = true; + } finally { + if (!success) { + postingsWriter.close(); + } + } + + success = false; + try { + FieldsConsumer ret = new PrefixCodedTermsWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator()); + success = true; + return ret; + } finally { + if (!success) { + try { + postingsWriter.close(); + } finally { + indexWriter.close(); + } + } + } + } + + @Override + public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException { + + final String seedFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.codecId, SEED_EXT); + final IndexInput in = state.dir.openInput(seedFileName); + final long seed = in.readLong(); + in.close(); + + final Random random = new Random(seed); + PostingsReaderBase postingsReader; + final int n = random.nextInt(4); + + if (n == 0) { + if (LuceneTestCase.VERBOSE) { + System.out.println("MockRandomCodec: reading MockSep postings"); + } + postingsReader = new SepPostingsReaderImpl(state.dir, state.segmentInfo, + state.readBufferSize, new MockSingleIntFactory(), state.codecId); + } else if (n == 1) { + final int blockSize = _TestUtil.nextInt(random, 1, 2000); + if (LuceneTestCase.VERBOSE) { + System.out.println("MockRandomCodec: reading MockFixedIntBlock(" + blockSize + ") postings"); + } + postingsReader = new SepPostingsReaderImpl(state.dir, state.segmentInfo, + state.readBufferSize, new MockFixedIntBlockCodec.MockIntFactory(blockSize), state.codecId); + } else if (n == 2) { + final int baseBlockSize = _TestUtil.nextInt(random, 1, 127); + if (LuceneTestCase.VERBOSE) { + System.out.println("MockRandomCodec: reading MockVariableIntBlock(" + baseBlockSize + ") postings"); + } + postingsReader = new SepPostingsReaderImpl(state.dir, state.segmentInfo, + state.readBufferSize, new MockVariableIntBlockCodec.MockIntFactory(baseBlockSize), state.codecId); + } else { + if (LuceneTestCase.VERBOSE) { + System.out.println("MockRandomCodec: reading Standard postings"); + } + postingsReader = new StandardPostingsReader(state.dir, state.segmentInfo, state.readBufferSize, state.codecId); + } + + if (random.nextBoolean()) { + final int freqCutoff = _TestUtil.nextInt(random, 1, 20); + if (LuceneTestCase.VERBOSE) { + System.out.println("MockRandomCodec: reading pulsing postings with freqCutoff=" + freqCutoff); + } + postingsReader = new PulsingPostingsReaderImpl(postingsReader); + } + + final TermsIndexReaderBase indexReader; + boolean success = false; + + try { + if (random.nextBoolean()) { + state.termsIndexDivisor = _TestUtil.nextInt(random, 1, 10); + if (LuceneTestCase.VERBOSE) { + System.out.println("MockRandomCodec: fixed-gap terms index (divisor=" + state.termsIndexDivisor + ")"); + } + indexReader = new FixedGapTermsIndexReader(state.dir, + state.fieldInfos, + state.segmentInfo.name, + state.termsIndexDivisor, + BytesRef.getUTF8SortedAsUnicodeComparator(), + state.codecId); + } else { + if (!random.nextBoolean()) { + random.nextLong(); + } + if (LuceneTestCase.VERBOSE) { + System.out.println("MockRandomCodec: variable-gap terms index (divisor=" + state.termsIndexDivisor + ")"); + } + state.termsIndexDivisor = _TestUtil.nextInt(random, 1, 10); + indexReader = new VariableGapTermsIndexReader(state.dir, + state.fieldInfos, + state.segmentInfo.name, + state.termsIndexDivisor, + state.codecId); + } + success = true; + } finally { + if (!success) { + postingsReader.close(); + } + } + + final int termsCacheSize = _TestUtil.nextInt(random, 1, 1024); + + success = false; + try { + FieldsProducer ret = new PrefixCodedTermsReader(indexReader, + state.dir, + state.fieldInfos, + state.segmentInfo.name, + postingsReader, + state.readBufferSize, + BytesRef.getUTF8SortedAsUnicodeComparator(), + termsCacheSize, + state.codecId); + success = true; + return ret; + } finally { + if (!success) { + try { + postingsReader.close(); + } finally { + indexReader.close(); + } + } + } + } + + @Override + public void files(Directory dir, SegmentInfo segmentInfo, String codecId, Set files) throws IOException { + final String seedFileName = IndexFileNames.segmentFileName(segmentInfo.name, codecId, SEED_EXT); + files.add(seedFileName); + SepPostingsReaderImpl.files(segmentInfo, codecId, files); + StandardPostingsReader.files(dir, segmentInfo, codecId, files); + PrefixCodedTermsReader.files(dir, segmentInfo, codecId, files); + FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files); + VariableGapTermsIndexReader.files(dir, segmentInfo, codecId, files); + + // hackish! + Iterator it = files.iterator(); + while(it.hasNext()) { + final String file = it.next(); + if (!dir.fileExists(file)) { + it.remove(); + } + } + //System.out.println("MockRandom.files return " + files); + } + + @Override + public void getExtensions(Set extensions) { + SepPostingsWriterImpl.getExtensions(extensions); + PrefixCodedTermsReader.getExtensions(extensions); + FixedGapTermsIndexReader.getIndexExtensions(extensions); + VariableGapTermsIndexReader.getIndexExtensions(extensions); + extensions.add(SEED_EXT); + //System.out.println("MockRandom.getExtensions return " + extensions); + } +} Property changes on: lucene/src/test/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java ___________________________________________________________________ Added: svn:eol-style + native Index: lucene/src/test/org/apache/lucene/index/TestPerFieldCodecSupport.java =================================================================== --- lucene/src/test/org/apache/lucene/index/TestPerFieldCodecSupport.java (revision 1054574) +++ lucene/src/test/org/apache/lucene/index/TestPerFieldCodecSupport.java (working copy) @@ -56,6 +56,7 @@ conf.setMergePolicy(logByteSizeMergePolicy); final IndexWriter writer = new IndexWriter(dir, conf); + writer.setInfoStream(VERBOSE ? System.out : null); return writer; } @@ -110,12 +111,15 @@ } /* - * Test is hetrogenous index segements are merge sucessfully + * Test that heterogeneous index segments are merged sucessfully */ @Test public void testChangeCodecAndMerge() throws IOException { Directory dir = newDirectory(); CodecProvider provider = new MockCodecProvider(); + if (VERBOSE) { + System.out.println("TEST: make new index"); + } IndexWriterConfig iwconf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer()).setOpenMode(OpenMode.CREATE).setCodecProvider(provider); iwconf.setMaxBufferedDocs(IndexWriterConfig.DISABLE_AUTO_FLUSH); @@ -125,6 +129,9 @@ addDocs(writer, 10); writer.commit(); assertQuery(new Term("content", "aaa"), dir, 10, provider); + if (VERBOSE) { + System.out.println("TEST: addDocs3"); + } addDocs3(writer, 10); writer.commit(); writer.close(); @@ -144,6 +151,9 @@ iwconf.setCodecProvider(provider); writer = newWriter(dir, iwconf); // swap in new codec for currently written segments + if (VERBOSE) { + System.out.println("TEST: add docs w/ Standard codec for content field"); + } addDocs2(writer, 10); writer.commit(); Codec origContentCodec = provider.lookup("MockSep"); @@ -152,9 +162,12 @@ origContentCodec, origContentCodec, newContentCodec); assertEquals(30, writer.maxDoc()); assertQuery(new Term("content", "bbb"), dir, 10, provider); - assertQuery(new Term("content", "ccc"), dir, 10, provider); + assertQuery(new Term("content", "ccc"), dir, 10, provider); //// assertQuery(new Term("content", "aaa"), dir, 10, provider); + if (VERBOSE) { + System.out.println("TEST: add more docs w/ new codec"); + } addDocs2(writer, 10); writer.commit(); assertQuery(new Term("content", "ccc"), dir, 10, provider); @@ -162,6 +175,9 @@ assertQuery(new Term("content", "aaa"), dir, 10, provider); assertEquals(40, writer.maxDoc()); + if (VERBOSE) { + System.out.println("TEST: now optimize"); + } writer.optimize(); assertEquals(40, writer.maxDoc()); writer.close(); @@ -206,6 +222,9 @@ public void assertQuery(Term t, Directory dir, int num, CodecProvider codecs) throws CorruptIndexException, IOException { + if (VERBOSE) { + System.out.println("\nTEST: assertQuery " + t); + } IndexReader reader = IndexReader.open(dir, null, true, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, codecs); IndexSearcher searcher = new IndexSearcher(reader); Index: lucene/src/test/org/apache/lucene/util/LuceneTestCase.java =================================================================== --- lucene/src/test/org/apache/lucene/util/LuceneTestCase.java (revision 1054574) +++ lucene/src/test/org/apache/lucene/util/LuceneTestCase.java (working copy) @@ -42,6 +42,7 @@ import org.apache.lucene.index.codecs.mockintblock.MockFixedIntBlockCodec; import org.apache.lucene.index.codecs.mockintblock.MockVariableIntBlockCodec; import org.apache.lucene.index.codecs.mocksep.MockSepCodec; +import org.apache.lucene.index.codecs.mockrandom.MockRandomCodec; import org.apache.lucene.index.codecs.preflex.PreFlexCodec; import org.apache.lucene.index.codecs.preflexrw.PreFlexRWCodec; import org.apache.lucene.index.codecs.pulsing.PulsingCodec; @@ -199,7 +200,7 @@ private static Map stores; - private static final String[] TEST_CODECS = new String[] {"MockSep", "MockFixedIntBlock", "MockVariableIntBlock"}; + private static final String[] TEST_CODECS = new String[] {"MockSep", "MockFixedIntBlock", "MockVariableIntBlock", "MockRandom"}; private static void swapCodec(Codec c, CodecProvider cp) { Codec prior = null; @@ -252,6 +253,7 @@ swapCodec(new MockFixedIntBlockCodec(codecHasParam && "MockFixedIntBlock".equals(codec) ? codecParam : _TestUtil.nextInt(random, 1, 2000)), cp); // baseBlockSize cannot be over 127: swapCodec(new MockVariableIntBlockCodec(codecHasParam && "MockVariableIntBlock".equals(codec) ? codecParam : _TestUtil.nextInt(random, 1, 127)), cp); + swapCodec(new MockRandomCodec(random), cp); return cp.lookup(codec); } @@ -268,9 +270,9 @@ cp.unregister(cp.lookup("MockSep")); cp.unregister(cp.lookup("MockFixedIntBlock")); cp.unregister(cp.lookup("MockVariableIntBlock")); + cp.unregister(cp.lookup("MockRandom")); swapCodec(new PulsingCodec(1), cp); cp.setDefaultFieldCodec(savedDefaultCodec); - } // randomly picks from core and test codecs Index: lucene/src/test/org/apache/lucene/util/automaton/fst/TestFSTs.java =================================================================== --- lucene/src/test/org/apache/lucene/util/automaton/fst/TestFSTs.java (revision 1054574) +++ lucene/src/test/org/apache/lucene/util/automaton/fst/TestFSTs.java (working copy) @@ -122,8 +122,10 @@ } public void testBasicFSA() throws IOException { - String[] strings = new String[] {"station", "commotion", "elation", "elastic", "plastic", "stop", "ftop", "ftation"}; + String[] strings = new String[] {"station", "commotion", "elation", "elastic", "plastic", "stop", "ftop", "ftation", "stat"}; + String[] strings2 = new String[] {"station", "commotion", "elation", "elastic", "plastic", "stop", "ftop", "ftation"}; IntsRef[] terms = new IntsRef[strings.length]; + IntsRef[] terms2 = new IntsRef[strings2.length]; for(int inputMode=0;inputMode<2;inputMode++) { if (VERBOSE) { System.out.println("TEST: inputMode=" + inputModeToString(inputMode)); @@ -132,6 +134,10 @@ for(int idx=0;idx outputs = NoOutputs.getSingleton(); final Object NO_OUTPUT = outputs.getNoOutput(); - final List> pairs = new ArrayList>(terms.length); - for(IntsRef term : terms) { + final List> pairs = new ArrayList>(terms2.length); + for(IntsRef term : terms2) { pairs.add(new FSTTester.InputOutput(term, NO_OUTPUT)); } FST fst = new FSTTester(random, dir, inputMode, pairs, outputs).doTest(0, 0); @@ -154,9 +160,9 @@ // FST ord pos int { final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true); - final List> pairs = new ArrayList>(terms.length); - for(int idx=0;idx(terms[idx], outputs.get(idx))); + final List> pairs = new ArrayList>(terms2.length); + for(int idx=0;idx(terms2[idx], outputs.get(idx))); } final FST fst = new FSTTester(random, dir, inputMode, pairs, outputs).doTest(0, 0); assertNotNull(fst); @@ -168,10 +174,10 @@ { final ByteSequenceOutputs outputs = ByteSequenceOutputs.getSingleton(); final BytesRef NO_OUTPUT = outputs.getNoOutput(); - final List> pairs = new ArrayList>(terms.length); - for(int idx=0;idx> pairs = new ArrayList>(terms2.length); + for(int idx=0;idx(terms[idx], output)); + pairs.add(new FSTTester.InputOutput(terms2[idx], output)); } final FST fst = new FSTTester(random, dir, inputMode, pairs, outputs).doTest(0, 0); assertNotNull(fst); @@ -322,19 +328,6 @@ } } - private String getRandomString() { - final String term; - if (random.nextBoolean()) { - term = _TestUtil.randomRealisticUnicodeString(random); - } else { - // we want to mix in limited-alphabet symbols so - // we get more sharing of the nodes given how few - // terms we are testing... - term = simpleRandomString(random); - } - return term; - } - public void doTest() throws IOException { // no pruning doTest(0, 0); @@ -346,156 +339,83 @@ doTest(0, _TestUtil.nextInt(random, 1, 1+pairs.size())); } - // NOTE: only copies the stuff this test needs!! - private FST.Arc copyArc(FST.Arc arc) { - final FST.Arc copy = new FST.Arc(); - copy.label = arc.label; - copy.target = arc.target; - copy.output = arc.output; - copy.nextFinalOutput = arc.nextFinalOutput; - return arc; - } + // runs the term, returning the output, or null if term + // isn't accepted. if prefixLength is non-null it must be + // length 1 int array; prefixLength[0] is set to the length + // of the term prefix that matches + private T run(FST fst, IntsRef term, int[] prefixLength) throws IOException { + assert prefixLength == null || prefixLength.length == 1; + final FST.Arc arc = fst.getFirstArc(new FST.Arc()); + final T NO_OUTPUT = fst.outputs.getNoOutput(); + T output = NO_OUTPUT; - // runs the term, returning the output, or null if term - // isn't accepted. if stopNode is non-null it must be - // length 2 int array; stopNode[0] will be the last - // matching node (-1 if the term is accepted) - // and stopNode[1] will be the length of the - // term prefix that matches - private T run(FST fst, IntsRef term, int[] stopNode) throws IOException { - if (term.length == 0) { - final T output = fst.getEmptyOutput(); - if (stopNode != null) { - stopNode[1] = 0; - if (output != null) { - // accepted - stopNode[0] = -1; - } else { - stopNode[0] = fst.getStartNode(); - } + for(int i=0;i<=term.length;i++) { + final int label; + if (i == term.length) { + label = FST.END_LABEL; + } else { + label = term.ints[term.offset+i]; } - return output; - } - - final FST.Arc arc = new FST.Arc(); - int node = fst.getStartNode(); - int lastNode = -1; - T output = fst.outputs.getNoOutput(); - //System.out.println("match?"); - for(int i=0;i fst, IntsRef in) throws IOException { - int node = fst.getStartNode(); + FST.Arc arc = fst.getFirstArc(new FST.Arc()); - if (fst.noNodes()) { - // degenerate FST: only accepts the empty string - assertTrue(fst.getEmptyOutput() != null); - in.length = 0; - return fst.getEmptyOutput(); - } final List> arcs = new ArrayList>(); in.length = 0; in.offset = 0; - T output = fst.outputs.getNoOutput(); - //System.out.println("get random"); + final T NO_OUTPUT = fst.outputs.getNoOutput(); + T output = NO_OUTPUT; + while(true) { // read all arcs: - //System.out.println(" n=" + node); - int arcAddress = node; - FST.Arc arc = new FST.Arc(); - fst.readFirstArc(arcAddress, arc); - arcs.add(copyArc(arc)); + fst.readFirstTargetArc(arc, arc); + arcs.add(new FST.Arc().copyFrom(arc)); while(!arc.isLast()) { fst.readNextArc(arc); - arcs.add(copyArc(arc)); + arcs.add(new FST.Arc().copyFrom(arc)); } // pick one arc = arcs.get(random.nextInt(arcs.size())); - arcs.clear(); + // accumulate output + output = fst.outputs.add(output, arc.output); + // append label + if (arc.label == FST.END_LABEL) { + break; + } + if (in.ints.length == in.length) { in.grow(1+in.length); } in.ints[in.length++] = arc.label; - - output = fst.outputs.add(output, arc.output); - - // maybe stop - if (arc.isFinal()) { - if (fst.hasArcs(arc.target)) { - // final state but it also has outgoing edges - if (random.nextBoolean()) { - output = fst.outputs.add(output, arc.nextFinalOutput); - break; - } - } else { - break; - } - } - - node = arc.target; } return output; } - private FST doTest(int prune1, int prune2) throws IOException { + FST doTest(int prune1, int prune2) throws IOException { if (VERBOSE) { System.out.println("TEST: prune1=" + prune1 + " prune2=" + prune2); } @@ -524,7 +444,7 @@ if (VERBOSE && pairs.size() <= 20 && fst != null) { PrintStream ps = new PrintStream("out.dot"); - fst.toDot(ps); + Util.toDot(fst, ps); ps.close(); System.out.println("SAVED out.dot"); } @@ -566,11 +486,19 @@ assertNotNull(fst); - // make sure all words are accepted + // visit valid paris in order -- make sure all words + // are accepted, and FSTEnum's next() steps through + // them correctly + if (VERBOSE) { + System.out.println("TEST: check valid terms/next()"); + } { IntsRefFSTEnum fstEnum = new IntsRefFSTEnum(fst); for(InputOutput pair : pairs) { IntsRef term = pair.input; + if (VERBOSE) { + System.out.println("TEST: check term=" + inputToString(inputMode, term) + " output=" + fst.outputs.outputToString(pair.output)); + } Object output = run(fst, term, null); assertNotNull("term " + inputToString(inputMode, term) + " is not accepted", output); @@ -578,8 +506,8 @@ // verify enum's next IntsRefFSTEnum.InputOutput t = fstEnum.next(); - - assertEquals(term, t.input); + assertNotNull(t); + assertEquals("expected input=" + inputToString(inputMode, term) + " but fstEnum returned " + inputToString(inputMode, t.input), term, t.input); assertEquals(pair.output, t.output); } assertNull(fstEnum.next()); @@ -591,6 +519,9 @@ } // find random matching word and make sure it's valid + if (VERBOSE) { + System.out.println("TEST: verify random accepted terms"); + } final IntsRef scratch = new IntsRef(10); for(int iter=0;iter<500*RANDOM_MULTIPLIER;iter++) { T output = randomAcceptedWord(fst, scratch); @@ -598,10 +529,15 @@ assertEquals(termsMap.get(scratch), output); } - // test single IntsRefFSTEnum.advance: - //System.out.println("TEST: verify advance"); + // test IntsRefFSTEnum.seek: + if (VERBOSE) { + System.out.println("TEST: verify seek"); + } + IntsRefFSTEnum fstEnum = new IntsRefFSTEnum(fst); for(int iter=0;iter<100*RANDOM_MULTIPLIER;iter++) { - final IntsRefFSTEnum fstEnum = new IntsRefFSTEnum(fst); + if (VERBOSE) { + System.out.println("TEST: iter=" + iter); + } if (random.nextBoolean()) { // seek to term that doesn't exist: while(true) { @@ -611,15 +547,35 @@ pos = -(pos+1); // ok doesn't exist //System.out.println(" seek " + inputToString(inputMode, term)); - final IntsRefFSTEnum.InputOutput seekResult = fstEnum.advance(term); - if (pos < pairs.size()) { + final IntsRefFSTEnum.InputOutput seekResult; + if (random.nextBoolean()) { + if (VERBOSE) { + System.out.println(" do non-exist seekFloor term=" + inputToString(inputMode, term)); + } + seekResult = fstEnum.seekFloor(term); + pos--; + } else { + if (VERBOSE) { + System.out.println(" do non-exist seekCeil term=" + inputToString(inputMode, term)); + } + seekResult = fstEnum.seekCeil(term); + } + + if (pos != -1 && pos < pairs.size()) { //System.out.println(" got " + inputToString(inputMode,seekResult.input) + " output=" + fst.outputs.outputToString(seekResult.output)); - assertEquals(pairs.get(pos).input, seekResult.input); + assertNotNull("got null but expected term=" + inputToString(inputMode, pairs.get(pos).input), seekResult); + if (VERBOSE) { + System.out.println(" got " + inputToString(inputMode, seekResult.input)); + } + assertEquals("expected " + inputToString(inputMode, pairs.get(pos).input) + " but got " + inputToString(inputMode, seekResult.input), pairs.get(pos).input, seekResult.input); assertEquals(pairs.get(pos).output, seekResult.output); } else { - // seeked beyond end + // seeked before start or beyond end //System.out.println("seek=" + seekTerm); assertNull("expected null but got " + (seekResult==null ? "null" : inputToString(inputMode, seekResult.input)), seekResult); + if (VERBOSE) { + System.out.println(" got null"); + } } break; @@ -627,24 +583,36 @@ } } else { // seek to term that does exist: - InputOutput pair = pairs.get(random.nextInt(pairs.size())); - //System.out.println(" seek " + inputToString(inputMode, pair.input)); - final IntsRefFSTEnum.InputOutput seekResult = fstEnum.advance(pair.input); - assertEquals(pair.input, seekResult.input); + InputOutput pair = pairs.get(random.nextInt(pairs.size())); + final IntsRefFSTEnum.InputOutput seekResult; + if (random.nextBoolean()) { + if (VERBOSE) { + System.out.println(" do exists seekFloor " + inputToString(inputMode, pair.input)); + } + seekResult = fstEnum.seekFloor(pair.input); + } else { + if (VERBOSE) { + System.out.println(" do exists seekCeil " + inputToString(inputMode, pair.input)); + } + seekResult = fstEnum.seekCeil(pair.input); + } + assertNotNull(seekResult); + assertEquals("got " + inputToString(inputMode, seekResult.input) + " but expected " + inputToString(inputMode, pair.input), pair.input, seekResult.input); assertEquals(pair.output, seekResult.output); } } if (VERBOSE) { - System.out.println("TEST: mixed next/advance"); + System.out.println("TEST: mixed next/seek"); } - // test mixed next/advance + // test mixed next/seek for(int iter=0;iter<100*RANDOM_MULTIPLIER;iter++) { if (VERBOSE) { System.out.println("TEST: iter " + iter); } - final IntsRefFSTEnum fstEnum = new IntsRefFSTEnum(fst); + // reset: + fstEnum = new IntsRefFSTEnum(fst); int upto = -1; while(true) { boolean isDone = false; @@ -660,13 +628,24 @@ for(;attempt<10;attempt++) { IntsRef term = toIntsRef(getRandomString(), inputMode); if (!termsMap.containsKey(term) && term.compareTo(pairs.get(upto).input) > 0) { - if (VERBOSE) { - System.out.println(" do non-exist advance(" + inputToString(inputMode, term) + "]"); - } int pos = Collections.binarySearch(pairs, new InputOutput(term, null)); assert pos < 0; upto = -(pos+1); - isDone = fstEnum.advance(term) == null; + + if (random.nextBoolean()) { + upto--; + assertTrue(upto != -1); + if (VERBOSE) { + System.out.println(" do non-exist seekFloor(" + inputToString(inputMode, term) + ")"); + } + isDone = fstEnum.seekFloor(term) == null; + } else { + if (VERBOSE) { + System.out.println(" do non-exist seekCeil(" + inputToString(inputMode, term) + ")"); + } + isDone = fstEnum.seekCeil(term) == null; + } + break; } } @@ -681,10 +660,17 @@ upto = 0; } - if (VERBOSE) { - System.out.println(" do advance(" + inputToString(inputMode, pairs.get(upto).input) + "]"); + if (random.nextBoolean()) { + if (VERBOSE) { + System.out.println(" do advanceCeil(" + inputToString(inputMode, pairs.get(upto).input) + ")"); + } + isDone = fstEnum.seekCeil(pairs.get(upto).input) == null; + } else { + if (VERBOSE) { + System.out.println(" do advanceFloor(" + inputToString(inputMode, pairs.get(upto).input) + ")"); + } + isDone = fstEnum.seekFloor(pairs.get(upto).input) == null; } - isDone = fstEnum.advance(pairs.get(upto).input) == null; } if (VERBOSE) { if (!isDone) { @@ -701,6 +687,24 @@ assertFalse(isDone); assertEquals(pairs.get(upto).input, fstEnum.current().input); assertEquals(pairs.get(upto).output, fstEnum.current().output); + + /* + if (upto < pairs.size()-1) { + int tryCount = 0; + while(tryCount < 10) { + final IntsRef t = toIntsRef(getRandomString(), inputMode); + if (pairs.get(upto).input.compareTo(t) < 0) { + final boolean expected = t.compareTo(pairs.get(upto+1).input) < 0; + if (VERBOSE) { + System.out.println("TEST: call beforeNext(" + inputToString(inputMode, t) + "); current=" + inputToString(inputMode, pairs.get(upto).input) + " next=" + inputToString(inputMode, pairs.get(upto+1).input) + " expected=" + expected); + } + assertEquals(expected, fstEnum.beforeNext(t)); + break; + } + tryCount++; + } + } + */ } } } @@ -757,7 +761,9 @@ } } - //System.out.println("TEST: now prune"); + if (VERBOSE) { + System.out.println("TEST: now prune"); + } // prune 'em final Iterator>> it = prefixes.entrySet().iterator(); @@ -765,7 +771,9 @@ Map.Entry> ent = it.next(); final IntsRef prefix = ent.getKey(); final CountMinOutput cmo = ent.getValue(); - //System.out.println(" term=" + inputToString(inputMode, prefix) + " count=" + cmo.count + " isLeaf=" + cmo.isLeaf); + if (VERBOSE) { + System.out.println(" term=" + inputToString(inputMode, prefix) + " count=" + cmo.count + " isLeaf=" + cmo.isLeaf + " output=" + outputs.outputToString(cmo.output) + " isFinal=" + cmo.isFinal); + } final boolean keep; if (prune1 > 0) { keep = cmo.count >= prune1; @@ -824,14 +832,20 @@ assertNotNull(fst); // make sure FST only enums valid prefixes + if (VERBOSE) { + System.out.println("TEST: check pruned enum"); + } IntsRefFSTEnum fstEnum = new IntsRefFSTEnum(fst); - IntsRefFSTEnum.InputOutput current; + IntsRefFSTEnum.InputOutput current; while((current = fstEnum.next()) != null) { - //System.out.println(" fst enum term=" + inputToString(inputMode, current.input) + " output=" + outputs.outputToString(current.output)); + if (VERBOSE) { + System.out.println(" fstEnum.next term=" + inputToString(inputMode, current.input) + " output=" + outputs.outputToString(current.output)); + } final CountMinOutput cmo = prefixes.get(current.input); assertNotNull(cmo); assertTrue(cmo.isLeaf || cmo.isFinal); - if (cmo.isFinal && !cmo.isLeaf) { + //if (cmo.isFinal && !cmo.isLeaf) { + if (cmo.isFinal) { assertEquals(cmo.finalOutput, current.output); } else { assertEquals(cmo.output, current.output); @@ -839,19 +853,24 @@ } // make sure all non-pruned prefixes are present in the FST - final int[] stopNode = new int[2]; + if (VERBOSE) { + System.out.println("TEST: verify all prefixes"); + } + final int[] stopNode = new int[1]; for(Map.Entry> ent : prefixes.entrySet()) { if (ent.getKey().length > 0) { final CountMinOutput cmo = ent.getValue(); final T output = run(fst, ent.getKey(), stopNode); - //System.out.println(" term=" + inputToString(inputMode, ent.getKey()) + " output=" + outputs.outputToString(cmo.output)); + if (VERBOSE) { + System.out.println("TEST: verify term=" + inputToString(inputMode, ent.getKey()) + " output=" + outputs.outputToString(cmo.output)); + } // if (cmo.isFinal && !cmo.isLeaf) { if (cmo.isFinal) { assertEquals(cmo.finalOutput, output); } else { assertEquals(cmo.output, output); } - assertEquals(ent.getKey().length, stopNode[1]); + assertEquals(ent.getKey().length, stopNode[0]); } } } @@ -859,7 +878,7 @@ public void testRandomWords() throws IOException { testRandomWords(1000, 5 * RANDOM_MULTIPLIER); - //testRandomWords(10, 100); + //testRandomWords(20, 100); } private String inputModeToString(int mode) { @@ -888,7 +907,7 @@ } } - private String getRandomString() { + static String getRandomString() { final String term; if (random.nextBoolean()) { term = _TestUtil.randomRealisticUnicodeString(random); @@ -909,10 +928,10 @@ private static String inputToString(int inputMode, IntsRef term) { if (inputMode == 0) { // utf8 - return toBytesRef(term).utf8ToString(); + return toBytesRef(term).utf8ToString() + " " + term; } else { // utf32 - return UnicodeUtil.newString(term.ints, term.offset, term.length); + return UnicodeUtil.newString(term.ints, term.offset, term.length) + " " + term; } } @@ -986,18 +1005,17 @@ // same: final BytesRefFSTEnum fstEnum = new BytesRefFSTEnum(fst); for(int iter=0;iter<1000*RANDOM_MULTIPLIER;iter++) { - fstEnum.reset(); final BytesRef randomTerm = new BytesRef(getRandomString()); final TermsEnum.SeekStatus seekResult = termsEnum.seek(randomTerm); - final BytesRefFSTEnum.InputOutput fstSeekResult = fstEnum.advance(randomTerm); + final BytesRefFSTEnum.InputOutput fstSeekResult = fstEnum.seekCeil(randomTerm); if (VERBOSE) { - System.out.println("TEST: seek " + randomTerm.utf8ToString()); + System.out.println("TEST: seek " + randomTerm.utf8ToString() + " " + randomTerm); } if (seekResult == TermsEnum.SeekStatus.END) { - assertNull(fstSeekResult); + assertNull("got " + (fstSeekResult == null ? "null" : fstSeekResult.input.utf8ToString()) + " but expected null", fstSeekResult); } else { assertSame(termsEnum, fstEnum, storeOrd); for(int nextIter=0;nextIter<10;nextIter++) { @@ -1032,7 +1050,8 @@ if (termsEnum.term() == null) { assertNull(fstEnum.current()); } else { - assertEquals(termsEnum.term(), fstEnum.current().input); + assertNotNull(fstEnum.current()); + assertEquals(termsEnum.term().utf8ToString() + " != " + fstEnum.current().input.utf8ToString(), termsEnum.term(), fstEnum.current().input); if (storeOrd) { // fst stored the ord assertEquals(termsEnum.ord(), ((Long) fstEnum.current().output).longValue()); @@ -1095,7 +1114,7 @@ System.out.println(ord + " terms; " + fst.getNodeCount() + " nodes; " + fst.getArcCount() + " arcs; " + fst.getArcWithOutputCount() + " arcs w/ output; tot size " + fst.sizeInBytes()); if (fst.getNodeCount() < 100) { PrintStream ps = new PrintStream("out.dot"); - fst.toDot(ps); + Util.toDot(fst, ps); ps.close(); System.out.println("Wrote FST to out.dot"); } @@ -1121,7 +1140,7 @@ } toIntsRef(w, inputMode, intsRef); T expected = getOutput(intsRef, ord); - T actual = fst.get(intsRef); + T actual = Util.get(fst, intsRef); if (actual == null) { throw new RuntimeException("unexpected null output on input=" + w); } @@ -1233,4 +1252,13 @@ }.run(limit); } } + + public void testSingleString() throws Exception { + final Outputs outputs = NoOutputs.getSingleton(); + final Builder b = new Builder(FST.INPUT_TYPE.BYTE1, 0, 0, true, outputs); + b.add(new BytesRef("foobar"), outputs.getNoOutput()); + final BytesRefFSTEnum fstEnum = new BytesRefFSTEnum(b.finish()); + assertNull(fstEnum.seekFloor(new BytesRef("foo"))); + assertNull(fstEnum.seekCeil(new BytesRef("foobaz"))); + } } Index: lucene/src/test/org/apache/lucene/util/_TestUtil.java =================================================================== --- lucene/src/test/org/apache/lucene/util/_TestUtil.java (revision 1054574) +++ lucene/src/test/org/apache/lucene/util/_TestUtil.java (working copy) @@ -109,7 +109,6 @@ final char[] buffer = new char[end]; for (int i = 0; i < end; i++) { int t = r.nextInt(5); - if (0 == t && i < end - 1) { // Make a surrogate pair // High surrogate Index: lucene/src/java/org/apache/lucene/search/ExactPhraseScorer.java =================================================================== --- lucene/src/java/org/apache/lucene/search/ExactPhraseScorer.java (revision 1054574) +++ lucene/src/java/org/apache/lucene/search/ExactPhraseScorer.java (working copy) @@ -134,7 +134,7 @@ // this doc has all the terms -- now test whether // phrase occurs docID = doc; - + freq = phraseFreq(); if (freq != 0) { return docID; Index: lucene/src/java/org/apache/lucene/index/SegmentCodecs.java =================================================================== --- lucene/src/java/org/apache/lucene/index/SegmentCodecs.java (revision 1054574) +++ lucene/src/java/org/apache/lucene/index/SegmentCodecs.java (working copy) @@ -124,7 +124,6 @@ for (int i = 0; i < codecArray.length; i++) { codecArray[i].files(dir, info, ""+i, files); } - } @Override Index: lucene/src/java/org/apache/lucene/index/DocumentsWriter.java =================================================================== --- lucene/src/java/org/apache/lucene/index/DocumentsWriter.java (revision 1054574) +++ lucene/src/java/org/apache/lucene/index/DocumentsWriter.java (working copy) @@ -906,7 +906,8 @@ final static int BYTE_BLOCK_NOT_MASK = ~BYTE_BLOCK_MASK; /* if you increase this, you must fix field cache impl for - * getTerms/getTermsIndex requires <= 32768 */ + * getTerms/getTermsIndex requires <= 32768. Also fix + * PrefixCodedTermsWriter's EOF_MARKER if necessary. */ final static int MAX_TERM_LENGTH_UTF8 = BYTE_BLOCK_SIZE-2; /* Initial chunks size of the shared int[] blocks used to Index: lucene/src/java/org/apache/lucene/index/SegmentReadState.java =================================================================== --- lucene/src/java/org/apache/lucene/index/SegmentReadState.java (revision 1054574) +++ lucene/src/java/org/apache/lucene/index/SegmentReadState.java (working copy) @@ -33,7 +33,7 @@ // terms index on init (preflex is the only once currently // that must do so), then it should negate this value to // get the app's terms divisor: - public final int termsIndexDivisor; + public int termsIndexDivisor; public final String codecId; public SegmentReadState(Directory dir, SegmentInfo info, Index: lucene/src/java/org/apache/lucene/index/SegmentWriteState.java =================================================================== --- lucene/src/java/org/apache/lucene/index/SegmentWriteState.java (revision 1054574) +++ lucene/src/java/org/apache/lucene/index/SegmentWriteState.java (working copy) @@ -43,7 +43,9 @@ * faster, while larger values use less memory and make searching slightly * slower. Searching is typically not dominated by dictionary lookup, so * tweaking this is rarely useful.*/ - public final int termIndexInterval; + // nocommit -- remove this here and from IndexWriter? + // codec should control it + public int termIndexInterval; /** Expert: The fraction of TermDocs entries stored in skip tables, * used to accelerate {@link DocsEnum#advance(int)}. Larger values result in Index: lucene/src/java/org/apache/lucene/index/codecs/PrefixCodedTermsWriter.java =================================================================== --- lucene/src/java/org/apache/lucene/index/codecs/PrefixCodedTermsWriter.java (revision 1054574) +++ lucene/src/java/org/apache/lucene/index/codecs/PrefixCodedTermsWriter.java (working copy) @@ -93,7 +93,7 @@ } @Override - public TermsConsumer addField(FieldInfo field) { + public TermsConsumer addField(FieldInfo field) throws IOException { assert currentField == null || currentField.name.compareTo(field.name) < 0; currentField = field; TermsIndexWriterBase.FieldWriter fieldIndexWriter = termsIndexWriter.addField(field); @@ -173,12 +173,25 @@ public void finishTerm(BytesRef text, int numDocs) throws IOException { assert numDocs > 0; + //System.out.println("finishTerm term=" + fieldInfo.name + ":" + text.utf8ToString() + " fp=" + out.getFilePointer()); final boolean isIndexTerm = fieldIndexWriter.checkIndexTerm(text, numDocs); termWriter.write(text); - out.writeVInt(numDocs); + final int highBit = isIndexTerm ? 0x80 : 0; + //System.out.println(" isIndex=" + isIndexTerm); + // This is a vInt, except, we steal top bit to record + // whether this was an indexed term: + if ((numDocs & ~0x3F) == 0) { + // Fast case -- docFreq fits in 6 bits + out.writeByte((byte) (highBit | numDocs)); + } else { + // Write bottom 6 bits of docFreq, then write the + // remainder as vInt: + out.writeByte((byte) (highBit | 0x40 | (numDocs & 0x3F))); + out.writeVInt(numDocs >>> 6); + } postingsWriter.finishTerm(numDocs, isIndexTerm); numTerms++; } @@ -186,6 +199,8 @@ // Finishes all terms in this field @Override public void finish() throws IOException { + // EOF marker: + out.writeVInt(DeltaBytesWriter.TERM_EOF); fieldIndexWriter.finish(); } } Index: lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexReader.java =================================================================== --- lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexReader.java (revision 1054574) +++ lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexReader.java (working copy) @@ -33,29 +33,6 @@ import java.util.Comparator; import java.io.IOException; -/** - * Uses a simplistic format to record terms dict index - * information. Limititations: - * - * - Index for all fields is loaded entirely into RAM up - * front - * - Index is stored in RAM using shared byte[] that - * wastefully expand every term. Using FST to share - * common prefix & suffix would save RAM. - * - Index is taken at regular numTerms (every 128 by - * default); might be better to do it by "net docFreqs" - * encountered, so that for spans of low-freq terms we - * take index less often. - * - * A better approach might be something similar to how - * postings are encoded, w/ multi-level skips. Ie, load all - * terms index data into memory, as a single large compactly - * encoded stream (eg delta bytes + delta offset). Index - * that w/ multi-level skipper. Then to look up a term is - * the equivalent binary search, using the skipper instead, - * while data remains compressed in memory. - */ - import org.apache.lucene.index.IndexFileNames; /** @lucene.experimental */ @@ -74,7 +51,7 @@ final private int indexInterval; // Closed if indexLoaded is true: - final private IndexInput in; + private IndexInput in; private volatile boolean indexLoaded; private final Comparator termComp; @@ -85,7 +62,7 @@ private final PagedBytes termBytes = new PagedBytes(PAGED_BYTES_BITS); private PagedBytes.Reader termBytesReader; - final HashMap fields = new HashMap(); + final HashMap fields = new HashMap(); // start of the field info data protected long dirOffset; @@ -95,7 +72,7 @@ this.termComp = termComp; - IndexInput in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, FixedGapTermsIndexWriter.TERMS_INDEX_EXTENSION)); + in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, FixedGapTermsIndexWriter.TERMS_INDEX_EXTENSION)); boolean success = false; @@ -116,50 +93,138 @@ seekDir(in, dirOffset); // Read directory - final int numFields = in.readInt(); - + final int numFields = in.readVInt(); for(int i=0;i= indexStart: "packedStart=" + packedIndexStart + " indexStart=" + indexStart + " numIndexTerms=" + numIndexTerms + " seg=" + segment; - if (numIndexTerms > 0) { - final FieldInfo fieldInfo = fieldInfos.fieldInfo(field); - fields.put(fieldInfo, new FieldIndexReader(in, fieldInfo, numIndexTerms, indexStart, termsStart, packedIndexStart, packedOffsetsStart)); - } + final FieldInfo fieldInfo = fieldInfos.fieldInfo(field); + fields.put(fieldInfo, new FieldIndexData(fieldInfo, numIndexTerms, indexStart, termsStart, packedIndexStart, packedOffsetsStart)); } success = true; } finally { if (indexDivisor > 0) { in.close(); - this.in = null; + in = null; if (success) { indexLoaded = true; } termBytesReader = termBytes.freeze(true); - } else { - this.in = in; } } } + @Override + public int getDivisor() { + return indexDivisor; + } + protected void readHeader(IndexInput input) throws IOException { CodecUtil.checkHeader(input, FixedGapTermsIndexWriter.CODEC_NAME, FixedGapTermsIndexWriter.VERSION_START, FixedGapTermsIndexWriter.VERSION_START); dirOffset = input.readLong(); } - private final class FieldIndexReader extends FieldReader { + private class IndexEnum extends FieldIndexEnum { + private final FieldIndexData.CoreFieldIndex fieldIndex; + private final BytesRef term = new BytesRef(); + private final BytesRef nextTerm = new BytesRef(); + private long ord; - final private FieldInfo fieldInfo; + public IndexEnum(FieldIndexData.CoreFieldIndex fieldIndex) { + this.fieldIndex = fieldIndex; + } - private volatile CoreFieldIndex coreIndex; + @Override + public BytesRef term() { + return term; + } - private final IndexInput in; + @Override + public long seek(BytesRef target) { + int lo = 0; // binary search + int hi = fieldIndex.numIndexTerms - 1; + assert totalIndexInterval > 0 : "totalIndexInterval=" + totalIndexInterval; + while (hi >= lo) { + int mid = (lo + hi) >>> 1; + + final long offset = fieldIndex.termOffsets.get(mid); + final int length = (int) (fieldIndex.termOffsets.get(1+mid) - offset); + termBytesReader.fillSlice(term, fieldIndex.termBytesStart + offset, length); + + int delta = termComp.compare(target, term); + if (delta < 0) { + hi = mid - 1; + } else if (delta > 0) { + lo = mid + 1; + } else { + assert mid >= 0; + ord = mid*totalIndexInterval; + return fieldIndex.termsStart + fieldIndex.termsDictOffsets.get(mid); + } + } + + if (hi < 0) { + assert hi == -1; + hi = 0; + } + + final long offset = fieldIndex.termOffsets.get(hi); + final int length = (int) (fieldIndex.termOffsets.get(1+hi) - offset); + termBytesReader.fillSlice(term, fieldIndex.termBytesStart + offset, length); + + ord = hi*totalIndexInterval; + return fieldIndex.termsStart + fieldIndex.termsDictOffsets.get(hi); + } + + @Override + public long next() { + final int idx = 1 + (int) (ord / totalIndexInterval); + if (idx >= fieldIndex.numIndexTerms) { + return -1; + } + ord += totalIndexInterval; + + final long offset = fieldIndex.termOffsets.get(idx); + final int length = (int) (fieldIndex.termOffsets.get(1+idx) - offset); + termBytesReader.fillSlice(nextTerm, fieldIndex.termBytesStart + offset, length); + return fieldIndex.termsStart + fieldIndex.termsDictOffsets.get(idx); + } + + @Override + public long ord() { + return ord; + } + + @Override + public long seek(long ord) { + int idx = (int) (ord / totalIndexInterval); + // caller must ensure ord is in bounds + assert idx < fieldIndex.numIndexTerms; + final long offset = fieldIndex.termOffsets.get(idx); + final int length = (int) (fieldIndex.termOffsets.get(1+idx) - offset); + termBytesReader.fillSlice(term, fieldIndex.termBytesStart + offset, length); + this.ord = idx * totalIndexInterval; + return fieldIndex.termsStart + fieldIndex.termsDictOffsets.get(idx); + } + } + + @Override + public boolean supportsOrd() { + return true; + } + + private final class FieldIndexData { + + final private FieldInfo fieldInfo; + + volatile CoreFieldIndex coreIndex; + private final long indexStart; private final long termsStart; private final long packedIndexStart; @@ -167,11 +232,10 @@ private final int numIndexTerms; - public FieldIndexReader(IndexInput in, FieldInfo fieldInfo, int numIndexTerms, long indexStart, long termsStart, long packedIndexStart, - long packedOffsetsStart) throws IOException { + public FieldIndexData(FieldInfo fieldInfo, int numIndexTerms, long indexStart, long termsStart, long packedIndexStart, + long packedOffsetsStart) throws IOException { this.fieldInfo = fieldInfo; - this.in = in; this.termsStart = termsStart; this.indexStart = indexStart; this.packedIndexStart = packedIndexStart; @@ -182,12 +246,7 @@ // is -1, so that PrefixCodedTermsReader can call // isIndexTerm for each field: if (indexDivisor > 0) { - coreIndex = new CoreFieldIndex(indexStart, - termsStart, - packedIndexStart, - packedOffsetsStart, - numIndexTerms); - + loadTermsIndex(); } } @@ -197,46 +256,11 @@ } } - @Override - public boolean isIndexTerm(long ord, int docFreq, boolean onlyLoaded) { - if (onlyLoaded) { - return ord % totalIndexInterval == 0; - } else { - return ord % indexInterval == 0; - } - } - - @Override - public boolean nextIndexTerm(long ord, TermsIndexResult result) throws IOException { - if (coreIndex == null) { - throw new IllegalStateException("terms index was not loaded"); - } else { - return coreIndex.nextIndexTerm(ord, result); - } - } - - @Override - public void getIndexOffset(BytesRef term, TermsIndexResult result) throws IOException { - // You must call loadTermsIndex if you had specified -1 for indexDivisor - if (coreIndex == null) { - throw new IllegalStateException("terms index was not loaded"); - } - coreIndex.getIndexOffset(term, result); - } - - @Override - public void getIndexOffset(long ord, TermsIndexResult result) throws IOException { - // You must call loadTermsIndex if you had specified - // indexDivisor < 0 to ctor - if (coreIndex == null) { - throw new IllegalStateException("terms index was not loaded"); - } - coreIndex.getIndexOffset(ord, result); - } - private final class CoreFieldIndex { - final private long termBytesStart; + // where this field's terms begin in the packed byte[] + // data + final long termBytesStart; // offset into index termBytes final PackedInts.Reader termOffsets; @@ -245,7 +269,6 @@ final PackedInts.Reader termsDictOffsets; final int numIndexTerms; - final long termsStart; public CoreFieldIndex(long indexStart, long termsStart, long packedIndexStart, long packedOffsetsStart, int numIndexTerms) throws IOException { @@ -315,7 +338,6 @@ termsDictOffsetsM.set(upto, termsDictOffsetsIter.next()); termOffsetsM.set(upto, termOffsetUpto); - upto++; long termOffset = termOffsetsIter.next(); long nextTermOffset = termOffsetsIter.next(); @@ -328,6 +350,11 @@ termBytes.copy(clone, numTermBytes); termOffsetUpto += numTermBytes; + upto++; + if (upto == this.numIndexTerms) { + break; + } + // skip terms: termsDictOffsetsIter.next(); for(int i=0;i 0 : "totalIndexInterval=" + totalIndexInterval; - - while (hi >= lo) { - int mid = (lo + hi) >>> 1; - - final long offset = termOffsets.get(mid); - final int length = (int) (termOffsets.get(1+mid) - offset); - termBytesReader.fillSlice(result.term, termBytesStart + offset, length); - - int delta = termComp.compare(term, result.term); - if (delta < 0) { - hi = mid - 1; - } else if (delta > 0) { - lo = mid + 1; - } else { - assert mid >= 0; - result.position = mid*totalIndexInterval; - result.offset = termsStart + termsDictOffsets.get(mid); - return; - } - } - if (hi < 0) { - assert hi == -1; - hi = 0; - } - - final long offset = termOffsets.get(hi); - final int length = (int) (termOffsets.get(1+hi) - offset); - termBytesReader.fillSlice(result.term, termBytesStart + offset, length); - - result.position = hi*totalIndexInterval; - result.offset = termsStart + termsDictOffsets.get(hi); - } - - public void getIndexOffset(long ord, TermsIndexResult result) throws IOException { - int idx = (int) (ord / totalIndexInterval); - // caller must ensure ord is in bounds - assert idx < numIndexTerms; - fillResult(idx, result); - } } } + // Externally synced in IndexWriter @Override public void loadTermsIndex(int indexDivisor) throws IOException { if (!indexLoaded) { @@ -420,7 +386,7 @@ } this.totalIndexInterval = indexInterval * this.indexDivisor; - Iterator it = fields.values().iterator(); + Iterator it = fields.values().iterator(); while(it.hasNext()) { it.next().loadTermsIndex(); } @@ -432,8 +398,13 @@ } @Override - public FieldReader getField(FieldInfo fieldInfo) { - return fields.get(fieldInfo); + public FieldIndexEnum getFieldEnum(FieldInfo fieldInfo) { + final FieldIndexData fieldData = fields.get(fieldInfo); + if (fieldData.coreIndex == null) { + return null; + } else { + return new IndexEnum(fieldData.coreIndex); + } } public static void files(Directory dir, SegmentInfo info, String id, Collection files) { Index: lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java =================================================================== --- lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java (revision 1054574) +++ lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java (working copy) @@ -130,9 +130,8 @@ public SeekStatus seek(BytesRef text, boolean useCache /* ignored */) throws IOException { - fstEnum.reset(); //System.out.println("seek to text=" + text.utf8ToString()); - final BytesRefFSTEnum.InputOutput> result = fstEnum.advance(text); + final BytesRefFSTEnum.InputOutput> result = fstEnum.seekCeil(text); if (result == null) { //System.out.println(" end"); return SeekStatus.END; Index: lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java =================================================================== --- lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java (revision 1054574) +++ lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java (working copy) @@ -157,7 +157,7 @@ final int delta = position - lastPosition; - assert delta > 0 || position == 0 || position == -1: "position=" + position + " lastPosition=" + lastPosition; // not quite right (if pos=0 is repeated twice we don't catch it) + assert delta > 0 || position == 0: "position=" + position + " lastPosition=" + lastPosition; // not quite right (if pos=0 is repeated twice we don't catch it) lastPosition = position; Index: lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java =================================================================== --- lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java (revision 1054574) +++ lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java (working copy) @@ -33,6 +33,8 @@ import org.apache.lucene.index.codecs.TermsIndexReaderBase; import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter; import org.apache.lucene.index.codecs.FixedGapTermsIndexReader; +import org.apache.lucene.index.codecs.VariableGapTermsIndexWriter; +import org.apache.lucene.index.codecs.VariableGapTermsIndexReader; import org.apache.lucene.index.codecs.PrefixCodedTermsWriter; import org.apache.lucene.index.codecs.PrefixCodedTermsReader; import org.apache.lucene.store.Directory; @@ -56,7 +58,8 @@ TermsIndexWriterBase indexWriter; boolean success = false; try { - indexWriter = new FixedGapTermsIndexWriter(state); + //indexWriter = new FixedGapTermsIndexWriter(state); + indexWriter = new VariableGapTermsIndexWriter(state, new VariableGapTermsIndexWriter.EveryNTermSelector(32)); success = true; } finally { if (!success) { @@ -89,12 +92,19 @@ boolean success = false; try { + indexReader = new VariableGapTermsIndexReader(state.dir, + state.fieldInfos, + state.segmentInfo.name, + state.termsIndexDivisor, + state.codecId); + /* indexReader = new FixedGapTermsIndexReader(state.dir, state.fieldInfos, state.segmentInfo.name, state.termsIndexDivisor, BytesRef.getUTF8SortedAsUnicodeComparator(), state.codecId); + */ success = true; } finally { if (!success) { @@ -136,7 +146,8 @@ public void files(Directory dir, SegmentInfo segmentInfo, String id, Set files) throws IOException { StandardPostingsReader.files(dir, segmentInfo, id, files); PrefixCodedTermsReader.files(dir, segmentInfo, id, files); - FixedGapTermsIndexReader.files(dir, segmentInfo, id, files); + VariableGapTermsIndexReader.files(dir, segmentInfo, id, files); + //FixedGapTermsIndexReader.files(dir, segmentInfo, id, files); } @Override @@ -148,6 +159,7 @@ extensions.add(FREQ_EXTENSION); extensions.add(PROX_EXTENSION); PrefixCodedTermsReader.getExtensions(extensions); - FixedGapTermsIndexReader.getIndexExtensions(extensions); + VariableGapTermsIndexReader.getIndexExtensions(extensions); + //FixedGapTermsIndexReader.getIndexExtensions(extensions); } } Index: lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexWriter.java =================================================================== --- lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexWriter.java (revision 1054574) +++ lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexWriter.java (working copy) @@ -31,7 +31,14 @@ import java.util.ArrayList; import java.io.IOException; -/** @lucene.experimental */ +/** + * Selects every Nth term as and index term, and hold term + * bytes fully expanded in memory. This terms index + * supports seeking by ord. See {@link + * VariableGapTermsIndexWriter} for a more memory efficient + * terms index that does not support seeking by ord. + * + * @lucene.experimental */ public class FixedGapTermsIndexWriter extends TermsIndexWriterBase { protected final IndexOutput out; @@ -203,16 +210,26 @@ final long dirStart = out.getFilePointer(); final int fieldCount = fields.size(); - out.writeInt(fieldCount); + int nonNullFieldCount = 0; for(int i=0;i 0) { + nonNullFieldCount++; + } } + + out.writeVInt(nonNullFieldCount); + for(int i=0;i 0) { + out.writeVInt(field.fieldInfo.number); + out.writeVInt(field.numIndexTerms); + out.writeVLong(field.termsStart); + out.writeVLong(field.indexStart); + out.writeVLong(field.packedIndexStart); + out.writeVLong(field.packedOffsetsStart); + } + } writeTrailer(dirStart); out.close(); } Index: lucene/src/java/org/apache/lucene/index/codecs/TermsIndexReaderBase.java =================================================================== --- lucene/src/java/org/apache/lucene/index/codecs/TermsIndexReaderBase.java (revision 1054574) +++ lucene/src/java/org/apache/lucene/index/codecs/TermsIndexReaderBase.java (working copy) @@ -21,6 +21,7 @@ import org.apache.lucene.util.BytesRef; import java.io.IOException; +import java.io.Closeable; import java.util.Collection; @@ -38,39 +39,37 @@ * text. * @lucene.experimental */ -public abstract class TermsIndexReaderBase { +public abstract class TermsIndexReaderBase implements Closeable { - static class TermsIndexResult { - long position; - final BytesRef term = new BytesRef(); - long offset; - }; + public abstract FieldIndexEnum getFieldEnum(FieldInfo fieldInfo); - public abstract class FieldReader { - /** Returns position of "largest" index term that's <= - * text. Returned TermsIndexResult may be reused - * across calls. This resets internal state, and - * expects that you'll then scan the file and - * sequentially call isIndexTerm for each term - * encountered. */ - public abstract void getIndexOffset(BytesRef term, TermsIndexResult result) throws IOException; + public abstract void loadTermsIndex(int indexDivisor) throws IOException; - public abstract void getIndexOffset(long ord, TermsIndexResult result) throws IOException; + public abstract void close() throws IOException; - /** Call this sequentially for each term encountered, - * after calling {@link #getIndexOffset}. */ - public abstract boolean isIndexTerm(long ord, int docFreq, boolean onlyLoaded) throws IOException; + public abstract void getExtensions(Collection extensions); - /** Finds the next index term, after the specified - * ord. Returns true if one exists. */ - public abstract boolean nextIndexTerm(long ord, TermsIndexResult result) throws IOException; - } + public abstract boolean supportsOrd(); - public abstract FieldReader getField(FieldInfo fieldInfo); + public abstract int getDivisor(); - public abstract void loadTermsIndex(int indexDivisor) throws IOException; + // Similar to TermsEnum, except, the only "metadata" it + // reports for a given indexed term is the long fileOffset + // into the main terms dict (_X.tis) file: + public static abstract class FieldIndexEnum { - public abstract void close() throws IOException; + /** Seeks to "largest" indexed term that's <= + * term; retruns file pointer index (into the main + * terms index file) for that term */ + public abstract long seek(BytesRef term) throws IOException; - public abstract void getExtensions(Collection extensions); -} \ No newline at end of file + /** Returns -1 at end */ + public abstract long next() throws IOException; + + public abstract BytesRef term(); + + // Only impl'd if supportsOrd() returns true! + public abstract long seek(long ord) throws IOException; + public abstract long ord(); + } +} Index: lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexReader.java =================================================================== --- lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexReader.java (revision 0) +++ lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexReader.java (revision 0) @@ -0,0 +1,241 @@ +package org.apache.lucene.index.codecs; + +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import java.io.IOException; +import java.util.Collection; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Iterator; + +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.SegmentInfo; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.CodecUtil; +import org.apache.lucene.util.automaton.fst.BytesRefFSTEnum; +import org.apache.lucene.util.automaton.fst.FST; +import org.apache.lucene.util.automaton.fst.PositiveIntOutputs; + +/** See {@link VariableGapTermsIndexWriter} + * + * @lucene.experimental */ +public class VariableGapTermsIndexReader extends TermsIndexReaderBase { + + private final PositiveIntOutputs fstOutputs = PositiveIntOutputs.getSingleton(true); + private int indexDivisor; + + // Closed if indexLoaded is true: + private IndexInput in; + private volatile boolean indexLoaded; + + final HashMap fields = new HashMap(); + + // start of the field info data + protected long dirOffset; + + // nocommit -- must get divisor working + public VariableGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor, String codecId) + throws IOException { + + in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, VariableGapTermsIndexWriter.TERMS_INDEX_EXTENSION)); + + boolean success = false; + + try { + + readHeader(in); + this.indexDivisor = indexDivisor; + + seekDir(in, dirOffset); + + // Read directory + final int numFields = in.readVInt(); + + for(int i=0;i 0) { + in.close(); + in = null; + if (success) { + indexLoaded = true; + } + } + } + } + + @Override + public int getDivisor() { + // nocommit fix + return 1; + } + + protected void readHeader(IndexInput input) throws IOException { + CodecUtil.checkHeader(input, VariableGapTermsIndexWriter.CODEC_NAME, + VariableGapTermsIndexWriter.VERSION_START, VariableGapTermsIndexWriter.VERSION_START); + dirOffset = input.readLong(); + } + + private static class IndexEnum extends FieldIndexEnum { + private final BytesRefFSTEnum fstEnum; + private BytesRefFSTEnum.InputOutput current; + + public IndexEnum(FST fst) { + fstEnum = new BytesRefFSTEnum(fst); + } + + @Override + public BytesRef term() { + if (current == null) { + return null; + } else { + return current.input; + } + } + + @Override + public long seek(BytesRef target) throws IOException { + //System.out.println("VGR: seek field=" + fieldInfo.name + " target=" + target); + current = fstEnum.seekFloor(target); + //System.out.println(" got input=" + current.input + " output=" + current.output); + return current.output; + } + + @Override + public long next() throws IOException { + //System.out.println("VGR: next field=" + fieldInfo.name); + current = fstEnum.next(); + if (current == null) { + //System.out.println(" eof"); + return -1; + } else { + return current.output; + } + } + + @Override + public long ord() { + throw new UnsupportedOperationException(); + } + + @Override + public long seek(long ord) { + throw new UnsupportedOperationException(); + } + } + + @Override + public boolean supportsOrd() { + return false; + } + + private final class FieldIndexData { + + private final FieldInfo fieldInfo; + private final long indexStart; + + // Set only if terms index is loaded: + private volatile FST fst; + + public FieldIndexData(FieldInfo fieldInfo, long indexStart) throws IOException { + + this.fieldInfo = fieldInfo; + this.indexStart = indexStart; + + // We still create the indexReader when indexDivisor + // is -1, so that PrefixCodedTermsReader can call + // isIndexTerm for each field: + if (indexDivisor > 0) { + loadTermsIndex(); + } + } + + public void loadTermsIndex() throws IOException { + if (fst == null) { + IndexInput clone = (IndexInput) in.clone(); + clone.seek(indexStart); + fst = new FST(clone, fstOutputs); + clone.close(); + } + } + } + + // Externally synced in IndexWriter + @Override + public void loadTermsIndex(int indexDivisor) throws IOException { + if (!indexLoaded) { + + if (indexDivisor < 0) { + this.indexDivisor = -indexDivisor; + } else { + this.indexDivisor = indexDivisor; + } + + Iterator it = fields.values().iterator(); + while(it.hasNext()) { + it.next().loadTermsIndex(); + } + + indexLoaded = true; + in.close(); + } + } + + @Override + public FieldIndexEnum getFieldEnum(FieldInfo fieldInfo) { + final FieldIndexData fieldData = fields.get(fieldInfo); + if (fieldData.fst == null) { + return null; + } else { + return new IndexEnum(fieldData.fst); + } + } + + public static void files(Directory dir, SegmentInfo info, String id, Collection files) { + files.add(IndexFileNames.segmentFileName(info.name, id, VariableGapTermsIndexWriter.TERMS_INDEX_EXTENSION)); + } + + public static void getIndexExtensions(Collection extensions) { + extensions.add(VariableGapTermsIndexWriter.TERMS_INDEX_EXTENSION); + } + + @Override + public void getExtensions(Collection extensions) { + getIndexExtensions(extensions); + } + + @Override + public void close() throws IOException { + if (in != null && !indexLoaded) { + in.close(); + } + } + + protected void seekDir(IndexInput input, long dirOffset) throws IOException { + input.seek(dirOffset); + } +} Property changes on: lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexReader.java ___________________________________________________________________ Added: svn:eol-style + native Index: lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java =================================================================== --- lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java (revision 1054574) +++ lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java (working copy) @@ -594,6 +594,7 @@ } final int code = posReader.next(); + assert code >= 0; if (storePayloads) { if ((code & 1) != 0) { // Payload length has changed Index: lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java =================================================================== --- lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java (revision 1054574) +++ lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java (working copy) @@ -211,6 +211,7 @@ assert !omitTF; final int delta = position - lastPosition; + assert delta > 0 || position == 0: "position=" + position + " lastPosition=" + lastPosition; // not quite right (if pos=0 is repeated twice we don't catch it) lastPosition = position; if (storePayloads) { Index: lucene/src/java/org/apache/lucene/index/codecs/TermsIndexWriterBase.java =================================================================== --- lucene/src/java/org/apache/lucene/index/codecs/TermsIndexWriterBase.java (revision 1054574) +++ lucene/src/java/org/apache/lucene/index/codecs/TermsIndexWriterBase.java (working copy) @@ -32,7 +32,7 @@ public abstract void finish() throws IOException; } - public abstract FieldWriter addField(FieldInfo fieldInfo); + public abstract FieldWriter addField(FieldInfo fieldInfo) throws IOException; public abstract void close() throws IOException; -} \ No newline at end of file +} Index: lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexWriter.java =================================================================== --- lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexWriter.java (revision 0) +++ lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexWriter.java (revision 0) @@ -0,0 +1,215 @@ +package org.apache.lucene.index.codecs; + +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.CodecUtil; +import org.apache.lucene.util.automaton.fst.Builder; +import org.apache.lucene.util.automaton.fst.FST; +import org.apache.lucene.util.automaton.fst.PositiveIntOutputs; + +/** + * Selects index terms according to provided pluggable + * IndexTermPolicy, and stores them in a prefix trie that's + * loaded entirely in RAM stored as an FST. This terms + * index only supports unsigned byte term sort order + * (unicode codepoint order when the bytes are UTF8). + * + * @lucene.experimental */ +public class VariableGapTermsIndexWriter extends TermsIndexWriterBase { + protected final IndexOutput out; + + /** Extension of terms index file */ + static final String TERMS_INDEX_EXTENSION = "tiv"; + + final static String CODEC_NAME = "VARIABLE_GAP_TERMS_INDEX"; + final static int VERSION_START = 0; + final static int VERSION_CURRENT = VERSION_START; + + private final List fields = new ArrayList(); + private final FieldInfos fieldInfos; // unread + private IndexOutput termsOut; + private final IndexTermSelector policy; + + /** @lucene.experimental */ + public static abstract class IndexTermSelector { + // Called sequentially on every term being written, + // returning true if this term should be indexed + public abstract boolean isIndexTerm(BytesRef term, int docFreq); + } + + /** Same policy as {@link FixedGapTermsIndexWriter} */ + public static final class EveryNTermSelector extends IndexTermSelector { + private int termCount; + private final int interval; + + public EveryNTermSelector(int interval) { + this.interval = interval; + } + + @Override + public boolean isIndexTerm(BytesRef term, int docFreq) { + // First term is first indexed term: + return 0 == (termCount++ % interval); + } + } + + // nocommit add a "by high DF or every N" selector + + // nocommit add a "FST decides" selector!! hmm + + public VariableGapTermsIndexWriter(SegmentWriteState state, IndexTermSelector policy) throws IOException { + final String indexFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, TERMS_INDEX_EXTENSION); + state.flushedFiles.add(indexFileName); + out = state.directory.createOutput(indexFileName); + fieldInfos = state.fieldInfos; + this.policy = policy; + writeHeader(out); + } + + protected void writeHeader(IndexOutput out) throws IOException { + CodecUtil.writeHeader(out, CODEC_NAME, VERSION_CURRENT); + // Placeholder for dir offset + out.writeLong(0); + } + + @Override + public void setTermsOutput(IndexOutput termsOut) { + this.termsOut = termsOut; + } + + @Override + public FieldWriter addField(FieldInfo field) throws IOException { + //System.out.println("VGW: field=" + field.name); + FSTFieldWriter writer = new FSTFieldWriter(field); + fields.add(writer); + return writer; + } + + /** NOTE: if your codec does not sort in unicode code + * point order, you must override this method, to simply + * return indexedTerm.length. */ + protected int indexedTermPrefixLength(final BytesRef priorTerm, final BytesRef indexedTerm) { + // As long as codec sorts terms in unicode codepoint + // order, we can safely strip off the non-distinguishing + // suffix to save RAM in the loaded terms index. + final int idxTermOffset = indexedTerm.offset; + final int priorTermOffset = priorTerm.offset; + final int limit = Math.min(priorTerm.length, indexedTerm.length); + for(int byteIdx=0;byteIdx fstBuilder; + private final PositiveIntOutputs fstOutputs; + + final FieldInfo fieldInfo; + int numIndexTerms; + FST fst; + final long indexStart; + + private final BytesRef lastTerm = new BytesRef(); + private boolean first = true; + + public FSTFieldWriter(FieldInfo fieldInfo) throws IOException { + this.fieldInfo = fieldInfo; + fstOutputs = PositiveIntOutputs.getSingleton(true); + fstBuilder = new Builder(FST.INPUT_TYPE.BYTE1, + 0, 0, true, + fstOutputs); + indexStart = out.getFilePointer(); + //System.out.println("VGW: field=" + fieldInfo.name); + + // Always put empty string in + fstBuilder.add(new BytesRef(), fstOutputs.get(termsOut.getFilePointer())); + } + + @Override + public boolean checkIndexTerm(BytesRef text, int docFreq) throws IOException { + if (policy.isIndexTerm(text, docFreq) || first) { + first = false; + //System.out.println("VGW: index term=" + text.utf8ToString() + " fp=" + termsOut.getFilePointer()); + final int lengthSave = text.length; + text.length = indexedTermPrefixLength(lastTerm, text); + try { + fstBuilder.add(text, fstOutputs.get(termsOut.getFilePointer())); + } finally { + text.length = lengthSave; + } + lastTerm.copy(text); + return true; + } else { + //System.out.println("VGW: not index term=" + text.utf8ToString() + " fp=" + termsOut.getFilePointer()); + lastTerm.copy(text); + return false; + } + } + + @Override + public void finish() throws IOException { + fst = fstBuilder.finish(); + if (fst != null) { + fst.save(out); + } + } + } + + @Override + public void close() throws IOException { + final long dirStart = out.getFilePointer(); + final int fieldCount = fields.size(); + + int nonNullFieldCount = 0; + for(int i=0;i 0) { - // Optimization: requested term is within the - // same index block we are now in; skip seeking - // (but do scanning): - doSeek = false; + // Target term is after current term + if (!didIndexNext) { + if (indexEnum.next() == -1) { + nextIndexTerm = null; + } else { + nextIndexTerm = indexEnum.term(); + } + didIndexNext = true; + } + + if (nextIndexTerm == null || termComp.compare(term, nextIndexTerm) < 0) { + // Optimization: requested term is within the + // same index block we are now in; skip seeking + // (but do scanning): + doSeek = false; + } } } - // Used only for assert: - final long startOrd; - if (doSeek) { - // As index to find biggest index term that's <= + positioned = true; + + // Ask index to find biggest index term that's <= // our text: - fieldIndexReader.getIndexOffset(term, indexResult); - - in.seek(indexResult.offset); + in.seek(indexEnum.seek(term)); + didIndexNext = false; seekPending = false; // NOTE: the first next() after an index seek is @@ -373,22 +415,17 @@ // those bytes in the primary file, but then when // scanning over an index term we'd have to // special case it: - bytesReader.reset(indexResult.term); - - state.ord = indexResult.position-1; - assert state.ord >= -1: "ord=" + state.ord + " pos=" + indexResult.position; - - startOrd = indexResult.position; - } else { - startOrd = -1; + bytesReader.reset(indexEnum.term()); } + assert startSeek(); + // Now scan: - while(next() != null) { + while (next() != null) { final int cmp = termComp.compare(bytesReader.term, term); if (cmp == 0) { - - if (doSeek && useCache) { + // Done! + if (useCache) { // Store in cache FieldAndTerm entryKey = new FieldAndTerm(fieldTerm); cachedState = (TermState) state.clone(); @@ -396,54 +433,262 @@ cachedState.filePointer = in.getFilePointer(); termsCache.put(entryKey, cachedState); } - + return SeekStatus.FOUND; } else if (cmp > 0) { return SeekStatus.NOT_FOUND; } + // The purpose of the terms dict index is to seek // the enum to the closest index term before the // term we are looking for. So, we should never // cross another index term (besides the first // one) while we are scanning: - assert state.ord == startOrd || !fieldIndexReader.isIndexTerm(state.ord, state.docFreq, true): "state.ord=" + state.ord + " startOrd=" + startOrd + " ir.isIndexTerm=" + fieldIndexReader.isIndexTerm(state.ord, state.docFreq, true) + " state.docFreq=" + state.docFreq; + assert checkSeekScan(); } + positioned = false; return SeekStatus.END; } @Override + public BytesRef term() { + return bytesReader.term; + } + + @Override + public BytesRef next() throws IOException { + + if (seekPending) { + seekPending = false; + in.seek(state.filePointer); + indexEnum.seek(bytesReader.term); + } + + if (!bytesReader.read()) { + return null; + } + + final byte b = in.readByte(); + isIndexTerm = (b & 0x80) != 0; + + if ((b & 0x40) == 0) { + // Fast case -- docFreq fits in 6 bits + state.docFreq = b & 0x3F; + } else { + state.docFreq = (in.readVInt() << 6) | (b & 0x3F); + } + + postingsReader.readTerm(in, + fieldInfo, state, + isIndexTerm); + positioned = true; + + return bytesReader.term; + } + + @Override + public int docFreq() { + return state.docFreq; + } + + @Override + public DocsEnum docs(Bits skipDocs, DocsEnum reuse) throws IOException { + DocsEnum docsEnum = postingsReader.docs(fieldInfo, state, skipDocs, reuse); + assert docsEnum != null; + return docsEnum; + } + + @Override + public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException { + if (fieldInfo.omitTermFreqAndPositions) { + return null; + } else { + return postingsReader.docsAndPositions(fieldInfo, state, skipDocs, reuse); + } + } + + @Override public SeekStatus seek(long ord) throws IOException { + throw new UnsupportedOperationException(); + } - // TODO: should we cache term lookup by ord as well...? + @Override + public long ord() { + throw new UnsupportedOperationException(); + } + } - if (ord >= numTerms) { - state.ord = numTerms-1; - return SeekStatus.END; + // Iterates through terms in this field, supporting ord() + private class SegmentTermsEnumWithOrd extends TermsEnum { + private final IndexInput in; + private final DeltaBytesReader bytesReader; + private final TermState state; + private boolean seekPending; + private final FieldAndTerm fieldTerm = new FieldAndTerm(); + private final TermsIndexReaderBase.FieldIndexEnum indexEnum; + private boolean positioned; + private boolean didIndexNext; + private BytesRef nextIndexTerm; + private boolean isIndexTerm; + + SegmentTermsEnumWithOrd() throws IOException { + in = (IndexInput) PrefixCodedTermsReader.this.in.clone(); + in.seek(termsStartPointer); + indexEnum = indexReader.getFieldEnum(fieldInfo); + bytesReader = new DeltaBytesReader(in); + fieldTerm.field = fieldInfo.name; + state = postingsReader.newTermState(); + state.ord = -1; + } + + @Override + public Comparator getComparator() { + return termComp; + } + + @Override + public void cacheCurrentTerm() { + TermState stateCopy = (TermState) state.clone(); + stateCopy.filePointer = in.getFilePointer(); + termsCache.put(new FieldAndTerm(fieldInfo.name, bytesReader.term), + stateCopy); + } + + // called only from assert + private boolean first; + private int indexTermCount; + + private boolean startSeek() { + first = true; + indexTermCount = 0; + //System.out.println("check: start"); + return true; + } + + private boolean checkSeekScan() { + if (!first && isIndexTerm) { + indexTermCount++; + if (indexTermCount >= indexReader.getDivisor()) { + //System.out.println("now fail count=" + indexTermCount); + return false; + } } + //System.out.println("check: scan=" + bytesReader.term); + first = false; + return true; + } - fieldIndexReader.getIndexOffset(ord, indexResult); - in.seek(indexResult.offset); - seekPending = false; + /** Seeks until the first term that's >= the provided + * text; returns SeekStatus.FOUND if the exact term + * is found, SeekStatus.NOT_FOUND if a different term + * was found, SeekStatus.END if we hit EOF */ + @Override + public SeekStatus seek(BytesRef term, boolean useCache) throws IOException { - // NOTE: the first next() after an index seek is - // wasteful, since it redundantly reads the same - // bytes into the buffer - bytesReader.reset(indexResult.term); + //System.out.println("te.seek target=" + term + " current=" + bytesReader.term + " positioned=" + positioned + " this=" + this); + + if (indexEnum == null) { + throw new IllegalStateException("terms index was not loaded"); + } - state.ord = indexResult.position-1; - assert state.ord >= -1: "ord=" + state.ord; + // Check cache + fieldTerm.term = term; + TermState cachedState; + if (useCache) { + cachedState = termsCache.get(fieldTerm); + if (cachedState != null) { + state.copy(cachedState); + seekPending = true; + positioned = false; + bytesReader.term.copy(term); + return SeekStatus.FOUND; + } + } else { + cachedState = null; + } - // Now, scan: - int left = (int) (ord - state.ord); - while(left > 0) { - final BytesRef term = next(); - assert term != null; - left--; + boolean doSeek = true; + + if (positioned) { + + final int cmp = termComp.compare(bytesReader.term, term); + + if (cmp == 0) { + // already at the requested term + return SeekStatus.FOUND; + } else if (cmp < 0) { + + // Target term is after current term + if (!didIndexNext) { + if (indexEnum.next() == -1) { + nextIndexTerm = null; + } else { + nextIndexTerm = indexEnum.term(); + } + didIndexNext = true; + } + + if (nextIndexTerm != null && termComp.compare(term, nextIndexTerm) < 0) { + // Optimization: requested term is within the + // same index block we are now in; skip seeking + // (but do scanning): + doSeek = false; + } + } } - // always found - return SeekStatus.FOUND; + if (doSeek) { + + positioned = true; + + // Ask terms index to find biggest index term that's <= + // our text: + in.seek(indexEnum.seek(term)); + didIndexNext = false; + state.ord = indexEnum.ord()-1; + seekPending = false; + + // NOTE: the first next() after an index seek is + // wasteful, since it redundantly reads the same + // bytes into the buffer. We could avoid storing + // those bytes in the primary file, but then when + // scanning over an index term we'd have to + // special case it: + bytesReader.reset(indexEnum.term()); + } + + assert startSeek(); + + // Now scan: + while (next() != null) { + final int cmp = termComp.compare(bytesReader.term, term); + if (cmp == 0) { + // Done! + if (useCache) { + // Store in cache + FieldAndTerm entryKey = new FieldAndTerm(fieldTerm); + cachedState = (TermState) state.clone(); + // this is fp after current term + cachedState.filePointer = in.getFilePointer(); + termsCache.put(entryKey, cachedState); + } + + return SeekStatus.FOUND; + } else if (cmp > 0) { + return SeekStatus.NOT_FOUND; + } + + // The purpose of the terms dict index is to seek + // the enum to the closest index term before the + // term we are looking for. So, we should never + // cross another index term (besides the first + // one) while we are scanning: + assert checkSeekScan(); + } + + positioned = false; + return SeekStatus.END; } @Override @@ -452,38 +697,37 @@ } @Override - public long ord() { - return state.ord; - } - - @Override public BytesRef next() throws IOException { if (seekPending) { seekPending = false; in.seek(state.filePointer); + indexEnum.seek(bytesReader.term); } if (state.ord >= numTerms-1) { return null; } - bytesReader.read(); - state.docFreq = in.readVInt(); + boolean result = bytesReader.read(); + assert result: "ord=" + state.ord + " vs " + (numTerms-1); - // TODO: would be cleaner, but space-wasting, to - // simply record a bit into each index entry as to - // whether it's an index entry or not, rather than - // re-compute that information... or, possibly store - // a "how many terms until next index entry" in each - // index entry, but that'd require some tricky - // lookahead work when writing the index + final byte b = in.readByte(); + isIndexTerm = (b & 0x80) != 0; + + if ((b & 0x40) == 0) { + // Fast case -- docFreq fits in 6 bits + state.docFreq = b & 0x3F; + } else { + state.docFreq = (in.readVInt() << 6) | (b & 0x3F); + } + postingsReader.readTerm(in, fieldInfo, state, - fieldIndexReader.isIndexTerm(1+state.ord, state.docFreq, false)); + isIndexTerm); state.ord++; - + positioned = true; return bytesReader.term; } @@ -507,6 +751,47 @@ return postingsReader.docsAndPositions(fieldInfo, state, skipDocs, reuse); } } + + @Override + public SeekStatus seek(long ord) throws IOException { + + if (indexEnum == null) { + throw new IllegalStateException("terms index was not loaded"); + } + + if (ord >= numTerms) { + state.ord = numTerms-1; + return SeekStatus.END; + } + + in.seek(indexEnum.seek(ord)); + seekPending = false; + positioned = true; + + // NOTE: the first next() after an index seek is + // wasteful, since it redundantly reads the same + // bytes into the buffer + bytesReader.reset(indexEnum.term()); + + state.ord = indexEnum.ord()-1; + assert state.ord >= -1: "ord=" + state.ord; + + // Now, scan: + int left = (int) (ord - state.ord); + while(left > 0) { + final BytesRef term = next(); + assert term != null; + left--; + } + + // always found + return SeekStatus.FOUND; + } + + @Override + public long ord() { + return state.ord; + } } } } Index: lucene/src/java/org/apache/lucene/index/codecs/DeltaBytesWriter.java =================================================================== --- lucene/src/java/org/apache/lucene/index/codecs/DeltaBytesWriter.java (revision 1054574) +++ lucene/src/java/org/apache/lucene/index/codecs/DeltaBytesWriter.java (working copy) @@ -20,11 +20,18 @@ import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.store.IndexOutput; import org.apache.lucene.util.BytesRef; +import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE; import java.io.IOException; final class DeltaBytesWriter { + // Must be bigger than + // DocumentsWriter.MAX_TERM_LENGTH_UTF8. If you change + // this it's an index format change, so that change must be + // versioned: + final static int TERM_EOF = BYTE_BLOCK_SIZE; + private byte[] lastBytes = new byte[10]; private int lastLength; final IndexOutput out; @@ -45,8 +52,9 @@ final int limit = length < lastLength ? length : lastLength; while(start < limit) { - if (bytes[upto] != lastBytes[start]) + if (bytes[upto] != lastBytes[start]) { break; + } start++; upto++; } Index: lucene/src/java/org/apache/lucene/util/automaton/fst/FSTEnum.java =================================================================== --- lucene/src/java/org/apache/lucene/util/automaton/fst/FSTEnum.java (revision 0) +++ lucene/src/java/org/apache/lucene/util/automaton/fst/FSTEnum.java (revision 0) @@ -0,0 +1,340 @@ +package org.apache.lucene.util.automaton.fst; + +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.RamUsageEstimator; + +import java.io.IOException; + +/** Can next() and advance() through the terms in an FST + * @lucene.experimental +*/ + +abstract class FSTEnum { + protected final FST fst; + + @SuppressWarnings("unchecked") protected FST.Arc[] arcs = new FST.Arc[10]; + // outputs are cumulative + @SuppressWarnings("unchecked") protected T[] output = (T[]) new Object[10]; + + protected final T NO_OUTPUT; + protected final FST.Arc scratchArc = new FST.Arc(); + + protected int upto; + protected 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 fst) { + this.fst = fst; + NO_OUTPUT = fst.outputs.getNoOutput(); + fst.getFirstArc(getArc(0)); + output[0] = NO_OUTPUT; + } + + protected abstract int getTargetLabel(); + protected abstract int getCurrentLabel(); + + protected abstract void setCurrentLabel(int label); + protected abstract void grow(); + + /** Rewinds enum state to match the shared prefix between + * current term and target term */ + protected final void rewindPrefix() throws IOException { + if (upto == 0) { + //System.out.println(" init"); + upto = 1; + fst.readFirstTargetArc(getArc(0), getArc(1)); + return; + } + //System.out.println(" rewind upto=" + upto + " vs targetLength=" + targetLength); + + final int currentLimit = upto; + upto = 1; + while (upto < currentLimit && upto <= targetLength+1) { + final int cmp = getCurrentLabel() - getTargetLabel(); + if (cmp < 0) { + // seek forward + break; + } else if (cmp > 0) { + // seek backwards -- reset this arc to the first arc + final FST.Arc arc = getArc(upto); + fst.readFirstTargetArc(getArc(upto-1), arc); + //System.out.println(" seek first arc"); + break; + } + upto++; + } + } + + protected void doNext() throws IOException { + //System.out.println("FE: next upto=" + upto); + if (upto == 0) { + //System.out.println(" init"); + upto = 1; + fst.readFirstTargetArc(getArc(0), getArc(1)); + } else { + // pop + //System.out.println(" check pop curArc target=" + arcs[upto].target + " label=" + arcs[upto].label + " isLast?=" + arcs[upto].isLast()); + while (arcs[upto].isLast()) { + upto--; + if (upto == 0) { + //System.out.println(" eof"); + return; + } + } + fst.readNextArc(arcs[upto]); + } + + pushFirst(); + } + + // TODO: should we return a status here (SEEK_FOUND / SEEK_NOT_FOUND / + // SEEK_END)? saves the eq check above? + + /** Seeks to smallest term that's >= target. */ + protected void doSeekCeil() throws IOException { + + //System.out.println(" advance len=" + target.length + " curlen=" + current.length); + + // TODO: possibly caller could/should provide common + // prefix length? ie this work may be redundant if + // caller is in fact intersecting against its own + // automaton + + //System.out.println("FE.seekCeil upto=" + upto); + + // Save time by starting at the end of the shared prefix + // b/w our current term & the target: + rewindPrefix(); + //System.out.println(" after rewind upto=" + upto); + + final int limit = targetLength + 1; + + FST.Arc arc = getArc(upto); + int targetLabel = getTargetLabel(); + //System.out.println(" init targetLabel=" + targetLabel); + + // Now scan forward, matching the new suffix of the target + while(true) { + + // nocommit -- make use of arc array here -- need a + // "findArc" that returns eg the ceil or floor on miss + //System.out.println(" cycle upto=" + upto + " arc.label=" + arc.label + " (" + (char) arc.label + ") vs targetLabel=" + targetLabel); + + if (arc.label == targetLabel) { + // recurse + output[upto] = fst.outputs.add(output[upto-1], arc.output); + if (targetLabel == FST.END_LABEL) { + return; + } + setCurrentLabel(arc.label); + incr(); + arc = fst.readFirstTargetArc(arc, getArc(upto)); + if (upto == limit) { + targetLabel = FST.END_LABEL; + } else { + targetLabel = getTargetLabel(); + } + } else if (arc.label > targetLabel) { + pushFirst(); + return; + } else if (arc.isLast()) { + // Dead end -- rollback to last fork then push + upto--; + while(true) { + if (upto == 0) { + return; + } + final FST.Arc prevArc = getArc(upto); + //System.out.println(" rollback upto=" + upto + " arc.label=" + prevArc.label + " isLast?=" + prevArc.isLast()); + if (!prevArc.isLast()) { + fst.readNextArc(prevArc); + pushFirst(); + return; + } + upto--; + } + } else { + // keep scanning + //System.out.println(" next scan"); + fst.readNextArc(arc); + } + } + } + + // 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 { + + // TODO: possibly caller could/should provide common + // prefix length? ie this work may be redundant if + // caller is in fact intersecting against its own + // automaton + //System.out.println("FE: seek floor upto=" + upto); + + // Save CPU by starting at the end of the shared prefix + // b/w our current term & the target: + rewindPrefix(); + + //System.out.println("FE: after rewind upto=" + upto); + + final int limit = targetLength + 1; + + FST.Arc arc = getArc(upto); + int targetLabel = getTargetLabel(); + + //System.out.println("FE: init targetLabel=" + targetLabel); + + // Now scan forward, matching the new suffix of the target + while(true) { + //System.out.println(" cycle upto=" + upto + " arc.label=" + arc.label + " (" + (char) arc.label + ") targetLabel=" + targetLabel + " isLast?=" + arc.isLast()); + + // nocommit -- make use of arc array here -- need a + // "findArc" that returns eg the ceil or floor on miss + + if (arc.label == targetLabel) { + // recurse + output[upto] = fst.outputs.add(output[upto-1], arc.output); + if (targetLabel == FST.END_LABEL) { + return; + } + setCurrentLabel(arc.label); + incr(); + arc = fst.readFirstTargetArc(arc, getArc(upto)); + if (upto == limit) { + targetLabel = FST.END_LABEL; + } else { + targetLabel = getTargetLabel(); + } + } 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 + // instead: + while(true) { + // First, walk backwards until we find a first arc + // that's before our target label: + fst.readFirstTargetArc(getArc(upto-1), arc); + if (arc.label < targetLabel) { + // Then, scan forwards to the arc just before + // the targetLabel: + while(!arc.isLast() && fst.readNextArcLabel(arc) < targetLabel) { + fst.readNextArc(arc); + } + pushLast(); + return; + } + upto--; + if (upto == 0) { + return; + } + targetLabel = getTargetLabel(); + arc = getArc(upto); + } + } else if (!arc.isLast()) { + //System.out.println(" check next label=" + fst.readNextArcLabel(arc) + " (" + (char) fst.readNextArcLabel(arc) + ")"); + if (fst.readNextArcLabel(arc) > targetLabel) { + pushLast(); + return; + } else { + // keep scanning + fst.readNextArc(arc); + } + } else { + pushLast(); + return; + } + } + } + + private void incr() { + upto++; + grow(); + if (arcs.length <= upto) { + @SuppressWarnings("unchecked") final FST.Arc[] newArcs = + new FST.Arc[ArrayUtil.oversize(1+upto, RamUsageEstimator.NUM_BYTES_OBJECT_REF)]; + System.arraycopy(arcs, 0, newArcs, 0, arcs.length); + arcs = newArcs; + } + if (output.length <= upto) { + @SuppressWarnings("unchecked") final T[] newOutput = + (T[]) new Object[ArrayUtil.oversize(1+upto, RamUsageEstimator.NUM_BYTES_OBJECT_REF)]; + System.arraycopy(output, 0, newOutput, 0, output.length); + output = newOutput; + } + } + + // Appends current arc, and then recurses from its target, + // appending first arc all the way to the final node + private void pushFirst() throws IOException { + + FST.Arc arc = arcs[upto]; + assert arc != null; + + while (true) { + 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); + incr(); + + final FST.Arc nextArc = getArc(upto); + fst.readFirstTargetArc(arc, nextArc); + arc = nextArc; + } + } + + // Recurses from current arc, appending last arc all the + // way to the first final node + private void pushLast() throws IOException { + + FST.Arc arc = arcs[upto]; + assert arc != null; + + while (true) { + setCurrentLabel(arc.label); + output[upto] = fst.outputs.add(output[upto-1], arc.output); + if (arc.label == FST.END_LABEL) { + // Final node + break; + } + incr(); + + final FST.Arc nextArc = getArc(upto); + fst.readFirstTargetArc(arc, nextArc); + arc = nextArc; + while(!arc.isLast()) { + fst.readNextArc(arc); + } + } + } + + private FST.Arc getArc(int idx) { + if (arcs[idx] == null) { + arcs[idx] = new FST.Arc(); + } + return arcs[idx]; + } +} Property changes on: lucene/src/java/org/apache/lucene/util/automaton/fst/FSTEnum.java ___________________________________________________________________ Added: svn:eol-style + native Index: lucene/src/java/org/apache/lucene/util/automaton/fst/BytesRefFSTEnum.java =================================================================== --- lucene/src/java/org/apache/lucene/util/automaton/fst/BytesRefFSTEnum.java (revision 1054574) +++ lucene/src/java/org/apache/lucene/util/automaton/fst/BytesRefFSTEnum.java (working copy) @@ -17,299 +17,91 @@ * limitations under the License. */ -import org.apache.lucene.util.ArrayUtil; -import org.apache.lucene.util.BytesRef; -import org.apache.lucene.util.RamUsageEstimator; - import java.io.IOException; +import org.apache.lucene.util.BytesRef; + /** Can next() and advance() through the terms in an FST * @lucene.experimental */ -public class BytesRefFSTEnum { - private final FST fst; - - private BytesRef current = new BytesRef(10); - @SuppressWarnings("unchecked") private FST.Arc[] arcs = new FST.Arc[10]; - // outputs are cumulative - @SuppressWarnings("unchecked") private T[] output = (T[]) new Object[10]; - - private boolean lastFinal; - private boolean didEmpty; - private final T NO_OUTPUT; +public final class BytesRefFSTEnum extends FSTEnum { + private final BytesRef current = new BytesRef(10); private final InputOutput result = new InputOutput(); + private BytesRef target; public static class InputOutput { public BytesRef input; public T output; } - + + /** doFloor controls the behavior of advance: if it's true + * doFloor is true, advance positions to the biggest + * term before target. */ public BytesRefFSTEnum(FST fst) { - this.fst = fst; + super(fst); result.input = current; - NO_OUTPUT = fst.outputs.getNoOutput(); + current.offset = 1; } - public void reset() { - lastFinal = false; - didEmpty = false; - current.length = 0; - result.output = NO_OUTPUT; - } - - /** NOTE: target must be >= where we are already - * positioned */ - public InputOutput advance(BytesRef target) throws IOException { - - assert target.compareTo(current) >= 0; - - //System.out.println(" advance len=" + target.length + " curlen=" + current.length); - - // special case empty string - if (current.length == 0) { - if (target.length == 0) { - final T output = fst.getEmptyOutput(); - if (output != null) { - if (!didEmpty) { - current.length = 0; - lastFinal = true; - result.output = output; - didEmpty = true; - } - return result; - } else { - return next(); - } - } - - if (fst.noNodes()) { - return null; - } - } - - // TODO: possibly caller could/should provide common - // prefix length? ie this work may be redundant if - // caller is in fact intersecting against its own - // automaton - - // what prefix does target share w/ current - int idx = 0; - while (idx < current.length && idx < target.length) { - if (current.bytes[idx] != target.bytes[target.offset + idx]) { - break; - } - idx++; - } - - //System.out.println(" shared " + idx); - - FST.Arc arc; - if (current.length == 0) { - // new enum (no seek/next yet) - arc = fst.readFirstArc(fst.getStartNode(), getArc(0)); - //System.out.println(" new enum"); - } else if (idx < current.length) { - // roll back to shared point - lastFinal = false; - current.length = idx; - arc = arcs[idx]; - if (arc.isLast()) { - if (idx == 0) { - return null; - } else { - return next(); - } - } - arc = fst.readNextArc(arc); - } else if (idx == target.length) { - // degenerate case -- seek to term we are already on - assert target.equals(current); - return result; - } else { - // current is a full prefix of target - if (lastFinal) { - arc = fst.readFirstArc(arcs[current.length-1].target, getArc(current.length)); - } else { - return next(); - } - } - - lastFinal = false; - - assert arc == arcs[current.length]; - int targetLabel = target.bytes[target.offset+current.length] & 0xFF; - - while(true) { - //System.out.println(" cycle len=" + current.length + " target=" + ((char) targetLabel) + " vs " + ((char) arc.label)); - if (arc.label == targetLabel) { - grow(); - current.bytes[current.length] = (byte) arc.label; - appendOutput(arc.output); - current.length++; - grow(); - if (current.length == target.length) { - result.output = output[current.length-1]; - if (arc.isFinal()) { - // target is exact match - if (fst.hasArcs(arc.target)) { - // target is also a proper prefix of other terms - lastFinal = true; - appendFinalOutput(arc.nextFinalOutput); - } - } else { - // target is not a match but is a prefix of - // other terms - current.length--; - push(); - } - return result; - } else if (!fst.hasArcs(arc.target)) { - // we only match a prefix of the target - return next(); - } else { - targetLabel = target.bytes[target.offset+current.length] & 0xFF; - arc = fst.readFirstArc(arc.target, getArc(current.length)); - } - } else if (arc.label > targetLabel) { - // we are now past the target - push(); - return result; - } else if (arc.isLast()) { - if (current.length == 0) { - return null; - } - return next(); - } else { - arc = fst.readNextArc(getArc(current.length)); - } - } - } - public InputOutput current() { return result; } public InputOutput next() throws IOException { //System.out.println(" enum.next"); + doNext(); + return setResult(); + } - if (current.length == 0) { - final T output = fst.getEmptyOutput(); - if (output != null) { - if (!didEmpty) { - current.length = 0; - lastFinal = true; - result.output = output; - didEmpty = true; - return result; - } else { - lastFinal = false; - } - } - if (fst.noNodes()) { - return null; - } - fst.readFirstArc(fst.getStartNode(), getArc(0)); - push(); - } else if (lastFinal) { - lastFinal = false; - assert current.length > 0; - // resume pushing - fst.readFirstArc(arcs[current.length-1].target, getArc(current.length)); - push(); - } else { - //System.out.println(" pop/push"); - pop(); - if (current.length == 0) { - // enum done - return null; - } else { - current.length--; - fst.readNextArc(arcs[current.length]); - push(); - } - } - - return result; + /** Seeks to smallest term that's >= target. */ + public InputOutput seekCeil(BytesRef target) throws IOException { + this.target = target; + targetLength = target.length; + super.doSeekCeil(); + return setResult(); } - private void grow() { - final int l = current.length + 1; - current.grow(l); - if (arcs.length < l) { - @SuppressWarnings("unchecked") final FST.Arc[] newArcs = - new FST.Arc[ArrayUtil.oversize(l, RamUsageEstimator.NUM_BYTES_OBJECT_REF)]; - System.arraycopy(arcs, 0, newArcs, 0, arcs.length); - arcs = newArcs; - } - if (output.length < l) { - @SuppressWarnings("unchecked") final T[] newOutput = - (T[]) new Object[ArrayUtil.oversize(l, RamUsageEstimator.NUM_BYTES_OBJECT_REF)]; - System.arraycopy(output, 0, newOutput, 0, output.length); - output = newOutput; - } + /** Seeks to biggest term that's <= target. */ + public InputOutput seekFloor(BytesRef target) throws IOException { + this.target = target; + targetLength = target.length; + super.doSeekFloor(); + return setResult(); } - private void appendOutput(T addedOutput) { - T newOutput; - if (current.length == 0) { - newOutput = addedOutput; - } else if (addedOutput == NO_OUTPUT) { - output[current.length] = output[current.length-1]; - return; + @Override + protected int getTargetLabel() { + if (upto-1 == target.length) { + return FST.END_LABEL; } else { - newOutput = fst.outputs.add(output[current.length-1], addedOutput); + return target.bytes[target.offset + upto - 1] & 0xFF; } - output[current.length] = newOutput; } - private void appendFinalOutput(T addedOutput) { - if (current.length == 0) { - result.output = addedOutput; - } else { - result.output = fst.outputs.add(output[current.length-1], addedOutput); - } + @Override + protected int getCurrentLabel() { + // current.offset fixed at 1 + return current.bytes[upto] & 0xFF; } - private void push() throws IOException { - - FST.Arc arc = arcs[current.length]; - assert arc != null; - - while(true) { - grow(); - - current.bytes[current.length] = (byte) arc.label; - appendOutput(arc.output); - //System.out.println(" push: append label=" + ((char) arc.label) + " output=" + fst.outputs.outputToString(arc.output)); - current.length++; - grow(); - - if (!fst.hasArcs(arc.target)) { - break; - } - - if (arc.isFinal()) { - appendFinalOutput(arc.nextFinalOutput); - lastFinal = true; - return; - } - - arc = fst.readFirstArc(arc.target, getArc(current.length)); - } - result.output = output[current.length-1]; + @Override + protected void setCurrentLabel(int label) { + current.bytes[upto] = (byte) label; } - private void pop() { - while (current.length > 0 && arcs[current.length-1].isLast()) { - current.length--; - } + @Override + protected void grow() { + current.grow(upto+1); } - private FST.Arc getArc(int idx) { - if (arcs[idx] == null) { - arcs[idx] = new FST.Arc(); + private InputOutput setResult() { + if (upto == 0) { + return null; + } else { + current.length = upto-1; + result.output = output[upto]; + return result; } - return arcs[idx]; } } Index: lucene/src/java/org/apache/lucene/util/automaton/fst/Builder.java =================================================================== --- lucene/src/java/org/apache/lucene/util/automaton/fst/Builder.java (revision 1054574) +++ lucene/src/java/org/apache/lucene/util/automaton/fst/Builder.java (working copy) @@ -256,7 +256,7 @@ } public void add(IntsRef input, T output) throws IOException { - //System.out.println("\nADD: " + input.utf8ToString()); + //System.out.println("\nFST ADD: input=" + input + " output=" + fst.outputs.outputToString(output)); assert lastInput.length == 0 || input.compareTo(lastInput) > 0: "inputs are added out of order lastInput=" + lastInput + " vs input=" + input; assert validOutput(output); @@ -361,7 +361,7 @@ compilePrevTail(1); //System.out.println("finish: inputCount=" + frontier[0].inputCount); if (frontier[0].inputCount < minSuffixCount1 || frontier[0].inputCount < minSuffixCount2 || frontier[0].numArcs == 0) { - if (fst.getEmptyOutput() == null) { + if (fst.emptyOutput == null) { return null; } else if (minSuffixCount1 > 0 || minSuffixCount2 > 0) { // empty string got pruned Index: lucene/src/java/org/apache/lucene/util/automaton/fst/NodeHash.java =================================================================== --- lucene/src/java/org/apache/lucene/util/automaton/fst/NodeHash.java (revision 1054574) +++ lucene/src/java/org/apache/lucene/util/automaton/fst/NodeHash.java (working copy) @@ -35,7 +35,7 @@ } private boolean nodesEqual(Builder.UnCompiledNode node, int address) throws IOException { - fst.readFirstArc(address, scratchArc); + fst.readFirstRealArc(address, scratchArc); if (scratchArc.bytesPerArc != 0 && node.numArcs != scratchArc.numArcs) { return false; } @@ -56,7 +56,7 @@ return false; } } - fst.readNextArc(scratchArc); + fst.readNextRealArc(scratchArc); } return false; @@ -89,7 +89,7 @@ final int PRIME = 31; //System.out.println("hash frozen"); int h = 0; - fst.readFirstArc(node, scratchArc); + fst.readFirstRealArc(node, scratchArc); 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()); h = PRIME * h + scratchArc.label; @@ -102,7 +102,7 @@ if (scratchArc.isLast()) { break; } - fst.readNextArc(scratchArc); + fst.readNextRealArc(scratchArc); } //System.out.println(" ret " + (h&Integer.MAX_VALUE)); return h & Integer.MAX_VALUE; Index: lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java =================================================================== --- lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java (revision 1054574) +++ lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java (working copy) @@ -18,20 +18,13 @@ */ import java.io.IOException; -import java.io.PrintStream; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Set; import org.apache.lucene.store.DataInput; import org.apache.lucene.store.DataOutput; import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.IndexOutput; import org.apache.lucene.util.ArrayUtil; -import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.CodecUtil; -import org.apache.lucene.util.IntsRef; /** Represents an FST using a compact byte[] format. *

The format is similar to what's used by Morfologik @@ -40,7 +33,7 @@ */ public class FST { public static enum INPUT_TYPE {BYTE1, BYTE2, BYTE4}; - private final INPUT_TYPE inputType; + public final INPUT_TYPE inputType; private final static int BIT_FINAL_ARC = 1 << 0; private final static int BIT_LAST_ARC = 1 << 1; @@ -76,7 +69,7 @@ // if non-null, this FST accepts the empty string and // produces this output - private T emptyOutput; + T emptyOutput; private byte[] emptyOutputBytes; private byte[] bytes; @@ -94,11 +87,16 @@ public int arcCount; public int arcWithOutputCount; + // If arc has this label then that arc is final/accepted + public static int END_LABEL = -1; + public final static class Arc { - int label; // really a "unsigned" byte + public int label; + public T output; + int target; + byte flags; - T output; T nextFinalOutput; int nextArc; @@ -108,13 +106,26 @@ int arcIdx; int numArcs; - // Must call this before re-using an Arc instance on a - // new node - public void reset() { - bytesPerArc = 0; + /** Returns this */ + public Arc copyFrom(Arc other) { + label = other.label; + target = other.target; + flags = other.flags; + output = other.output; + nextFinalOutput = other.nextFinalOutput; + nextArc = other.nextArc; + if (other.bytesPerArc != 0) { + bytesPerArc = other.bytesPerArc; + posArcsStart = other.posArcsStart; + arcIdx = other.arcIdx; + numArcs = other.numArcs; + } else { + bytesPerArc = 0; + } + return this; } - public boolean flag(int flag) { + boolean flag(int flag) { return FST.flag(flags, flag); } @@ -122,7 +133,7 @@ return flag(BIT_LAST_ARC); } - public boolean isFinal() { + boolean isFinal() { return flag(BIT_FINAL_ARC); } }; @@ -203,9 +214,9 @@ this.startNode = startNode; } - public void setEmptyOutput(T v) throws IOException { - if (emptyOutput != null) { - throw new IllegalStateException("empty output is already set"); + void setEmptyOutput(T v) throws IOException { + if (emptyOutput != null && !emptyOutput.equals(v)) { + throw new IllegalStateException("empty output is already set: " + outputs.outputToString(emptyOutput) + " vs " + outputs.outputToString(v)); } emptyOutput = v; @@ -285,23 +296,10 @@ // returns true if the node at this address has any // outgoing arcs - public boolean hasArcs(int address) { - return address != FINAL_END_NODE && address != NON_FINAL_END_NODE; + public boolean targetHasArcs(Arc arc) { + return arc.target > 0; } - public int getStartNode() { - if (startNode == -1) { - throw new IllegalStateException("call finish first"); - } - return startNode; - } - - // returns null if this FST does not accept the empty - // string, else, the output for the empty string - public T getEmptyOutput() { - return emptyOutput; - } - // serializes new node by appending its bytes to the end // of the current byte[] int addNode(Builder.UnCompiledNode node) throws IOException { @@ -364,7 +362,7 @@ assert arc.nextFinalOutput == NO_OUTPUT; } - boolean targetHasArcs = hasArcs(target.address); + boolean targetHasArcs = target.address > 0; if (!targetHasArcs) { flags += BIT_STOP_NODE; @@ -453,9 +451,48 @@ return endAddress-1; } - public Arc readFirstArc(int address, Arc arc) throws IOException { - //System.out.println("readFirstArc addr=" + address); + /** Fills virtual 'start' arc, ie, an empty incoming arc to + * the FST's start node */ + public Arc getFirstArc(Arc arc) { + if (emptyOutput != null) { + arc.flags = BIT_FINAL_ARC | BIT_LAST_ARC; + arc.nextFinalOutput = emptyOutput; + } else { + arc.flags = BIT_LAST_ARC; + } + + // If there are no nodes, ie, the FST only accepts the + // empty string, then startNode is 0, and then readFirstTargetArc + arc.target = startNode; + return arc; + } + + /** Follow the follow arc and read the first arc of its + * target; this changes the provide arc (2nd arg) in-place + * and returns it. */ + public Arc readFirstTargetArc(Arc follow, Arc arc) throws IOException { //int pos = address; + //System.out.println(" readFirstTarget follow.target=" + follow.target + " isFinal=" + follow.isFinal()); + if (follow.isFinal()) { + // Insert "fake" final first arc: + arc.label = -1; + arc.output = follow.nextFinalOutput; + if (follow.target <= 0) { + arc.flags = BIT_LAST_ARC; + } else { + arc.flags = 0; + arc.nextArc = follow.target; + } + //System.out.println(" insert isFinal; nextArc=" + follow.target + " isLast=" + arc.isLast() + " output=" + outputs.outputToString(arc.output)); + return arc; + } else { + return readFirstRealArc(follow.target, arc); + } + } + + // Not private beacaus NodeHash needs access: + Arc readFirstRealArc(int address, Arc arc) throws IOException { + final BytesReader in = new BytesReader(address); arc.flags = in.readByte(); @@ -473,10 +510,57 @@ arc.bytesPerArc = 0; } arc.nextArc = in.pos; + arc.label = 0; return readNextArc(arc); } + /** In-place read; returns the arc. */ public Arc readNextArc(Arc arc) throws IOException { + if (arc.label == -1) { + // This was a fake inserted "final" arc + if (arc.nextArc <= 0) { + // This arc went to virtual final node, ie has no outgoing arcs + return null; + } + return readFirstRealArc(arc.nextArc, arc); + } else { + return readNextRealArc(arc); + } + } + + /** Peeks at next arc's label; does not alter arc. Do + * not call this if arc.isLast()! */ + public int readNextArcLabel(Arc arc) throws IOException { + assert !arc.isLast(); + + final BytesReader in; + if (arc.label == END_LABEL) { + //System.out.println(" nextArc fake " + arc.nextArc); + in = new BytesReader(arc.nextArc); + byte flags = bytes[in.pos]; + if (flag(flags, BIT_ARCS_AS_FIXED_ARRAY)) { + //System.out.println(" nextArc fake array"); + in.pos--; + in.readVInt(); + in.readByte(); + } + } else { + if (arc.bytesPerArc != 0) { + //System.out.println(" nextArc real array"); + // arcs are at fixed entries + in = new BytesReader(arc.posArcsStart - (1+arc.arcIdx)*arc.bytesPerArc); + } else { + // arcs are packed + //System.out.println(" nextArc real packed"); + in = new BytesReader(arc.nextArc); + } + } + // skip flags + in.readByte(); + return readLabel(in); + } + + Arc readNextRealArc(Arc arc) throws IOException { // this is a continuing arc in a fixed array final BytesReader in; if (arc.bytesPerArc != 0) { @@ -504,6 +588,8 @@ if (arc.flag(BIT_STOP_NODE)) { arc.target = FINAL_END_NODE; + // nocommit -- shouldn't this have happened during construction? + arc.flags |= BIT_FINAL_ARC; arc.nextArc = in.pos; } else if (arc.flag(BIT_TARGET_NEXT)) { arc.nextArc = in.pos; @@ -524,14 +610,30 @@ return arc; } - public Arc findArc(int address, int labelToMatch, Arc arc) throws IOException { + /** Finds an arc leaving the incoming arc, replacing the arc in place. + * This returns null if the arc was not found, else the incoming arc. */ + public Arc findTargetArc(int labelToMatch, Arc follow, Arc arc) throws IOException { + + if (labelToMatch == END_LABEL) { + if (follow.isFinal()) { + arc.output = follow.nextFinalOutput; + arc.label = END_LABEL; + return arc; + } else { + return null; + } + } + + if (!targetHasArcs(follow)) { + return null; + } + // TODO: maybe make an explicit thread state that holds // reusable stuff eg BytesReader: - final BytesReader in = new BytesReader(address); + final BytesReader in = new BytesReader(follow.target); if ((in.readByte() & BIT_ARCS_AS_FIXED_ARRAY) != 0) { // Arcs are full array; do binary search: - //System.out.println("findArc: array label=" + labelToMatch); arc.numArcs = in.readVInt(); arc.bytesPerArc = in.readByte() & 0xFF; arc.posArcsStart = in.pos; @@ -554,13 +656,14 @@ return null; } - //System.out.println("findArc: scan"); - readFirstArc(address, arc); - + // Linear scan + readFirstTargetArc(follow, arc); while(true) { if (arc.label == labelToMatch) { return arc; + } else if (arc.label > labelToMatch) { + return null; } else if (arc.isLast()) { return null; } else { @@ -569,191 +672,6 @@ } } - /** Looks up the output for this input, or null if the - * input is not accepted. FST must be - * INPUT_TYPE.BYTE4. */ - public T get(IntsRef input) throws IOException { - assert inputType == INPUT_TYPE.BYTE4; - - if (input.length == 0) { - return getEmptyOutput(); - } - - // TODO: would be nice not to alloc this on every lookup - final FST.Arc arc = new FST.Arc(); - int node = getStartNode(); - T output = NO_OUTPUT; - for(int i=0;i arc = new FST.Arc(); - int node = getStartNode(); - int charIdx = offset; - final int charLimit = offset + length; - T output = NO_OUTPUT; - while(charIdx < charLimit) { - if (!hasArcs(node)) { - // hit end of FST before input end - return null; - } - - final int utf32 = Character.codePointAt(input, charIdx); - charIdx += Character.charCount(utf32); - - if (findArc(node, utf32, arc) != null) { - node = arc.target; - if (arc.output != NO_OUTPUT) { - output = outputs.add(output, arc.output); - } - } else { - return null; - } - } - - if (!arc.isFinal()) { - // hit input's end before end node - return null; - } - - if (arc.nextFinalOutput != NO_OUTPUT) { - output = outputs.add(output, arc.nextFinalOutput); - } - - return output; - } - - - /** Logically casts input to UTF32 ints then looks up the output - * or null if the input is not accepted. FST must be - * INPUT_TYPE.BYTE4. */ - public T get(CharSequence input) throws IOException { - assert inputType == INPUT_TYPE.BYTE4; - - final int len = input.length(); - if (len == 0) { - return getEmptyOutput(); - } - - // TODO: would be nice not to alloc this on every lookup - final FST.Arc arc = new FST.Arc(); - int node = getStartNode(); - int charIdx = 0; - final int charLimit = input.length(); - T output = NO_OUTPUT; - while(charIdx < charLimit) { - if (!hasArcs(node)) { - // hit end of FST before input end - return null; - } - - final int utf32 = Character.codePointAt(input, charIdx); - charIdx += Character.charCount(utf32); - - if (findArc(node, utf32, arc) != null) { - node = arc.target; - if (arc.output != NO_OUTPUT) { - output = outputs.add(output, arc.output); - } - } else { - return null; - } - } - - if (!arc.isFinal()) { - // hit input's end before end node - return null; - } - - if (arc.nextFinalOutput != NO_OUTPUT) { - output = outputs.add(output, arc.nextFinalOutput); - } - - return output; - } - - /** Looks up the output for this input, or null if the - * input is not accepted */ - public T get(BytesRef input) throws IOException { - assert inputType == INPUT_TYPE.BYTE1; - - if (input.length == 0) { - return getEmptyOutput(); - } - - // TODO: would be nice not to alloc this on every lookup - final FST.Arc arc = new FST.Arc(); - int node = getStartNode(); - T output = NO_OUTPUT; - for(int i=0;i /x/tmp/out.png - */ - public void toDot(PrintStream out) throws IOException { - - final List queue = new ArrayList(); - queue.add(startNode); - - final Set seen = new HashSet(); - seen.add(startNode); - - out.println("digraph FST {"); - out.println(" rankdir = LR;"); - //out.println(" " + startNode + " [shape=circle label=" + startNode + "];"); - out.println(" " + startNode + " [label=\"\" shape=circle];"); - out.println(" initial [shape=point color=white label=\"\"];"); - if (emptyOutput != null) { - out.println(" initial -> " + startNode + " [arrowhead=tee label=\"(" + outputs.outputToString(emptyOutput) + ")\"];"); - } else { - out.println(" initial -> " + startNode); - } - - final Arc arc = new Arc(); - - while(queue.size() != 0) { - Integer node = queue.get(queue.size()-1); - queue.remove(queue.size()-1); - - if (node == FINAL_END_NODE || node == NON_FINAL_END_NODE) { - continue; - } - - // scan all arcs - readFirstArc(node, arc); - while(true) { - - if (!seen.contains(arc.target)) { - //out.println(" " + arc.target + " [label=" + arc.target + "];"); - out.println(" " + arc.target + " [label=\"\" shape=circle];"); - seen.add(arc.target); - queue.add(arc.target); - } - String outs; - if (arc.output != NO_OUTPUT) { - outs = "/" + outputs.outputToString(arc.output); - } else { - outs = ""; - } - if (arc.isFinal() && arc.nextFinalOutput != NO_OUTPUT) { - outs += " (" + outputs.outputToString(arc.nextFinalOutput) + ")"; - } - out.print(" " + node + " -> " + arc.target + " [label=\"" + arc.label + outs + "\""); - if (arc.isFinal()) { - out.print(" arrowhead=tee"); - } - if (arc.flag(BIT_TARGET_NEXT)) { - out.print(" color=blue"); - } - out.println("];"); - - if (arc.isLast()) { - break; - } else { - readNextArc(arc); - } - } - } - out.println("}"); - } - public int getNodeCount() { // 1+ in order to count the -1 implicit final node return 1+nodeCount; Index: lucene/src/java/org/apache/lucene/util/automaton/fst/IntsRefFSTEnum.java =================================================================== --- lucene/src/java/org/apache/lucene/util/automaton/fst/IntsRefFSTEnum.java (revision 1054574) +++ lucene/src/java/org/apache/lucene/util/automaton/fst/IntsRefFSTEnum.java (working copy) @@ -17,9 +17,7 @@ * limitations under the License. */ -import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.IntsRef; -import org.apache.lucene.util.RamUsageEstimator; import java.io.IOException; @@ -27,289 +25,83 @@ * @lucene.experimental */ -public class IntsRefFSTEnum { - private final FST fst; - - private IntsRef current = new IntsRef(10); - @SuppressWarnings("unchecked") private FST.Arc[] arcs = new FST.Arc[10]; - // outputs are cumulative - @SuppressWarnings("unchecked") private T[] output = (T[]) new Object[10]; - - private boolean lastFinal; - private boolean didEmpty; - private final T NO_OUTPUT; +public final class IntsRefFSTEnum extends FSTEnum { + private final IntsRef current = new IntsRef(10); private final InputOutput result = new InputOutput(); + private IntsRef target; public static class InputOutput { public IntsRef input; public T output; } - + + /** doFloor controls the behavior of advance: if it's true + * doFloor is true, advance positions to the biggest + * term before target. */ public IntsRefFSTEnum(FST fst) { - this.fst = fst; + super(fst); result.input = current; - NO_OUTPUT = fst.outputs.getNoOutput(); + current.offset = 1; } - public void reset() { - lastFinal = false; - didEmpty = false; - current.length = 0; - result.output = NO_OUTPUT; - } - - /** NOTE: target must be >= where we are already - * positioned */ - public InputOutput advance(IntsRef target) throws IOException { - - assert target.compareTo(current) >= 0; - - //System.out.println(" advance len=" + target.length + " curlen=" + current.length); - - // special case empty string - if (current.length == 0) { - if (target.length == 0) { - final T output = fst.getEmptyOutput(); - if (output != null) { - if (!didEmpty) { - current.length = 0; - lastFinal = true; - result.output = output; - didEmpty = true; - } - return result; - } else { - return next(); - } - } - - if (fst.noNodes()) { - return null; - } - } - - // TODO: possibly caller could/should provide common - // prefix length? ie this work may be redundant if - // caller is in fact intersecting against its own - // automaton - - // what prefix does target share w/ current - int idx = 0; - while (idx < current.length && idx < target.length) { - if (current.ints[idx] != target.ints[target.offset + idx]) { - break; - } - idx++; - } - - //System.out.println(" shared " + idx); - - FST.Arc arc; - if (current.length == 0) { - // new enum (no seek/next yet) - arc = fst.readFirstArc(fst.getStartNode(), getArc(0)); - //System.out.println(" new enum"); - } else if (idx < current.length) { - // roll back to shared point - lastFinal = false; - current.length = idx; - arc = arcs[idx]; - if (arc.isLast()) { - if (idx == 0) { - return null; - } else { - return next(); - } - } - arc = fst.readNextArc(arc); - } else if (idx == target.length) { - // degenerate case -- seek to term we are already on - assert target.equals(current); - return result; - } else { - // current is a full prefix of target - if (lastFinal) { - arc = fst.readFirstArc(arcs[current.length-1].target, getArc(current.length)); - } else { - return next(); - } - } - - lastFinal = false; - - assert arc == arcs[current.length]; - int targetLabel = target.ints[target.offset+current.length]; - - while(true) { - //System.out.println(" cycle len=" + current.length + " target=" + ((char) targetLabel) + " vs " + ((char) arc.label)); - if (arc.label == targetLabel) { - grow(); - current.ints[current.length] = arc.label; - appendOutput(arc.output); - current.length++; - grow(); - if (current.length == target.length) { - result.output = output[current.length-1]; - if (arc.isFinal()) { - // target is exact match - if (fst.hasArcs(arc.target)) { - // target is also a proper prefix of other terms - lastFinal = true; - appendFinalOutput(arc.nextFinalOutput); - } - } else { - // target is not a match but is a prefix of - // other terms - current.length--; - push(); - } - return result; - } else if (!fst.hasArcs(arc.target)) { - // we only match a prefix of the target - return next(); - } else { - targetLabel = target.ints[target.offset+current.length]; - arc = fst.readFirstArc(arc.target, getArc(current.length)); - } - } else if (arc.label > targetLabel) { - // we are now past the target - push(); - return result; - } else if (arc.isLast()) { - if (current.length == 0) { - return null; - } - return next(); - } else { - arc = fst.readNextArc(getArc(current.length)); - } - } - } - public InputOutput current() { return result; } public InputOutput next() throws IOException { //System.out.println(" enum.next"); + doNext(); + return setResult(); + } - if (current.length == 0) { - final T output = fst.getEmptyOutput(); - if (output != null) { - if (!didEmpty) { - current.length = 0; - lastFinal = true; - result.output = output; - didEmpty = true; - return result; - } else { - lastFinal = false; - } - } - if (fst.noNodes()) { - return null; - } - fst.readFirstArc(fst.getStartNode(), getArc(0)); - push(); - } else if (lastFinal) { - lastFinal = false; - assert current.length > 0; - // resume pushing - fst.readFirstArc(arcs[current.length-1].target, getArc(current.length)); - push(); - } else { - //System.out.println(" pop/push"); - pop(); - if (current.length == 0) { - // enum done - return null; - } else { - current.length--; - fst.readNextArc(arcs[current.length]); - push(); - } - } - - return result; + /** Seeks to smallest term that's >= target. */ + public InputOutput seekCeil(IntsRef target) throws IOException { + this.target = target; + targetLength = target.length; + super.doSeekCeil(); + return setResult(); } - private void grow() { - final int l = current.length + 1; - current.grow(l); - if (arcs.length < l) { - @SuppressWarnings("unchecked") final FST.Arc[] newArcs = - new FST.Arc[ArrayUtil.oversize(l, RamUsageEstimator.NUM_BYTES_OBJECT_REF)]; - System.arraycopy(arcs, 0, newArcs, 0, arcs.length); - arcs = newArcs; - } - if (output.length < l) { - @SuppressWarnings("unchecked") final T[] newOutput = - (T[]) new Object[ArrayUtil.oversize(l, RamUsageEstimator.NUM_BYTES_OBJECT_REF)]; - System.arraycopy(output, 0, newOutput, 0, output.length); - output = newOutput; - } + /** Seeks to biggest term that's <= target. */ + public InputOutput seekFloor(IntsRef target) throws IOException { + this.target = target; + targetLength = target.length; + super.doSeekFloor(); + return setResult(); } - private void appendOutput(T addedOutput) { - T newOutput; - if (current.length == 0) { - newOutput = addedOutput; - } else if (addedOutput == NO_OUTPUT) { - output[current.length] = output[current.length-1]; - return; + @Override + protected int getTargetLabel() { + if (upto-1 == target.length) { + return FST.END_LABEL; } else { - newOutput = fst.outputs.add(output[current.length-1], addedOutput); + return target.ints[target.offset + upto - 1]; } - output[current.length] = newOutput; } - private void appendFinalOutput(T addedOutput) { - if (current.length == 0) { - result.output = addedOutput; - } else { - result.output = fst.outputs.add(output[current.length-1], addedOutput); - } + @Override + protected int getCurrentLabel() { + // current.offset fixed at 1 + return current.ints[upto]; } - private void push() throws IOException { - - FST.Arc arc = arcs[current.length]; - assert arc != null; - - while(true) { - grow(); - - current.ints[current.length] = arc.label; - appendOutput(arc.output); - //System.out.println(" push: append label=" + ((char) arc.label) + " output=" + fst.outputs.outputToString(arc.output)); - current.length++; - grow(); - - if (!fst.hasArcs(arc.target)) { - break; - } - - if (arc.isFinal()) { - appendFinalOutput(arc.nextFinalOutput); - lastFinal = true; - return; - } - - arc = fst.readFirstArc(arc.target, getArc(current.length)); - } - result.output = output[current.length-1]; + @Override + protected void setCurrentLabel(int label) { + current.ints[upto] = label; } - private void pop() { - while (current.length > 0 && arcs[current.length-1].isLast()) { - current.length--; - } + @Override + protected void grow() { + current.grow(upto+1); } - private FST.Arc getArc(int idx) { - if (arcs[idx] == null) { - arcs[idx] = new FST.Arc(); + private InputOutput setResult() { + if (upto == 0) { + return null; + } else { + current.length = upto-1; + result.output = output[upto]; + return result; } - return arcs[idx]; } } Index: lucene/src/java/org/apache/lucene/util/automaton/fst/Util.java =================================================================== --- lucene/src/java/org/apache/lucene/util/automaton/fst/Util.java (revision 0) +++ lucene/src/java/org/apache/lucene/util/automaton/fst/Util.java (revision 0) @@ -0,0 +1,249 @@ +package org.apache.lucene.util.automaton.fst; + +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import java.io.IOException; +import java.io.PrintStream; +import java.util.List; +import java.util.ArrayList; +import java.util.Set; +import java.util.HashSet; + +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.IntsRef; + +/** Static helper methods */ +public final class Util { + private Util() { + } + + /** Looks up the output for this input, or null if the + * input is not accepted. FST must be + * INPUT_TYPE.BYTE4. */ + public static T get(FST fst, IntsRef input) throws IOException { + assert fst.inputType == FST.INPUT_TYPE.BYTE4; + + // TODO: would be nice not to alloc this on every lookup + final FST.Arc arc = fst.getFirstArc(new FST.Arc()); + + // Accumulate output as we go + final T NO_OUTPUT = fst.outputs.getNoOutput(); + T output = NO_OUTPUT; + for(int i=0;i T get(FST fst, char[] input, int offset, int length) throws IOException { + assert fst.inputType == FST.INPUT_TYPE.BYTE4; + + // TODO: would be nice not to alloc this on every lookup + final FST.Arc arc = fst.getFirstArc(new FST.Arc()); + + int charIdx = offset; + final int charLimit = offset + length; + + // Accumulate output as we go + final T NO_OUTPUT = fst.outputs.getNoOutput(); + T output = NO_OUTPUT; + while(charIdx < charLimit) { + final int utf32 = Character.codePointAt(input, charIdx); + charIdx += Character.charCount(utf32); + + if (fst.findTargetArc(utf32, arc, arc) == null) { + return null; + } else if (arc.output != NO_OUTPUT) { + output = fst.outputs.add(output, arc.output); + } + } + + if (fst.findTargetArc(FST.END_LABEL, arc, arc) == null) { + return null; + } else if (arc.output != NO_OUTPUT) { + return fst.outputs.add(output, arc.output); + } else { + return output; + } + } + + + /** Logically casts input to UTF32 ints then looks up the output + * or null if the input is not accepted. FST must be + * INPUT_TYPE.BYTE4. */ + public static T get(FST fst, CharSequence input) throws IOException { + assert fst.inputType == FST.INPUT_TYPE.BYTE4; + + // TODO: would be nice not to alloc this on every lookup + final FST.Arc arc = fst.getFirstArc(new FST.Arc()); + + int charIdx = 0; + final int charLimit = input.length(); + + // Accumulate output as we go + final T NO_OUTPUT = fst.outputs.getNoOutput(); + T output = NO_OUTPUT; + + while(charIdx < charLimit) { + final int utf32 = Character.codePointAt(input, charIdx); + charIdx += Character.charCount(utf32); + + if (fst.findTargetArc(utf32, arc, arc) == null) { + return null; + } else if (arc.output != NO_OUTPUT) { + output = fst.outputs.add(output, arc.output); + } + } + + if (fst.findTargetArc(FST.END_LABEL, arc, arc) == null) { + return null; + } else if (arc.output != NO_OUTPUT) { + return fst.outputs.add(output, arc.output); + } else { + return output; + } + } + + /** Looks up the output for this input, or null if the + * input is not accepted */ + public static T get(FST fst, BytesRef input) throws IOException { + assert fst.inputType == FST.INPUT_TYPE.BYTE1; + + // TODO: would be nice not to alloc this on every lookup + final FST.Arc arc = fst.getFirstArc(new FST.Arc()); + + // Accumulate output as we go + final T NO_OUTPUT = fst.outputs.getNoOutput(); + T output = NO_OUTPUT; + for(int i=0;i /x/tmp/out.png + */ + + public static void toDot(FST fst, PrintStream out) throws IOException { + + final FST.Arc startArc = fst.getFirstArc(new FST.Arc()); + + final List> queue = new ArrayList>(); + queue.add(startArc); + + final Set seen = new HashSet(); + seen.add(startArc.target); + + out.println("digraph FST {"); + out.println(" rankdir = LR;"); + //out.println(" " + startNode + " [shape=circle label=" + startNode + "];"); + out.println(" " + startArc.target + " [label=\"\" shape=circle];"); + out.println(" initial [shape=point color=white label=\"\"];"); + out.println(" initial -> " + startArc.target); + + final T NO_OUTPUT = fst.outputs.getNoOutput(); + + while(queue.size() != 0) { + FST.Arc arc = queue.get(queue.size()-1); + queue.remove(queue.size()-1); + //System.out.println("dot cycle target=" + arc.target); + + if (fst.targetHasArcs(arc)) { + + // scan all arcs + final int node = arc.target; + fst.readFirstTargetArc(arc, arc); + while(true) { + + //System.out.println(" cycle label=" + arc.label + " (" + (char) arc.label + ") target=" + arc.target); + if (!seen.contains(arc.target)) { + final String shape; + if (arc.target == -1) { + shape = "doublecircle"; + } else { + shape = "circle"; + } + out.println(" " + arc.target + " [shape=" + shape + "];"); + seen.add(arc.target); + queue.add(new FST.Arc().copyFrom(arc)); + //System.out.println(" new!"); + } + String outs; + if (arc.output != NO_OUTPUT) { + outs = "/" + fst.outputs.outputToString(arc.output); + } else { + outs = ""; + } + final char cl; + if (arc.label == FST.END_LABEL) { + cl = '~'; + } else { + cl = (char) arc.label; + } + out.println(" " + node + " -> " + arc.target + " [label=\"" + cl + outs + "\"]"); + //if (arc.flag(FST.BIT_TARGET_NEXT)) { + //out.print(" color=blue"); + //} + //out.println("];"); + + if (arc.isLast()) { + break; + } else { + fst.readNextArc(arc); + } + } + } + } + out.println("}"); + } +} Property changes on: lucene/src/java/org/apache/lucene/util/automaton/fst/Util.java ___________________________________________________________________ Added: svn:eol-style + native