You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2014/07/20 14:08:33 UTC

svn commit: r1612080 [3/3] - in /lucene/dev/trunk/lucene: ./ codecs/src/java/org/apache/lucene/codecs/blocktreeords/ codecs/src/resources/META-INF/services/ codecs/src/test/org/apache/lucene/codecs/blocktreeords/ core/src/java/org/apache/lucene/codecs/...

Added: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsSegmentTermsEnumFrame.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsSegmentTermsEnumFrame.java?rev=1612080&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsSegmentTermsEnumFrame.java (added)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsSegmentTermsEnumFrame.java Sun Jul 20 12:08:32 2014
@@ -0,0 +1,851 @@
+package org.apache.lucene.codecs.blocktreeords;
+
+/*
+ * 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 org.apache.lucene.codecs.BlockTermState;
+import org.apache.lucene.codecs.blocktreeords.FSTOrdsOutputs.Output;
+import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.TermsEnum.SeekStatus;
+import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.fst.FST;
+
+final class OrdsSegmentTermsEnumFrame {
+  // Our index in stack[]:
+  final int ord;
+  // final boolean DEBUG = true;
+
+  boolean hasTerms;
+  boolean hasTermsOrig;
+  boolean isFloor;
+
+  // static boolean DEBUG = OrdsSegmentTermsEnum.DEBUG;
+
+  FST.Arc<Output> arc;
+
+  // File pointer where this block was loaded from
+  long fp;
+  long fpOrig;
+  long fpEnd;
+
+  byte[] suffixBytes = new byte[128];
+  final ByteArrayDataInput suffixesReader = new ByteArrayDataInput();
+
+  byte[] statBytes = new byte[64];
+  final ByteArrayDataInput statsReader = new ByteArrayDataInput();
+
+  byte[] floorData = new byte[32];
+  final ByteArrayDataInput floorDataReader = new ByteArrayDataInput();
+
+  // Length of prefix shared by all terms in this block
+  int prefix;
+
+  // Number of entries (term or sub-block) in this block
+  int entCount;
+
+  // Which term we will next read, or -1 if the block
+  // isn't loaded yet
+  int nextEnt;
+
+  // Starting termOrd for this frame, used to reset termOrd in rewind()
+  long termOrdOrig;
+
+  // 1 + ordinal of the current term
+  long termOrd;
+
+  // True if this block is either not a floor block,
+  // or, it's the last sub-block of a floor block
+  boolean isLastInFloor;
+
+  // True if all entries are terms
+  boolean isLeafBlock;
+
+  long lastSubFP;
+
+  // Starting byte of next floor block:
+  int nextFloorLabel;
+
+  // Starting termOrd of next floor block:
+  long nextFloorTermOrd;
+
+  int numFollowFloorBlocks;
+
+  // Next term to decode metaData; we decode metaData
+  // lazily so that scanning to find the matching term is
+  // fast and only if you find a match and app wants the
+  // stats or docs/positions enums, will we decode the
+  // metaData
+  int metaDataUpto;
+
+  final BlockTermState state;
+
+  // metadata buffer, holding monotonic values
+  public long[] longs;
+  // metadata buffer, holding general values
+  public byte[] bytes;
+  ByteArrayDataInput bytesReader;
+
+  private final OrdsSegmentTermsEnum ste;
+
+  public OrdsSegmentTermsEnumFrame(OrdsSegmentTermsEnum ste, int ord) throws IOException {
+    this.ste = ste;
+    this.ord = ord;
+    this.state = ste.fr.parent.postingsReader.newTermState();
+    this.state.totalTermFreq = -1;
+    this.longs = new long[ste.fr.longsSize];
+  }
+
+  public void setFloorData(ByteArrayDataInput in, BytesRef source) {
+    final int numBytes = source.length - (in.getPosition() - source.offset);
+    assert numBytes > 0;
+    if (numBytes > floorData.length) {
+      floorData = new byte[ArrayUtil.oversize(numBytes, 1)];
+    }
+    System.arraycopy(source.bytes, source.offset+in.getPosition(), floorData, 0, numBytes);
+    floorDataReader.reset(floorData, 0, numBytes);
+    numFollowFloorBlocks = floorDataReader.readVInt();
+    nextFloorLabel = floorDataReader.readByte() & 0xff;
+    nextFloorTermOrd = termOrdOrig + floorDataReader.readVLong();
+    // System.out.println("  setFloorData ord=" + ord + " nextFloorTermOrd=" + nextFloorTermOrd + " shift=" + (nextFloorTermOrd-termOrdOrig));
+
+    //if (DEBUG) {
+    //System.out.println("    setFloorData fpOrig=" + fpOrig + " bytes=" + new BytesRef(source.bytes, source.offset + in.getPosition(), numBytes) + " numFollowFloorBlocks=" + numFollowFloorBlocks + " nextFloorLabel=" + toHex(nextFloorLabel));
+    //}
+  }
+
+  public int getTermBlockOrd() {
+    return isLeafBlock ? nextEnt : state.termBlockOrd;
+  }
+
+  void loadNextFloorBlock() throws IOException {
+    //if (DEBUG) {
+    //System.out.println("    loadNextFloorBlock fp=" + fp + " fpEnd=" + fpEnd);
+    //}
+    assert arc == null || isFloor: "arc=" + arc + " isFloor=" + isFloor;
+    // NOTE: we don't need to touch termOrd here, because we fully scanned this current frame
+    fp = fpEnd;
+    nextEnt = -1;
+    loadBlock();
+  }
+
+  /* Does initial decode of next block of terms; this
+     doesn't actually decode the docFreq, totalTermFreq,
+     postings details (frq/prx offset, etc.) metadata;
+     it just loads them as byte[] blobs which are then      
+     decoded on-demand if the metadata is ever requested
+     for any term in this block.  This enables terms-only
+     intensive consumes (eg certain MTQs, respelling) to
+     not pay the price of decoding metadata they won't
+     use. */
+  void loadBlock() throws IOException {
+
+    // Clone the IndexInput lazily, so that consumers
+    // that just pull a TermsEnum to
+    // seekExact(TermState) don't pay this cost:
+    ste.initIndexInput();
+
+    if (nextEnt != -1) {
+      // Already loaded
+      return;
+    }
+    // System.out.println("loadBlock ord=" + ord + " termOrdOrig=" + termOrdOrig + " termOrd=" + termOrd + " fp=" + fp);
+
+    ste.in.seek(fp);
+    int code = ste.in.readVInt();
+    entCount = code >>> 1;
+    assert entCount > 0;
+    isLastInFloor = (code & 1) != 0;
+    assert arc == null || (isLastInFloor || isFloor);
+
+    // TODO: if suffixes were stored in random-access
+    // array structure, then we could do binary search
+    // instead of linear scan to find target term; eg
+    // we could have simple array of offsets
+
+    // term suffixes:
+    code = ste.in.readVInt();
+    isLeafBlock = (code & 1) != 0;
+    int numBytes = code >>> 1;
+    if (suffixBytes.length < numBytes) {
+      suffixBytes = new byte[ArrayUtil.oversize(numBytes, 1)];
+    }
+    ste.in.readBytes(suffixBytes, 0, numBytes);
+    suffixesReader.reset(suffixBytes, 0, numBytes);
+
+    /*if (DEBUG) {
+      if (arc == null) {
+      System.out.println("    loadBlock (next) fp=" + fp + " entCount=" + entCount + " prefixLen=" + prefix + " isLastInFloor=" + isLastInFloor + " leaf?=" + isLeafBlock);
+      } else {
+      System.out.println("    loadBlock (seek) fp=" + fp + " entCount=" + entCount + " prefixLen=" + prefix + " hasTerms?=" + hasTerms + " isFloor?=" + isFloor + " isLastInFloor=" + isLastInFloor + " leaf?=" + isLeafBlock);
+      }
+      }*/
+
+    // stats
+    numBytes = ste.in.readVInt();
+    if (statBytes.length < numBytes) {
+      statBytes = new byte[ArrayUtil.oversize(numBytes, 1)];
+    }
+    // System.out.println("READ stats numBytes=" + numBytes + " fp=" + ste.in.getFilePointer());
+    ste.in.readBytes(statBytes, 0, numBytes);
+    statsReader.reset(statBytes, 0, numBytes);
+    metaDataUpto = 0;
+
+    state.termBlockOrd = 0;
+    nextEnt = 0;
+    lastSubFP = -1;
+
+    // TODO: we could skip this if !hasTerms; but
+    // that's rare so won't help much
+    // metadata
+    numBytes = ste.in.readVInt();
+    if (bytes == null) {
+      bytes = new byte[ArrayUtil.oversize(numBytes, 1)];
+      bytesReader = new ByteArrayDataInput();
+    } else if (bytes.length < numBytes) {
+      bytes = new byte[ArrayUtil.oversize(numBytes, 1)];
+    }
+    ste.in.readBytes(bytes, 0, numBytes);
+    bytesReader.reset(bytes, 0, numBytes);
+
+    // Sub-blocks of a single floor block are always
+    // written one after another -- tail recurse:
+    fpEnd = ste.in.getFilePointer();
+    // if (DEBUG) {
+    //   System.out.println("      fpEnd=" + fpEnd);
+    // }
+  }
+
+  void rewind() {
+
+    // Force reload:
+    fp = fpOrig;
+    termOrd = termOrdOrig;
+    nextEnt = -1;
+    hasTerms = hasTermsOrig;
+    if (isFloor) {
+      floorDataReader.rewind();
+      numFollowFloorBlocks = floorDataReader.readVInt();
+      assert numFollowFloorBlocks > 0;
+      nextFloorLabel = floorDataReader.readByte() & 0xff;
+      nextFloorTermOrd = termOrdOrig + floorDataReader.readVLong();
+      //System.out.println("  frame.rewind nextFloorTermOrd=" + nextFloorTermOrd);
+    }
+
+    /*
+    //System.out.println("rewind");
+    // Keeps the block loaded, but rewinds its state:
+    if (nextEnt > 0 || fp != fpOrig) {
+    if (DEBUG) {
+    System.out.println("      rewind frame ord=" + ord + " fpOrig=" + fpOrig + " fp=" + fp + " hasTerms?=" + hasTerms + " isFloor?=" + isFloor + " nextEnt=" + nextEnt + " prefixLen=" + prefix);
+    }
+    if (fp != fpOrig) {
+    fp = fpOrig;
+    nextEnt = -1;
+    } else {
+    nextEnt = 0;
+    }
+    hasTerms = hasTermsOrig;
+    if (isFloor) {
+    floorDataReader.rewind();
+    numFollowFloorBlocks = floorDataReader.readVInt();
+    nextFloorLabel = floorDataReader.readByte() & 0xff;
+    }
+    assert suffixBytes != null;
+    suffixesReader.rewind();
+    assert statBytes != null;
+    statsReader.rewind();
+    metaDataUpto = 0;
+    state.termBlockOrd = 0;
+    // TODO: skip this if !hasTerms?  Then postings
+    // impl wouldn't have to write useless 0 byte
+    postingsReader.resetTermsBlock(fieldInfo, state);
+    lastSubFP = -1;
+    } else if (DEBUG) {
+    System.out.println("      skip rewind fp=" + fp + " fpOrig=" + fpOrig + " nextEnt=" + nextEnt + " ord=" + ord);
+    }
+    */
+  }
+
+  public boolean next() {
+    return isLeafBlock ? nextLeaf() : nextNonLeaf();
+  }
+
+  // Decodes next entry; returns true if it's a sub-block
+  public boolean nextLeaf() {
+    //if (DEBUG) System.out.println("  frame.next ord=" + ord + " nextEnt=" + nextEnt + " entCount=" + entCount);
+    assert nextEnt != -1 && nextEnt < entCount: "nextEnt=" + nextEnt + " entCount=" + entCount + " fp=" + fp + " termOrd=" + termOrd;
+    nextEnt++;
+    termOrd++;
+    suffix = suffixesReader.readVInt();
+    startBytePos = suffixesReader.getPosition();
+    ste.term.length = prefix + suffix;
+    if (ste.term.bytes.length < ste.term.length) {
+      ste.term.grow(ste.term.length);
+    }
+    suffixesReader.readBytes(ste.term.bytes, prefix, suffix);
+    // A normal term
+    ste.termExists = true;
+    return false;
+  }
+
+  public boolean nextNonLeaf() {
+    // if (DEBUG) System.out.println("  frame.next ord=" + ord + " nextEnt=" + nextEnt + " entCount=" + entCount);
+    assert nextEnt != -1 && nextEnt < entCount: "nextEnt=" + nextEnt + " entCount=" + entCount + " fp=" + fp;
+    nextEnt++;
+    final int code = suffixesReader.readVInt();
+    suffix = code >>> 1;
+    startBytePos = suffixesReader.getPosition();
+    ste.term.length = prefix + suffix;
+    if (ste.term.bytes.length < ste.term.length) {
+      ste.term.grow(ste.term.length);
+    }
+    suffixesReader.readBytes(ste.term.bytes, prefix, suffix);
+    if ((code & 1) == 0) {
+      // A normal term
+      ste.termExists = true;
+      subCode = 0;
+      state.termBlockOrd++;
+      termOrd++;
+      return false;
+    } else {
+      // A sub-block; make sub-FP absolute:
+      ste.termExists = false;
+      subCode = suffixesReader.readVLong();
+      termOrd += suffixesReader.readVLong();
+      lastSubFP = fp - subCode;
+      // if (DEBUG) {
+      //   System.out.println("    lastSubFP=" + lastSubFP);
+      // }
+      return true;
+    }
+  }
+        
+  // TODO: make this array'd so we can do bin search?
+  // likely not worth it?  need to measure how many
+  // floor blocks we "typically" get
+  public void scanToFloorFrame(BytesRef target) {
+
+    if (!isFloor || target.length <= prefix) {
+      // if (DEBUG) {
+      //    System.out.println("    scanToFloorFrame skip: isFloor=" + isFloor + " target.length=" + target.length + " vs prefix=" + prefix);
+      //  }
+      return;
+    }
+
+    final int targetLabel = target.bytes[target.offset + prefix] & 0xFF;
+
+    // if (DEBUG) {
+    //    System.out.println("    scanToFloorFrame fpOrig=" + fpOrig + " targetLabel=" + ((char) targetLabel) + " vs nextFloorLabel=" + ((char) nextFloorLabel) + " numFollowFloorBlocks=" + numFollowFloorBlocks);
+    //  }
+
+    if (targetLabel < nextFloorLabel) {
+      // if (DEBUG) {
+      //    System.out.println("      already on correct block");
+      //  }
+      return;
+    }
+
+    assert numFollowFloorBlocks != 0;
+
+    long newFP = fpOrig;
+    long lastFloorTermOrd = nextFloorTermOrd;
+    while (true) {
+      final long code = floorDataReader.readVLong();
+      newFP = fpOrig + (code >>> 1);
+      hasTerms = (code & 1) != 0;
+      // if (DEBUG) {
+      //    System.out.println("      label=" + ((char) nextFloorLabel) + " fp=" + newFP + " hasTerms?=" + hasTerms + " numFollowFloor=" + numFollowFloorBlocks);
+      //  }
+            
+      isLastInFloor = numFollowFloorBlocks == 1;
+      numFollowFloorBlocks--;
+
+      lastFloorTermOrd = nextFloorTermOrd;
+
+      if (isLastInFloor) {
+        nextFloorLabel = 256;
+        nextFloorTermOrd = Long.MAX_VALUE;
+        // if (DEBUG) {
+        //    System.out.println("        stop!  last block nextFloorLabel=" + ((char) nextFloorLabel));
+        //  }
+        break;
+      } else {
+        nextFloorLabel = floorDataReader.readByte() & 0xff;
+        nextFloorTermOrd += floorDataReader.readVLong();
+        //System.out.println("  scanToFloorFrame: nextFloorTermOrd=" + nextFloorTermOrd);
+        if (targetLabel < nextFloorLabel) {
+          // if (DEBUG) {
+          //    System.out.println("        stop!  nextFloorLabel=" + ((char) nextFloorLabel));
+          //  }
+          break;
+        }
+      }
+    }
+
+    if (newFP != fp) {
+      // Force re-load of the block:
+      // if (DEBUG) {
+      //    System.out.println("      force switch to fp=" + newFP + " oldFP=" + fp);
+      //  }
+      nextEnt = -1;
+      termOrd = lastFloorTermOrd;
+      fp = newFP;
+    } else {
+      // if (DEBUG) {
+      //    System.out.println("      stay on same fp=" + newFP);
+      //  }
+    }
+  }
+
+  // TODO: make this array'd so we can do bin search?
+  // likely not worth it?  need to measure how many
+  // floor blocks we "typically" get
+  public void scanToFloorFrame(long targetOrd) {
+    // System.out.println("  scanToFloorFrame targetOrd=" + targetOrd + " vs nextFloorTermOrd=" + nextFloorTermOrd + " numFollowFloorBlocks=" + numFollowFloorBlocks + " termOrdOrig=" + termOrdOrig);
+
+    if (!isFloor || targetOrd < nextFloorTermOrd) {
+      return;
+    }
+
+    assert numFollowFloorBlocks != 0;
+    long lastFloorTermOrd = nextFloorTermOrd;
+
+    long newFP = fpOrig;
+    while (true) {
+      final long code = floorDataReader.readVLong();
+      newFP = fpOrig + (code >>> 1);
+      hasTerms = (code & 1) != 0;
+      // if (DEBUG) {
+      //    System.out.println("      label=" + ((char) nextFloorLabel) + " fp=" + newFP + " hasTerms?=" + hasTerms + " numFollowFloor=" + numFollowFloorBlocks);
+      //  }
+            
+      isLastInFloor = numFollowFloorBlocks == 1;
+      numFollowFloorBlocks--;
+
+      lastFloorTermOrd = nextFloorTermOrd;
+
+      if (isLastInFloor) {
+        nextFloorLabel = 256;
+        nextFloorTermOrd = Long.MAX_VALUE;
+        // if (DEBUG) {
+        //    System.out.println("        stop!  last block nextFloorLabel=" + ((char) nextFloorLabel));
+        //  }
+        break;
+      } else {
+        nextFloorLabel = floorDataReader.readByte() & 0xff;
+        nextFloorTermOrd += floorDataReader.readVLong();
+        if (targetOrd < nextFloorTermOrd) {
+          // if (DEBUG) {
+          //    System.out.println("        stop!  nextFloorLabel=" + ((char) nextFloorLabel));
+          //  }
+          break;
+        }
+      }
+    }
+    // System.out.println("  after: lastFloorTermOrd=" + lastFloorTermOrd + " newFP=" + newFP + " vs fp=" + fp + " lastFloorTermOrd=" + lastFloorTermOrd);
+
+    if (newFP != fp) {
+      // Force re-load of the block:
+      // if (DEBUG) {
+      //    System.out.println("      force switch to fp=" + newFP + " oldFP=" + fp);
+      //  }
+      nextEnt = -1;
+      termOrd = lastFloorTermOrd;
+      fp = newFP;
+    } else {
+      // if (DEBUG) {
+      //    System.out.println("      stay on same fp=" + newFP);
+      //  }
+    }
+  }
+    
+  public void decodeMetaData() throws IOException {
+
+    assert nextEnt >= 0;
+
+    // lazily catch up on metadata decode:
+    final int limit = getTermBlockOrd();
+    boolean absolute = metaDataUpto == 0;
+    assert limit > 0: "limit=" + limit + " isLeafBlock=" + isLeafBlock + " nextEnt=" + nextEnt;
+
+    // if (DEBUG) System.out.println("\nBTTR.decodeMetadata seg=" + ste.fr.parent.segment + " mdUpto=" + metaDataUpto + " vs termBlockOrd=" + state.termBlockOrd + " limit=" + limit);
+
+    // TODO: better API would be "jump straight to term=N"???
+    while (metaDataUpto < limit) {
+
+      // TODO: we could make "tiers" of metadata, ie,
+      // decode docFreq/totalTF but don't decode postings
+      // metadata; this way caller could get
+      // docFreq/totalTF w/o paying decode cost for
+      // postings
+
+      // TODO: if docFreq were bulk decoded we could
+      // just skipN here:
+
+      // stats
+      state.docFreq = statsReader.readVInt();
+      //if (DEBUG) System.out.println("    dF=" + state.docFreq);
+      if (ste.fr.fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
+        state.totalTermFreq = state.docFreq + statsReader.readVLong();
+        //if (DEBUG) System.out.println("    totTF=" + state.totalTermFreq);
+      }
+      //if (DEBUG) System.out.println("    longsSize=" + ste.fr.longsSize);
+
+      // metadata 
+      for (int i = 0; i < ste.fr.longsSize; i++) {
+        longs[i] = bytesReader.readVLong();
+      }
+      ste.fr.parent.postingsReader.decodeTerm(longs, bytesReader, ste.fr.fieldInfo, state, absolute);
+
+      metaDataUpto++;
+      absolute = false;
+    }
+    state.termBlockOrd = metaDataUpto;
+  }
+
+  // Used only by assert
+  private boolean prefixMatches(BytesRef target) {
+    for(int bytePos=0;bytePos<prefix;bytePos++) {
+      if (target.bytes[target.offset + bytePos] != ste.term.bytes[bytePos]) {
+        return false;
+      }
+    }
+
+    return true;
+  }
+
+  // Scans to sub-block that has this target fp; only
+  // called by next(); NOTE: does not set
+  // startBytePos/suffix as a side effect
+  public void scanToSubBlock(long subFP) {
+    assert !isLeafBlock;
+    //if (DEBUG) System.out.println("  scanToSubBlock fp=" + fp + " subFP=" + subFP + " entCount=" + entCount + " lastSubFP=" + lastSubFP);
+    //assert nextEnt == 0;
+    if (lastSubFP == subFP) {
+      //if (DEBUG) System.out.println("    already positioned");
+      return;
+    }
+    assert subFP < fp : "fp=" + fp + " subFP=" + subFP;
+    final long targetSubCode = fp - subFP;
+    //if (DEBUG) System.out.println("    targetSubCode=" + targetSubCode);
+    while(true) {
+      assert nextEnt < entCount;
+      nextEnt++;
+      final int code = suffixesReader.readVInt();
+      suffixesReader.skipBytes(isLeafBlock ? code : code >>> 1);
+      //if (DEBUG) System.out.println("    " + nextEnt + " (of " + entCount + ") ent isSubBlock=" + ((code&1)==1));
+      if ((code & 1) != 0) {
+        final long subCode = suffixesReader.readVLong();
+        termOrd += suffixesReader.readVLong();
+
+        //if (DEBUG) System.out.println("      subCode=" + subCode);
+        if (targetSubCode == subCode) {
+          //if (DEBUG) System.out.println("        match!");
+          lastSubFP = subFP;
+          return;
+        }
+      } else {
+        state.termBlockOrd++;
+        termOrd++;
+      }
+    }
+  }
+
+  // NOTE: sets startBytePos/suffix as a side effect
+  public SeekStatus scanToTerm(BytesRef target, boolean exactOnly) throws IOException {
+    return isLeafBlock ? scanToTermLeaf(target, exactOnly) : scanToTermNonLeaf(target, exactOnly);
+  }
+
+  private int startBytePos;
+  private int suffix;
+  private long subCode;
+
+  // Target's prefix matches this block's prefix; we
+  // scan the entries check if the suffix matches.
+  public SeekStatus scanToTermLeaf(BytesRef target, boolean exactOnly) throws IOException {
+
+    // if (DEBUG) System.out.println("    scanToTermLeaf: block fp=" + fp + " prefix=" + prefix + " nextEnt=" + nextEnt + " (of " + entCount + ") target=" + OrdsSegmentTermsEnum.brToString(target) + " term=" + OrdsSegmentTermsEnum.brToString(ste.term));
+
+    assert nextEnt != -1;
+
+    ste.termExists = true;
+    subCode = 0;
+
+    if (nextEnt == entCount) {
+      if (exactOnly) {
+        fillTerm();
+      }
+      return SeekStatus.END;
+    }
+
+    assert prefixMatches(target);
+
+    // Loop over each entry (term or sub-block) in this block:
+    //nextTerm: while(nextEnt < entCount) {
+    nextTerm: while (true) {
+      nextEnt++;
+      termOrd++;
+
+      suffix = suffixesReader.readVInt();
+
+      // if (DEBUG) {
+      //    BytesRef suffixBytesRef = new BytesRef();
+      //    suffixBytesRef.bytes = suffixBytes;
+      //    suffixBytesRef.offset = suffixesReader.getPosition();
+      //    suffixBytesRef.length = suffix;
+      //    System.out.println("      cycle: term " + (nextEnt-1) + " (of " + entCount + ") suffix=" + OrdsSegmentTermsEnum.brToString(suffixBytesRef));
+      // }
+
+      final int termLen = prefix + suffix;
+      startBytePos = suffixesReader.getPosition();
+      suffixesReader.skipBytes(suffix);
+
+      final int targetLimit = target.offset + (target.length < termLen ? target.length : termLen);
+      int targetPos = target.offset + prefix;
+
+      // Loop over bytes in the suffix, comparing to
+      // the target
+      int bytePos = startBytePos;
+      while(true) {
+        final int cmp;
+        final boolean stop;
+        if (targetPos < targetLimit) {
+          cmp = (suffixBytes[bytePos++]&0xFF) - (target.bytes[targetPos++]&0xFF);
+          stop = false;
+        } else {
+          assert targetPos == targetLimit;
+          cmp = termLen - target.length;
+          stop = true;
+        }
+
+        if (cmp < 0) {
+          // Current entry is still before the target;
+          // keep scanning
+
+          if (nextEnt == entCount) {
+            if (exactOnly) {
+              fillTerm();
+            }
+            // We are done scanning this block
+            break nextTerm;
+          } else {
+            continue nextTerm;
+          }
+        } else if (cmp > 0) {
+
+          // Done!  Current entry is after target --
+          // return NOT_FOUND:
+          fillTerm();
+
+          if (!exactOnly && !ste.termExists) {
+            // We are on a sub-block, and caller wants
+            // us to position to the next term after
+            // the target, so we must recurse into the
+            // sub-frame(s):
+            ste.currentFrame = ste.pushFrame(null, ste.currentFrame.lastSubFP, termLen, ste.currentFrame.termOrd);
+            ste.currentFrame.loadBlock();
+            while (ste.currentFrame.next()) {
+              ste.currentFrame = ste.pushFrame(null, ste.currentFrame.lastSubFP, ste.term.length, ste.currentFrame.termOrd);
+              ste.currentFrame.loadBlock();
+            }
+          }
+                
+          //if (DEBUG) System.out.println("        not found");
+          return SeekStatus.NOT_FOUND;
+        } else if (stop) {
+          // Exact match!
+
+          // This cannot be a sub-block because we
+          // would have followed the index to this
+          // sub-block from the start:
+
+          assert ste.termExists;
+          fillTerm();
+          //if (DEBUG) System.out.println("        found!");
+          return SeekStatus.FOUND;
+        }
+      }
+    }
+
+    // It is possible (and OK) that terms index pointed us
+    // at this block, but, we scanned the entire block and
+    // did not find the term to position to.  This happens
+    // when the target is after the last term in the block
+    // (but, before the next term in the index).  EG
+    // target could be foozzz, and terms index pointed us
+    // to the foo* block, but the last term in this block
+    // was fooz (and, eg, first term in the next block will
+    // bee fop).
+    //if (DEBUG) System.out.println("      block end");
+    if (exactOnly) {
+      fillTerm();
+    }
+
+    // TODO: not consistent that in the
+    // not-exact case we don't next() into the next
+    // frame here
+    return SeekStatus.END;
+  }
+
+  // Target's prefix matches this block's prefix; we
+  // scan the entries check if the suffix matches.
+  public SeekStatus scanToTermNonLeaf(BytesRef target, boolean exactOnly) throws IOException {
+
+    // if (DEBUG) System.out.println("    scanToTermNonLeaf: block fp=" + fp + " prefix=" + prefix + " nextEnt=" + nextEnt + " (of " + entCount + ") target=" + OrdsSegmentTermsEnum.brToString(target) + " term=" + OrdsSegmentTermsEnum.brToString(ste.term));
+
+    assert nextEnt != -1;
+
+    if (nextEnt == entCount) {
+      if (exactOnly) {
+        fillTerm();
+        ste.termExists = subCode == 0;
+      }
+      return SeekStatus.END;
+    }
+
+    assert prefixMatches(target);
+
+    // Loop over each entry (term or sub-block) in this block:
+    //nextTerm: while(nextEnt < entCount) {
+    nextTerm: while (true) {
+      nextEnt++;
+
+      final int code = suffixesReader.readVInt();
+      suffix = code >>> 1;
+      // if (DEBUG) {
+      //   BytesRef suffixBytesRef = new BytesRef();
+      //   suffixBytesRef.bytes = suffixBytes;
+      //   suffixBytesRef.offset = suffixesReader.getPosition();
+      //   suffixBytesRef.length = suffix;
+      //   System.out.println("      cycle: " + ((code&1)==1 ? "sub-block" : "term") + " " + (nextEnt-1) + " (of " + entCount + ") suffix=" + brToString(suffixBytesRef));
+      // }
+
+      ste.termExists = (code & 1) == 0;
+      final int termLen = prefix + suffix;
+      startBytePos = suffixesReader.getPosition();
+      suffixesReader.skipBytes(suffix);
+      if (ste.termExists) {
+        state.termBlockOrd++;
+        termOrd++;
+        subCode = 0;
+      } else {
+        subCode = suffixesReader.readVLong();
+        termOrd += suffixesReader.readVLong();
+        lastSubFP = fp - subCode;
+      }
+
+      final int targetLimit = target.offset + (target.length < termLen ? target.length : termLen);
+      int targetPos = target.offset + prefix;
+
+      // Loop over bytes in the suffix, comparing to
+      // the target
+      int bytePos = startBytePos;
+      while(true) {
+        final int cmp;
+        final boolean stop;
+        if (targetPos < targetLimit) {
+          cmp = (suffixBytes[bytePos++]&0xFF) - (target.bytes[targetPos++]&0xFF);
+          stop = false;
+        } else {
+          assert targetPos == targetLimit;
+          cmp = termLen - target.length;
+          stop = true;
+        }
+
+        if (cmp < 0) {
+          // Current entry is still before the target;
+          // keep scanning
+
+          if (nextEnt == entCount) {
+            if (exactOnly) {
+              fillTerm();
+              //termExists = true;
+            }
+            // We are done scanning this block
+            break nextTerm;
+          } else {
+            continue nextTerm;
+          }
+        } else if (cmp > 0) {
+
+          // Done!  Current entry is after target --
+          // return NOT_FOUND:
+          fillTerm();
+
+          if (!exactOnly && !ste.termExists) {
+            // We are on a sub-block, and caller wants
+            // us to position to the next term after
+            // the target, so we must recurse into the
+            // sub-frame(s):
+            ste.currentFrame = ste.pushFrame(null, ste.currentFrame.lastSubFP, termLen, ste.currentFrame.termOrd);
+            ste.currentFrame.loadBlock();
+            while (ste.currentFrame.next()) {
+              ste.currentFrame = ste.pushFrame(null, ste.currentFrame.lastSubFP, ste.term.length, ste.currentFrame.termOrd);
+              ste.currentFrame.loadBlock();
+            }
+          }
+                
+          //if (DEBUG) System.out.println("        not found");
+          return SeekStatus.NOT_FOUND;
+        } else if (stop) {
+          // Exact match!
+
+          // This cannot be a sub-block because we
+          // would have followed the index to this
+          // sub-block from the start:
+
+          assert ste.termExists;
+          fillTerm();
+          //if (DEBUG) System.out.println("        found!");
+          return SeekStatus.FOUND;
+        }
+      }
+    }
+
+    // It is possible (and OK) that terms index pointed us
+    // at this block, but, we scanned the entire block and
+    // did not find the term to position to.  This happens
+    // when the target is after the last term in the block
+    // (but, before the next term in the index).  EG
+    // target could be foozzz, and terms index pointed us
+    // to the foo* block, but the last term in this block
+    // was fooz (and, eg, first term in the next block will
+    // bee fop).
+    //if (DEBUG) System.out.println("      block end");
+    if (exactOnly) {
+      fillTerm();
+    }
+
+    // TODO: not consistent that in the
+    // not-exact case we don't next() into the next
+    // frame here
+    return SeekStatus.END;
+  }
+
+  private void fillTerm() {
+    final int termLength = prefix + suffix;
+    ste.term.length = prefix + suffix;
+    if (ste.term.bytes.length < termLength) {
+      ste.term.grow(termLength);
+    }
+    System.arraycopy(suffixBytes, startBytePos, ste.term.bytes, prefix, suffix);
+  }
+}

Added: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/package.html?rev=1612080&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/package.html (added)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/package.html Sun Jul 20 12:08:32 2014
@@ -0,0 +1,27 @@
+<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<html>
+<head>
+   <meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
+</head>
+<body>
+Same postings format as Lucene41, except the terms dictionary also
+supports ords, i.e. returning which ord the enum is seeked to, and
+seeking by ord.
+</body>
+</html>

Modified: lucene/dev/trunk/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat?rev=1612080&r1=1612079&r2=1612080&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat (original)
+++ lucene/dev/trunk/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat Sun Jul 20 12:08:32 2014
@@ -13,12 +13,13 @@
 #  See the License for the specific language governing permissions and
 #  limitations under the License.
 
-org.apache.lucene.codecs.pulsing.Pulsing41PostingsFormat
-org.apache.lucene.codecs.simpletext.SimpleTextPostingsFormat
-org.apache.lucene.codecs.memory.MemoryPostingsFormat
+org.apache.lucene.codecs.blocktreeords.Ords41PostingsFormat
 org.apache.lucene.codecs.bloom.BloomFilteringPostingsFormat
 org.apache.lucene.codecs.memory.DirectPostingsFormat
-org.apache.lucene.codecs.memory.FSTPulsing41PostingsFormat
+org.apache.lucene.codecs.memory.FSTOrdPostingsFormat
 org.apache.lucene.codecs.memory.FSTOrdPulsing41PostingsFormat
 org.apache.lucene.codecs.memory.FSTPostingsFormat
-org.apache.lucene.codecs.memory.FSTOrdPostingsFormat
+org.apache.lucene.codecs.memory.FSTPulsing41PostingsFormat
+org.apache.lucene.codecs.memory.MemoryPostingsFormat
+org.apache.lucene.codecs.pulsing.Pulsing41PostingsFormat
+org.apache.lucene.codecs.simpletext.SimpleTextPostingsFormat

Added: lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/blocktreeords/TestOrdsBlockTree.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/blocktreeords/TestOrdsBlockTree.java?rev=1612080&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/blocktreeords/TestOrdsBlockTree.java (added)
+++ lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/blocktreeords/TestOrdsBlockTree.java Sun Jul 20 12:08:32 2014
@@ -0,0 +1,360 @@
+package org.apache.lucene.codecs.blocktreeords;
+
+/*
+ * 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.Collections;
+import java.util.List;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.index.BasePostingsFormatTestCase;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.MultiFields;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.TestUtil;
+
+public class TestOrdsBlockTree extends BasePostingsFormatTestCase {
+  private final Codec codec = TestUtil.alwaysPostingsFormat(new Ords41PostingsFormat());
+
+  @Override
+  protected Codec getCodec() {
+    return codec;
+  }
+
+  public void testBasic() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+    Document doc = new Document();
+    doc.add(newTextField("field", "a b c", Field.Store.NO));
+    w.addDocument(doc);
+    IndexReader r = w.getReader();
+    TermsEnum te = MultiFields.getTerms(r, "field").iterator(null);
+
+    // Test next()
+    assertEquals(new BytesRef("a"), te.next());
+    assertEquals(0L, te.ord());
+    assertEquals(new BytesRef("b"), te.next());
+    assertEquals(1L, te.ord());
+    assertEquals(new BytesRef("c"), te.next());
+    assertEquals(2L, te.ord());
+    assertNull(te.next());
+
+    // Test seekExact by term
+    assertTrue(te.seekExact(new BytesRef("b")));
+    assertEquals(1, te.ord());
+    assertTrue(te.seekExact(new BytesRef("a")));
+    assertEquals(0, te.ord());
+    assertTrue(te.seekExact(new BytesRef("c")));
+    assertEquals(2, te.ord());
+
+    // Test seekExact by ord
+    te.seekExact(1);
+    assertEquals(new BytesRef("b"), te.term());
+    te.seekExact(0);
+    assertEquals(new BytesRef("a"), te.term());
+    te.seekExact(2);
+    assertEquals(new BytesRef("c"), te.term());
+
+    r.close();
+    w.close();
+    dir.close();
+  }
+
+  public void testTwoBlocks() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+    List<String> terms = new ArrayList<>();
+    for(int i=0;i<36;i++) {
+      Document doc = new Document();
+      String term = "" + (char) (97+i);
+      terms.add(term);
+      if (VERBOSE) {
+        System.out.println("i=" + i + " term=" + term);
+      }
+      doc.add(newTextField("field", term, Field.Store.NO));
+      w.addDocument(doc);
+    }
+    for(int i=0;i<36;i++) {
+      Document doc = new Document();
+      String term = "m" + (char) (97+i);
+      terms.add(term);
+      if (VERBOSE) {
+        System.out.println("i=" + i + " term=" + term);
+      }
+      doc.add(newTextField("field", term, Field.Store.NO));
+      w.addDocument(doc);
+    }
+    w.forceMerge(1);
+    IndexReader r = w.getReader();
+    TermsEnum te = MultiFields.getTerms(r, "field").iterator(null);
+
+    assertTrue(te.seekExact(new BytesRef("mo")));
+    assertEquals(27, te.ord());
+
+    te.seekExact(54);
+    assertEquals(new BytesRef("s"), te.term());
+
+    Collections.sort(terms);
+
+    for(int i=terms.size()-1;i>=0;i--) {
+      te.seekExact(i);
+      assertEquals(i, te.ord());
+      assertEquals(terms.get(i), te.term().utf8ToString());
+    }
+
+    int iters = atLeast(1000);
+    for(int iter=0;iter<iters;iter++) {
+      int ord = random().nextInt(terms.size());
+      BytesRef term = new BytesRef(terms.get(ord));
+      if (random().nextBoolean()) {
+        if (VERBOSE) {
+          System.out.println("TEST: iter=" + iter + " seek to ord=" + ord + " of " + terms.size());
+        }
+        te.seekExact(ord);
+      } else {
+        if (VERBOSE) {
+          System.out.println("TEST: iter=" + iter + " seek to term=" + terms.get(ord) + " ord=" + ord + " of " + terms.size());
+        }
+        te.seekExact(term);
+      }
+      assertEquals(ord, te.ord());
+      assertEquals(term, te.term());
+    }
+
+    r.close();
+    w.close();
+    dir.close();
+  }
+
+  public void testThreeBlocks() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+    List<String> terms = new ArrayList<>();
+    for(int i=0;i<36;i++) {
+      Document doc = new Document();
+      String term = "" + (char) (97+i);
+      terms.add(term);
+      if (VERBOSE) {
+        System.out.println("i=" + i + " term=" + term);
+      }
+      doc.add(newTextField("field", term, Field.Store.NO));
+      w.addDocument(doc);
+    }
+    for(int i=0;i<36;i++) {
+      Document doc = new Document();
+      String term = "m" + (char) (97+i);
+      terms.add(term);
+      if (VERBOSE) {
+        System.out.println("i=" + i + " term=" + term);
+      }
+      doc.add(newTextField("field", term, Field.Store.NO));
+      w.addDocument(doc);
+    }
+    for(int i=0;i<36;i++) {
+      Document doc = new Document();
+      String term = "mo" + (char) (97+i);
+      terms.add(term);
+      if (VERBOSE) {
+        System.out.println("i=" + i + " term=" + term);
+      }
+      doc.add(newTextField("field", term, Field.Store.NO));
+      w.addDocument(doc);
+    }
+    w.forceMerge(1);
+    IndexReader r = w.getReader();
+    TermsEnum te = MultiFields.getTerms(r, "field").iterator(null);
+
+    if (VERBOSE) {
+      while (te.next() != null) {
+        System.out.println("TERM: " + te.ord() + " " + te.term().utf8ToString());
+      }
+    }
+
+    assertTrue(te.seekExact(new BytesRef("mo")));
+    assertEquals(27, te.ord());
+
+    te.seekExact(90);
+    assertEquals(new BytesRef("s"), te.term());
+
+    testEnum(te, terms);
+
+    r.close();
+    w.close();
+    dir.close();
+  }
+
+  private void testEnum(TermsEnum te, List<String> terms) throws IOException {
+    Collections.sort(terms);
+    for(int i=terms.size()-1;i>=0;i--) {
+      if (VERBOSE) {
+        System.out.println("TEST: seek to ord=" + i);
+      }
+      te.seekExact(i);
+      assertEquals(i, te.ord());
+      assertEquals(terms.get(i), te.term().utf8ToString());
+    }
+
+    int iters = atLeast(1000);
+    for(int iter=0;iter<iters;iter++) {
+      int ord = random().nextInt(terms.size());
+      if (random().nextBoolean()) {
+        te.seekExact(ord);
+        assertEquals(terms.get(ord), te.term().utf8ToString());
+      } else {
+        te.seekExact(new BytesRef(terms.get(ord)));
+        assertEquals(ord, te.ord());
+      }
+    }
+  }
+
+  public void testFloorBlocks() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+    IndexWriter w = new IndexWriter(dir, iwc);
+    for(int i=0;i<128;i++) {
+      Document doc = new Document();
+      String term = "" + (char) i;
+      if (VERBOSE) {
+        System.out.println("i=" + i + " term=" + term + " bytes=" + new BytesRef(term));
+      }
+      doc.add(newStringField("field", term, Field.Store.NO));
+      w.addDocument(doc);
+    }
+    w.forceMerge(1);
+    IndexReader r = DirectoryReader.open(w, true);
+    TermsEnum te = MultiFields.getTerms(r, "field").iterator(null);
+
+    if (VERBOSE) {
+      BytesRef term;
+      while ((term = te.next()) != null) {
+        System.out.println("  " + te.ord() + ": " + term.utf8ToString());
+      }
+    }
+
+    assertTrue(te.seekExact(new BytesRef("a")));
+    assertEquals(97, te.ord());
+
+    te.seekExact(98);
+    assertEquals(new BytesRef("b"), te.term());
+
+    assertTrue(te.seekExact(new BytesRef("z")));
+    assertEquals(122, te.ord());
+
+    r.close();
+    w.close();
+    dir.close();
+  }
+
+  public void testNonRootFloorBlocks() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+    IndexWriter w = new IndexWriter(dir, iwc);
+    List<String> terms = new ArrayList<>();
+    for(int i=0;i<36;i++) {
+      Document doc = new Document();
+      String term = "" + (char) (97+i);
+      terms.add(term);
+      if (VERBOSE) {
+        System.out.println("i=" + i + " term=" + term);
+      }
+      doc.add(newTextField("field", term, Field.Store.NO));
+      w.addDocument(doc);
+    }
+    for(int i=0;i<128;i++) {
+      Document doc = new Document();
+      String term = "m" + (char) i;
+      terms.add(term);
+      if (VERBOSE) {
+        System.out.println("i=" + i + " term=" + term + " bytes=" + new BytesRef(term));
+      }
+      doc.add(newStringField("field", term, Field.Store.NO));
+      w.addDocument(doc);
+    }
+    w.forceMerge(1);
+    IndexReader r = DirectoryReader.open(w, true);
+    TermsEnum te = MultiFields.getTerms(r, "field").iterator(null);
+
+    BytesRef term;
+    int ord = 0;
+    while ((term = te.next()) != null) {
+      if (VERBOSE) {
+        System.out.println("TEST: " + te.ord() + ": " + term.utf8ToString());
+      }
+      assertEquals(ord, te.ord());
+      ord++;
+    }
+
+    testEnum(te, terms);
+
+    r.close();
+    w.close();
+    dir.close();
+  }
+
+  public void testSeveralNonRootBlocks() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+    IndexWriter w = new IndexWriter(dir, iwc);
+    List<String> terms = new ArrayList<>();
+    for(int i=0;i<30;i++) {
+      for(int j=0;j<30;j++) {
+        Document doc = new Document();
+        String term = "" + (char) (97+i) + (char) (97+j);
+        terms.add(term);
+        if (VERBOSE) {
+          System.out.println("term=" + term);
+        }
+        doc.add(newTextField("body", term, Field.Store.NO));
+        w.addDocument(doc);
+      }
+    }
+    w.forceMerge(1);
+    IndexReader r = DirectoryReader.open(w, true);
+    TermsEnum te = MultiFields.getTerms(r, "body").iterator(null);
+
+    for(int i=0;i<30;i++) {
+      for(int j=0;j<30;j++) {
+        String term = "" + (char) (97+i) + (char) (97+j);
+        if (VERBOSE) {
+          System.out.println("TEST: check term=" + term);
+        }
+        assertEquals(term, te.next().utf8ToString());
+        assertEquals(30*i+j, te.ord());
+      }
+    }
+
+    testEnum(te, terms);
+
+    te.seekExact(0);
+    assertEquals("aa", te.term().utf8ToString());
+
+    r.close();
+    w.close();
+    dir.close();
+  }
+}

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java?rev=1612080&r1=1612079&r2=1612080&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnumFrame.java Sun Jul 20 12:08:32 2014
@@ -222,6 +222,7 @@ final class SegmentTermsEnumFrame {
     if (isFloor) {
       floorDataReader.rewind();
       numFollowFloorBlocks = floorDataReader.readVInt();
+      assert numFollowFloorBlocks > 0;
       nextFloorLabel = floorDataReader.readByte() & 0xff;
     }
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java?rev=1612080&r1=1612079&r2=1612080&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java Sun Jul 20 12:08:32 2014
@@ -43,6 +43,7 @@ import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CommandLineUtil;
 import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LongBitSet;
 import org.apache.lucene.util.StringHelper;
 
@@ -329,6 +330,20 @@ public class CheckIndex {
     return crossCheckTermVectors;
   }
 
+  private boolean failFast;
+
+  /** If true, just throw the original exception immediately when
+   *  corruption is detected, rather than continuing to iterate to other
+   *  segments looking for more corruption.  */
+  public void setFailFast(boolean v) {
+    failFast = v;
+  }
+
+  /** See {@link #setFailFast}. */
+  public boolean getFailFast() {
+    return failFast;
+  }
+
   private boolean verbose;
 
   /** Set infoStream where messages should go.  If null, no
@@ -382,6 +397,9 @@ public class CheckIndex {
     try {
       sis.read(dir);
     } catch (Throwable t) {
+      if (failFast) {
+        IOUtils.reThrow(t);
+      }
       msg(infoStream, "ERROR: could not read any segments file in directory");
       result.missingSegments = true;
       if (infoStream != null)
@@ -417,6 +435,9 @@ public class CheckIndex {
     try {
       input = dir.openInput(segmentsFileName, IOContext.READONCE);
     } catch (Throwable t) {
+      if (failFast) {
+        IOUtils.reThrow(t);
+      }
       msg(infoStream, "ERROR: could not open segments file in directory");
       if (infoStream != null)
         t.printStackTrace(infoStream);
@@ -427,6 +448,9 @@ public class CheckIndex {
     try {
       format = input.readInt();
     } catch (Throwable t) {
+      if (failFast) {
+        IOUtils.reThrow(t);
+      }
       msg(infoStream, "ERROR: could not read segment file version in directory");
       if (infoStream != null)
         t.printStackTrace(infoStream);
@@ -607,18 +631,18 @@ public class CheckIndex {
         segInfoStat.numFields = fieldInfos.size();
         
         // Test Field Norms
-        segInfoStat.fieldNormStatus = testFieldNorms(reader, infoStream);
+        segInfoStat.fieldNormStatus = testFieldNorms(reader, infoStream, failFast);
 
         // Test the Term Index
-        segInfoStat.termIndexStatus = testPostings(reader, infoStream, verbose);
+        segInfoStat.termIndexStatus = testPostings(reader, infoStream, verbose, failFast);
 
         // Test Stored Fields
-        segInfoStat.storedFieldStatus = testStoredFields(reader, infoStream);
+        segInfoStat.storedFieldStatus = testStoredFields(reader, infoStream, failFast);
 
         // Test Term Vectors
-        segInfoStat.termVectorStatus = testTermVectors(reader, infoStream, verbose, crossCheckTermVectors);
+        segInfoStat.termVectorStatus = testTermVectors(reader, infoStream, verbose, crossCheckTermVectors, failFast);
 
-        segInfoStat.docValuesStatus = testDocValues(reader, infoStream);
+        segInfoStat.docValuesStatus = testDocValues(reader, infoStream, failFast);
 
         // Rethrow the first exception we encountered
         //  This will cause stats for failed segments to be incremented properly
@@ -637,6 +661,9 @@ public class CheckIndex {
         msg(infoStream, "");
 
       } catch (Throwable t) {
+        if (failFast) {
+          IOUtils.reThrow(t);
+        }
         msg(infoStream, "FAILED");
         String comment;
         comment = "fixIndex() would remove reference to this segment";
@@ -678,7 +705,7 @@ public class CheckIndex {
    * Test field norms.
    * @lucene.experimental
    */
-  public static Status.FieldNormStatus testFieldNorms(AtomicReader reader, PrintStream infoStream) {
+  public static Status.FieldNormStatus testFieldNorms(AtomicReader reader, PrintStream infoStream, boolean failFast) throws IOException {
     final Status.FieldNormStatus status = new Status.FieldNormStatus();
 
     try {
@@ -699,6 +726,9 @@ public class CheckIndex {
 
       msg(infoStream, "OK [" + status.totFields + " fields]");
     } catch (Throwable e) {
+      if (failFast) {
+        IOUtils.reThrow(e);
+      }
       msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
       status.error = e;
       if (infoStream != null) {
@@ -825,6 +855,7 @@ public class CheckIndex {
       
       long sumTotalTermFreq = 0;
       long sumDocFreq = 0;
+      long upto = 0;
       FixedBitSet visitedDocs = new FixedBitSet(maxDoc);
       while(true) {
         
@@ -832,7 +863,7 @@ public class CheckIndex {
         if (term == null) {
           break;
         }
-        
+
         assert term.isValid();
         
         // make sure terms arrive in order according to
@@ -1267,15 +1298,15 @@ public class CheckIndex {
    * Test the term index.
    * @lucene.experimental
    */
-  public static Status.TermIndexStatus testPostings(AtomicReader reader, PrintStream infoStream) {
-    return testPostings(reader, infoStream, false);
+  public static Status.TermIndexStatus testPostings(AtomicReader reader, PrintStream infoStream) throws IOException {
+    return testPostings(reader, infoStream, false, false);
   }
   
   /**
    * Test the term index.
    * @lucene.experimental
    */
-  public static Status.TermIndexStatus testPostings(AtomicReader reader, PrintStream infoStream, boolean verbose) {
+  public static Status.TermIndexStatus testPostings(AtomicReader reader, PrintStream infoStream, boolean verbose, boolean failFast) throws IOException {
 
     // TODO: we should go and verify term vectors match, if
     // crossCheckTermVectors is on...
@@ -1299,6 +1330,9 @@ public class CheckIndex {
         checkFields(fields, null, maxDoc, fieldInfos, true, false, infoStream, verbose);
       }
     } catch (Throwable e) {
+      if (failFast) {
+        IOUtils.reThrow(e);
+      }
       msg(infoStream, "ERROR: " + e);
       status = new Status.TermIndexStatus();
       status.error = e;
@@ -1314,7 +1348,7 @@ public class CheckIndex {
    * Test stored fields.
    * @lucene.experimental
    */
-  public static Status.StoredFieldStatus testStoredFields(AtomicReader reader, PrintStream infoStream) {
+  public static Status.StoredFieldStatus testStoredFields(AtomicReader reader, PrintStream infoStream, boolean failFast) throws IOException {
     final Status.StoredFieldStatus status = new Status.StoredFieldStatus();
 
     try {
@@ -1342,6 +1376,9 @@ public class CheckIndex {
       msg(infoStream, "OK [" + status.totFields + " total field count; avg " + 
           NumberFormat.getInstance(Locale.ROOT).format((((float) status.totFields)/status.docCount)) + " fields per doc]");      
     } catch (Throwable e) {
+      if (failFast) {
+        IOUtils.reThrow(e);
+      }
       msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
       status.error = e;
       if (infoStream != null) {
@@ -1357,7 +1394,8 @@ public class CheckIndex {
    * @lucene.experimental
    */
   public static Status.DocValuesStatus testDocValues(AtomicReader reader,
-                                                     PrintStream infoStream) {
+                                                     PrintStream infoStream,
+                                                     boolean failFast) throws IOException {
     final Status.DocValuesStatus status = new Status.DocValuesStatus();
     try {
       if (infoStream != null) {
@@ -1385,6 +1423,9 @@ public class CheckIndex {
                              + status.totalSortedNumericFields + " SORTED_NUMERIC; "
                              + status.totalSortedSetFields + " SORTED_SET]");
     } catch (Throwable e) {
+      if (failFast) {
+        IOUtils.reThrow(e);
+      }
       msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
       status.error = e;
       if (infoStream != null) {
@@ -1624,15 +1665,15 @@ public class CheckIndex {
    * Test term vectors.
    * @lucene.experimental
    */
-  public static Status.TermVectorStatus testTermVectors(AtomicReader reader, PrintStream infoStream) {
-    return testTermVectors(reader, infoStream, false, false);
+  public static Status.TermVectorStatus testTermVectors(AtomicReader reader, PrintStream infoStream) throws IOException {
+    return testTermVectors(reader, infoStream, false, false, false);
   }
 
   /**
    * Test term vectors.
    * @lucene.experimental
    */
-  public static Status.TermVectorStatus testTermVectors(AtomicReader reader, PrintStream infoStream, boolean verbose, boolean crossCheckTermVectors) {
+  public static Status.TermVectorStatus testTermVectors(AtomicReader reader, PrintStream infoStream, boolean verbose, boolean crossCheckTermVectors, boolean failFast) throws IOException {
     final Status.TermVectorStatus status = new Status.TermVectorStatus();
     final FieldInfos fieldInfos = reader.getFieldInfos();
     final Bits onlyDocIsDeleted = new FixedBitSet(1);
@@ -1844,6 +1885,9 @@ public class CheckIndex {
       msg(infoStream, "OK [" + status.totVectors + " total vector count; avg " + 
           NumberFormat.getInstance(Locale.ROOT).format(vectorAvg) + " term/freq vector fields per doc]");
     } catch (Throwable e) {
+      if (failFast) {
+        IOUtils.reThrow(e);
+      }
       msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
       status.error = e;
       if (infoStream != null) {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/ByteSequenceOutputs.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/ByteSequenceOutputs.java?rev=1612080&r1=1612079&r2=1612080&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/ByteSequenceOutputs.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/ByteSequenceOutputs.java Sun Jul 20 12:08:32 2014
@@ -23,6 +23,7 @@ import org.apache.lucene.store.DataInput
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.StringHelper;
 
 /**
  * An FST {@link Outputs} implementation where each output
@@ -80,13 +81,16 @@ public final class ByteSequenceOutputs e
     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);
+      assert StringHelper.startsWith(output, inc);
+      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);
+      }
     }
   }
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/FST.java?rev=1612080&r1=1612079&r2=1612080&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/FST.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/FST.java Sun Jul 20 12:08:32 2014
@@ -85,7 +85,10 @@ public final class FST<T> implements Acc
 
   // TODO: we can free up a bit if we can nuke this:
   final static int BIT_STOP_NODE = 1 << 3;
-  final static int BIT_ARC_HAS_OUTPUT = 1 << 4;
+
+  /** This flag is set if the arc has an output. */
+  public final static int BIT_ARC_HAS_OUTPUT = 1 << 4;
+
   final static int BIT_ARC_HAS_FINAL_OUTPUT = 1 << 5;
 
   // Arcs are stored as fixed-size (per entry) array, so
@@ -196,11 +199,22 @@ public final class FST<T> implements Acc
     // address (into the byte[]), or ord/address if label == END_LABEL
     long nextArc;
 
-    // This is non-zero if current arcs are fixed array:
-    long posArcsStart;
-    int bytesPerArc;
-    int arcIdx;
-    int numArcs;
+    /** Where the first arc in the array starts; only valid if
+     *  bytesPerArc != 0 */
+    public long posArcsStart;
+    
+    /** Non-zero if this arc is part of an array, which means all
+     *  arcs for the node are encoded with a fixed number of bytes so
+     *  that we can random access by index.  We do when there are enough
+     *  arcs leaving one node.  It wastes some bytes but gives faster
+     *  lookups. */
+    public int bytesPerArc;
+
+    /** Where we are in the array; only valid if bytesPerArc != 0. */
+    public int arcIdx;
+
+    /** How many arcs in the array; only valid if bytesPerArc != 0. */
+    public int numArcs;
 
     /** Returns this */
     public Arc<T> copyFrom(Arc<T> other) {
@@ -644,7 +658,8 @@ public final class FST<T> implements Acc
     }
   }
 
-  int readLabel(DataInput in) throws IOException {
+  /** Reads one BYTE1/2/4 label from the provided {@link DataInput}. */
+  public int readLabel(DataInput in) throws IOException {
     final int v;
     if (inputType == INPUT_TYPE.BYTE1) {
       // Unsigned byte:

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java?rev=1612080&r1=1612079&r2=1612080&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java Sun Jul 20 12:08:32 2014
@@ -36,6 +36,8 @@ import org.apache.lucene.codecs.blockter
 import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexWriter;
 import org.apache.lucene.codecs.blocktree.BlockTreeTermsReader;
 import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter;
+import org.apache.lucene.codecs.blocktreeords.OrdsBlockTreeTermsReader;
+import org.apache.lucene.codecs.blocktreeords.OrdsBlockTreeTermsWriter;
 import org.apache.lucene.codecs.lucene41.Lucene41PostingsReader;
 import org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter;
 import org.apache.lucene.codecs.memory.FSTOrdTermsReader;
@@ -128,7 +130,7 @@ public final class MockRandomPostingsFor
     }
 
     final FieldsConsumer fields;
-    final int t1 = random.nextInt(4);
+    final int t1 = random.nextInt(5);
 
     if (t1 == 0) {
       boolean success = false;
@@ -171,7 +173,7 @@ public final class MockRandomPostingsFor
           postingsWriter.close();
         }
       }
-    } else {
+    } else if (t1 == 3) {
 
       if (LuceneTestCase.VERBOSE) {
         System.out.println("MockRandomCodec: writing Block terms dict");
@@ -241,6 +243,30 @@ public final class MockRandomPostingsFor
           }
         }
       }
+    } else if (t1 == 4) {
+      // Use OrdsBlockTree terms dict
+      if (LuceneTestCase.VERBOSE) {
+        System.out.println("MockRandomCodec: writing OrdsBlockTree");
+      }
+
+      // TODO: would be nice to allow 1 but this is very
+      // slow to write
+      final int minTermsInBlock = TestUtil.nextInt(random, 2, 100);
+      final int maxTermsInBlock = Math.max(2, (minTermsInBlock-1)*2 + random.nextInt(100));
+
+      boolean success = false;
+      try {
+        fields = new OrdsBlockTreeTermsWriter(state, postingsWriter, minTermsInBlock, maxTermsInBlock);
+        success = true;
+      } finally {
+        if (!success) {
+          postingsWriter.close();
+        }
+      }
+      
+    } else {
+      // BUG!
+      throw new AssertionError();
     }
 
     return fields;
@@ -275,7 +301,7 @@ public final class MockRandomPostingsFor
     }
 
     final FieldsProducer fields;
-    final int t1 = random.nextInt(4);
+    final int t1 = random.nextInt(5);
     if (t1 == 0) {
       boolean success = false;
       try {
@@ -316,7 +342,7 @@ public final class MockRandomPostingsFor
           postingsReader.close();
         }
       }
-    } else {
+    } else if (t1 == 3) {
 
       if (LuceneTestCase.VERBOSE) {
         System.out.println("MockRandomCodec: reading Block terms dict");
@@ -380,6 +406,29 @@ public final class MockRandomPostingsFor
           }
         }
       }
+    } else if (t1 == 4) {
+      // Use OrdsBlockTree terms dict
+      if (LuceneTestCase.VERBOSE) {
+        System.out.println("MockRandomCodec: reading OrdsBlockTree terms dict");
+      }
+
+      boolean success = false;
+      try {
+        fields = new OrdsBlockTreeTermsReader(state.directory,
+                                              state.fieldInfos,
+                                              state.segmentInfo,
+                                              postingsReader,
+                                              state.context,
+                                              state.segmentSuffix);
+        success = true;
+      } finally {
+        if (!success) {
+          postingsReader.close();
+        }
+      }
+    } else {
+      // BUG!
+      throw new AssertionError();
     }
 
     return fields;

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java?rev=1612080&r1=1612079&r2=1612080&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java Sun Jul 20 12:08:32 2014
@@ -59,6 +59,11 @@ import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LineFileDocs;
 import org.apache.lucene.util.RamUsageTester;
 import org.apache.lucene.util.TestUtil;
+import org.apache.lucene.util.UnicodeUtil;
+import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.AutomatonTestUtil.RandomAcceptedStrings;
+import org.apache.lucene.util.automaton.AutomatonTestUtil;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 
@@ -293,17 +298,28 @@ public abstract class BasePostingsFormat
   }
   
   private static class FieldAndTerm {
-    String field;
-    BytesRef term;
+    final String field;
+    final BytesRef term;
+    final long ord;
 
-    public FieldAndTerm(String field, BytesRef term) {
+    public FieldAndTerm(String field, BytesRef term, long ord) {
       this.field = field;
       this.term = BytesRef.deepCopyOf(term);
+      this.ord = ord;
+    }
+  }
+
+  private static class SeedAndOrd {
+    final long seed;
+    long ord;
+
+    public SeedAndOrd(long seed) {
+      this.seed = seed;
     }
   }
 
   // Holds all postings:
-  private static Map<String,SortedMap<BytesRef,Long>> fields;
+  private static Map<String,SortedMap<BytesRef,SeedAndOrd>> fields;
 
   private static FieldInfos fieldInfos;
 
@@ -359,7 +375,7 @@ public abstract class BasePostingsFormat
                                                 null, DocValuesType.NUMERIC, -1, null);
       fieldUpto++;
 
-      SortedMap<BytesRef,Long> postings = new TreeMap<>();
+      SortedMap<BytesRef,SeedAndOrd> postings = new TreeMap<>();
       fields.put(field, postings);
       Set<String> seenTerms = new HashSet<>();
 
@@ -370,7 +386,9 @@ public abstract class BasePostingsFormat
         numTerms = TestUtil.nextInt(random(), 2, 20);
       }
 
-      for(int termUpto=0;termUpto<numTerms;termUpto++) {
+      while (postings.size() < numTerms) {
+        int termUpto = postings.size();
+        // Cannot contain surrogates else default Java string sort order (by UTF16 code unit) is different from Lucene:
         String term = TestUtil.randomSimpleString(random());
         if (seenTerms.contains(term)) {
           continue;
@@ -392,7 +410,7 @@ public abstract class BasePostingsFormat
         }
 
         long termSeed = random().nextLong();
-        postings.put(new BytesRef(term), termSeed);
+        postings.put(new BytesRef(term), new SeedAndOrd(termSeed));
 
         // NOTE: sort of silly: we enum all the docs just to
         // get the maxDoc
@@ -404,6 +422,12 @@ public abstract class BasePostingsFormat
         }
         maxDoc = Math.max(lastDoc, maxDoc);
       }
+
+      // assign ords
+      long ord = 0;
+      for(SeedAndOrd ent : postings.values()) {
+        ent.ord = ord++;
+      }
     }
 
     fieldInfos = new FieldInfos(fieldInfoArray);
@@ -420,10 +444,11 @@ public abstract class BasePostingsFormat
     }
 
     allTerms = new ArrayList<>();
-    for(Map.Entry<String,SortedMap<BytesRef,Long>> fieldEnt : fields.entrySet()) {
+    for(Map.Entry<String,SortedMap<BytesRef,SeedAndOrd>> fieldEnt : fields.entrySet()) {
       String field = fieldEnt.getKey();
-      for(Map.Entry<BytesRef,Long> termEnt : fieldEnt.getValue().entrySet()) {
-        allTerms.add(new FieldAndTerm(field, termEnt.getKey()));
+      long ord = 0;
+      for(Map.Entry<BytesRef,SeedAndOrd> termEnt : fieldEnt.getValue().entrySet()) {
+        allTerms.add(new FieldAndTerm(field, termEnt.getKey(), ord++));
       }
     }
 
@@ -441,12 +466,12 @@ public abstract class BasePostingsFormat
   }
 
   private static class SeedFields extends Fields {
-    final Map<String,SortedMap<BytesRef,Long>> fields;
+    final Map<String,SortedMap<BytesRef,SeedAndOrd>> fields;
     final FieldInfos fieldInfos;
     final IndexOptions maxAllowed;
     final boolean allowPayloads;
 
-    public SeedFields(Map<String,SortedMap<BytesRef,Long>> fields, FieldInfos fieldInfos, IndexOptions maxAllowed, boolean allowPayloads) {
+    public SeedFields(Map<String,SortedMap<BytesRef,SeedAndOrd>> fields, FieldInfos fieldInfos, IndexOptions maxAllowed, boolean allowPayloads) {
       this.fields = fields;
       this.fieldInfos = fieldInfos;
       this.maxAllowed = maxAllowed;
@@ -460,7 +485,7 @@ public abstract class BasePostingsFormat
 
     @Override
     public Terms terms(String field) {
-      SortedMap<BytesRef,Long> terms = fields.get(field);
+      SortedMap<BytesRef,SeedAndOrd> terms = fields.get(field);
       if (terms == null) {
         return null;
       } else {
@@ -475,12 +500,12 @@ public abstract class BasePostingsFormat
   }
 
   private static class SeedTerms extends Terms {
-    final SortedMap<BytesRef,Long> terms;
+    final SortedMap<BytesRef,SeedAndOrd> terms;
     final FieldInfo fieldInfo;
     final IndexOptions maxAllowed;
     final boolean allowPayloads;
 
-    public SeedTerms(SortedMap<BytesRef,Long> terms, FieldInfo fieldInfo, IndexOptions maxAllowed, boolean allowPayloads) {
+    public SeedTerms(SortedMap<BytesRef,SeedAndOrd> terms, FieldInfo fieldInfo, IndexOptions maxAllowed, boolean allowPayloads) {
       this.terms = terms;
       this.fieldInfo = fieldInfo;
       this.maxAllowed = maxAllowed;
@@ -545,15 +570,15 @@ public abstract class BasePostingsFormat
   }
 
   private static class SeedTermsEnum extends TermsEnum {
-    final SortedMap<BytesRef,Long> terms;
+    final SortedMap<BytesRef,SeedAndOrd> terms;
     final IndexOptions maxAllowed;
     final boolean allowPayloads;
 
-    private Iterator<Map.Entry<BytesRef,Long>> iterator;
+    private Iterator<Map.Entry<BytesRef,SeedAndOrd>> iterator;
 
-    private Map.Entry<BytesRef,Long> current;
+    private Map.Entry<BytesRef,SeedAndOrd> current;
 
-    public SeedTermsEnum(SortedMap<BytesRef,Long> terms, IndexOptions maxAllowed, boolean allowPayloads) {
+    public SeedTermsEnum(SortedMap<BytesRef,SeedAndOrd> terms, IndexOptions maxAllowed, boolean allowPayloads) {
       this.terms = terms;
       this.maxAllowed = maxAllowed;
       this.allowPayloads = allowPayloads;
@@ -565,7 +590,7 @@ public abstract class BasePostingsFormat
 
     @Override
     public SeekStatus seekCeil(BytesRef text) {
-      SortedMap<BytesRef,Long> tailMap = terms.tailMap(text);
+      SortedMap<BytesRef,SeedAndOrd> tailMap = terms.tailMap(text);
       if (tailMap.isEmpty()) {
         return SeekStatus.END;
       } else {
@@ -600,7 +625,7 @@ public abstract class BasePostingsFormat
 
     @Override
     public long ord() {
-      throw new UnsupportedOperationException();
+      return current.getValue().ord;
     }
 
     @Override
@@ -621,7 +646,7 @@ public abstract class BasePostingsFormat
       if ((flags & DocsEnum.FLAG_FREQS) != 0 && maxAllowed.compareTo(IndexOptions.DOCS_AND_FREQS) < 0) {
         return null;
       }
-      return getSeedPostings(current.getKey().utf8ToString(), current.getValue(), false, maxAllowed, allowPayloads);
+      return getSeedPostings(current.getKey().utf8ToString(), current.getValue().seed, false, maxAllowed, allowPayloads);
     }
 
     @Override
@@ -638,7 +663,7 @@ public abstract class BasePostingsFormat
       if ((flags & DocsAndPositionsEnum.FLAG_PAYLOADS) != 0 && allowPayloads == false) {
         return null;
       }
-      return getSeedPostings(current.getKey().utf8ToString(), current.getValue(), false, maxAllowed, allowPayloads);
+      return getSeedPostings(current.getKey().utf8ToString(), current.getValue().seed, false, maxAllowed, allowPayloads);
     }
   }
 
@@ -766,7 +791,7 @@ public abstract class BasePostingsFormat
 
     // NOTE: can be empty list if we are using liveDocs:
     SeedPostings expected = getSeedPostings(term.utf8ToString(), 
-                                            fields.get(field).get(term),
+                                            fields.get(field).get(term).seed,
                                             useLiveDocs,
                                             maxIndexOptions,
                                             true);
@@ -1104,12 +1129,16 @@ public abstract class BasePostingsFormat
     // Test random terms/fields:
     List<TermState> termStates = new ArrayList<>();
     List<FieldAndTerm> termStateTerms = new ArrayList<>();
+
+    boolean supportsOrds = true;
     
     Collections.shuffle(allTerms, random());
     int upto = 0;
     while (upto < allTerms.size()) {
 
       boolean useTermState = termStates.size() != 0 && random().nextInt(5) == 1;
+      boolean useTermOrd = supportsOrds && useTermState == false && random().nextInt(5) == 1;
+
       FieldAndTerm fieldAndTerm;
       TermsEnum termsEnum;
 
@@ -1119,7 +1148,11 @@ public abstract class BasePostingsFormat
         // Seek by random field+term:
         fieldAndTerm = allTerms.get(upto++);
         if (VERBOSE) {
-          System.out.println("\nTEST: seek to term=" + fieldAndTerm.field + ":" + fieldAndTerm.term.utf8ToString() );
+          if (useTermOrd) {
+            System.out.println("\nTEST: seek to term=" + fieldAndTerm.field + ":" + fieldAndTerm.term.utf8ToString() + " using ord=" + fieldAndTerm.ord);
+          } else {
+            System.out.println("\nTEST: seek to term=" + fieldAndTerm.field + ":" + fieldAndTerm.term.utf8ToString() );
+          }
         }
       } else {
         // Seek by previous saved TermState
@@ -1136,11 +1169,38 @@ public abstract class BasePostingsFormat
       termsEnum = terms.iterator(null);
 
       if (!useTermState) {
-        assertTrue(termsEnum.seekExact(fieldAndTerm.term));
+        if (useTermOrd) {
+          // Try seek by ord sometimes:
+          try {
+            termsEnum.seekExact(fieldAndTerm.ord);
+          } catch (UnsupportedOperationException uoe) {
+            supportsOrds = false;
+            assertTrue(termsEnum.seekExact(fieldAndTerm.term));
+          }
+        } else {
+          assertTrue(termsEnum.seekExact(fieldAndTerm.term));
+        }
       } else {
         termsEnum.seekExact(fieldAndTerm.term, termState);
       }
 
+      long termOrd;
+      if (supportsOrds) {
+        try {
+          termOrd = termsEnum.ord();
+        } catch (UnsupportedOperationException uoe) {
+          supportsOrds = false;
+          termOrd = -1;
+        }
+      } else {
+        termOrd = -1;
+      }
+
+      if (termOrd != -1) {
+        // PostingsFormat supports ords
+        assertEquals(fieldAndTerm.ord, termsEnum.ord());
+      }
+
       boolean savedTermState = false;
 
       if (options.contains(Option.TERM_STATE) && !useTermState && random().nextInt(5) == 1) {
@@ -1185,6 +1245,71 @@ public abstract class BasePostingsFormat
                    alwaysTestMax);
       }
     }
+
+    // Test Terms.intersect:
+    for(String field : fields.keySet()) {
+      while (true) {
+        Automaton a = AutomatonTestUtil.randomAutomaton(random());
+        CompiledAutomaton ca = new CompiledAutomaton(a);
+        if (ca.type != CompiledAutomaton.AUTOMATON_TYPE.NORMAL) {
+          // Keep retrying until we get an A that will really "use" the PF's intersect code:
+          continue;
+        }
+        // System.out.println("A:\n" + a.toDot());
+
+        BytesRef startTerm = null;
+        if (random().nextBoolean()) {
+          RandomAcceptedStrings ras = new RandomAcceptedStrings(a);
+          for (int iter=0;iter<100;iter++) {
+            int[] codePoints = ras.getRandomAcceptedString(random());
+            if (codePoints.length == 0) {
+              continue;
+            }
+            startTerm = new BytesRef(UnicodeUtil.newString(codePoints, 0, codePoints.length));
+            break;
+          }
+          // Don't allow empty string startTerm:
+          if (startTerm == null) {
+            continue;
+          }
+        }
+        TermsEnum intersected = fieldsSource.terms(field).intersect(ca, startTerm);
+
+        Set<BytesRef> intersectedTerms = new HashSet<BytesRef>();
+        BytesRef term;
+        while ((term = intersected.next()) != null) {
+          if (startTerm != null) {
+            // NOTE: not <=
+            assertTrue(startTerm.compareTo(term) < 0);
+          }
+          intersectedTerms.add(BytesRef.deepCopyOf(term));     
+          verifyEnum(threadState,
+                     field,
+                     term,
+                     intersected,
+                     maxTestOptions,
+                     maxIndexOptions,
+                     options,
+                     alwaysTestMax);
+        }
+
+        if (ca.runAutomaton == null) {
+          assertTrue(intersectedTerms.isEmpty());
+        } else {
+          for(BytesRef term2 : fields.get(field).keySet()) {
+            boolean expected;
+            if (startTerm != null && startTerm.compareTo(term2) >= 0) {
+              expected = false;
+            } else {
+              expected = ca.runAutomaton.run(term2.bytes, term2.offset, term2.length);
+            }
+            assertEquals("term=" + term2, expected, intersectedTerms.contains(term2));
+          }
+        }
+
+        break;
+      }
+    }
   }
   
   private void testFields(Fields fields) throws Exception {
@@ -1284,7 +1409,7 @@ public abstract class BasePostingsFormat
     }
   }
   
-  public void testEmptyField() throws Exception {
+  public void testJustEmptyField() throws Exception {
     Directory dir = newDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, null);
     iwc.setCodec(getCodec());
@@ -1449,6 +1574,7 @@ public abstract class BasePostingsFormat
                     DocsEnum docs = null;
                     while(termsEnum.next() != null) {
                       BytesRef term = termsEnum.term();
+
                       if (random().nextBoolean()) {
                         docs = termsEnum.docs(null, docs, DocsEnum.FLAG_FREQS);
                       } else if (docs instanceof DocsAndPositionsEnum) {
@@ -1584,11 +1710,24 @@ public abstract class BasePostingsFormat
 
     TermsEnum termsEnum = terms.iterator(null);
     long termCount = 0;
+    boolean supportsOrds = true;
     while(termsEnum.next() != null) {
       BytesRef term = termsEnum.term();
-      termCount++;
       assertEquals(termFreqs.get(term.utf8ToString()).docFreq, termsEnum.docFreq());
       assertEquals(termFreqs.get(term.utf8ToString()).totalTermFreq, termsEnum.totalTermFreq());
+      if (supportsOrds) {
+        long ord;
+        try {
+          ord = termsEnum.ord();
+        } catch (UnsupportedOperationException uoe) {
+          supportsOrds = false;
+          ord = -1;
+        }
+        if (ord != -1) {
+          assertEquals(termCount, ord);
+        }
+      }
+      termCount++;
     }
     assertEquals(termFreqs.size(), termCount);
 

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java?rev=1612080&r1=1612079&r2=1612080&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java Sun Jul 20 12:08:32 2014
@@ -692,7 +692,7 @@ public class MockDirectoryWrapper extend
         if (LuceneTestCase.VERBOSE) {
           System.out.println("\nNOTE: MockDirectoryWrapper: now run CheckIndex");
         } 
-        TestUtil.checkIndex(this, getCrossCheckTermVectorsOnClose());
+        TestUtil.checkIndex(this, getCrossCheckTermVectorsOnClose(), true);
 
         // TODO: factor this out / share w/ TestIW.assertNoUnreferencedFiles
         if (assertNoUnreferencedFilesOnClose) {

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java?rev=1612080&r1=1612079&r2=1612080&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java Sun Jul 20 12:08:32 2014
@@ -28,7 +28,6 @@ import java.io.PrintStream;
 import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.nio.CharBuffer;
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Enumeration;
 import java.util.HashMap;
@@ -194,9 +193,16 @@ public final class TestUtil {
   }
 
   public static CheckIndex.Status checkIndex(Directory dir, boolean crossCheckTermVectors) throws IOException {
+    return checkIndex(dir, crossCheckTermVectors, false);
+  }
+
+  /** If failFast is true, then throw the first exception when index corruption is hit, instead of moving on to other fields/segments to
+   *  look for any other corruption.  */
+  public static CheckIndex.Status checkIndex(Directory dir, boolean crossCheckTermVectors, boolean failFast) throws IOException {
     ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
     CheckIndex checker = new CheckIndex(dir);
     checker.setCrossCheckTermVectors(crossCheckTermVectors);
+    checker.setFailFast(failFast);
     checker.setInfoStream(new PrintStream(bos, false, IOUtils.UTF_8), false);
     CheckIndex.Status indexStatus = checker.checkIndex(null);
     if (indexStatus == null || indexStatus.clean == false) {
@@ -224,24 +230,14 @@ public final class TestUtil {
     PrintStream infoStream = new PrintStream(bos, false, IOUtils.UTF_8);
 
     reader.checkIntegrity();
-    FieldNormStatus fieldNormStatus = CheckIndex.testFieldNorms(reader, infoStream);
-    TermIndexStatus termIndexStatus = CheckIndex.testPostings(reader, infoStream);
-    StoredFieldStatus storedFieldStatus = CheckIndex.testStoredFields(reader, infoStream);
-    TermVectorStatus termVectorStatus = CheckIndex.testTermVectors(reader, infoStream, false, crossCheckTermVectors);
-    DocValuesStatus docValuesStatus = CheckIndex.testDocValues(reader, infoStream);
+    FieldNormStatus fieldNormStatus = CheckIndex.testFieldNorms(reader, infoStream, true);
+    TermIndexStatus termIndexStatus = CheckIndex.testPostings(reader, infoStream, false, true);
+    StoredFieldStatus storedFieldStatus = CheckIndex.testStoredFields(reader, infoStream, true);
+    TermVectorStatus termVectorStatus = CheckIndex.testTermVectors(reader, infoStream, false, crossCheckTermVectors, true);
+    DocValuesStatus docValuesStatus = CheckIndex.testDocValues(reader, infoStream, true);
     
-    if (fieldNormStatus.error != null || 
-      termIndexStatus.error != null ||
-      storedFieldStatus.error != null ||
-      termVectorStatus.error != null ||
-      docValuesStatus.error != null) {
-      System.out.println("CheckReader failed");
+    if (LuceneTestCase.INFOSTREAM) {
       System.out.println(bos.toString(IOUtils.UTF_8));
-      throw new RuntimeException("CheckReader failed");
-    } else {
-      if (LuceneTestCase.INFOSTREAM) {
-        System.out.println(bos.toString(IOUtils.UTF_8));
-      }
     }
   }