You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2013/02/11 03:54:54 UTC

svn commit: r1444647 [1/2] - in /lucene/dev/branches/lucene4765: lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/ lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/ lucene/core/src/java/org/apache/lucene/codecs/ lucene/core/src/java/org...

Author: rmuir
Date: Mon Feb 11 02:54:53 2013
New Revision: 1444647

URL: http://svn.apache.org/r1444647
Log:
commit current state

Added:
    lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/document/SortedSetDocValuesField.java   (with props)
    lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValues.java   (with props)
    lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesTermsEnum.java   (with props)
    lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesWriter.java   (with props)
    lucene/dev/branches/lucene4765/lucene/core/src/test/org/apache/lucene/TestDemoDocValue.java   (with props)
Modified:
    lucene/dev/branches/lucene4765/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesConsumer.java
    lucene/dev/branches/lucene4765/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java
    lucene/dev/branches/lucene4765/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java
    lucene/dev/branches/lucene4765/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java
    lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
    lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java
    lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java
    lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java
    lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java
    lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosReader.java
    lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosWriter.java
    lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java
    lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java
    lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java
    lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
    lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/DocValuesProcessor.java
    lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
    lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java
    lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java
    lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java
    lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java
    lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java
    lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
    lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
    lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
    lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java
    lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/util/packed/AppendingLongBuffer.java
    lucene/dev/branches/lucene4765/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
    lucene/dev/branches/lucene4765/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingDocValuesFormat.java
    lucene/dev/branches/lucene4765/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardDocValuesProducer.java
    lucene/dev/branches/lucene4765/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesWriter.java
    lucene/dev/branches/lucene4765/lucene/test-framework/src/java/org/apache/lucene/index/AssertingAtomicReader.java
    lucene/dev/branches/lucene4765/solr/core/src/test/org/apache/solr/search/TestDocSet.java

Modified: lucene/dev/branches/lucene4765/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesConsumer.java?rev=1444647&r1=1444646&r2=1444647&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesConsumer.java (original)
+++ lucene/dev/branches/lucene4765/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesConsumer.java Mon Feb 11 02:54:53 2013
@@ -124,6 +124,11 @@ public class DiskDocValuesConsumer exten
   }
   
   @Override
+  public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrdCount, Iterable<Number> ords) throws IOException {
+    throw new UnsupportedOperationException(); // nocommit
+  }
+
+  @Override
   public void close() throws IOException {
     boolean success = false;
     try {

Modified: lucene/dev/branches/lucene4765/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java?rev=1444647&r1=1444646&r2=1444647&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java (original)
+++ lucene/dev/branches/lucene4765/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java Mon Feb 11 02:54:53 2013
@@ -31,6 +31,7 @@ import org.apache.lucene.index.IndexFile
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
@@ -252,6 +253,11 @@ class DiskDocValuesProducer extends DocV
   }
 
   @Override
+  public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
+    throw new UnsupportedOperationException(); // nocommit
+  }
+
+  @Override
   public void close() throws IOException {
     data.close();
   }

Modified: lucene/dev/branches/lucene4765/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java?rev=1444647&r1=1444646&r2=1444647&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java (original)
+++ lucene/dev/branches/lucene4765/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java Mon Feb 11 02:54:53 2013
@@ -36,6 +36,7 @@ import org.apache.lucene.index.NumericDo
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.FieldInfo.DocValuesType;
+import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.StringHelper;
@@ -285,6 +286,11 @@ class SimpleTextDocValuesReader extends 
   }
 
   @Override
+  public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
+    throw new UnsupportedOperationException(); // nocommit
+  }
+
+  @Override
   public void close() throws IOException {
     data.close();
   }

Modified: lucene/dev/branches/lucene4765/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java?rev=1444647&r1=1444646&r2=1444647&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java (original)
+++ lucene/dev/branches/lucene4765/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java Mon Feb 11 02:54:53 2013
@@ -250,6 +250,11 @@ class SimpleTextDocValuesWriter extends 
     }
   }
 
+  @Override
+  public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrdCount, Iterable<Number> ords) throws IOException {
+    throw new UnsupportedOperationException(); // nocommit
+  }
+
   /** write the header for this field */
   private void writeFieldEntry(FieldInfo field, FieldInfo.DocValuesType type) throws IOException {
     SimpleTextUtil.write(data, FIELD);

Modified: lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java?rev=1444647&r1=1444646&r2=1444647&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java (original)
+++ lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java Mon Feb 11 02:54:53 2013
@@ -19,7 +19,6 @@ package org.apache.lucene.codecs;
 
 import java.io.Closeable;
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 import java.util.NoSuchElementException;
@@ -27,15 +26,20 @@ import java.util.NoSuchElementException;
 import org.apache.lucene.index.AtomicReader;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FilteredTermsEnum;
 import org.apache.lucene.index.MergeState;
+import org.apache.lucene.index.MultiDocValues.OrdinalMap;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedDocValuesTermsEnum;
+import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.index.SortedSetDocValues.OrdIterator;
+import org.apache.lucene.index.SortedSetDocValuesTermsEnum;
+import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FixedBitSet;
-import org.apache.lucene.util.PriorityQueue;
-import org.apache.lucene.util.packed.AppendingLongBuffer;
 
 /** 
  * Abstract API that consumes numeric, binary and
@@ -90,6 +94,16 @@ public abstract class DocValuesConsumer 
   public abstract void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException;
 
   /**
+   * Writes pre-sorted set docvalues for a field
+   * @param field field information
+   * @param values Iterable of binary values in sorted order (deduplicated).
+   * @param docToOrdCount Iterable of the number of values for each document. 
+   * @param ords Iterable of ordinal occurrences (docToOrdCount*maxDoc total).
+   * @throws IOException if an I/O error occurred.
+   */
+  public abstract void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrdCount, Iterable<Number> ords) throws IOException;
+  
+  /**
    * Merges the numeric docvalues from <code>toMerge</code>.
    * <p>
    * The default implementation calls {@link #addNumericField}, passing
@@ -237,257 +251,375 @@ public abstract class DocValuesConsumer 
                    });
   }
 
-  static class SortedBytesMerger {
-
-    public int numMergedTerms;
 
-    final AppendingLongBuffer ordToReaderId = new AppendingLongBuffer();
-    final List<SegmentState> segStates = new ArrayList<SegmentState>();
-
-    private static class SegmentState {
-      int segmentID;
-      AtomicReader reader;
-      FixedBitSet liveTerms;
-      int ord = -1;
-      SortedDocValues values;
-      BytesRef scratch = new BytesRef();
-      AppendingLongBuffer ordDeltas = new AppendingLongBuffer();
-
-      // TODO: use another scheme?
-      // currently we +/- delta merged-ord from segment-ord (is this good? makes sense to me?)
-      // but we have a good idea "roughly" what
-      // the ord should be (linear projection) so we only
-      // need to encode the delta from that ...:        
-      AppendingLongBuffer segOrdToMergedOrd = new AppendingLongBuffer();
-
-      public BytesRef nextTerm() {
-        while (ord < values.getValueCount()-1) {
-          ord++;
-          if (liveTerms == null || liveTerms.get(ord)) {
-            values.lookupOrd(ord, scratch);
-            return scratch;
+  /**
+   * Merges the sorted docvalues from <code>toMerge</code>.
+   * <p>
+   * The default implementation calls {@link #addSortedField}, passing
+   * an Iterable that merges ordinals and values and filters deleted documents .
+   */
+  public void mergeSortedField(FieldInfo fieldInfo, final MergeState mergeState, List<SortedDocValues> toMerge) throws IOException {
+    final AtomicReader readers[] = mergeState.readers.toArray(new AtomicReader[toMerge.size()]);
+    final SortedDocValues dvs[] = toMerge.toArray(new SortedDocValues[toMerge.size()]);
+    
+    // step 1: iterate thru each sub and mark terms still in use
+    TermsEnum liveTerms[] = new TermsEnum[dvs.length];
+    for (int sub = 0; sub < liveTerms.length; sub++) {
+      AtomicReader reader = readers[sub];
+      SortedDocValues dv = dvs[sub];
+      Bits liveDocs = reader.getLiveDocs();
+      if (liveDocs == null) {
+        liveTerms[sub] = new SortedDocValuesTermsEnum(dv);
+      } else {
+        FixedBitSet bitset = new FixedBitSet(dv.getValueCount());
+        for (int i = 0; i < reader.maxDoc(); i++) {
+          if (liveDocs.get(i)) {
+            bitset.set(dv.getOrd(i));
           }
         }
-
-        return null;
-      }
-    }
-
-    private static class TermMergeQueue extends PriorityQueue<SegmentState> {
-      public TermMergeQueue(int maxSize) {
-        super(maxSize);
-      }
-
-      @Override
-      protected boolean lessThan(SegmentState a, SegmentState b) {
-        return a.scratch.compareTo(b.scratch) <= 0;
+        liveTerms[sub] = new BitsFilteredTermsEnum(new SortedDocValuesTermsEnum(dv), bitset);
       }
     }
-
-    public void merge(MergeState mergeState, List<SortedDocValues> toMerge) throws IOException {
-
-      // First pass: mark "live" terms
-      for (int readerIDX=0;readerIDX<toMerge.size();readerIDX++) {
-        AtomicReader reader = mergeState.readers.get(readerIDX);      
-        int maxDoc = reader.maxDoc();
-
-        SegmentState state = new SegmentState();
-        state.segmentID = readerIDX;
-        state.reader = reader;
-        state.values = toMerge.get(readerIDX);
-
-        segStates.add(state);
-        assert state.values.getValueCount() < Integer.MAX_VALUE;
-        if (reader.hasDeletions()) {
-          state.liveTerms = new FixedBitSet(state.values.getValueCount());
-          Bits liveDocs = reader.getLiveDocs();
-          assert liveDocs != null;
-          for(int docID=0;docID<maxDoc;docID++) {
-            if (liveDocs.get(docID)) {
-              state.liveTerms.set(state.values.getOrd(docID));
-            }
+    
+    // step 2: create ordinal map (this conceptually does the "merging")
+    final OrdinalMap map = new OrdinalMap(this, liveTerms);
+    
+    // step 3: add field
+    addSortedField(fieldInfo,
+        // ord -> value
+        new Iterable<BytesRef>() {
+          @Override
+          public Iterator<BytesRef> iterator() {
+            return new Iterator<BytesRef>() {
+              final BytesRef scratch = new BytesRef();
+              int currentOrd;
+
+              @Override
+              public boolean hasNext() {
+                return currentOrd < map.getValueCount();
+              }
+
+              @Override
+              public BytesRef next() {
+                if (!hasNext()) {
+                  throw new NoSuchElementException();
+                }
+                int segmentNumber = map.getSegmentNumber(currentOrd);
+                int segmentOrd = (int)map.getSegmentOrd(segmentNumber, currentOrd);
+                dvs[segmentNumber].lookupOrd(segmentOrd, scratch);
+                currentOrd++;
+                return scratch;
+              }
+
+              @Override
+              public void remove() {
+                throw new UnsupportedOperationException();
+              }
+            };
           }
-        }
-
-        // TODO: we can unload the bits/packed ints to disk to reduce
-        // transient ram spike... most of these just require iterators
-      }
-
-      // Second pass: merge only the live terms
-
-      TermMergeQueue q = new TermMergeQueue(segStates.size());
-      for(SegmentState segState : segStates) {
-        if (segState.nextTerm() != null) {
-          q.add(segState);
-        }
-      }
-
-      int lastOrds[] = new int[segStates.size()];
-      BytesRef lastTerm = null;
-      int ord = 0;
-      while (q.size() != 0) {
-        SegmentState top = q.top();
-        if (lastTerm == null || !lastTerm.equals(top.scratch)) {
-          // a new unique term: record its segment ID / sourceOrd pair
-          int readerId = top.segmentID;
-          ordToReaderId.add(readerId);
-
-          int sourceOrd = top.ord;             
-          int delta = sourceOrd - lastOrds[readerId];
-          lastOrds[readerId] = sourceOrd;
-          top.ordDeltas.add(delta);
-          
-          if (lastTerm == null) {
-            lastTerm = BytesRef.deepCopyOf(top.scratch);
-          } else {
-            lastTerm.copyBytes(top.scratch);
+        },
+        // doc -> ord
+        new Iterable<Number>() {
+          @Override
+          public Iterator<Number> iterator() {
+            return new Iterator<Number>() {
+              int readerUpto = -1;
+              int docIDUpto;
+              int nextValue;
+              AtomicReader currentReader;
+              Bits currentLiveDocs;
+              boolean nextIsSet;
+
+              @Override
+              public boolean hasNext() {
+                return nextIsSet || setNext();
+              }
+
+              @Override
+              public void remove() {
+                throw new UnsupportedOperationException();
+              }
+
+              @Override
+              public Number next() {
+                if (!hasNext()) {
+                  throw new NoSuchElementException();
+                }
+                assert nextIsSet;
+                nextIsSet = false;
+                // TODO make a mutable number
+                return nextValue;
+              }
+
+              private boolean setNext() {
+                while (true) {
+                  if (readerUpto == readers.length) {
+                    return false;
+                  }
+
+                  if (currentReader == null || docIDUpto == currentReader.maxDoc()) {
+                    readerUpto++;
+                    if (readerUpto < readers.length) {
+                      currentReader = readers[readerUpto];
+                      currentLiveDocs = currentReader.getLiveDocs();
+                    }
+                    docIDUpto = 0;
+                    continue;
+                  }
+
+                  if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
+                    nextIsSet = true;
+                    int segOrd = dvs[readerUpto].getOrd(docIDUpto);
+                    nextValue = (int) map.getGlobalOrd(readerUpto, segOrd);
+                    docIDUpto++;
+                    return true;
+                  }
+
+                  docIDUpto++;
+                }
+              }
+            };
           }
-          ord++;
-        }
-
-        long signedDelta = (ord-1) - top.ord; // global ord space - segment ord space
-        // fill in any holes for unused ords, then finally the value we want (segOrdToMergedOrd[top.ord])
-        // TODO: is there a better way...
-        while (top.segOrdToMergedOrd.size() <= top.ord) {
-          top.segOrdToMergedOrd.add(signedDelta);
-        }
-        if (top.nextTerm() == null) {
-          q.pop();
-        } else {
-          q.updateTop();
         }
-      }
-
-      numMergedTerms = ord;
-      // clear our bitsets for GC: we dont need them anymore (e.g. while flushing merged stuff to codec)
-      for (SegmentState state : segStates) {
-        state.liveTerms = null;
-      }
-    }
+    );
   }
-
+  
   /**
-   * Merges the sorted docvalues from <code>toMerge</code>.
+   * Merges the sortedset docvalues from <code>toMerge</code>.
    * <p>
-   * The default implementation calls {@link #addSortedField}, passing
+   * The default implementation calls {@link #addSortedSetField}, passing
    * an Iterable that merges ordinals and values and filters deleted documents .
    */
-  public void mergeSortedField(FieldInfo fieldInfo, final MergeState mergeState, List<SortedDocValues> toMerge) throws IOException {
-    final SortedBytesMerger merger = new SortedBytesMerger();
-
-    // Does the heavy lifting to merge sort all "live" ords:
-    merger.merge(mergeState, toMerge);
-
-    addSortedField(fieldInfo,
-
-                   // ord -> value
-                   new Iterable<BytesRef>() {
-                     @Override
-                     public Iterator<BytesRef> iterator() {
-                       // for each next(), tells us what reader to go to
-                       final AppendingLongBuffer.Iterator readerIDs = merger.ordToReaderId.iterator();
-                       // for each next(), gives us the original ord
-                       final AppendingLongBuffer.Iterator ordDeltas[] = new AppendingLongBuffer.Iterator[merger.segStates.size()];
-                       final int lastOrds[] = new int[ordDeltas.length];
-                       
-                       for (int i = 0; i < ordDeltas.length; i++) {
-                         ordDeltas[i] = merger.segStates.get(i).ordDeltas.iterator();
-                       }
-
-                       final BytesRef scratch = new BytesRef();
-                       
-                       return new Iterator<BytesRef>() {
-                         int ordUpto;
-
-                         @Override
-                         public boolean hasNext() {
-                           return ordUpto < merger.numMergedTerms;
-                         }
-
-                         @Override
-                         public void remove() {
-                           throw new UnsupportedOperationException();
-                         }
-
-                         @Override
-                         public BytesRef next() {
-                           if (!hasNext()) {
-                             throw new NoSuchElementException();
-                           }
-                           int readerID = (int) readerIDs.next();
-                           int ord = lastOrds[readerID] + (int) ordDeltas[readerID].next();
-                           merger.segStates.get(readerID).values.lookupOrd(ord, scratch);
-                           lastOrds[readerID] = ord;
-                           ordUpto++;
-                           return scratch;
-                         }
-                       };
-                     }
-                   },
-
-                   // doc -> ord
-                    new Iterable<Number>() {
-                      @Override
-                      public Iterator<Number> iterator() {
-                        return new Iterator<Number>() {
-                          int readerUpto = -1;
-                          int docIDUpto;
-                          int nextValue;
-                          SortedBytesMerger.SegmentState currentReader;
-                          Bits currentLiveDocs;
-                          boolean nextIsSet;
-
-                          @Override
-                          public boolean hasNext() {
-                            return nextIsSet || setNext();
-                          }
-
-                          @Override
-                          public void remove() {
-                            throw new UnsupportedOperationException();
-                          }
-
-                          @Override
-                          public Number next() {
-                            if (!hasNext()) {
-                              throw new NoSuchElementException();
-                            }
-                            assert nextIsSet;
-                            nextIsSet = false;
-                            // TODO make a mutable number
-                            return nextValue;
-                          }
-
-                          private boolean setNext() {
-                            while (true) {
-                              if (readerUpto == merger.segStates.size()) {
-                                return false;
-                              }
-
-                              if (currentReader == null || docIDUpto == currentReader.reader.maxDoc()) {
-                                readerUpto++;
-                                if (readerUpto < merger.segStates.size()) {
-                                  currentReader = merger.segStates.get(readerUpto);
-                                  currentLiveDocs = currentReader.reader.getLiveDocs();
-                                }
-                                docIDUpto = 0;
-                                continue;
-                              }
-
-                              if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
-                                nextIsSet = true;
-                                int segOrd = currentReader.values.getOrd(docIDUpto);
-                                nextValue = (int) (segOrd + currentReader.segOrdToMergedOrd.get(segOrd));
-                                docIDUpto++;
-                                return true;
-                              }
-
-                              docIDUpto++;
-                            }
-                          }
-                        };
-                      }
-                    });
-
+  public void mergeSortedSetField(FieldInfo fieldInfo, final MergeState mergeState, List<SortedSetDocValues> toMerge) throws IOException {
+    final AtomicReader readers[] = mergeState.readers.toArray(new AtomicReader[toMerge.size()]);
+    final SortedSetDocValues dvs[] = toMerge.toArray(new SortedSetDocValues[toMerge.size()]);
+    
+    // step 1: iterate thru each sub and mark terms still in use
+    TermsEnum liveTerms[] = new TermsEnum[dvs.length];
+    for (int sub = 0; sub < liveTerms.length; sub++) {
+      AtomicReader reader = readers[sub];
+      SortedSetDocValues dv = dvs[sub];
+      Bits liveDocs = reader.getLiveDocs();
+      if (liveDocs == null) {
+        liveTerms[sub] = new SortedSetDocValuesTermsEnum(dv);
+      } else {
+        // nocommit: need a "pagedbits"
+        if (dv.getValueCount() > Integer.MAX_VALUE) {
+          throw new UnsupportedOperationException();
+        }
+        FixedBitSet bitset = new FixedBitSet((int)dv.getValueCount());
+        OrdIterator iterator = null;
+        for (int i = 0; i < reader.maxDoc(); i++) {
+          if (liveDocs.get(i)) {
+            iterator = dv.getOrds(i, iterator);
+            long ord;
+            while ((ord = iterator.nextOrd()) != OrdIterator.NO_MORE_ORDS) {
+              bitset.set((int)ord); // nocommit
+            }
+          }
+        }
+        liveTerms[sub] = new BitsFilteredTermsEnum(new SortedSetDocValuesTermsEnum(dv), bitset);
+      }
+    }
+    
+    // step 2: create ordinal map (this conceptually does the "merging")
+    final OrdinalMap map = new OrdinalMap(this, liveTerms);
+    
+    // step 3: add field
+    addSortedSetField(fieldInfo,
+        // ord -> value
+        new Iterable<BytesRef>() {
+          @Override
+          public Iterator<BytesRef> iterator() {
+            return new Iterator<BytesRef>() {
+              final BytesRef scratch = new BytesRef();
+              long currentOrd;
+
+              @Override
+              public boolean hasNext() {
+                return currentOrd < map.getValueCount();
+              }
+
+              @Override
+              public BytesRef next() {
+                if (!hasNext()) {
+                  throw new NoSuchElementException();
+                }
+                int segmentNumber = map.getSegmentNumber(currentOrd);
+                long segmentOrd = map.getSegmentOrd(segmentNumber, currentOrd);
+                dvs[segmentNumber].lookupOrd(segmentOrd, scratch);
+                currentOrd++;
+                return scratch;
+              }
+
+              @Override
+              public void remove() {
+                throw new UnsupportedOperationException();
+              }
+            };
+          }
+        },
+        // doc -> ord count
+        new Iterable<Number>() {
+          @Override
+          public Iterator<Number> iterator() {
+            return new Iterator<Number>() {
+              int readerUpto = -1;
+              int docIDUpto;
+              int nextValue;
+              AtomicReader currentReader;
+              OrdIterator iterator;
+              Bits currentLiveDocs;
+              boolean nextIsSet;
+
+              @Override
+              public boolean hasNext() {
+                return nextIsSet || setNext();
+              }
+
+              @Override
+              public void remove() {
+                throw new UnsupportedOperationException();
+              }
+
+              @Override
+              public Number next() {
+                if (!hasNext()) {
+                  throw new NoSuchElementException();
+                }
+                assert nextIsSet;
+                nextIsSet = false;
+                // TODO make a mutable number
+                return nextValue;
+              }
+
+              private boolean setNext() {
+                while (true) {
+                  if (readerUpto == readers.length) {
+                    return false;
+                  }
+
+                  if (currentReader == null || docIDUpto == currentReader.maxDoc()) {
+                    readerUpto++;
+                    if (readerUpto < readers.length) {
+                      currentReader = readers[readerUpto];
+                      currentLiveDocs = currentReader.getLiveDocs();
+                    }
+                    docIDUpto = 0;
+                    continue;
+                  }
+
+                  if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
+                    nextIsSet = true;
+                    iterator = dvs[readerUpto].getOrds(docIDUpto, iterator);
+                    nextValue = 0;
+                    while (iterator.nextOrd() != OrdIterator.NO_MORE_ORDS) {
+                      nextValue++;
+                    }
+                    docIDUpto++;
+                    return true;
+                  }
+
+                  docIDUpto++;
+                }
+              }
+            };
+          }
+        },
+        // ords
+        new Iterable<Number>() {
+          @Override
+          public Iterator<Number> iterator() {
+            return new Iterator<Number>() {
+              int readerUpto = -1;
+              int docIDUpto;
+              long nextValue;
+              AtomicReader currentReader;
+              OrdIterator iterator;
+              Bits currentLiveDocs;
+              boolean nextIsSet;
+
+              @Override
+              public boolean hasNext() {
+                return nextIsSet || setNext();
+              }
+
+              @Override
+              public void remove() {
+                throw new UnsupportedOperationException();
+              }
+
+              @Override
+              public Number next() {
+                if (!hasNext()) {
+                  throw new NoSuchElementException();
+                }
+                assert nextIsSet;
+                nextIsSet = false;
+                // TODO make a mutable number
+                return nextValue;
+              }
+
+              private boolean setNext() {
+                while (true) {
+                  if (readerUpto == readers.length) {
+                    return false;
+                  }
+                  
+                  if (iterator != null) {
+                    final long segmentOrd = iterator.nextOrd();
+                    if (segmentOrd != OrdIterator.NO_MORE_ORDS) {
+                      nextValue = map.getGlobalOrd(readerUpto, segmentOrd);
+                      nextIsSet = true;
+                      return true;
+                    } else {
+                      docIDUpto++;
+                    }
+                  }
+
+                  if (currentReader == null || docIDUpto == currentReader.maxDoc()) {
+                    readerUpto++;
+                    if (readerUpto < readers.length) {
+                      currentReader = readers[readerUpto];
+                      currentLiveDocs = currentReader.getLiveDocs();
+                    }
+                    docIDUpto = 0;
+                    iterator = null;
+                    continue;
+                  }
+                  
+                  if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
+                    assert docIDUpto < currentReader.maxDoc();
+                    iterator = dvs[readerUpto].getOrds(docIDUpto, iterator);
+                    continue;
+                  }
+
+                  docIDUpto++;
+                }
+              }
+            };
+          }
+        }
+     );
+  }
+  
+  // nocommit: need a "pagedbits"
+  static class BitsFilteredTermsEnum extends FilteredTermsEnum {
+    final Bits liveTerms;
+    
+    BitsFilteredTermsEnum(TermsEnum in, Bits liveTerms) {
+      super(in, false); // <-- not passing false here wasted about 3 hours of my time!!!!!!!!!!!!!
+      assert liveTerms != null;
+      this.liveTerms = liveTerms;
+    }
+    
+    @Override
+    protected AcceptStatus accept(BytesRef term) throws IOException {
+      if (liveTerms.get((int) ord())) {
+        return AcceptStatus.YES;
+      } else {
+        return AcceptStatus.NO;
+      }
+    }
   }
 }

Modified: lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java?rev=1444647&r1=1444646&r2=1444647&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java (original)
+++ lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java Mon Feb 11 02:54:53 2013
@@ -24,6 +24,7 @@ import org.apache.lucene.index.BinaryDoc
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedSetDocValues;
 
 /** Abstract API that produces numeric, binary and
  * sorted docvalues.
@@ -50,4 +51,9 @@ public abstract class DocValuesProducer 
    *  The returned instance need not be thread-safe: it will only be
    *  used by a single thread. */
   public abstract SortedDocValues getSorted(FieldInfo field) throws IOException;
+  
+  /** Returns {@link SortedSetDocValues} for this field.
+   *  The returned instance need not be thread-safe: it will only be
+   *  used by a single thread. */
+  public abstract SortedSetDocValues getSortedSet(FieldInfo field) throws IOException;
 }

Modified: lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java?rev=1444647&r1=1444646&r2=1444647&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java (original)
+++ lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java Mon Feb 11 02:54:53 2013
@@ -31,6 +31,7 @@ import org.apache.lucene.index.IndexFile
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
@@ -615,6 +616,11 @@ final class Lucene40DocValuesReader exte
   }
   
   @Override
+  public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
+    throw new IllegalStateException("Lucene 4.0 does not support SortedSet: how did you pull this off?");
+  }
+
+  @Override
   public void close() throws IOException {
     dir.close();
   }

Modified: lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java?rev=1444647&r1=1444646&r2=1444647&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java (original)
+++ lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java Mon Feb 11 02:54:53 2013
@@ -20,13 +20,17 @@ package org.apache.lucene.codecs.lucene4
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
 
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.store.ByteArrayDataOutput;
 import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.IntsRef;
@@ -195,13 +199,8 @@ class Lucene42DocValuesConsumer extends 
       writer.finish();
     }
   }
-
-  @Override
-  public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
-    // write the ordinals as numerics
-    addNumericField(field, docToOrd);
-    
-    // write the values as FST
+  
+  private void writeFST(FieldInfo field, Iterable<BytesRef> values) throws IOException {
     meta.writeVInt(field.number);
     meta.writeByte(FST);
     meta.writeLong(data.getFilePointer());
@@ -215,6 +214,91 @@ class Lucene42DocValuesConsumer extends 
     }
     FST<Long> fst = builder.finish();
     fst.save(data);
-    meta.writeVInt((int)ord);
+    meta.writeVLong(ord);
+  }
+
+  @Override
+  public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
+    // write the ordinals as numerics
+    addNumericField(field, docToOrd);
+    
+    // write the values as FST
+    writeFST(field, values);
+  }
+
+  // note: this might not be the most efficient... but its fairly simple
+  @Override
+  public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, final Iterable<Number> docToOrdCount, final Iterable<Number> ords) throws IOException {
+    // write the ordinals as a binary field
+    addBinaryField(field, new Iterable<BytesRef>() {
+      @Override
+      public Iterator<BytesRef> iterator() {
+        return new SortedSetIterator(docToOrdCount.iterator(), ords.iterator());
+      }
+    });
+      
+    // write the values as FST
+    writeFST(field, values);
+  }
+  
+  // per-document vint-encoded byte[]
+  static class SortedSetIterator implements Iterator<BytesRef> {
+    byte[] buffer = new byte[10];
+    ByteArrayDataOutput out = new ByteArrayDataOutput();
+    BytesRef ref = new BytesRef();
+    
+    final Iterator<Number> counts;
+    final Iterator<Number> ords;
+    
+    SortedSetIterator(Iterator<Number> counts, Iterator<Number> ords) {
+      this.counts = counts;
+      this.ords = ords;
+    }
+    
+    @Override
+    public boolean hasNext() {
+      return counts.hasNext();
+    }
+
+    @Override
+    public BytesRef next() {
+      if (!hasNext()) {
+        throw new NoSuchElementException();
+      }
+      
+      int count = counts.next().intValue();
+      int maxSize = count*9; // worst case
+      if (maxSize > buffer.length) {
+        buffer = ArrayUtil.grow(buffer, maxSize);
+      }
+      
+      try {
+        encodeValues(count);
+      } catch (IOException bogus) {
+        throw new RuntimeException(bogus);
+      }
+      
+      ref.bytes = buffer;
+      ref.offset = 0;
+      ref.length = out.getPosition();
+
+      return ref;
+    }
+    
+    // encodes count values to buffer
+    private void encodeValues(int count) throws IOException {
+      out.reset(buffer);
+      long lastOrd = 0;
+      for (int i = 0; i < count; i++) {
+        long ord = ords.next().longValue();
+        out.writeVLong(ord - lastOrd);
+        lastOrd = ord;
+      }
+    }
+
+    @Override
+    public void remove() {
+      throw new UnsupportedOperationException();
+    }
   }
 }

Modified: lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java?rev=1444647&r1=1444646&r2=1444647&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java (original)
+++ lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java Mon Feb 11 02:54:53 2013
@@ -31,6 +31,9 @@ import org.apache.lucene.index.IndexFile
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.index.SortedSetDocValues.OrdIterator;
+import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
@@ -123,7 +126,7 @@ class Lucene42DocValuesProducer extends 
       } else if (fieldType == Lucene42DocValuesConsumer.FST) {
         FSTEntry entry = new FSTEntry();
         entry.offset = meta.readLong();
-        entry.numOrds = meta.readVInt();
+        entry.numOrds = meta.readVLong();
         fsts.put(fieldNumber, entry);
       } else {
         throw new CorruptIndexException("invalid entry type: " + fieldType + ", input=" + meta);
@@ -281,11 +284,111 @@ class Lucene42DocValuesProducer extends 
 
       @Override
       public int getValueCount() {
+        return (int)entry.numOrds;
+      }
+    };
+  }
+  
+  @Override
+  public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
+    final FSTEntry entry = fsts.get(field.number);
+    FST<Long> instance;
+    synchronized(this) {
+      instance = fstInstances.get(field.number);
+      if (instance == null) {
+        data.seek(entry.offset);
+        instance = new FST<Long>(data, PositiveIntOutputs.getSingleton(true));
+        fstInstances.put(field.number, instance);
+      }
+    }
+    final BinaryDocValues docToOrds = getBinary(field);
+    final FST<Long> fst = instance;
+    
+    // per-thread resources
+    final BytesReader in = fst.getBytesReader();
+    final Arc<Long> firstArc = new Arc<Long>();
+    final Arc<Long> scratchArc = new Arc<Long>();
+    final IntsRef scratchInts = new IntsRef();
+    final BytesRefFSTEnum<Long> fstEnum = new BytesRefFSTEnum<Long>(fst); 
+    return new SortedSetDocValues() {
+
+      @Override
+      public OrdIterator getOrds(int docID, OrdIterator reuse) {
+        final Lucene42OrdsIterator iterator;
+        if (reuse instanceof Lucene42OrdsIterator) {
+          iterator = (Lucene42OrdsIterator) reuse;
+        } else {
+          iterator = new Lucene42OrdsIterator(docToOrds);
+        }
+        iterator.reset(docID);
+        return iterator;
+      }
+
+      @Override
+      public void lookupOrd(long ord, BytesRef result) {
+        try {
+          in.setPosition(0);
+          fst.getFirstArc(firstArc);
+          IntsRef output = Util.getByOutput(fst, ord, in, firstArc, scratchArc, scratchInts);
+          result.bytes = new byte[output.length];
+          result.offset = 0;
+          result.length = 0;
+          Util.toBytesRef(output, result);
+        } catch (IOException bogus) {
+          throw new RuntimeException(bogus);
+        }
+      }
+
+      @Override
+      public long lookupTerm(BytesRef key) {
+        try {
+          InputOutput<Long> o = fstEnum.seekCeil(key);
+          if (o == null) {
+            return -getValueCount()-1;
+          } else if (o.input.equals(key)) {
+            return o.output.intValue();
+          } else {
+            return -o.output-1;
+          }
+        } catch (IOException bogus) {
+          throw new RuntimeException(bogus);
+        }
+      }
+
+      @Override
+      public long getValueCount() {
         return entry.numOrds;
       }
     };
   }
   
+  static class Lucene42OrdsIterator extends OrdIterator {
+    final BinaryDocValues data;
+    final BytesRef ref = new BytesRef();
+    final ByteArrayDataInput input = new ByteArrayDataInput();
+    long currentOrd;
+    
+    Lucene42OrdsIterator(BinaryDocValues data) {
+      this.data = data;
+    }
+    
+    @Override
+    public long nextOrd() {
+      if (input.eof()) {
+        return NO_MORE_ORDS;
+      } else {
+        currentOrd += input.readVLong();
+        return currentOrd;
+      }
+    }
+    
+    void reset(int docid) {
+      data.get(docid, ref);
+      input.reset(ref.bytes, ref.offset, ref.length);
+      currentOrd = 0;
+    }
+  }
+
   @Override
   public void close() throws IOException {
     data.close();
@@ -308,6 +411,6 @@ class Lucene42DocValuesProducer extends 
   
   static class FSTEntry {
     long offset;
-    int numOrds;
+    long numOrds;
   }
 }

Modified: lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosReader.java?rev=1444647&r1=1444646&r2=1444647&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosReader.java (original)
+++ lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosReader.java Mon Feb 11 02:54:53 2013
@@ -114,6 +114,8 @@ final class Lucene42FieldInfosReader ext
       return DocValuesType.BINARY;
     } else if (b == 3) {
       return DocValuesType.SORTED;
+    } else if (b == 4) {
+      return DocValuesType.SORTED_SET;
     } else {
       throw new CorruptIndexException("invalid docvalues byte: " + b + " (resource=" + input + ")");
     }

Modified: lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosWriter.java?rev=1444647&r1=1444646&r2=1444647&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosWriter.java (original)
+++ lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosWriter.java Mon Feb 11 02:54:53 2013
@@ -99,6 +99,8 @@ final class Lucene42FieldInfosWriter ext
       return 2;
     } else if (type == DocValuesType.SORTED) {
       return 3;
+    } else if (type == DocValuesType.SORTED_SET) {
+      return 4;
     } else {
       throw new AssertionError();
     }

Modified: lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java?rev=1444647&r1=1444646&r2=1444647&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java (original)
+++ lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java Mon Feb 11 02:54:53 2013
@@ -35,6 +35,7 @@ import org.apache.lucene.index.NumericDo
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 
@@ -114,6 +115,11 @@ public abstract class PerFieldDocValuesF
       getInstance(field).addSortedField(field, values, docToOrd);
     }
 
+    @Override
+    public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrdCount, Iterable<Number> ords) throws IOException {
+      getInstance(field).addSortedSetField(field, values, docToOrdCount, ords);
+    }
+
     private DocValuesConsumer getInstance(FieldInfo field) throws IOException {
       final DocValuesFormat format = getDocValuesFormatForField(field.name);
       if (format == null) {
@@ -255,6 +261,12 @@ public abstract class PerFieldDocValuesF
     }
 
     @Override
+    public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
+      DocValuesProducer producer = fields.get(field.name);
+      return producer == null ? null : producer.getSortedSet(field);
+    }
+
+    @Override
     public void close() throws IOException {
       IOUtils.close(formats.values());
     }

Added: lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/document/SortedSetDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/document/SortedSetDocValuesField.java?rev=1444647&view=auto
==============================================================================
--- lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/document/SortedSetDocValuesField.java (added)
+++ lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/document/SortedSetDocValuesField.java Mon Feb 11 02:54:53 2013
@@ -0,0 +1,61 @@
+package org.apache.lucene.document;
+
+/*
+ * 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.index.FieldInfo;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * <p>
+ * Field that stores
+ * a set of per-document {@link BytesRef} values, indexed for
+ * faceting,grouping,joining.  Here's an example usage:
+ * 
+ * <pre class="prettyprint">
+ *   document.add(new SortedSetDocValuesField(name, new BytesRef("hello")));
+ *   document.add(new SortedSetDocValuesField(name, new BytesRef("world")));
+ * </pre>
+ * 
+ * <p>
+ * If you also need to store the value, you should add a
+ * separate {@link StoredField} instance.
+ * 
+ * */
+
+public class SortedSetDocValuesField extends StoredField {
+
+  /**
+   * Type for sorted bytes DocValues
+   */
+  public static final FieldType TYPE = new FieldType();
+  static {
+    TYPE.setDocValueType(FieldInfo.DocValuesType.SORTED_SET);
+    TYPE.freeze();
+  }
+
+  /**
+   * Create a new sorted DocValues field.
+   * @param name field name
+   * @param bytes binary content
+   * @throws IllegalArgumentException if the field name is null
+   */
+  public SortedSetDocValuesField(String name, BytesRef bytes) {
+    super(name, TYPE);
+    fieldsData = bytes;
+  }
+}

Modified: lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java?rev=1444647&r1=1444646&r2=1444647&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java (original)
+++ lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java Mon Feb 11 02:54:53 2013
@@ -174,6 +174,12 @@ public abstract class AtomicReader exten
    *  this field.  The returned instance should only be
    *  used by a single thread. */
   public abstract SortedDocValues getSortedDocValues(String field) throws IOException;
+  
+  /** Returns {@link SortedSetDocValues} for this field, or
+   *  null if no {@link SortedSetDocValues} were indexed for
+   *  this field.  The returned instance should only be
+   *  used by a single thread. */
+  public abstract SortedSetDocValues getSortedSetDocValues(String field) throws IOException;
 
   /** Returns {@link NumericDocValues} representing norms
    *  for this field, or null if no {@link NumericDocValues}

Modified: lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java?rev=1444647&r1=1444646&r2=1444647&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java (original)
+++ lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java Mon Feb 11 02:54:53 2013
@@ -91,7 +91,7 @@ class BinaryDocValuesWriter extends DocV
   private class BytesIterator implements Iterator<BytesRef> {
     final BytesRef value = new BytesRef();
     final AppendingLongBuffer.Iterator lengthsIterator = lengths.iterator();
-    final int size = lengths.size();
+    final int size = (int) lengths.size();
     final int maxDoc;
     int upto;
     long byteOffset;

Modified: lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java?rev=1444647&r1=1444646&r2=1444647&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java Mon Feb 11 02:54:53 2013
@@ -33,6 +33,7 @@ import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.PostingsFormat; // javadocs
 import org.apache.lucene.document.FieldType; // for javadocs
 import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.SortedSetDocValues.OrdIterator;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
@@ -1333,6 +1334,50 @@ public class CheckIndex {
     }
   }
   
+  private static void checkSortedSetDocValues(String fieldName, AtomicReader reader, SortedSetDocValues dv) {
+    final long maxOrd = dv.getValueCount()-1;
+    // nocommit
+    FixedBitSet seenOrds = new FixedBitSet((int)dv.getValueCount());
+    long maxOrd2 = -1;
+    OrdIterator iterator = null;
+    for (int i = 0; i < reader.maxDoc(); i++) {
+      iterator = dv.getOrds(i, iterator);
+      long lastOrd = -1;
+      long ord;
+      while ((ord = iterator.nextOrd()) != OrdIterator.NO_MORE_ORDS) {
+        if (ord <= lastOrd) {
+          throw new RuntimeException("ords out of order: " + ord + " <= " + lastOrd + " for doc: " + i);
+        }
+        if (ord < 0 || ord > maxOrd) {
+          throw new RuntimeException("ord out of bounds: " + ord);
+        }
+        lastOrd = ord;
+        maxOrd2 = Math.max(maxOrd2, ord);
+        // nocommit
+        seenOrds.set((int)ord);
+      }
+    }
+    if (maxOrd != maxOrd2) {
+      throw new RuntimeException("dv for field: " + fieldName + " reports wrong maxOrd=" + maxOrd + " but this is not the case: " + maxOrd2);
+    }
+    if (seenOrds.cardinality() != dv.getValueCount()) {
+      throw new RuntimeException("dv for field: " + fieldName + " has holes in its ords, valueCount=" + dv.getValueCount() + " but only used: " + seenOrds.cardinality());
+    }
+    
+    BytesRef lastValue = null;
+    BytesRef scratch = new BytesRef();
+    for (long i = 0; i <= maxOrd; i++) {
+      dv.lookupOrd(i, scratch);
+      assert scratch.isValid();
+      if (lastValue != null) {
+        if (scratch.compareTo(lastValue) <= 0) {
+          throw new RuntimeException("dv for field: " + fieldName + " has ords out of order: " + lastValue + " >=" + scratch);
+        }
+      }
+      lastValue = BytesRef.deepCopyOf(scratch);
+    }
+  }
+
   private static void checkNumericDocValues(String fieldName, AtomicReader reader, NumericDocValues ndv) {
     for (int i = 0; i < reader.maxDoc(); i++) {
       ndv.get(i);
@@ -1344,6 +1389,9 @@ public class CheckIndex {
       case SORTED:
         checkSortedDocValues(fi.name, reader, reader.getSortedDocValues(fi.name));
         break;
+      case SORTED_SET:
+        checkSortedSetDocValues(fi.name, reader, reader.getSortedSetDocValues(fi.name));
+        break;
       case BINARY:
         checkBinaryDocValues(fi.name, reader, reader.getBinaryDocValues(fi.name));
         break;

Modified: lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/DocValuesProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/DocValuesProcessor.java?rev=1444647&r1=1444646&r2=1444647&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/DocValuesProcessor.java (original)
+++ lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/DocValuesProcessor.java Mon Feb 11 02:54:53 2013
@@ -57,6 +57,8 @@ final class DocValuesProcessor extends S
         addBinaryField(fieldInfo, docID, field.binaryValue());
       } else if (dvType == DocValuesType.SORTED) {
         addSortedField(fieldInfo, docID, field.binaryValue());
+      } else if (dvType == DocValuesType.SORTED_SET) {
+        addSortedSetField(fieldInfo, docID, field.binaryValue());
       } else if (dvType == DocValuesType.NUMERIC) {
         if (!(field.numericValue() instanceof Long)) {
           throw new IllegalArgumentException("illegal type " + field.numericValue().getClass() + ": DocValues types must be Long");
@@ -122,6 +124,20 @@ final class DocValuesProcessor extends S
     }
     sortedWriter.addValue(docID, value);
   }
+  
+  void addSortedSetField(FieldInfo fieldInfo, int docID, BytesRef value) {
+    DocValuesWriter writer = writers.get(fieldInfo.name);
+    SortedSetDocValuesWriter sortedSetWriter;
+    if (writer == null) {
+      sortedSetWriter = new SortedSetDocValuesWriter(fieldInfo, bytesUsed);
+      writers.put(fieldInfo.name, sortedSetWriter);
+    } else if (!(writer instanceof SortedSetDocValuesWriter)) {
+      throw new IllegalArgumentException("Incompatible DocValues type: field \"" + fieldInfo.name + "\" changed from " + getTypeDesc(writer) + " to sorted");
+    } else {
+      sortedSetWriter = (SortedSetDocValuesWriter) writer;
+    }
+    sortedSetWriter.addValue(docID, value);
+  }
 
   void addNumericField(FieldInfo fieldInfo, int docID, long value) {
     DocValuesWriter writer = writers.get(fieldInfo.name);

Modified: lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java?rev=1444647&r1=1444646&r2=1444647&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java (original)
+++ lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java Mon Feb 11 02:54:53 2013
@@ -101,7 +101,14 @@ public final class FieldInfo {
      * byte[]. The stored byte[] is presorted and allows access via document id, 
      * ordinal and by-value.
      */
-    SORTED
+    SORTED,
+    /** 
+     * A pre-sorted Set<byte[]>. Fields with this type only store distinct byte values 
+     * and store additional offset pointers per document to dereference the shared 
+     * byte[]s. The stored byte[] is presorted and allows access via document id, 
+     * ordinal and by-value.
+     */
+    SORTED_SET
   };
 
   /**

Modified: lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java?rev=1444647&r1=1444646&r2=1444647&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java (original)
+++ lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java Mon Feb 11 02:54:53 2013
@@ -424,6 +424,12 @@ public class FilterAtomicReader extends 
   }
 
   @Override
+  public SortedSetDocValues getSortedSetDocValues(String field) throws IOException {
+    ensureOpen();
+    return in.getSortedSetDocValues(field);
+  }
+
+  @Override
   public NumericDocValues getNormValues(String field) throws IOException {
     ensureOpen();
     return in.getNormValues(field);

Modified: lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java?rev=1444647&r1=1444646&r2=1444647&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java (original)
+++ lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java Mon Feb 11 02:54:53 2013
@@ -18,6 +18,7 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.List;
 
 import org.apache.lucene.index.MultiTermsEnum.TermsEnumIndex;
@@ -214,14 +215,62 @@ public class MultiDocValues {
     if (!anyReal) {
       return null;
     } else {
-      OrdinalMap mapping = new OrdinalMap(r.getCoreCacheKey(), values);
+      TermsEnum enums[] = new TermsEnum[values.length];
+      for (int i = 0; i < values.length; i++) {
+        enums[i] = new SortedDocValuesTermsEnum(values[i]);
+      }
+      OrdinalMap mapping = new OrdinalMap(r.getCoreCacheKey(), enums);
       return new MultiSortedDocValues(values, starts, mapping);
     }
   }
   
+  /** Returns a SortedSetDocValues for a reader's docvalues (potentially doing extremely slow things).
+   * <p>
+   * This is an extremely slow way to access sorted values. Instead, access them per-segment
+   * with {@link AtomicReader#getSortedSetDocValues(String)}
+   * </p>  
+   */
+  public static SortedSetDocValues getSortedSetValues(final IndexReader r, final String field) throws IOException {
+    final List<AtomicReaderContext> leaves = r.leaves();
+    final int size = leaves.size();
+    
+    if (size == 0) {
+      return null;
+    } else if (size == 1) {
+      return leaves.get(0).reader().getSortedSetDocValues(field);
+    }
+    
+    boolean anyReal = false;
+    final SortedSetDocValues[] values = new SortedSetDocValues[size];
+    final int[] starts = new int[size+1];
+    for (int i = 0; i < size; i++) {
+      AtomicReaderContext context = leaves.get(i);
+      SortedSetDocValues v = context.reader().getSortedSetDocValues(field);
+      if (v == null) {
+        v = SortedSetDocValues.EMPTY;
+      } else {
+        anyReal = true;
+      }
+      values[i] = v;
+      starts[i] = context.docBase;
+    }
+    starts[size] = r.maxDoc();
+    
+    if (!anyReal) {
+      return null;
+    } else {
+      TermsEnum enums[] = new TermsEnum[values.length];
+      for (int i = 0; i < values.length; i++) {
+        enums[i] = new SortedSetDocValuesTermsEnum(values[i]);
+      }
+      OrdinalMap mapping = new OrdinalMap(r.getCoreCacheKey(), enums);
+      return new MultiSortedSetDocValues(values, starts, mapping);
+    }
+  }
+  
   /** maps per-segment ordinals to/from global ordinal space */
-  // TODO: use more efficient packed ints structures (these are all positive values!)
-  static class OrdinalMap {
+  // TODO: use more efficient packed ints structures?
+  public static class OrdinalMap {
     // cache key of whoever asked for this aweful thing
     final Object owner;
     // globalOrd -> (globalOrd - segmentOrd)
@@ -231,7 +280,7 @@ public class MultiDocValues {
     // segmentOrd -> (globalOrd - segmentOrd)
     final AppendingLongBuffer ordDeltas[];
     
-    OrdinalMap(Object owner, SortedDocValues subs[]) throws IOException {
+    public OrdinalMap(Object owner, TermsEnum subs[]) throws IOException {
       // create the ordinal mappings by pulling a termsenum over each sub's 
       // unique terms, and walking a multitermsenum over those
       this.owner = owner;
@@ -241,34 +290,52 @@ public class MultiDocValues {
       for (int i = 0; i < ordDeltas.length; i++) {
         ordDeltas[i] = new AppendingLongBuffer();
       }
-      int segmentOrds[] = new int[subs.length];
+      long segmentOrds[] = new long[subs.length];
       ReaderSlice slices[] = new ReaderSlice[subs.length];
       TermsEnumIndex indexes[] = new TermsEnumIndex[slices.length];
       for (int i = 0; i < slices.length; i++) {
         slices[i] = new ReaderSlice(0, 0, i);
-        indexes[i] = new TermsEnumIndex(new SortedDocValuesTermsEnum(subs[i]), i);
+        indexes[i] = new TermsEnumIndex(subs[i], i);
       }
       MultiTermsEnum mte = new MultiTermsEnum(slices);
       mte.reset(indexes);
-      int globalOrd = 0;
+      long globalOrd = 0;
       while (mte.next() != null) {        
         TermsEnumWithSlice matches[] = mte.getMatchArray();
         for (int i = 0; i < mte.getMatchCount(); i++) {
           int subIndex = matches[i].index;
-          int delta = globalOrd - segmentOrds[subIndex];
-          assert delta >= 0;
+          long segmentOrd = matches[i].terms.ord();
+          long delta = globalOrd - segmentOrd;
           // for each unique term, just mark the first subindex/delta where it occurs
           if (i == 0) {
             subIndexes.add(subIndex);
             globalOrdDeltas.add(delta);
           }
           // for each per-segment ord, map it back to the global term.
-          ordDeltas[subIndex].add(delta);
-          segmentOrds[subIndex]++;
+          while (segmentOrds[subIndex] <= segmentOrd) {
+            ordDeltas[subIndex].add(delta);
+            segmentOrds[subIndex]++;
+          }
         }
         globalOrd++;
       }
     }
+    
+    public long getGlobalOrd(int subIndex, long segmentOrd) {
+      return segmentOrd + ordDeltas[subIndex].get(segmentOrd);
+    }
+
+    public long getSegmentOrd(int subIndex, long globalOrd) {
+      return globalOrd - globalOrdDeltas.get(globalOrd);
+    }
+    
+    public int getSegmentNumber(long globalOrd) {
+      return (int) subIndexes.get(globalOrd);
+    }
+    
+    public long getValueCount() {
+      return globalOrdDeltas.size();
+    }
   }
   
   /** implements SortedDocValues over n subs, using an OrdinalMap */
@@ -289,19 +356,78 @@ public class MultiDocValues {
     public int getOrd(int docID) {
       int subIndex = ReaderUtil.subIndex(docID, docStarts);
       int segmentOrd = values[subIndex].getOrd(docID - docStarts[subIndex]);
-      return (int) (segmentOrd + mapping.ordDeltas[subIndex].get(segmentOrd));
+      return (int) mapping.getGlobalOrd(subIndex, segmentOrd);
     }
  
     @Override
     public void lookupOrd(int ord, BytesRef result) {
-      int subIndex = (int) mapping.subIndexes.get(ord);
-      int segmentOrd = (int) (ord - mapping.globalOrdDeltas.get(ord));
+      int subIndex = mapping.getSegmentNumber(ord);
+      int segmentOrd = (int) mapping.getSegmentOrd(subIndex, ord);
       values[subIndex].lookupOrd(segmentOrd, result);
     }
  
     @Override
     public int getValueCount() {
-      return mapping.globalOrdDeltas.size();
+      return (int) mapping.getValueCount();
+    }
+  }
+  
+  /** implements MultiSortedDocValues over n subs, using an OrdinalMap */
+  static class MultiSortedSetDocValues extends SortedSetDocValues {
+    final int docStarts[];
+    final SortedSetDocValues values[];
+    final OrdinalMap mapping;
+    
+    MultiSortedSetDocValues(SortedSetDocValues values[], int docStarts[], OrdinalMap mapping) throws IOException {
+      assert values.length == mapping.ordDeltas.length;
+      assert docStarts.length == values.length + 1;
+      this.values = values;
+      this.docStarts = docStarts;
+      this.mapping = mapping;
+    }
+       
+    @Override
+    public OrdIterator getOrds(int docID, OrdIterator reuse) {
+      MultiOrdIterator iterator;
+      if (reuse instanceof MultiOrdIterator) {
+        iterator = (MultiOrdIterator) reuse;
+      } else {
+        iterator = new MultiOrdIterator();
+      }
+      iterator.reset(docID);
+      return iterator;
+    }
+ 
+    @Override
+    public void lookupOrd(long ord, BytesRef result) {
+      int subIndex = mapping.getSegmentNumber(ord);
+      long segmentOrd = mapping.getSegmentOrd(subIndex, ord);
+      values[subIndex].lookupOrd(segmentOrd, result);
+    }
+ 
+    @Override
+    public long getValueCount() {
+      return mapping.getValueCount();
+    }
+    
+    class MultiOrdIterator extends OrdIterator {
+      private OrdIterator inner;
+      private int subIndex;
+
+      @Override
+      public long nextOrd() {
+        long segmentOrd = inner.nextOrd();
+        if (segmentOrd == NO_MORE_ORDS) {
+          return NO_MORE_ORDS;
+        } else {
+          return mapping.getGlobalOrd(subIndex, segmentOrd);
+        }
+      }
+      
+      void reset(int docID) {
+        subIndex = ReaderUtil.subIndex(docID, docStarts);
+        inner = values[subIndex].getOrds(docID - docStarts[subIndex], inner);
+      }
     }
   }
 }

Modified: lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java?rev=1444647&r1=1444646&r2=1444647&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java (original)
+++ lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java Mon Feb 11 02:54:53 2013
@@ -497,7 +497,7 @@ public final class MultiTermsEnum extend
 
   final static class TermsEnumWithSlice {
     private final ReaderSlice subSlice;
-    private TermsEnum terms;
+    TermsEnum terms;
     public BytesRef current;
     final int index;
 

Modified: lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java?rev=1444647&r1=1444646&r2=1444647&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java (original)
+++ lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java Mon Feb 11 02:54:53 2013
@@ -50,7 +50,7 @@ class NumericDocValuesWriter extends Doc
     }
 
     // Fill in any holes:
-    for (int i = pending.size(); i < docID; ++i) {
+    for (int i = (int)pending.size(); i < docID; ++i) {
       pending.add(MISSING);
     }
 
@@ -90,7 +90,7 @@ class NumericDocValuesWriter extends Doc
   // iterates over the values we have in ram
   private class NumericIterator implements Iterator<Number> {
     final AppendingLongBuffer.Iterator iter = pending.iterator();
-    final int size = pending.size();
+    final int size = (int)pending.size();
     final int maxDoc;
     int upto;
     

Modified: lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java?rev=1444647&r1=1444646&r2=1444647&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java (original)
+++ lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java Mon Feb 11 02:54:53 2013
@@ -285,6 +285,13 @@ public final class ParallelAtomicReader 
   }
 
   @Override
+  public SortedSetDocValues getSortedSetDocValues(String field) throws IOException {
+    ensureOpen();
+    AtomicReader reader = fieldToReader.get(field);
+    return reader == null ? null : reader.getSortedSetDocValues(field);
+  }
+
+  @Override
   public NumericDocValues getNormValues(String field) throws IOException {
     ensureOpen();
     AtomicReader reader = fieldToReader.get(field);

Modified: lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java?rev=1444647&r1=1444646&r2=1444647&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java (original)
+++ lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java Mon Feb 11 02:54:53 2013
@@ -253,6 +253,34 @@ final class SegmentCoreReaders {
 
     return dvs;
   }
+  
+  SortedSetDocValues getSortedSetDocValues(String field) throws IOException {
+    FieldInfo fi = fieldInfos.fieldInfo(field);
+    if (fi == null) {
+      // Field does not exist
+      return null;
+    }
+    if (fi.getDocValuesType() == null) {
+      // Field was not indexed with doc values
+      return null;
+    }
+    if (fi.getDocValuesType() != DocValuesType.SORTED_SET) {
+      // DocValues were not sorted
+      return null;
+    }
+
+    assert dvProducer != null;
+
+    Map<String,Object> dvFields = docValuesLocal.get();
+
+    SortedSetDocValues dvs = (SortedSetDocValues) dvFields.get(field);
+    if (dvs == null) {
+      dvs = dvProducer.getSortedSet(fi);
+      dvFields.put(field, dvs);
+    }
+
+    return dvs;
+  }
 
   NumericDocValues getNormValues(String field) throws IOException {
     FieldInfo fi = fieldInfos.fieldInfo(field);

Modified: lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1444647&r1=1444646&r2=1444647&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java Mon Feb 11 02:54:53 2013
@@ -199,6 +199,16 @@ final class SegmentMerger {
                 toMerge.add(values);
               }
               consumer.mergeSortedField(field, mergeState, toMerge);
+            } else if (type == DocValuesType.SORTED_SET) {
+              List<SortedSetDocValues> toMerge = new ArrayList<SortedSetDocValues>();
+              for (AtomicReader reader : mergeState.readers) {
+                SortedSetDocValues values = reader.getSortedSetDocValues(field.name);
+                if (values == null) {
+                  values = SortedSetDocValues.EMPTY;
+                }
+                toMerge.add(values);
+              }
+              consumer.mergeSortedSetField(field, mergeState, toMerge);
             } else {
               throw new AssertionError("type=" + type);
             }

Modified: lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java?rev=1444647&r1=1444646&r2=1444647&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java Mon Feb 11 02:54:53 2013
@@ -248,6 +248,12 @@ public final class SegmentReader extends
   }
 
   @Override
+  public SortedSetDocValues getSortedSetDocValues(String field) throws IOException {
+    ensureOpen();
+    return core.getSortedSetDocValues(field);
+  }
+
+  @Override
   public NumericDocValues getNormValues(String field) throws IOException {
     ensureOpen();
     return core.getNormValues(field);

Modified: lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java?rev=1444647&r1=1444646&r2=1444647&view=diff
==============================================================================
--- lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java (original)
+++ lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java Mon Feb 11 02:54:53 2013
@@ -25,6 +25,7 @@ import org.apache.lucene.util.Bits;
 
 import org.apache.lucene.index.DirectoryReader; // javadoc
 import org.apache.lucene.index.MultiDocValues.MultiSortedDocValues;
+import org.apache.lucene.index.MultiDocValues.MultiSortedSetDocValues;
 import org.apache.lucene.index.MultiDocValues.OrdinalMap;
 import org.apache.lucene.index.MultiReader; // javadoc
 
@@ -131,6 +132,42 @@ public final class SlowCompositeReaderWr
     return new MultiSortedDocValues(values, starts, map);
   }
   
+  @Override
+  public SortedSetDocValues getSortedSetDocValues(String field) throws IOException {
+    ensureOpen();
+    OrdinalMap map = null;
+    synchronized (cachedOrdMaps) {
+      map = cachedOrdMaps.get(field);
+      if (map == null) {
+        // uncached, or not a multi dv
+        SortedSetDocValues dv = MultiDocValues.getSortedSetValues(in, field);
+        if (dv instanceof MultiSortedSetDocValues) {
+          map = ((MultiSortedSetDocValues)dv).mapping;
+          if (map.owner == getCoreCacheKey()) {
+            cachedOrdMaps.put(field, map);
+          }
+        }
+        return dv;
+      }
+    }
+    // cached multi dv
+    assert map != null;
+    int size = in.leaves().size();
+    final SortedSetDocValues[] values = new SortedSetDocValues[size];
+    final int[] starts = new int[size+1];
+    for (int i = 0; i < size; i++) {
+      AtomicReaderContext context = in.leaves().get(i);
+      SortedSetDocValues v = context.reader().getSortedSetDocValues(field);
+      if (v == null) {
+        v = SortedSetDocValues.EMPTY;
+      }
+      values[i] = v;
+      starts[i] = context.docBase;
+    }
+    starts[size] = maxDoc();
+    return new MultiSortedSetDocValues(values, starts, map);
+  }
+  
   // TODO: this could really be a weak map somewhere else on the coreCacheKey,
   // but do we really need to optimize slow-wrapper any more?
   private final Map<String,OrdinalMap> cachedOrdMaps = new HashMap<String,OrdinalMap>();

Added: lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValues.java?rev=1444647&view=auto
==============================================================================
--- lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValues.java (added)
+++ lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValues.java Mon Feb 11 02:54:53 2013
@@ -0,0 +1,120 @@
+package org.apache.lucene.index;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * A per-document set of presorted byte[] values.
+ * <p>
+ * Per-Document values in a SortedDocValues are deduplicated, dereferenced,
+ * and sorted into a dictionary of unique values. A pointer to the
+ * dictionary value (ordinal) can be retrieved for each document. Ordinals
+ * are dense and in increasing sorted order.
+ */
+public abstract class SortedSetDocValues {
+  
+  /** Sole constructor. (For invocation by subclass 
+   * constructors, typically implicit.) */
+  protected SortedSetDocValues() {}
+
+  /**
+   * Returns an iterator over the ordinals for the specified docID.
+   * @param  docID document ID to lookup
+   * @return iterator over ordinals for the document: these are dense, 
+   *         start at 0, then increment by 1 for the next value in sorted order. 
+   */
+  public abstract OrdIterator getOrds(int docID, OrdIterator reuse);
+
+  /** Retrieves the value for the specified ordinal.
+   * @param ord ordinal to lookup
+   * @param result will be populated with the ordinal's value
+   * @see #getOrds
+   */
+  public abstract void lookupOrd(long ord, BytesRef result);
+
+  /**
+   * Returns the number of unique values.
+   * @return number of unique values in this SortedDocValues. This is
+   *         also equivalent to one plus the maximum ordinal.
+   */
+  public abstract long getValueCount();
+
+
+  /** An empty SortedDocValues which returns {@link OrdIterator#EMPTY} for every document */
+  public static final SortedSetDocValues EMPTY = new SortedSetDocValues() {
+    @Override
+    public OrdIterator getOrds(int docID, OrdIterator reuse) {
+      return OrdIterator.EMPTY;
+    }
+
+    @Override
+    public void lookupOrd(long ord, BytesRef result) {
+      throw new IndexOutOfBoundsException();
+    }
+
+    @Override
+    public long getValueCount() {
+      return 0;
+    }
+  };
+
+  /** If {@code key} exists, returns its ordinal, else
+   *  returns {@code -insertionPoint-1}, like {@code
+   *  Arrays.binarySearch}.
+   *
+   *  @param key Key to look up
+   **/
+  public long lookupTerm(BytesRef key) {
+    BytesRef spare = new BytesRef();
+    long low = 0;
+    long high = getValueCount()-1;
+
+    while (low <= high) {
+      long mid = (low + high) >>> 1;
+      lookupOrd(mid, spare);
+      int cmp = spare.compareTo(key);
+
+      if (cmp < 0) {
+        low = mid + 1;
+      } else if (cmp > 0) {
+        high = mid - 1;
+      } else {
+        return mid; // key found
+      }
+    }
+
+    return -(low + 1);  // key not found.
+  }
+  
+  /** An iterator over the ordinals in a document (in increasing order) */
+  public static abstract class OrdIterator {
+    /** Indicates enumeration has ended: no more ordinals for this document */
+    public static final long NO_MORE_ORDS = Long.MAX_VALUE;
+    /** An iterator that always returns {@link #NO_MORE_ORDS} */
+    public static final OrdIterator EMPTY = new OrdIterator() {
+      @Override
+      public long nextOrd() {
+        return NO_MORE_ORDS;
+      }
+    };
+    
+    /** Returns next ordinal, or {@link #NO_MORE_ORDS} */
+    public abstract long nextOrd();
+  }
+}