Return-Path: Delivered-To: apmail-lucene-commits-archive@www.apache.org Received: (qmail 51180 invoked from network); 12 Dec 2010 15:36:43 -0000 Received: from unknown (HELO mail.apache.org) (140.211.11.3) by 140.211.11.9 with SMTP; 12 Dec 2010 15:36:43 -0000 Received: (qmail 11395 invoked by uid 500); 12 Dec 2010 15:36:43 -0000 Mailing-List: contact commits-help@lucene.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@lucene.apache.org Delivered-To: mailing list commits@lucene.apache.org Received: (qmail 11387 invoked by uid 99); 12 Dec 2010 15:36:42 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Sun, 12 Dec 2010 15:36:42 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=10.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Sun, 12 Dec 2010 15:36:32 +0000 Received: by eris.apache.org (Postfix, from userid 65534) id B379B23889B2; Sun, 12 Dec 2010 15:36:09 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1044834 [1/2] - in /lucene/dev/trunk: ./ lucene/src/java/org/apache/lucene/index/codecs/simpletext/ lucene/src/java/org/apache/lucene/util/ lucene/src/java/org/apache/lucene/util/automaton/ lucene/src/java/org/apache/lucene/util/automaton/... Date: Sun, 12 Dec 2010 15:36:09 -0000 To: commits@lucene.apache.org From: mikemccand@apache.org X-Mailer: svnmailer-1.0.8 Message-Id: <20101212153609.B379B23889B2@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: mikemccand Date: Sun Dec 12 15:36:08 2010 New Revision: 1044834 URL: http://svn.apache.org/viewvc?rev=1044834&view=rev Log: LUCENE-2792: add FST impl Added: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/Builder.java (with props) lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/ByteSequenceOutputs.java (with props) lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/BytesRefFSTEnum.java (with props) lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java (with props) lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/IntSequenceOutputs.java (with props) lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/IntsRefFSTEnum.java (with props) lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/NoOutputs.java (with props) lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/NodeHash.java (with props) lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/Outputs.java (with props) lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/PairOutputs.java (with props) lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/PositiveIntOutputs.java (with props) lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/TODO lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/fst/ lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/automaton/fst/TestFSTs.java (with props) Modified: lucene/dev/trunk/.hgignore lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/ArrayUtil.java lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/IntsRef.java lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/RecyclingByteBlockAllocator.java lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/Automaton.java lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/BasicOperations.java lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/State.java lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/UTF32ToUTF8.java Modified: lucene/dev/trunk/.hgignore URL: http://svn.apache.org/viewvc/lucene/dev/trunk/.hgignore?rev=1044834&r1=1044833&r2=1044834&view=diff ============================================================================== --- lucene/dev/trunk/.hgignore (original) +++ lucene/dev/trunk/.hgignore Sun Dec 12 15:36:08 2010 @@ -1,2 +1,4 @@ syntax: glob */build/* +*.class + Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java?rev=1044834&r1=1044833&r2=1044834&view=diff ============================================================================== --- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java (original) +++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java Sun Dec 12 15:36:08 2010 @@ -31,15 +31,16 @@ import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.Bits; import org.apache.lucene.util.StringHelper; import org.apache.lucene.util.UnicodeUtil; +import org.apache.lucene.util.automaton.fst.Builder; +import org.apache.lucene.util.automaton.fst.BytesRefFSTEnum; +import org.apache.lucene.util.automaton.fst.FST; +import org.apache.lucene.util.automaton.fst.PositiveIntOutputs; +import org.apache.lucene.util.automaton.fst.PairOutputs; import java.io.IOException; import java.util.Comparator; import java.util.Map; -import java.util.Set; import java.util.HashMap; -import java.util.TreeMap; -import java.util.SortedMap; -import java.util.Iterator; class SimpleTextFieldsReader extends FieldsProducer { @@ -116,73 +117,39 @@ class SimpleTextFieldsReader extends Fie private class SimpleTextTermsEnum extends TermsEnum { private final IndexInput in; private final boolean omitTF; - private BytesRef current; private int docFreq; private long docsStart; private boolean ended; - private final TreeMap allTerms; - private Iterator> iter; + private final BytesRefFSTEnum> fstEnum; - public SimpleTextTermsEnum(TreeMap allTerms, boolean omitTF) throws IOException { + public SimpleTextTermsEnum(FST> fst, boolean omitTF) throws IOException { this.in = (IndexInput) SimpleTextFieldsReader.this.in.clone(); - this.allTerms = allTerms; this.omitTF = omitTF; - iter = allTerms.entrySet().iterator(); + fstEnum = new BytesRefFSTEnum>(fst); } public SeekStatus seek(BytesRef text, boolean useCache /* ignored */) throws IOException { - - final SortedMap tailMap = allTerms.tailMap(text); - if (tailMap.isEmpty()) { - current = null; + fstEnum.reset(); + //System.out.println("seek to text=" + text.utf8ToString()); + final BytesRefFSTEnum.InputOutput> result = fstEnum.advance(text); + if (result == null) { + //System.out.println(" end"); return SeekStatus.END; } else { - current = tailMap.firstKey(); - final TermData td = tailMap.get(current); - docsStart = td.docsStart; - docFreq = td.docFreq; - iter = tailMap.entrySet().iterator(); - assert iter.hasNext(); - iter.next(); - if (current.equals(text)) { - return SeekStatus.FOUND; - } else { - return SeekStatus.NOT_FOUND; - } - } - - /* - if (current != null) { - final int cmp = current.compareTo(text); - if (cmp == 0) { - return SeekStatus.FOUND; - } else if (cmp > 0) { - ended = false; - in.seek(fieldStart); - } - } else { - ended = false; - in.seek(fieldStart); - } + //System.out.println(" got text=" + term.utf8ToString()); + PairOutputs.Pair pair = result.output; + docsStart = pair.output1; + docFreq = pair.output2.intValue(); - // Naive!! This just scans... would be better to do - // up-front scan to build in-RAM index - BytesRef b; - while((b = next()) != null) { - final int cmp = b.compareTo(text); - if (cmp == 0) { - ended = false; + if (result.input.equals(text)) { + //System.out.println(" match docsStart=" + docsStart); return SeekStatus.FOUND; - } else if (cmp > 0) { - ended = false; + } else { + //System.out.println(" not match docsStart=" + docsStart); return SeekStatus.NOT_FOUND; } } - current = null; - ended = true; - return SeekStatus.END; - */ } @Override @@ -192,56 +159,20 @@ class SimpleTextFieldsReader extends Fie @Override public BytesRef next() throws IOException { assert !ended; - - if (iter.hasNext()) { - Map.Entry ent = iter.next(); - current = ent.getKey(); - TermData td = ent.getValue(); - docFreq = td.docFreq; - docsStart = td.docsStart; - return current; + final BytesRefFSTEnum.InputOutput> result = fstEnum.next(); + if (result != null) { + final PairOutputs.Pair pair = result.output; + docsStart = pair.output1; + docFreq = pair.output2.intValue(); + return result.input; } else { - current = null; - return null; - } - - /* - readLine(in, scratch); - if (scratch.equals(END) || scratch.startsWith(FIELD)) { - ended = true; - current = null; return null; - } else { - assert scratch.startsWith(TERM): "got " + scratch.utf8ToString(); - docsStart = in.getFilePointer(); - final int len = scratch.length - TERM.length; - if (len > scratch2.length) { - scratch2.grow(len); - } - System.arraycopy(scratch.bytes, TERM.length, scratch2.bytes, 0, len); - scratch2.length = len; - current = scratch2; - docFreq = 0; - long lineStart = 0; - while(true) { - lineStart = in.getFilePointer(); - readLine(in, scratch); - if (scratch.equals(END) || scratch.startsWith(FIELD) || scratch.startsWith(TERM)) { - break; - } - if (scratch.startsWith(DOC)) { - docFreq++; - } - } - in.seek(lineStart); - return current; } - */ } @Override public BytesRef term() { - return current; + return fstEnum.current().input; } @Override @@ -512,10 +443,7 @@ class SimpleTextFieldsReader extends Fie private final String field; private final long termsStart; private final boolean omitTF; - - // NOTE: horribly, horribly RAM consuming, but then - // SimpleText should never be used in production - private final TreeMap allTerms = new TreeMap(); + private FST> fst; private final BytesRef scratch = new BytesRef(10); @@ -527,6 +455,8 @@ class SimpleTextFieldsReader extends Fie } private void loadTerms() throws IOException { + PositiveIntOutputs posIntOutputs = PositiveIntOutputs.getSingleton(false); + Builder> b = new Builder>(FST.INPUT_TYPE.BYTE1, 0, 0, true, new PairOutputs(posIntOutputs, posIntOutputs)); IndexInput in = (IndexInput) SimpleTextFieldsReader.this.in.clone(); in.seek(termsStart); final BytesRef lastTerm = new BytesRef(10); @@ -536,16 +466,14 @@ class SimpleTextFieldsReader extends Fie readLine(in, scratch); if (scratch.equals(END) || scratch.startsWith(FIELD)) { if (lastDocsStart != -1) { - allTerms.put(new BytesRef(lastTerm), - new TermData(lastDocsStart, docFreq)); + b.add(lastTerm, new PairOutputs.Pair(lastDocsStart, Long.valueOf(docFreq))); } break; } else if (scratch.startsWith(DOC)) { docFreq++; } else if (scratch.startsWith(TERM)) { if (lastDocsStart != -1) { - allTerms.put(new BytesRef(lastTerm), - new TermData(lastDocsStart, docFreq)); + b.add(lastTerm, new PairOutputs.Pair(lastDocsStart, Long.valueOf(docFreq))); } lastDocsStart = in.getFilePointer(); final int len = scratch.length - TERM.length; @@ -557,11 +485,23 @@ class SimpleTextFieldsReader extends Fie docFreq = 0; } } + fst = b.finish(); + /* + PrintStream ps = new PrintStream("out.dot"); + fst.toDot(ps); + ps.close(); + System.out.println("SAVED out.dot"); + */ + //System.out.println("FST " + fst.sizeInBytes()); } @Override public TermsEnum iterator() throws IOException { - return new SimpleTextTermsEnum(allTerms, omitTF); + if (fst != null) { + return new SimpleTextTermsEnum(fst, omitTF); + } else { + return TermsEnum.EMPTY; + } } @Override Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/ArrayUtil.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/ArrayUtil.java?rev=1044834&r1=1044833&r2=1044834&view=diff ============================================================================== --- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/ArrayUtil.java (original) +++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/ArrayUtil.java Sun Dec 12 15:36:08 2010 @@ -19,6 +19,7 @@ package org.apache.lucene.util; import java.util.Collection; import java.util.Comparator; +import java.lang.reflect.Array; /** * Methods for manipulating arrays. @@ -392,7 +393,7 @@ public final class ArrayUtil { } /** - * Returns hash of chars in range start (inclusive) to + * Returns hash of bytes in range start (inclusive) to * end (inclusive) */ public static int hashCode(byte[] array, int start, int end) { @@ -429,6 +430,31 @@ public final class ArrayUtil { return false; } + public static T[] grow(T[] array, int minSize) { + if (array.length < minSize) { + @SuppressWarnings("unchecked") final T[] newArray = + (T[]) Array.newInstance(array.getClass().getComponentType(), oversize(minSize, RamUsageEstimator.NUM_BYTES_OBJ_REF)); + System.arraycopy(array, 0, newArray, 0, array.length); + return newArray; + } else + return array; + } + + public static T[] grow(T[] array) { + return grow(array, 1 + array.length); + } + + public static T[] shrink(T[] array, int targetSize) { + final int newSize = getShrinkSize(array.length, targetSize, RamUsageEstimator.NUM_BYTES_OBJ_REF); + if (newSize != array.length) { + @SuppressWarnings("unchecked") final T[] newArray = + (T[]) Array.newInstance(array.getClass().getComponentType(), newSize); + System.arraycopy(array, 0, newArray, 0, newSize); + return newArray; + } else + return array; + } + // Since Arrays.equals doesn't implement offsets for equals /** * See if two array slices are the same. Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/IntsRef.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/IntsRef.java?rev=1044834&r1=1044833&r2=1044834&view=diff ============================================================================== --- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/IntsRef.java (original) +++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/IntsRef.java Sun Dec 12 15:36:08 2010 @@ -21,7 +21,7 @@ package org.apache.lucene.util; * existing int[]. * * @lucene.internal */ -public final class IntsRef { +public final class IntsRef implements Comparable { public int[] ints; public int offset; @@ -81,6 +81,31 @@ public final class IntsRef { } } + /** Signed int order comparison */ + public int compareTo(IntsRef other) { + if (this == other) return 0; + + final int[] aInts = this.ints; + int aUpto = this.offset; + final int[] bInts = other.ints; + int bUpto = other.offset; + + final int aStop = aUpto + Math.min(this.length, other.length); + + while(aUpto < aStop) { + int aInt = aInts[aUpto++]; + int bInt = bInts[bUpto++]; + if (aInt > bInt) { + return 1; + } else if (aInt < bInt) { + return -1; + } + } + + // One is a prefix of the other, or, they are equal: + return this.length - other.length; + } + public void copy(IntsRef other) { if (ints == null) { ints = new int[other.length]; @@ -97,4 +122,18 @@ public final class IntsRef { ints = ArrayUtil.grow(ints, newLength); } } + + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append('['); + final int end = offset + length; + for(int i=offset;i offset) { + sb.append(' '); + } + sb.append(Integer.toHexString(ints[i])); + } + sb.append(']'); + return sb.toString(); + } } Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/RecyclingByteBlockAllocator.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/RecyclingByteBlockAllocator.java?rev=1044834&r1=1044833&r2=1044834&view=diff ============================================================================== --- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/RecyclingByteBlockAllocator.java (original) +++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/RecyclingByteBlockAllocator.java Sun Dec 12 15:36:08 2010 @@ -93,13 +93,7 @@ public final class RecyclingByteBlockAll @Override public synchronized void recycleByteBlocks(byte[][] blocks, int start, int end) { final int numBlocks = Math.min(maxBufferedBlocks - freeBlocks, end - start); - final int size = freeBlocks + numBlocks; - if (size >= freeByteBlocks.length) { - final byte[][] newBlocks = new byte[ArrayUtil.oversize(size, - RamUsageEstimator.NUM_BYTES_OBJ_REF)][]; - System.arraycopy(freeByteBlocks, 0, newBlocks, 0, freeBlocks); - freeByteBlocks = newBlocks; - } + freeByteBlocks = ArrayUtil.grow(freeByteBlocks, freeBlocks + numBlocks); final int stop = start + numBlocks; for (int i = start; i < stop; i++) { freeByteBlocks[freeBlocks++] = blocks[i]; Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/Automaton.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/Automaton.java?rev=1044834&r1=1044833&r2=1044834&view=diff ============================================================================== --- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/Automaton.java (original) +++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/Automaton.java Sun Dec 12 15:36:08 2010 @@ -40,7 +40,6 @@ import java.util.List; import java.util.Set; import org.apache.lucene.util.ArrayUtil; -import org.apache.lucene.util.RamUsageEstimator; /** * Finite-state automaton with regular expression operations. @@ -281,9 +280,7 @@ public class Automaton implements Serial worklist.add(t.to); t.to.number = upto; if (upto == numberedStates.length) { - final State[] newArray = new State[ArrayUtil.oversize(1+upto, RamUsageEstimator.NUM_BYTES_OBJ_REF)]; - System.arraycopy(numberedStates, 0, newArray, 0, upto); - numberedStates = newArray; + numberedStates = ArrayUtil.grow(numberedStates); } numberedStates[upto] = t.to; upto++; Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/BasicOperations.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/BasicOperations.java?rev=1044834&r1=1044833&r2=1044834&view=diff ============================================================================== --- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/BasicOperations.java (original) +++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/BasicOperations.java Sun Dec 12 15:36:08 2010 @@ -30,7 +30,6 @@ package org.apache.lucene.util.automaton; import org.apache.lucene.util.ArrayUtil; -import org.apache.lucene.util.RamUsageEstimator; import java.util.ArrayList; import java.util.BitSet; @@ -459,9 +458,7 @@ final public class BasicOperations { public void add(Transition t) { if (transitions.length == count) { - Transition[] newArray = new Transition[ArrayUtil.oversize(1+count, RamUsageEstimator.NUM_BYTES_OBJ_REF)]; - System.arraycopy(transitions, 0, newArray, 0, count); - transitions = newArray; + transitions = ArrayUtil.grow(transitions); } transitions[count++] = t; } @@ -503,9 +500,7 @@ final public class BasicOperations { private PointTransitions next(int point) { // 1st time we are seeing this point if (count == points.length) { - final PointTransitions[] newArray = new PointTransitions[ArrayUtil.oversize(1+count, RamUsageEstimator.NUM_BYTES_OBJ_REF)]; - System.arraycopy(points, 0, newArray, 0, count); - points = newArray; + points = ArrayUtil.grow(points); } PointTransitions points0 = points[count]; if (points0 == null) { @@ -650,9 +645,7 @@ final public class BasicOperations { final SortedIntSet.FrozenIntSet p = statesSet.freeze(q); worklist.add(p); if (newStateUpto == newStatesArray.length) { - final State[] newArray = new State[ArrayUtil.oversize(1+newStateUpto, RamUsageEstimator.NUM_BYTES_OBJ_REF)]; - System.arraycopy(newStatesArray, 0, newArray, 0, newStateUpto); - newStatesArray = newArray; + newStatesArray = ArrayUtil.grow(newStatesArray); } newStatesArray[newStateUpto] = q; q.number = newStateUpto; Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/State.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/State.java?rev=1044834&r1=1044833&r2=1044834&view=diff ============================================================================== --- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/State.java (original) +++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/State.java Sun Dec 12 15:36:08 2010 @@ -29,7 +29,6 @@ package org.apache.lucene.util.automaton; import org.apache.lucene.util.ArrayUtil; -import org.apache.lucene.util.RamUsageEstimator; import java.io.Serializable; import java.util.Collection; @@ -111,9 +110,7 @@ public class State implements Serializab */ public void addTransition(Transition t) { if (numTransitions == transitionsArray.length) { - final Transition[] newArray = new Transition[ArrayUtil.oversize(1+numTransitions, RamUsageEstimator.NUM_BYTES_OBJ_REF)]; - System.arraycopy(transitionsArray, 0, newArray, 0, numTransitions); - transitionsArray = newArray; + transitionsArray = ArrayUtil.grow(transitionsArray); } transitionsArray[numTransitions++] = t; } Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/UTF32ToUTF8.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/UTF32ToUTF8.java?rev=1044834&r1=1044833&r2=1044834&view=diff ============================================================================== --- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/UTF32ToUTF8.java (original) +++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/UTF32ToUTF8.java Sun Dec 12 15:36:08 2010 @@ -17,7 +17,6 @@ package org.apache.lucene.util.automaton * limitations under the License. */ -import org.apache.lucene.util.RamUsageEstimator; import org.apache.lucene.util.ArrayUtil; import java.util.List; @@ -299,9 +298,7 @@ final class UTF32ToUTF8 { private State newUTF8State() { State s = new State(); if (utf8StateCount == utf8States.length) { - final State[] newArray = new State[ArrayUtil.oversize(1+utf8StateCount, RamUsageEstimator.NUM_BYTES_OBJ_REF)]; - System.arraycopy(utf8States, 0, newArray, 0, utf8StateCount); - utf8States = newArray; + utf8States = ArrayUtil.grow(utf8States); } utf8States[utf8StateCount] = s; s.number = utf8StateCount; Added: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/Builder.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/Builder.java?rev=1044834&view=auto ============================================================================== --- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/Builder.java (added) +++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/Builder.java Sun Dec 12 15:36:08 2010 @@ -0,0 +1,506 @@ +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.BytesRef; +import org.apache.lucene.util.IntsRef; + +import java.io.IOException; + +/** + * Builds a compact FST (maps an IntsRef term to an arbitrary + * output) from pre-sorted terms with outputs (the FST + * becomes an FSA if you use NoOutputs). The FST is written + * on-the-fly into a compact serialized format byte array, which can + * be saved to / loaded from a Directory or used directly + * for traversal. The FST is always finite (no cycles). + * + *

NOTE: The algorithm is described at + * http://citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.24.3698

+ * + * If your outputs are ByteSequenceOutput then the final FST + * will be minimal, but if you use PositiveIntOutput then + * it's only "near minimal". For example, aa/0, aab/1, bbb/2 + * will produce 6 states when a 5 state fst is also + * possible. + * + * The parameterized type T is the output type. See the + * subclasses of {@link Outputs}. + * + * @lucene.experimental + */ + +public class Builder { + private final NodeHash dedupHash; + private final FST fst; + private final T NO_OUTPUT; + + // simplistic pruning: we prune node (and all following + // nodes) if less than this number of terms go through it: + private final int minSuffixCount1; + + // better pruning: we prune node (and all following + // nodes) if the prior node has less than this number of + // terms go through it: + private final int minSuffixCount2; + + private final IntsRef lastInput = new IntsRef(); + + // NOTE: cutting this over to ArrayList instead loses ~6% + // in build performance on 9.8M Wikipedia terms; so we + // left this as an array: + // current "frontier" + private UnCompiledNode[] frontier; + + public Builder(FST.INPUT_TYPE inputType, int minSuffixCount1, int minSuffixCount2, boolean doMinSuffix, Outputs outputs) { + this.minSuffixCount1 = minSuffixCount1; + this.minSuffixCount2 = minSuffixCount2; + fst = new FST(inputType, outputs); + if (doMinSuffix) { + dedupHash = new NodeHash(fst); + } else { + dedupHash = null; + } + NO_OUTPUT = outputs.getNoOutput(); + + @SuppressWarnings("unchecked") final UnCompiledNode[] f = (UnCompiledNode[]) new UnCompiledNode[10]; + frontier = f; + for(int idx=0;idx(this); + } + } + + public int getTotStateCount() { + return fst.nodeCount; + } + + public int getTermCount() { + return frontier[0].inputCount; + } + + public int getMappedStateCount() { + return dedupHash == null ? 0 : fst.nodeCount; + } + + private CompiledNode compileNode(UnCompiledNode n) throws IOException { + + final int address; + if (dedupHash != null) { + if (n.numArcs == 0) { + address = fst.addNode(n); + } else { + address = dedupHash.add(n); + } + } else { + address = fst.addNode(n); + } + assert address != -2; + + n.clear(); + + final CompiledNode fn = new CompiledNode(); + fn.address = address; + return fn; + } + + private void compilePrevTail(int prefixLenPlus1) throws IOException { + assert prefixLenPlus1 >= 1; + //System.out.println(" compileTail " + prefixLenPlus1); + for(int idx=lastInput.length; idx >= prefixLenPlus1; idx--) { + boolean doPrune = false; + boolean doCompile = false; + + final UnCompiledNode node = frontier[idx]; + final UnCompiledNode parent = frontier[idx-1]; + + if (node.inputCount < minSuffixCount1) { + doPrune = true; + doCompile = true; + } else if (idx > prefixLenPlus1) { + // prune if parent's inputCount is less than suffixMinCount2 + if (parent.inputCount < minSuffixCount2 || minSuffixCount2 == 1 && parent.inputCount == 1) { + // my parent, about to be compiled, doesn't make the cut, so + // I'm definitely pruned + + // if pruneCount2 is 1, we keep only up + // until the 'distinguished edge', ie we keep only the + // 'divergent' part of the FST. if my parent, about to be + // compiled, has inputCount 1 then we are already past the + // distinguished edge. NOTE: this only works if + // the FST outputs are not "compressible" (simple + // ords ARE compressible). + doPrune = true; + } else { + // my parent, about to be compiled, does make the cut, so + // I'm definitely not pruned + doPrune = false; + } + doCompile = true; + } else { + // if pruning is disabled (count is 0) we can always + // compile current node + doCompile = minSuffixCount2 == 0; + } + + //System.out.println(" label=" + ((char) lastInput.ints[lastInput.offset+idx-1]) + " idx=" + idx + " inputCount=" + frontier[idx].inputCount + " doCompile=" + doCompile + " doPrune=" + doPrune); + + if (node.inputCount < minSuffixCount2 || minSuffixCount2 == 1 && node.inputCount == 1) { + // drop all arcs + for(int arcIdx=0;arcIdx target = (UnCompiledNode) node.arcs[arcIdx].target; + target.clear(); + } + node.numArcs = 0; + } + + if (doPrune) { + // this node doesn't make it -- deref it + node.clear(); + parent.deleteLast(lastInput.ints[lastInput.offset+idx-1], node); + } else { + + if (minSuffixCount2 != 0) { + compileAllTargets(node); + } + final T nextFinalOutput = node.output; + final boolean isFinal = node.isFinal; + + if (doCompile) { + // this node makes it and we now compile it. first, + // compile any targets that were previously + // undecided: + parent.replaceLast(lastInput.ints[lastInput.offset + idx-1], + compileNode(node), + nextFinalOutput, + isFinal); + } else { + // replaceLast just to install + // nextFinalOutput/isFinal onto the arc + parent.replaceLast(lastInput.ints[lastInput.offset + idx-1], + node, + nextFinalOutput, + isFinal); + // this node will stay in play for now, since we are + // undecided on whether to prune it. later, it + // will be either compiled or pruned, so we must + // allocate a new node: + frontier[idx] = new UnCompiledNode(this); + } + } + } + } + + private final IntsRef scratchIntsRef = new IntsRef(10); + + public void add(BytesRef input, T output) throws IOException { + assert fst.getInputType() == FST.INPUT_TYPE.BYTE1; + scratchIntsRef.grow(input.length); + for(int i=0;i 0: "inputs are added out of order lastInput=" + lastInput + " vs input=" + input; + assert validOutput(output); + + //System.out.println("\nadd: " + input); + if (input.length == 0) { + // empty input: only allowed as first input. we have + // to special case this because the packed FST + // format cannot represent the empty input since + // 'finalness' is stored on the incoming arc, not on + // the node + frontier[0].inputCount++; + fst.setEmptyOutput(output); + return; + } + + // compare shared prefix length + int pos1 = 0; + int pos2 = input.offset; + final int pos1Stop = Math.min(lastInput.length, input.length); + while(true) { + //System.out.println(" incr " + pos1); + frontier[pos1].inputCount++; + if (pos1 >= pos1Stop || lastInput.ints[pos1] != input.ints[pos2]) { + break; + } + pos1++; + pos2++; + } + final int prefixLenPlus1 = pos1+1; + + if (frontier.length < input.length+1) { + final UnCompiledNode[] next = ArrayUtil.grow(frontier, input.length+1); + for(int idx=frontier.length;idx(this); + } + frontier = next; + } + + // minimize/compile states from previous input's + // orphan'd suffix + compilePrevTail(prefixLenPlus1); + + // init tail states for current input + for(int idx=prefixLenPlus1;idx<=input.length;idx++) { + frontier[idx-1].addArc(input.ints[input.offset + idx - 1], + frontier[idx]); + //System.out.println(" incr tail " + idx); + frontier[idx].inputCount++; + } + + final UnCompiledNode lastNode = frontier[input.length]; + lastNode.isFinal = true; + lastNode.output = NO_OUTPUT; + + // push conflicting outputs forward, only as far as + // needed + for(int idx=1;idx node = frontier[idx]; + final UnCompiledNode parentNode = frontier[idx-1]; + + final T lastOutput = parentNode.getLastOutput(input.ints[input.offset + idx - 1]); + assert validOutput(lastOutput); + + final T commonOutputPrefix; + final T wordSuffix; + + if (lastOutput != NO_OUTPUT) { + commonOutputPrefix = fst.outputs.common(output, lastOutput); + assert validOutput(commonOutputPrefix); + wordSuffix = fst.outputs.subtract(lastOutput, commonOutputPrefix); + assert validOutput(wordSuffix); + parentNode.setLastOutput(input.ints[input.offset + idx - 1], commonOutputPrefix); + node.prependOutput(wordSuffix); + } else { + commonOutputPrefix = wordSuffix = NO_OUTPUT; + } + + output = fst.outputs.subtract(output, commonOutputPrefix); + assert validOutput(output); + } + + // push remaining output: + frontier[prefixLenPlus1-1].setLastOutput(input.ints[input.offset + prefixLenPlus1-1], output); + + // save last input + lastInput.copy(input); + + //System.out.println(" count[0]=" + frontier[0].inputCount); + } + + private boolean validOutput(T output) { + return output == NO_OUTPUT || !output.equals(NO_OUTPUT); + } + + /** Returns final FST. NOTE: this will return null if + * nothing is accepted by the FST. */ + public FST finish() throws IOException { + + // minimize nodes in the last word's suffix + 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) { + return null; + } else if (minSuffixCount1 > 0 || minSuffixCount2 > 0) { + // empty string got pruned + return null; + } else { + fst.finish(compileNode(frontier[0]).address); + //System.out.println("compile addr = " + fst.getStartNode()); + return fst; + } + } else { + if (minSuffixCount2 != 0) { + compileAllTargets(frontier[0]); + } + //System.out.println("NOW: " + frontier[0].numArcs); + fst.finish(compileNode(frontier[0]).address); + } + + return fst; + } + + private void compileAllTargets(UnCompiledNode node) throws IOException { + for(int arcIdx=0;arcIdx arc = node.arcs[arcIdx]; + if (!arc.target.isCompiled()) { + // not yet compiled + @SuppressWarnings("unchecked") final UnCompiledNode n = (UnCompiledNode) arc.target; + arc.target = compileNode(n); + } + } + } + + static class Arc { + public int label; // really an "unsigned" byte + public Node target; + public boolean isFinal; + public T output; + public T nextFinalOutput; + } + + // NOTE: not many instances of Node or CompiledNode are in + // memory while the FST is being built; it's only the + // current "frontier": + + static interface Node { + boolean isCompiled(); + } + + static final class CompiledNode implements Node { + int address; + public boolean isCompiled() { + return true; + } + } + + static final class UnCompiledNode implements Node { + final Builder owner; + int numArcs; + Arc[] arcs; + T output; + boolean isFinal; + int inputCount; + + @SuppressWarnings("unchecked") + public UnCompiledNode(Builder owner) { + this.owner = owner; + arcs = (Arc[]) new Arc[1]; + arcs[0] = new Arc(); + output = owner.NO_OUTPUT; + } + + public boolean isCompiled() { + return false; + } + + public void clear() { + numArcs = 0; + isFinal = false; + output = owner.NO_OUTPUT; + inputCount = 0; + } + + public T getLastOutput(int labelToMatch) { + assert numArcs > 0; + assert arcs[numArcs-1].label == labelToMatch; + return arcs[numArcs-1].output; + } + + public void addArc(int label, Node target) { + assert label >= 0; + assert numArcs == 0 || label > arcs[numArcs-1].label: "arc[-1].label=" + arcs[numArcs-1].label + " new label=" + label + " numArcs=" + numArcs; + if (numArcs == arcs.length) { + final Arc[] newArcs = ArrayUtil.grow(arcs); + for(int arcIdx=numArcs;arcIdx(); + } + arcs = newArcs; + } + final Arc arc = arcs[numArcs++]; + arc.label = label; + arc.target = target; + arc.output = arc.nextFinalOutput = owner.NO_OUTPUT; + arc.isFinal = false; + } + + public void replaceLast(int labelToMatch, Node target, T nextFinalOutput, boolean isFinal) { + assert numArcs > 0; + final Arc arc = arcs[numArcs-1]; + assert arc.label == labelToMatch: "arc.label=" + arc.label + " vs " + labelToMatch; + arc.target = target; + //assert target.address != -2; + arc.nextFinalOutput = nextFinalOutput; + arc.isFinal = isFinal; + } + + public void deleteLast(int label, Node target) { + assert numArcs > 0; + assert label == arcs[numArcs-1].label; + assert target == arcs[numArcs-1].target; + numArcs--; + } + + public void setLastOutput(int labelToMatch, T newOutput) { + assert owner.validOutput(newOutput); + assert numArcs > 0; + final Arc arc = arcs[numArcs-1]; + assert arc.label == labelToMatch; + arc.output = newOutput; + } + + // pushes an output prefix forward onto all arcs + public void prependOutput(T outputPrefix) { + assert owner.validOutput(outputPrefix); + + for(int arcIdx=0;arcIdx { + + private final static BytesRef NO_OUTPUT = new BytesRef(); + + private ByteSequenceOutputs() { + } + + public static ByteSequenceOutputs getSingleton() { + return new ByteSequenceOutputs(); + } + + @Override + public BytesRef common(BytesRef output1, BytesRef output2) { + assert output1 != null; + assert output2 != null; + + int pos1 = output1.offset; + int pos2 = output2.offset; + int stopAt1 = pos1 + Math.min(output1.length, output2.length); + while(pos1 < stopAt1) { + if (output1.bytes[pos1] != output2.bytes[pos2]) { + break; + } + pos1++; + pos2++; + } + + if (pos1 == output1.offset) { + // no common prefix + return NO_OUTPUT; + } else if (pos1 == output1.offset + output1.length) { + // output1 is a prefix of output2 + return output1; + } else if (pos2 == output2.offset + output2.length) { + // output2 is a prefix of output1 + return output2; + } else { + return new BytesRef(output1.bytes, output1.offset, pos1-output1.offset); + } + } + + @Override + public BytesRef subtract(BytesRef output, BytesRef inc) { + assert output != null; + assert inc != null; + if (inc == NO_OUTPUT) { + // no prefix removed + return output; + } else if (inc.length == output.length) { + // entire output removed + return NO_OUTPUT; + } else { + assert inc.length < output.length: "inc.length=" + inc.length + " vs output.length=" + output.length; + assert inc.length > 0; + return new BytesRef(output.bytes, output.offset + inc.length, output.length-inc.length); + } + } + + @Override + public BytesRef add(BytesRef prefix, BytesRef output) { + assert prefix != null; + assert output != null; + if (prefix == NO_OUTPUT) { + return output; + } else if (output == NO_OUTPUT) { + return prefix; + } else { + assert prefix.length > 0; + assert output.length > 0; + BytesRef result = new BytesRef(prefix.length + output.length); + System.arraycopy(prefix.bytes, prefix.offset, result.bytes, 0, prefix.length); + System.arraycopy(output.bytes, output.offset, result.bytes, prefix.length, output.length); + result.length = prefix.length + output.length; + return result; + } + } + + @Override + public void write(BytesRef prefix, DataOutput out) throws IOException { + assert prefix != null; + out.writeVInt(prefix.length); + out.writeBytes(prefix.bytes, prefix.offset, prefix.length); + } + + @Override + public BytesRef read(DataInput in) throws IOException { + final int len = in.readVInt(); + if (len == 0) { + return NO_OUTPUT; + } else { + final BytesRef output = new BytesRef(len); + in.readBytes(output.bytes, 0, len); + output.length = len; + return output; + } + } + + @Override + public BytesRef getNoOutput() { + return NO_OUTPUT; + } + + @Override + public String outputToString(BytesRef output) { + return output.utf8ToString(); + } +} Added: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/BytesRefFSTEnum.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/BytesRefFSTEnum.java?rev=1044834&view=auto ============================================================================== --- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/BytesRefFSTEnum.java (added) +++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/BytesRefFSTEnum.java Sun Dec 12 15:36:08 2010 @@ -0,0 +1,304 @@ +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.BytesRef; + +import java.io.IOException; + +/** 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 = (FST.Arc[]) 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; + private final InputOutput result = new InputOutput(); + + public static class InputOutput { + public BytesRef input; + public T output; + } + + public BytesRefFSTEnum(FST fst) { + this.fst = fst; + result.input = current; + NO_OUTPUT = fst.outputs.getNoOutput(); + } + + 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"); + + 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; + } + + private void grow() { + final int l = current.length + 1; + current.grow(l); + arcs = ArrayUtil.grow(arcs, l); + output = ArrayUtil.grow(output, l); + } + + 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; + } else { + newOutput = fst.outputs.add(output[current.length-1], addedOutput); + } + 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); + } + } + + 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]; + } + + private void pop() { + while (current.length > 0 && arcs[current.length-1].isLast()) { + current.length--; + } + } + + private FST.Arc getArc(int idx) { + if (arcs[idx] == null) { + arcs[idx] = new FST.Arc(); + } + return arcs[idx]; + } +} Added: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java?rev=1044834&view=auto ============================================================================== --- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java (added) +++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java Sun Dec 12 15:36:08 2010 @@ -0,0 +1,922 @@ +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.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 + * (http://sourceforge.net/projects/morfologik). + * @lucene.experimental + */ +public class FST { + public static enum INPUT_TYPE {BYTE1, BYTE2, BYTE4}; + private final INPUT_TYPE inputType; + + private final static int BIT_FINAL_ARC = 1 << 0; + private final static int BIT_LAST_ARC = 1 << 1; + private final static int BIT_TARGET_NEXT = 1 << 2; + private final static int BIT_STOP_NODE = 1 << 3; + private final static int BIT_ARC_HAS_OUTPUT = 1 << 4; + private final static int BIT_ARC_HAS_FINAL_OUTPUT = 1 << 5; + + // Arcs are stored as fixed-size (per entry) array, so + // that we can find an arc using binary search. We do + // this when number of arcs is > NUM_ARCS_ARRAY: + private final static int BIT_ARCS_AS_FIXED_ARRAY = 1 << 6; + + // If the node has >= this number of arcs, the arcs are + // stored as a fixed array. Fixed array consumes more RAM + // but enables binary search on the arcs (instead of + // linear scan) on lookup by arc label: + private final static int NUM_ARCS_FIXED_ARRAY = 10; + private int[] bytesPerArc = new int[0]; + + // Increment version to change it + private final static String FILE_FORMAT_NAME = "FST"; + private final static int VERSION_START = 0; + private final static int VERSION_CURRENT = VERSION_START; + + // Never serialized; just used to represent the virtual + // final node w/ no arcs: + private final static int FINAL_END_NODE = -1; + + // Never serialized; just used to represent the virtual + // non-final node w/ no arcs: + private final static int NON_FINAL_END_NODE = 0; + + // if non-null, this FST accepts the empty string and + // produces this output + private T emptyOutput; + private byte[] emptyOutputBytes; + + private byte[] bytes; + int byteUpto = 0; + + private int startNode = -1; + + public final Outputs outputs; + + private int lastFrozenNode; + + private final T NO_OUTPUT; + + public int nodeCount; + public int arcCount; + public int arcWithOutputCount; + + public final static class Arc { + int label; // really a "unsigned" byte + int target; + byte flags; + T output; + T nextFinalOutput; + int nextArc; + + // This is non-zero if current arcs are fixed array: + int posArcsStart; + int bytesPerArc; + int arcIdx; + int numArcs; + + // Must call this before re-using an Arc instance on a + // new node + public void reset() { + bytesPerArc = 0; + } + + public boolean flag(int flag) { + return FST.flag(flags, flag); + } + + public boolean isLast() { + return flag(BIT_LAST_ARC); + } + + public boolean isFinal() { + return flag(BIT_FINAL_ARC); + } + }; + + static boolean flag(int flags, int bit) { + return (flags & bit) != 0; + } + + private final BytesWriter writer; + + // make a new empty FST, for building + public FST(INPUT_TYPE inputType, Outputs outputs) { + this.inputType = inputType; + this.outputs = outputs; + bytes = new byte[128]; + NO_OUTPUT = outputs.getNoOutput(); + + writer = new BytesWriter(); + + emptyOutput = null; + } + + // create an existing FST + public FST(IndexInput in, Outputs outputs) throws IOException { + this.outputs = outputs; + writer = null; + CodecUtil.checkHeader(in, FILE_FORMAT_NAME, VERSION_START, VERSION_START); + if (in.readByte() == 1) { + // accepts empty string + int numBytes = in.readVInt(); + // messy + bytes = new byte[numBytes]; + in.readBytes(bytes, 0, numBytes); + emptyOutput = outputs.read(new BytesReader(numBytes-1)); + } else { + emptyOutput = null; + } + final byte t = in.readByte(); + switch(t) { + case 0: + inputType = INPUT_TYPE.BYTE1; + break; + case 1: + inputType = INPUT_TYPE.BYTE2; + break; + case 2: + inputType = INPUT_TYPE.BYTE4; + break; + default: + throw new IllegalStateException("invalid input type " + t); + } + startNode = in.readVInt(); + nodeCount = in.readVInt(); + arcCount = in.readVInt(); + arcWithOutputCount = in.readVInt(); + + bytes = new byte[in.readVInt()]; + in.readBytes(bytes, 0, bytes.length); + NO_OUTPUT = outputs.getNoOutput(); + } + + public INPUT_TYPE getInputType() { + return inputType; + } + + /** Returns bytes used to represent the FST */ + public int sizeInBytes() { + return bytes.length; + } + + void finish(int startNode) { + if (this.startNode != -1) { + throw new IllegalStateException("already finished"); + } + byte[] finalBytes = new byte[writer.posWrite]; + System.arraycopy(bytes, 0, finalBytes, 0, writer.posWrite); + bytes = finalBytes; + this.startNode = startNode; + } + + public void setEmptyOutput(T v) throws IOException { + if (emptyOutput != null) { + throw new IllegalStateException("empty output is already set"); + } + emptyOutput = v; + + // TODO: this is messy -- replace with sillyBytesWriter; maybe make + // bytes private + final int posSave = writer.posWrite; + outputs.write(emptyOutput, writer); + emptyOutputBytes = new byte[writer.posWrite-posSave]; + + // reverse + final int stopAt = (writer.posWrite - posSave)/2; + int upto = 0; + while(upto < stopAt) { + final byte b = bytes[posSave + upto]; + bytes[posSave+upto] = bytes[writer.posWrite-upto-1]; + bytes[writer.posWrite-upto-1] = b; + upto++; + } + System.arraycopy(bytes, posSave, emptyOutputBytes, 0, writer.posWrite-posSave); + writer.posWrite = posSave; + } + + public void save(IndexOutput out) throws IOException { + if (startNode == -1) { + throw new IllegalStateException("call finish first"); + } + CodecUtil.writeHeader(out, FILE_FORMAT_NAME, VERSION_CURRENT); + if (emptyOutput != null) { + out.writeByte((byte) 1); + out.writeVInt(emptyOutputBytes.length); + out.writeBytes(emptyOutputBytes, 0, emptyOutputBytes.length); + } else { + out.writeByte((byte) 0); + } + final byte t; + if (inputType == INPUT_TYPE.BYTE1) { + t = 0; + } else if (inputType == INPUT_TYPE.BYTE2) { + t = 1; + } else { + t = 2; + } + out.writeByte(t); + out.writeVInt(startNode); + out.writeVInt(nodeCount); + out.writeVInt(arcCount); + out.writeVInt(arcWithOutputCount); + out.writeVInt(bytes.length); + out.writeBytes(bytes, 0, bytes.length); + } + + private void writeLabel(int v) throws IOException { + assert v >= 0: "v=" + v; + if (inputType == INPUT_TYPE.BYTE1) { + assert v <= 255: "v=" + v; + writer.writeByte((byte) v); + } else if (inputType == INPUT_TYPE.BYTE2) { + assert v <= 65535: "v=" + v; + writer.writeVInt(v); + } else { + //writeInt(v); + writer.writeVInt(v); + } + } + + private int readLabel(DataInput in) throws IOException { + final int v; + if (inputType == INPUT_TYPE.BYTE1) { + v = in.readByte()&0xFF; + } else if (inputType == INPUT_TYPE.BYTE2) { + v = in.readVInt(); + } else { + v = in.readVInt(); + } + return v; + } + + // 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 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 { + //System.out.println("FST.addNode pos=" + posWrite + " numArcs=" + node.numArcs); + if (node.numArcs == 0) { + if (node.isFinal) { + return FINAL_END_NODE; + } else { + return NON_FINAL_END_NODE; + } + } + + int startAddress = writer.posWrite; + //System.out.println(" startAddr=" + startAddress); + + final boolean doFixedArray = node.numArcs >= NUM_ARCS_FIXED_ARRAY; + final int fixedArrayStart; + if (doFixedArray) { + if (bytesPerArc.length < node.numArcs) { + bytesPerArc = new int[ArrayUtil.oversize(node.numArcs, 1)]; + } + // write a "false" first arc: + writer.writeByte((byte) BIT_ARCS_AS_FIXED_ARRAY); + writer.writeVInt(node.numArcs); + // placeholder -- we'll come back and write the number + // of bytes per arc here: + writer.writeByte((byte) 0); + fixedArrayStart = writer.posWrite; + //System.out.println(" do fixed arcs array arcsStart=" + fixedArrayStart); + } else { + fixedArrayStart = 0; + } + + nodeCount++; + arcCount += node.numArcs; + + final int lastArc = node.numArcs-1; + + int lastArcStart = writer.posWrite; + int maxBytesPerArc = 0; + for(int arcIdx=0;arcIdx arc = node.arcs[arcIdx]; + final Builder.CompiledNode target = (Builder.CompiledNode) arc.target; + int flags = 0; + + if (arcIdx == lastArc) { + flags += BIT_LAST_ARC; + } + + if (lastFrozenNode == target.address && !doFixedArray) { + flags += BIT_TARGET_NEXT; + } + + if (arc.isFinal) { + flags += BIT_FINAL_ARC; + if (arc.nextFinalOutput != NO_OUTPUT) { + flags += BIT_ARC_HAS_FINAL_OUTPUT; + } + } else { + assert arc.nextFinalOutput == NO_OUTPUT; + } + + boolean targetHasArcs = hasArcs(target.address); + + if (!targetHasArcs) { + flags += BIT_STOP_NODE; + } + + if (arc.output != NO_OUTPUT) { + flags += BIT_ARC_HAS_OUTPUT; + } + + writer.writeByte((byte) flags); + writeLabel(arc.label); + + //System.out.println(" write arc: label=" + arc.label + " flags=" + flags); + + if (arc.output != NO_OUTPUT) { + outputs.write(arc.output, writer); + arcWithOutputCount++; + } + if (arc.nextFinalOutput != NO_OUTPUT) { + outputs.write(arc.nextFinalOutput, writer); + } + + if (targetHasArcs && (doFixedArray || lastFrozenNode != target.address)) { + assert target.address > 0; + writer.writeInt(target.address); + } + + // just write the arcs "like normal" on first pass, + // but record how many bytes each one took, and max + // byte size: + if (doFixedArray) { + bytesPerArc[arcIdx] = writer.posWrite - lastArcStart; + lastArcStart = writer.posWrite; + maxBytesPerArc = Math.max(maxBytesPerArc, bytesPerArc[arcIdx]); + //System.out.println(" bytes=" + bytesPerArc[arcIdx]); + } + } + + if (doFixedArray) { + assert maxBytesPerArc > 0; + // 2nd pass just "expands" all arcs to take up a fixed + // byte size + final int sizeNeeded = fixedArrayStart + node.numArcs * maxBytesPerArc; + bytes = ArrayUtil.grow(bytes, sizeNeeded); + if (maxBytesPerArc > 255) { + throw new IllegalStateException("max arc size is too large (" + maxBytesPerArc + ")"); + } + bytes[fixedArrayStart-1] = (byte) maxBytesPerArc; + + // expand the arcs in place, backwards + int srcPos = writer.posWrite; + int destPos = fixedArrayStart + node.numArcs*maxBytesPerArc; + writer.posWrite = destPos; + for(int arcIdx=node.numArcs-1;arcIdx>=0;arcIdx--) { + //System.out.println(" repack arcIdx=" + arcIdx + " srcPos=" + srcPos + " destPos=" + destPos); + destPos -= maxBytesPerArc; + srcPos -= bytesPerArc[arcIdx]; + if (srcPos != destPos) { + assert destPos > srcPos; + System.arraycopy(bytes, srcPos, bytes, destPos, bytesPerArc[arcIdx]); + } + } + } + + // reverse bytes in-place; we do this so that the + // "BIT_TARGET_NEXT" opto can work, ie, it reads the + // node just before the current one + final int endAddress = writer.posWrite; + final int stopAt = (endAddress - startAddress)/2; + int upto = 0; + while (upto < stopAt) { + final byte b = bytes[startAddress+upto]; + bytes[startAddress+upto] = bytes[endAddress-upto-1]; + bytes[endAddress-upto-1] = b; + upto++; + } + + lastFrozenNode = endAddress - 1; + /* + System.out.println(" return node addr=" + (endAddress-1)); + for(int i=endAddress-1;i>=startAddress;i--) { + System.out.println(" bytes[" + i + "]=" + bytes[i]); + } + */ + + return endAddress-1; + } + + public Arc readFirstArc(int address, Arc arc) throws IOException { + //System.out.println("readFirstArc addr=" + address); + //int pos = address; + final BytesReader in = new BytesReader(address); + + arc.flags = in.readByte(); + + if (arc.flag(BIT_ARCS_AS_FIXED_ARRAY)) { + //System.out.println(" fixedArray"); + // this is first arc in a fixed-array + arc.numArcs = in.readVInt(); + arc.bytesPerArc = in.readByte() & 0xFF; + arc.arcIdx = -1; + arc.posArcsStart = in.pos; + //System.out.println(" bytesPer=" + arc.bytesPerArc + " numArcs=" + arc.numArcs + " arcsStart=" + pos); + } else { + in.pos++; + arc.bytesPerArc = 0; + } + arc.nextArc = in.pos; + return readNextArc(arc); + } + + public Arc readNextArc(Arc arc) throws IOException { + // this is a continuing arc in a fixed array + final BytesReader in; + if (arc.bytesPerArc != 0) { + // arcs are at fixed entries + arc.arcIdx++; + in = new BytesReader(arc.posArcsStart - arc.arcIdx*arc.bytesPerArc); + } else { + // arcs are packed + in = new BytesReader(arc.nextArc); + } + arc.flags = in.readByte(); + arc.label = readLabel(in); + + if (arc.flag(BIT_ARC_HAS_OUTPUT)) { + arc.output = outputs.read(in); + } else { + arc.output = outputs.getNoOutput(); + } + + if (arc.flag(BIT_ARC_HAS_FINAL_OUTPUT)) { + arc.nextFinalOutput = outputs.read(in); + } else { + arc.nextFinalOutput = outputs.getNoOutput(); + } + + if (arc.flag(BIT_STOP_NODE)) { + arc.target = FINAL_END_NODE; + arc.nextArc = in.pos; + } else if (arc.flag(BIT_TARGET_NEXT)) { + arc.nextArc = in.pos; + if (!arc.flag(BIT_LAST_ARC)) { + if (arc.bytesPerArc == 0) { + // must scan + seekToNextNode(in); + } else { + in.pos = arc.posArcsStart - arc.bytesPerArc * arc.numArcs; + } + } + arc.target = in.pos; + } else { + arc.target = in.readInt(); + arc.nextArc = in.pos; + } + + return arc; + } + + public Arc findArc(int address, int labelToMatch, Arc arc) throws IOException { + // TODO: maybe make an explicit thread state that holds + // reusable stuff eg BytesReader: + final BytesReader in = new BytesReader(address); + + 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; + int low = 0; + int high = arc.numArcs-1; + while (low <= high) { + int mid = (low + high) >>> 1; + in.pos = arc.posArcsStart - arc.bytesPerArc*mid - 1; + int midLabel = readLabel(in); + final int cmp = midLabel - labelToMatch; + if (cmp < 0) + low = mid + 1; + else if (cmp > 0) + high = mid - 1; + else { + arc.arcIdx = mid-1; + return readNextArc(arc); + } + } + + return null; + } + //System.out.println("findArc: scan"); + + readFirstArc(address, arc); + + while(true) { + if (arc.label == labelToMatch) { + return arc; + } else if (arc.isLast()) { + return null; + } else { + readNextArc(arc); + } + } + } + + /** 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; + } + + public int getArcCount() { + return arcCount; + } + + public int getArcWithOutputCount() { + return arcWithOutputCount; + } + + // Non-static: writes to FST's byte[] + private class BytesWriter extends DataOutput { + int posWrite; + + public BytesWriter() { + // pad: ensure no node gets address 0 which is reserved to mean + // the stop state w/ no arcs + posWrite = 1; + } + + @Override + public void writeByte(byte b) { + if (bytes.length == posWrite) { + bytes = ArrayUtil.grow(bytes); + } + assert posWrite < bytes.length: "posWrite=" + posWrite + " bytes.length=" + bytes.length; + bytes[posWrite++] = b; + } + + @Override + public void writeBytes(byte[] b, int offset, int length) { + final int size = posWrite + length; + bytes = ArrayUtil.grow(bytes, size); + System.arraycopy(b, offset, bytes, posWrite, length); + posWrite += length; + } + } + + // Non-static: reads byte[] from FST + private class BytesReader extends DataInput { + int pos; + + public BytesReader(int pos) { + this.pos = pos; + } + + @Override + public byte readByte() { + return bytes[pos--]; + } + + @Override + public void readBytes(byte[] b, int offset, int len) { + for(int i=0;i { + + private final static IntsRef NO_OUTPUT = new IntsRef(); + + private IntSequenceOutputs() { + } + + public static IntSequenceOutputs getSingleton() { + return new IntSequenceOutputs(); + } + + @Override + public IntsRef common(IntsRef output1, IntsRef output2) { + assert output1 != null; + assert output2 != null; + + int pos1 = output1.offset; + int pos2 = output2.offset; + int stopAt1 = pos1 + Math.min(output1.length, output2.length); + while(pos1 < stopAt1) { + if (output1.ints[pos1] != output2.ints[pos2]) { + break; + } + pos1++; + pos2++; + } + + if (pos1 == output1.offset) { + // no common prefix + return NO_OUTPUT; + } else if (pos1 == output1.offset + output1.length) { + // output1 is a prefix of output2 + return output1; + } else if (pos2 == output2.offset + output2.length) { + // output2 is a prefix of output1 + return output2; + } else { + return new IntsRef(output1.ints, output1.offset, pos1-output1.offset); + } + } + + @Override + public IntsRef subtract(IntsRef output, IntsRef inc) { + assert output != null; + assert inc != null; + if (inc == NO_OUTPUT) { + // no prefix removed + return output; + } else if (inc.length == output.length) { + // entire output removed + return NO_OUTPUT; + } else { + assert inc.length < output.length: "inc.length=" + inc.length + " vs output.length=" + output.length; + assert inc.length > 0; + return new IntsRef(output.ints, output.offset + inc.length, output.length-inc.length); + } + } + + @Override + public IntsRef add(IntsRef prefix, IntsRef output) { + assert prefix != null; + assert output != null; + if (prefix == NO_OUTPUT) { + return output; + } else if (output == NO_OUTPUT) { + return prefix; + } else { + assert prefix.length > 0; + assert output.length > 0; + IntsRef result = new IntsRef(prefix.length + output.length); + System.arraycopy(prefix.ints, prefix.offset, result.ints, 0, prefix.length); + System.arraycopy(output.ints, output.offset, result.ints, prefix.length, output.length); + result.length = prefix.length + output.length; + return result; + } + } + + @Override + public void write(IntsRef prefix, DataOutput out) throws IOException { + assert prefix != null; + out.writeVInt(prefix.length); + for(int idx=0;idx