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/20 05:29:44 UTC

svn commit: r1447999 [1/3] - in /lucene/dev/trunk: ./ dev-tools/ lucene/ lucene/codecs/ lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/ lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/ lucene/core/ lucene/core/src/java/org/apache/luc...

Author: rmuir
Date: Wed Feb 20 04:29:42 2013
New Revision: 1447999

URL: http://svn.apache.org/r1447999
Log:
LUCENE-4765: Multi-valued docvalues field

Added:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/SortedSetDocValuesField.java
      - copied unchanged from r1447981, lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/document/SortedSetDocValuesField.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SingletonSortedSetDocValues.java
      - copied unchanged from r1447981, lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/SingletonSortedSetDocValues.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValues.java
      - copied unchanged from r1447981, lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValues.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesTermsEnum.java
      - copied unchanged from r1447981, lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesTermsEnum.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesWriter.java
      - copied unchanged from r1447981, lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesWriter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/AbstractAppendingLongBuffer.java
      - copied unchanged from r1447981, lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/util/packed/AbstractAppendingLongBuffer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/MonotonicAppendingLongBuffer.java
      - copied unchanged from r1447981, lucene/dev/branches/lucene4765/lucene/core/src/java/org/apache/lucene/util/packed/MonotonicAppendingLongBuffer.java
Modified:
    lucene/dev/trunk/   (props changed)
    lucene/dev/trunk/dev-tools/   (props changed)
    lucene/dev/trunk/lucene/   (props changed)
    lucene/dev/trunk/lucene/CHANGES.txt   (contents, props changed)
    lucene/dev/trunk/lucene/codecs/   (props changed)
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesConsumer.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesFormat.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java
    lucene/dev/trunk/lucene/common-build.xml   (props changed)
    lucene/dev/trunk/lucene/core/   (props changed)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosWriter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/package.html
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocTermOrds.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocValuesProcessor.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/NumericDocValuesWriter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldCache.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/packed/AppendingLongBuffer.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDocTermOrds.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDocValuesIndexing.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestMultiDocValues.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFieldCache.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/packed/TestPackedInts.java
    lucene/dev/trunk/lucene/demo/   (props changed)
    lucene/dev/trunk/lucene/facet/   (props changed)
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/codecs/facet42/Facet42DocValuesConsumer.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/codecs/facet42/Facet42DocValuesProducer.java
    lucene/dev/trunk/lucene/grouping/   (props changed)
    lucene/dev/trunk/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java
    lucene/dev/trunk/lucene/join/   (props changed)
    lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/TermsCollector.java
    lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/TermsWithScoreCollector.java
    lucene/dev/trunk/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java
    lucene/dev/trunk/lucene/memory/   (props changed)
    lucene/dev/trunk/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
    lucene/dev/trunk/lucene/queries/   (props changed)
    lucene/dev/trunk/lucene/queries/src/test/org/apache/lucene/queries/function/TestDocValuesFieldSources.java
    lucene/dev/trunk/lucene/spatial/   (props changed)
    lucene/dev/trunk/lucene/test-framework/   (props changed)
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingDocValuesFormat.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardDocValuesProducer.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesWriter.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/AssertingAtomicReader.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
    lucene/dev/trunk/lucene/tools/   (props changed)
    lucene/dev/trunk/solr/   (props changed)
    lucene/dev/trunk/solr/CHANGES.txt   (props changed)
    lucene/dev/trunk/solr/core/   (props changed)
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestDocSet.java
    lucene/dev/trunk/solr/example/   (props changed)
    lucene/dev/trunk/solr/licenses/   (props changed)
    lucene/dev/trunk/solr/solrj/   (props changed)
    lucene/dev/trunk/solr/test-framework/   (props changed)

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1447999&r1=1447998&r2=1447999&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Wed Feb 20 04:29:42 2013
@@ -170,6 +170,12 @@ New Features
   
 * LUCENE-4778: Add a getter for the delegate in RateLimitedDirectoryWrapper.
   (Mark Miller)
+
+* LUCENE-4765: Add a multi-valued docvalues type (SORTED_SET). This is equivalent
+  to building a FieldCache.getDocTermOrds at index-time.  (Robert Muir)
+
+* LUCENE-4780: Add MonotonicAppendingLongBuffer: an append-only buffer for
+  monotonically increasing values.  (Adrien Grand)
  
 API Changes
 

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesConsumer.java?rev=1447999&r1=1447998&r2=1447999&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesConsumer.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesConsumer.java Wed Feb 20 04:29:42 2013
@@ -59,7 +59,7 @@ public class DiskDocValuesConsumer exten
   
   @Override
   public void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException {
-    int count = 0;
+    long count = 0;
     for (@SuppressWarnings("unused") Number nv : values) {
       ++count;
     }
@@ -68,7 +68,7 @@ public class DiskDocValuesConsumer exten
     meta.writeByte(DiskDocValuesFormat.NUMERIC);
     meta.writeVInt(PackedInts.VERSION_CURRENT);
     meta.writeLong(data.getFilePointer());
-    meta.writeVInt(count);
+    meta.writeVLong(count);
     meta.writeVInt(BLOCK_SIZE);
 
     final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE);
@@ -86,7 +86,7 @@ public class DiskDocValuesConsumer exten
     int minLength = Integer.MAX_VALUE;
     int maxLength = Integer.MIN_VALUE;
     final long startFP = data.getFilePointer();
-    int count = 0;
+    long count = 0;
     for(BytesRef v : values) {
       minLength = Math.min(minLength, v.length);
       maxLength = Math.max(maxLength, v.length);
@@ -95,7 +95,7 @@ public class DiskDocValuesConsumer exten
     }
     meta.writeVInt(minLength);
     meta.writeVInt(maxLength);
-    meta.writeVInt(count);
+    meta.writeVLong(count);
     meta.writeLong(startFP);
     
     // if minLength == maxLength, its a fixed-length byte[], we are done (the addresses are implicit)
@@ -124,6 +124,33 @@ public class DiskDocValuesConsumer exten
   }
   
   @Override
+  public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrdCount, Iterable<Number> ords) throws IOException {
+    meta.writeVInt(field.number);
+    meta.writeByte(DiskDocValuesFormat.SORTED_SET);
+    // write the ord -> byte[] as a binary field
+    addBinaryField(field, values);
+    // write the stream of ords as a numeric field
+    // NOTE: we could return an iterator that delta-encodes these within a doc
+    addNumericField(field, ords);
+    
+    // write the doc -> ord count as a absolute index to the stream
+    meta.writeVInt(field.number);
+    meta.writeByte(DiskDocValuesFormat.NUMERIC);
+    meta.writeVInt(PackedInts.VERSION_CURRENT);
+    meta.writeLong(data.getFilePointer());
+    meta.writeVLong(maxDoc);
+    meta.writeVInt(BLOCK_SIZE);
+
+    final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, BLOCK_SIZE);
+    long addr = 0;
+    for (Number v : docToOrdCount) {
+      addr += v.longValue();
+      writer.add(addr);
+    }
+    writer.finish();
+  }
+
+  @Override
   public void close() throws IOException {
     boolean success = false;
     try {

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java?rev=1447999&r1=1447998&r2=1447999&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java Wed Feb 20 04:29:42 2013
@@ -58,4 +58,5 @@ public final class DiskDocValuesFormat e
   public static final byte NUMERIC = 0;
   public static final byte BINARY = 1;
   public static final byte SORTED = 2;
+  public static final byte SORTED_SET = 3;
 }

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java?rev=1447999&r1=1447998&r2=1447999&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java Wed Feb 20 04:29:42 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;
@@ -41,11 +42,13 @@ class DiskDocValuesProducer extends DocV
   private final Map<Integer,NumericEntry> numerics;
   private final Map<Integer,BinaryEntry> binaries;
   private final Map<Integer,NumericEntry> ords;
+  private final Map<Integer,NumericEntry> ordIndexes;
   private final IndexInput data;
 
   // memory-resident structures
   private final Map<Integer,BlockPackedReader> ordinalInstances = new HashMap<Integer,BlockPackedReader>();
   private final Map<Integer,MonotonicBlockPackedReader> addressInstances = new HashMap<Integer,MonotonicBlockPackedReader>();
+  private final Map<Integer,MonotonicBlockPackedReader> ordIndexInstances = new HashMap<Integer,MonotonicBlockPackedReader>();
   
   DiskDocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
     String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
@@ -58,6 +61,7 @@ class DiskDocValuesProducer extends DocV
                                 DiskDocValuesFormat.VERSION_START);
       numerics = new HashMap<Integer,NumericEntry>();
       ords = new HashMap<Integer,NumericEntry>();
+      ordIndexes = new HashMap<Integer,NumericEntry>();
       binaries = new HashMap<Integer,BinaryEntry>();
       readFields(in, state.fieldInfos);
       success = true;
@@ -104,6 +108,36 @@ class DiskDocValuesProducer extends DocV
         }
         NumericEntry n = readNumericEntry(meta);
         ords.put(fieldNumber, n);
+      } else if (type == DiskDocValuesFormat.SORTED_SET) {
+        // sortedset = binary + numeric + ordIndex
+        if (meta.readVInt() != fieldNumber) {
+          throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt");
+        }
+        if (meta.readByte() != DiskDocValuesFormat.BINARY) {
+          throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt");
+        }
+        BinaryEntry b = readBinaryEntry(meta);
+        binaries.put(fieldNumber, b);
+        
+        if (meta.readVInt() != fieldNumber) {
+          throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt");
+        }
+        if (meta.readByte() != DiskDocValuesFormat.NUMERIC) {
+          throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt");
+        }
+        NumericEntry n1 = readNumericEntry(meta);
+        ords.put(fieldNumber, n1);
+        
+        if (meta.readVInt() != fieldNumber) {
+          throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt");
+        }
+        if (meta.readByte() != DiskDocValuesFormat.NUMERIC) {
+          throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt");
+        }
+        NumericEntry n2 = readNumericEntry(meta);
+        ordIndexes.put(fieldNumber, n2);
+      } else {
+        throw new CorruptIndexException("invalid type: " + type + ", resource=" + meta);
       }
       fieldNumber = meta.readVInt();
     }
@@ -113,7 +147,7 @@ class DiskDocValuesProducer extends DocV
     NumericEntry entry = new NumericEntry();
     entry.packedIntsVersion = meta.readVInt();
     entry.offset = meta.readLong();
-    entry.count = meta.readVInt();
+    entry.count = meta.readVLong();
     entry.blockSize = meta.readVInt();
     return entry;
   }
@@ -122,7 +156,7 @@ class DiskDocValuesProducer extends DocV
     BinaryEntry entry = new BinaryEntry();
     entry.minLength = meta.readVInt();
     entry.maxLength = meta.readVInt();
-    entry.count = meta.readVInt();
+    entry.count = meta.readVLong();
     entry.offset = meta.readLong();
     if (entry.minLength != entry.maxLength) {
       entry.addressesOffset = meta.readLong();
@@ -135,14 +169,18 @@ class DiskDocValuesProducer extends DocV
   @Override
   public NumericDocValues getNumeric(FieldInfo field) throws IOException {
     NumericEntry entry = numerics.get(field.number);
+    return getNumeric(entry);
+  }
+  
+  LongNumericDocValues getNumeric(NumericEntry entry) throws IOException {
     final IndexInput data = this.data.clone();
     data.seek(entry.offset);
 
     final BlockPackedReader reader = new BlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, entry.count, true);
-    return new NumericDocValues() {
+    return new LongNumericDocValues() {
       @Override
-      public long get(int docID) {
-        return reader.get(docID);
+      public long get(long id) {
+        return reader.get(id);
       }
     };
   }
@@ -160,10 +198,10 @@ class DiskDocValuesProducer extends DocV
   private BinaryDocValues getFixedBinary(FieldInfo field, final BinaryEntry bytes) {
     final IndexInput data = this.data.clone();
 
-    return new BinaryDocValues() {
+    return new LongBinaryDocValues() {
       @Override
-      public void get(int docID, BytesRef result) {
-        long address = bytes.offset + docID * (long)bytes.maxLength;
+      public void get(long id, BytesRef result) {
+        long address = bytes.offset + id * bytes.maxLength;
         try {
           data.seek(address);
           // NOTE: we could have one buffer, but various consumers (e.g. FieldComparatorSource) 
@@ -194,11 +232,11 @@ class DiskDocValuesProducer extends DocV
       addresses = addrInstance;
     }
 
-    return new BinaryDocValues() {
+    return new LongBinaryDocValues() {
       @Override
-      public void get(int docID, BytesRef result) {
-        long startAddress = bytes.offset + (docID == 0 ? 0 : + addresses.get(docID-1));
-        long endAddress = bytes.offset + addresses.get(docID);
+      public void get(long id, BytesRef result) {
+        long startAddress = bytes.offset + (id == 0 ? 0 : addresses.get(id-1));
+        long endAddress = bytes.offset + addresses.get(id);
         int length = (int) (endAddress - startAddress);
         try {
           data.seek(startAddress);
@@ -218,7 +256,7 @@ class DiskDocValuesProducer extends DocV
 
   @Override
   public SortedDocValues getSorted(FieldInfo field) throws IOException {
-    final int valueCount = binaries.get(field.number).count;
+    final int valueCount = (int) binaries.get(field.number).count;
     final BinaryDocValues binary = getBinary(field);
     final BlockPackedReader ordinals;
     synchronized (ordinalInstances) {
@@ -252,6 +290,59 @@ class DiskDocValuesProducer extends DocV
   }
 
   @Override
+  public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
+    final long valueCount = binaries.get(field.number).count;
+    // we keep the byte[]s and list of ords on disk, these could be large
+    final LongBinaryDocValues binary = (LongBinaryDocValues) getBinary(field);
+    final LongNumericDocValues ordinals = getNumeric(ords.get(field.number));
+    // but the addresses to the ord stream are in RAM
+    final MonotonicBlockPackedReader ordIndex;
+    synchronized (ordIndexInstances) {
+      MonotonicBlockPackedReader ordIndexInstance = ordIndexInstances.get(field.number);
+      if (ordIndexInstance == null) {
+        NumericEntry entry = ordIndexes.get(field.number);
+        IndexInput data = this.data.clone();
+        data.seek(entry.offset);
+        ordIndexInstance = new MonotonicBlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, entry.count, false);
+        ordIndexInstances.put(field.number, ordIndexInstance);
+      }
+      ordIndex = ordIndexInstance;
+    }
+    
+    return new SortedSetDocValues() {
+      long offset;
+      long endOffset;
+      
+      @Override
+      public long nextOrd() {
+        if (offset == endOffset) {
+          return NO_MORE_ORDS;
+        } else {
+          long ord = ordinals.get(offset);
+          offset++;
+          return ord;
+        }
+      }
+
+      @Override
+      public void setDocument(int docID) {
+        offset = (docID == 0 ? 0 : ordIndex.get(docID-1));
+        endOffset = ordIndex.get(docID);
+      }
+
+      @Override
+      public void lookupOrd(long ord, BytesRef result) {
+        binary.get(ord, result);
+      }
+
+      @Override
+      public long getValueCount() {
+        return valueCount;
+      }
+    };
+  }
+
+  @Override
   public void close() throws IOException {
     data.close();
   }
@@ -260,18 +351,37 @@ class DiskDocValuesProducer extends DocV
     long offset;
 
     int packedIntsVersion;
-    int count;
+    long count;
     int blockSize;
   }
   
   static class BinaryEntry {
     long offset;
 
-    int count;
+    long count;
     int minLength;
     int maxLength;
     long addressesOffset;
     int packedIntsVersion;
     int blockSize;
   }
+  
+  // internally we compose complex dv (sorted/sortedset) from other ones
+  static abstract class LongNumericDocValues extends NumericDocValues {
+    @Override
+    public final long get(int docID) {
+      return get((long) docID);
+    }
+    
+    abstract long get(long id);
+  }
+  
+  static abstract class LongBinaryDocValues extends BinaryDocValues {
+    @Override
+    public final void get(int docID, BytesRef result) {
+      get((long)docID, result);
+    }
+    
+    abstract void get(long id, BytesRef Result);
+  }
 }

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesFormat.java?rev=1447999&r1=1447998&r2=1447999&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesFormat.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesFormat.java Wed Feb 20 04:29:42 2013
@@ -82,6 +82,31 @@ import org.apache.lucene.index.SegmentWr
  *  so the "ord section" begins at startOffset + (9+pattern.length+maxlength)*numValues.
  *  a document's ord can be retrieved by seeking to "ord section" + (1+ordpattern.length())*docid
  *  an ord's value can be retrieved by seeking to startOffset + (9+pattern.length+maxlength)*ord
+ *  
+ *  for sorted set this is a fixed-width file very similar to the SORTED case, for example:
+ *  <pre>
+ *  field myField
+ *    type SORTED_SET
+ *    numvalues 10
+ *    maxLength 8
+ *    pattern 0
+ *    ordpattern XXXXX
+ *  length 6
+ *  foobar[space][space]
+ *  length 3
+ *  baz[space][space][space][space][space]
+ *  ...
+ *  0,3,5   
+ *  1,2
+ *  
+ *  10
+ *  ...
+ *  </pre>
+ *  so the "ord section" begins at startOffset + (9+pattern.length+maxlength)*numValues.
+ *  a document's ord list can be retrieved by seeking to "ord section" + (1+ordpattern.length())*docid
+ *  this is a comma-separated list, and its padded with spaces to be fixed width. so trim() and split() it.
+ *  and beware the empty string!
+ *  an ord's value can be retrieved by seeking to startOffset + (9+pattern.length+maxlength)*ord
  *   
  *  the reader can just scan this file when it opens, skipping over the data blocks
  *  and saving the offset/etc for each field. 

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java?rev=1447999&r1=1447998&r2=1447999&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java Wed Feb 20 04:29:42 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;
@@ -59,7 +60,7 @@ class SimpleTextDocValuesReader extends 
     int maxLength;
     boolean fixedLength;
     long minValue;
-    int numValues;
+    long numValues;
   };
 
   final int maxDoc;
@@ -109,10 +110,10 @@ class SimpleTextDocValuesReader extends 
         field.pattern = stripPrefix(PATTERN);
         field.dataStartFilePointer = data.getFilePointer();
         data.seek(data.getFilePointer() + (9+field.pattern.length()+field.maxLength) * maxDoc);
-      } else if (dvType == DocValuesType.SORTED) {
+      } else if (dvType == DocValuesType.SORTED || dvType == DocValuesType.SORTED_SET) {
         readLine();
         assert startsWith(NUMVALUES);
-        field.numValues = Integer.parseInt(stripPrefix(NUMVALUES));
+        field.numValues = Long.parseLong(stripPrefix(NUMVALUES));
         readLine();
         assert startsWith(MAXLENGTH);
         field.maxLength = Integer.parseInt(stripPrefix(MAXLENGTH));
@@ -279,6 +280,84 @@ class SimpleTextDocValuesReader extends 
 
       @Override
       public int getValueCount() {
+        return (int)field.numValues;
+      }
+    };
+  }
+
+  @Override
+  public SortedSetDocValues getSortedSet(FieldInfo fieldInfo) throws IOException {
+    final OneField field = fields.get(fieldInfo.name);
+
+    // SegmentCoreReaders already verifies this field is
+    // valid:
+    assert field != null;
+
+    final IndexInput in = data.clone();
+    final BytesRef scratch = new BytesRef();
+    final DecimalFormat decoder = new DecimalFormat(field.pattern, new DecimalFormatSymbols(Locale.ROOT));
+    
+    return new SortedSetDocValues() {
+      String[] currentOrds = new String[0];
+      int currentIndex = 0;
+      
+      @Override
+      public long nextOrd() {
+        if (currentIndex == currentOrds.length) {
+          return NO_MORE_ORDS;
+        } else {
+          return Long.parseLong(currentOrds[currentIndex++]);
+        }
+      }
+
+      @Override
+      public void setDocument(int docID) {
+        if (docID < 0 || docID >= maxDoc) {
+          throw new IndexOutOfBoundsException("docID must be 0 .. " + (maxDoc-1) + "; got " + docID);
+        }
+        try {
+          in.seek(field.dataStartFilePointer + field.numValues * (9 + field.pattern.length() + field.maxLength) + docID * (1 + field.ordPattern.length()));
+          SimpleTextUtil.readLine(in, scratch);
+          String ordList = scratch.utf8ToString().trim();
+          if (ordList.isEmpty()) {
+            currentOrds = new String[0];
+          } else {
+            currentOrds = ordList.split(",");
+          }
+          currentIndex = 0;
+        } catch (IOException ioe) {
+          throw new RuntimeException(ioe);
+        }
+      }
+
+      @Override
+      public void lookupOrd(long ord, BytesRef result) {
+        try {
+          if (ord < 0 || ord >= field.numValues) {
+            throw new IndexOutOfBoundsException("ord must be 0 .. " + (field.numValues-1) + "; got " + ord);
+          }
+          in.seek(field.dataStartFilePointer + ord * (9 + field.pattern.length() + field.maxLength));
+          SimpleTextUtil.readLine(in, scratch);
+          assert StringHelper.startsWith(scratch, LENGTH): "got " + scratch.utf8ToString() + " in=" + in;
+          int len;
+          try {
+            len = decoder.parse(new String(scratch.bytes, scratch.offset + LENGTH.length, scratch.length - LENGTH.length, "UTF-8")).intValue();
+          } catch (ParseException pe) {
+            CorruptIndexException e = new CorruptIndexException("failed to parse int length");
+            e.initCause(pe);
+            throw e;
+          }
+          result.bytes = new byte[len];
+          result.offset = 0;
+          result.length = len;
+          in.readBytes(result.bytes, 0, len);
+        } catch (IOException ioe) {
+          throw new RuntimeException(ioe);
+        }
+      }
+
+      @Override
+      public long getValueCount() {
         return field.numValues;
       }
     };

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java?rev=1447999&r1=1447998&r2=1447999&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java Wed Feb 20 04:29:42 2013
@@ -22,6 +22,7 @@ import java.math.BigInteger;
 import java.text.DecimalFormat;
 import java.text.DecimalFormatSymbols;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.Locale;
 import java.util.Set;
 
@@ -250,6 +251,115 @@ class SimpleTextDocValuesWriter extends 
     }
   }
 
+  @Override
+  public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrdCount, Iterable<Number> ords) throws IOException {
+    assert fieldSeen(field.name);
+    assert field.getDocValuesType() == DocValuesType.SORTED_SET;
+    writeFieldEntry(field, FieldInfo.DocValuesType.SORTED_SET);
+
+    long valueCount = 0;
+    int maxLength = 0;
+    for(BytesRef value : values) {
+      maxLength = Math.max(maxLength, value.length);
+      valueCount++;
+    }
+
+    // write numValues
+    SimpleTextUtil.write(data, NUMVALUES);
+    SimpleTextUtil.write(data, Long.toString(valueCount), scratch);
+    SimpleTextUtil.writeNewline(data);
+    
+    // write maxLength
+    SimpleTextUtil.write(data, MAXLENGTH);
+    SimpleTextUtil.write(data, Integer.toString(maxLength), scratch);
+    SimpleTextUtil.writeNewline(data);
+    
+    int maxBytesLength = Integer.toString(maxLength).length();
+    StringBuilder sb = new StringBuilder();
+    for (int i = 0; i < maxBytesLength; i++) {
+      sb.append('0');
+    }
+    
+    // write our pattern for encoding lengths
+    SimpleTextUtil.write(data, PATTERN);
+    SimpleTextUtil.write(data, sb.toString(), scratch);
+    SimpleTextUtil.writeNewline(data);
+    final DecimalFormat encoder = new DecimalFormat(sb.toString(), new DecimalFormatSymbols(Locale.ROOT));
+    
+    // compute ord pattern: this is funny, we encode all values for all docs to find the maximum length
+    int maxOrdListLength = 0;
+    StringBuilder sb2 = new StringBuilder();
+    Iterator<Number> ordStream = ords.iterator();
+    for (Number n : docToOrdCount) {
+      sb2.setLength(0);
+      int count = n.intValue();
+      for (int i = 0; i < count; i++) {
+        long ord = ordStream.next().longValue();
+        if (sb2.length() > 0) {
+          sb2.append(",");
+        }
+        sb2.append(Long.toString(ord));
+      }
+      maxOrdListLength = Math.max(maxOrdListLength, sb2.length());
+    }
+     
+    sb2.setLength(0);
+    for (int i = 0; i < maxOrdListLength; i++) {
+      sb2.append('X');
+    }
+    
+    // write our pattern for ord lists
+    SimpleTextUtil.write(data, ORDPATTERN);
+    SimpleTextUtil.write(data, sb2.toString(), scratch);
+    SimpleTextUtil.writeNewline(data);
+    
+    // for asserts:
+    long valuesSeen = 0;
+
+    for(BytesRef value : values) {
+      // write length
+      SimpleTextUtil.write(data, LENGTH);
+      SimpleTextUtil.write(data, encoder.format(value.length), scratch);
+      SimpleTextUtil.writeNewline(data);
+        
+      // write bytes -- don't use SimpleText.write
+      // because it escapes:
+      data.writeBytes(value.bytes, value.offset, value.length);
+
+      // pad to fit
+      for (int i = value.length; i < maxLength; i++) {
+        data.writeByte((byte)' ');
+      }
+      SimpleTextUtil.writeNewline(data);
+      valuesSeen++;
+      assert valuesSeen <= valueCount;
+    }
+
+    assert valuesSeen == valueCount;
+
+    ordStream = ords.iterator();
+    
+    // write the ords for each doc comma-separated
+    for(Number n : docToOrdCount) {
+      sb2.setLength(0);
+      int count = n.intValue();
+      for (int i = 0; i < count; i++) {
+        long ord = ordStream.next().longValue();
+        if (sb2.length() > 0) {
+          sb2.append(",");
+        }
+        sb2.append(Long.toString(ord));
+      }
+      // now pad to fit: these are numbers so spaces work well. reader calls trim()
+      int numPadding = maxOrdListLength - sb2.length();
+      for (int i = 0; i < numPadding; i++) {
+        sb2.append(' ');
+      }
+      SimpleTextUtil.write(data, sb2.toString(), scratch);
+      SimpleTextUtil.writeNewline(data);
+    }
+  }
+
   /** write the header for this field */
   private void writeFieldEntry(FieldInfo field, FieldInfo.DocValuesType type) throws IOException {
     SimpleTextUtil.write(data, FIELD);

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java?rev=1447999&r1=1447998&r2=1447999&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesConsumer.java Wed Feb 20 04:29:42 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.SortedSetDocValuesTermsEnum;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.util.ArrayUtil;
 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;
+import org.apache.lucene.util.OpenBitSet;
 
 /** 
  * 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,378 @@ 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 {
+        OpenBitSet bitset = new OpenBitSet(dv.getValueCount());
+        for (int i = 0; i < reader.maxDoc(); i++) {
+          if (liveDocs.get(i)) {
+            bitset.set(dv.getOrd(i));
           }
         }
-
-        return null;
+        liveTerms[sub] = new BitsFilteredTermsEnum(new SortedDocValuesTermsEnum(dv), bitset);
       }
     }
-
-    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;
-      }
-    }
-
-    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 {
+        OpenBitSet bitset = new OpenBitSet(dv.getValueCount());
+        for (int i = 0; i < reader.maxDoc(); i++) {
+          if (liveDocs.get(i)) {
+            dv.setDocument(i);
+            long ord;
+            while ((ord = dv.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
+              bitset.set(ord);
+            }
+          }
+        }
+        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;
+              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;
+                    SortedSetDocValues dv = dvs[readerUpto];
+                    dv.setDocument(docIDUpto);
+                    nextValue = 0;
+                    while (dv.nextOrd() != SortedSetDocValues.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;
+              Bits currentLiveDocs;
+              boolean nextIsSet;
+              long ords[] = new long[8];
+              int ordUpto;
+              int ordLength;
+
+              @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 (ordUpto < ordLength) {
+                    nextValue = ords[ordUpto];
+                    ordUpto++;
+                    nextIsSet = true;
+                    return true;
+                  }
+
+                  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)) {
+                    assert docIDUpto < currentReader.maxDoc();
+                    SortedSetDocValues dv = dvs[readerUpto];
+                    dv.setDocument(docIDUpto);
+                    ordUpto = ordLength = 0;
+                    long ord;
+                    while ((ord = dv.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
+                      if (ordLength == ords.length) {
+                        ords = ArrayUtil.grow(ords, ordLength+1);
                       }
-                    });
+                      ords[ordLength] = map.getGlobalOrd(readerUpto, ord);
+                      ordLength++;
+                    }
+                    docIDUpto++;
+                    continue;
+                  }
+
+                  docIDUpto++;
+                }
+              }
+            };
+          }
+        }
+     );
+  }
+  
+  // TODO: seek-by-ord to nextSetBit
+  static class BitsFilteredTermsEnum extends FilteredTermsEnum {
+    final OpenBitSet liveTerms;
+    
+    BitsFilteredTermsEnum(TermsEnum in, OpenBitSet 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(ord())) {
+        return AcceptStatus.YES;
+      } else {
+        return AcceptStatus.NO;
+      }
+    }
   }
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java?rev=1447999&r1=1447998&r2=1447999&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/DocValuesProducer.java Wed Feb 20 04:29:42 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/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java?rev=1447999&r1=1447998&r2=1447999&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesReader.java Wed Feb 20 04:29:42 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/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html?rev=1447999&r1=1447998&r2=1447999&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html Wed Feb 20 04:29:42 2013
@@ -375,7 +375,8 @@ can optionally be indexed into the posti
 term vectors.</li>
 <li>In version 4.1, the format of the postings list changed to use either
 of FOR compression or variable-byte encoding, depending upon the frequency
-of the term.</li>
+of the term. Terms appearing only once were changed to inline directly into
+the term dictionary. Stored fields are compressed by default. </li>
 </ul>
 <a name="Limitations" id="Limitations"></a>
 <h2>Limitations</h2>

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java?rev=1447999&r1=1447998&r2=1447999&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java Wed Feb 20 04:29:42 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());
@@ -214,7 +213,94 @@ class Lucene42DocValuesConsumer extends 
       ord++;
     }
     FST<Long> fst = builder.finish();
-    fst.save(data);
-    meta.writeVInt((int)ord);
+    if (fst != null) {
+      fst.save(data);
+    }
+    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/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java?rev=1447999&r1=1447998&r2=1447999&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java Wed Feb 20 04:29:42 2013
@@ -34,7 +34,7 @@ import org.apache.lucene.util.packed.Blo
 /**
  * Lucene 4.2 DocValues format.
  * <p>
- * Encodes the three per-document value types (Numeric,Binary,Sorted) with five basic strategies.
+ * Encodes the four per-document value types (Numeric,Binary,Sorted,SortedSet) with seven basic strategies.
  * <p>
  * <ul>
  *    <li>Delta-compressed Numerics: per-document integers written in blocks of 4096. For each block
@@ -51,7 +51,9 @@ import org.apache.lucene.util.packed.Blo
  *        start for the block, and the average (expected) delta per entry. For each document the 
  *        deviation from the delta (actual - expected) is written.
  *    <li>Sorted: an FST mapping deduplicated terms to ordinals is written, along with the per-document
- *        ordinals written using one of the numeric stratgies above.
+ *        ordinals written using one of the numeric strategies above.
+ *    <li>SortedSet: an FST mapping deduplicated terms to ordinals is written, along with the per-document
+ *        ordinal list written using one of the binary strategies above.  
  * </ul>
  * <p>
  * Files:
@@ -77,6 +79,8 @@ import org.apache.lucene.util.packed.Blo
  *   </ul>
  *   <p>Sorted fields have two entries: a SortedEntry with the FST metadata,
  *      and an ordinary NumericEntry for the document-to-ord metadata.</p>
+ *   <p>SortedSet fields have two entries: a SortedEntry with the FST metadata,
+ *      and an ordinary BinaryEntry for the document-to-ord-list metadata.</p>
  *   <p>FieldNumber of -1 indicates the end of metadata.</p>
  *   <p>EntryType is a 0 (NumericEntry), 1 (BinaryEntry, or 2 (SortedEntry)</p>
  *   <p>DataOffset is the pointer to the start of the data in the DocValues data (.dvd)</p>
@@ -107,6 +111,8 @@ import org.apache.lucene.util.packed.Blo
  *     <li>UncompressedNumerics --&gt; {@link DataOutput#writeByte Byte}<sup>maxdoc</sup></li>
  *     <li>Addresses --&gt; {@link MonotonicBlockPackedWriter MonotonicBlockPackedInts(blockSize=4096)}</li>
  *   </ul>
+ *   <p>SortedSet entries store the list of ordinals in their BinaryData as a
+ *      sequences of increasing {@link DataOutput#writeVLong vLong}s, delta-encoded.</p>       
  * </ol>
  */
 public final class Lucene42DocValuesFormat extends DocValuesFormat {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java?rev=1447999&r1=1447998&r2=1447999&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java Wed Feb 20 04:29:42 2013
@@ -31,6 +31,8 @@ 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.ByteArrayDataInput;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
@@ -123,7 +125,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,12 +283,96 @@ class Lucene42DocValuesProducer extends 
 
       @Override
       public int getValueCount() {
-        return entry.numOrds;
+        return (int)entry.numOrds;
       }
     };
   }
   
   @Override
+  public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
+    final FSTEntry entry = fsts.get(field.number);
+    if (entry.numOrds == 0) {
+      return SortedSetDocValues.EMPTY; // empty FST!
+    }
+    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); 
+    final BytesRef ref = new BytesRef();
+    final ByteArrayDataInput input = new ByteArrayDataInput();
+    return new SortedSetDocValues() {
+      long currentOrd;
+
+      @Override
+      public long nextOrd() {
+        if (input.eof()) {
+          return NO_MORE_ORDS;
+        } else {
+          currentOrd += input.readVLong();
+          return currentOrd;
+        }
+      }
+      
+      @Override
+      public void setDocument(int docID) {
+        docToOrds.get(docID, ref);
+        input.reset(ref.bytes, ref.offset, ref.length);
+        currentOrd = 0;
+      }
+
+      @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;
+      }
+    };
+  }
+
+  @Override
   public void close() throws IOException {
     data.close();
   }
@@ -308,6 +394,6 @@ class Lucene42DocValuesProducer extends 
   
   static class FSTEntry {
     long offset;
-    int numOrds;
+    long numOrds;
   }
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosReader.java?rev=1447999&r1=1447998&r2=1447999&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosReader.java Wed Feb 20 04:29:42 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/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosWriter.java?rev=1447999&r1=1447998&r2=1447999&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosWriter.java Wed Feb 20 04:29:42 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/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/package.html?rev=1447999&r1=1447998&r2=1447999&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/package.html (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene42/package.html Wed Feb 20 04:29:42 2013
@@ -375,7 +375,11 @@ can optionally be indexed into the posti
 term vectors.</li>
 <li>In version 4.1, the format of the postings list changed to use either
 of FOR compression or variable-byte encoding, depending upon the frequency
-of the term.</li>
+of the term. Terms appearing only once were changed to inline directly into
+the term dictionary. Stored fields are compressed by default. </li>
+<li>In version 4.2, term vectors are compressed by default. DocValues has 
+a new multi-valued type (SortedSet), that can be used for faceting/grouping/joining
+on multi-valued fields.</li>
 </ul>
 <a name="Limitations" id="Limitations"></a>
 <h2>Limitations</h2>

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java?rev=1447999&r1=1447998&r2=1447999&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java Wed Feb 20 04:29:42 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());
     }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java?rev=1447999&r1=1447998&r2=1447999&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java Wed Feb 20 04:29:42 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/trunk/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java?rev=1447999&r1=1447998&r2=1447999&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java Wed Feb 20 04:29:42 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/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=1447999&r1=1447998&r2=1447999&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 Wed Feb 20 04:29:42 2013
@@ -42,6 +42,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.OpenBitSet;
 import org.apache.lucene.util.StringHelper;
 
 /**
@@ -1275,7 +1276,8 @@ public class CheckIndex {
         } else {
           if (reader.getBinaryDocValues(fieldInfo.name) != null ||
               reader.getNumericDocValues(fieldInfo.name) != null ||
-              reader.getSortedDocValues(fieldInfo.name) != null) {
+              reader.getSortedDocValues(fieldInfo.name) != null || 
+              reader.getSortedSetDocValues(fieldInfo.name) != null) {
             throw new RuntimeException("field: " + fieldInfo.name + " has docvalues but should omit them!");
           }
         }
@@ -1333,6 +1335,47 @@ public class CheckIndex {
     }
   }
   
+  private static void checkSortedSetDocValues(String fieldName, AtomicReader reader, SortedSetDocValues dv) {
+    final long maxOrd = dv.getValueCount()-1;
+    OpenBitSet seenOrds = new OpenBitSet(dv.getValueCount());
+    long maxOrd2 = -1;
+    for (int i = 0; i < reader.maxDoc(); i++) {
+      dv.setDocument(i);
+      long lastOrd = -1;
+      long ord;
+      while ((ord = dv.nextOrd()) != SortedSetDocValues.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);
+        seenOrds.set(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);
@@ -1343,12 +1386,35 @@ public class CheckIndex {
     switch(fi.getDocValuesType()) {
       case SORTED:
         checkSortedDocValues(fi.name, reader, reader.getSortedDocValues(fi.name));
+        if (reader.getBinaryDocValues(fi.name) != null ||
+            reader.getNumericDocValues(fi.name) != null ||
+            reader.getSortedSetDocValues(fi.name) != null) {
+          throw new RuntimeException(fi.name + " returns multiple docvalues types!");
+        }
+        break;
+      case SORTED_SET:
+        checkSortedSetDocValues(fi.name, reader, reader.getSortedSetDocValues(fi.name));
+        if (reader.getBinaryDocValues(fi.name) != null ||
+            reader.getNumericDocValues(fi.name) != null ||
+            reader.getSortedDocValues(fi.name) != null) {
+          throw new RuntimeException(fi.name + " returns multiple docvalues types!");
+        }
         break;
       case BINARY:
         checkBinaryDocValues(fi.name, reader, reader.getBinaryDocValues(fi.name));
+        if (reader.getNumericDocValues(fi.name) != null ||
+            reader.getSortedDocValues(fi.name) != null ||
+            reader.getSortedSetDocValues(fi.name) != null) {
+          throw new RuntimeException(fi.name + " returns multiple docvalues types!");
+        }
         break;
       case NUMERIC:
         checkNumericDocValues(fi.name, reader, reader.getNumericDocValues(fi.name));
+        if (reader.getBinaryDocValues(fi.name) != null ||
+            reader.getSortedDocValues(fi.name) != null ||
+            reader.getSortedSetDocValues(fi.name) != null) {
+          throw new RuntimeException(fi.name + " returns multiple docvalues types!");
+        }
         break;
       default:
         throw new AssertionError();