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/08 04:26:21 UTC

svn commit: r1443834 [12/16] - in /lucene/dev/branches/branch_4x: ./ dev-tools/ lucene/ lucene/analysis/ lucene/analysis/icu/src/java/org/apache/lucene/collation/ lucene/analysis/icu/src/test/org/apache/lucene/collation/ lucene/backwards/ lucene/benchm...

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationsIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationsIterator.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationsIterator.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationsIterator.java Fri Feb  8 03:26:14 2013
@@ -3,8 +3,7 @@ package org.apache.lucene.facet.associat
 import java.io.IOException;
 
 import org.apache.lucene.index.AtomicReaderContext;
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.DocValues.Source;
+import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.util.BytesRef;
 
@@ -34,25 +33,19 @@ public abstract class AssociationsIterat
 
   private final T association;
   private final String dvField;
-  private final boolean useDirectSource;
   private final BytesRef bytes = new BytesRef(32);
   
-  private DocValues.Source current;
+  private BinaryDocValues current;
   
   /**
    * Construct a new associations iterator. The given
    * {@link CategoryAssociation} is used to deserialize the association values.
    * It is assumed that all association values can be deserialized with the
    * given {@link CategoryAssociation}.
-   * 
-   * <p>
-   * <b>NOTE:</b> if {@code useDirectSource} is {@code false}, then a
-   * {@link DocValues#getSource()} is used, which is an in-memory {@link Source}.
    */
-  public AssociationsIterator(String field, T association, boolean useDirectSource) throws IOException {
+  public AssociationsIterator(String field, T association) throws IOException {
     this.association = association;
     this.dvField = field + association.getCategoryListID();
-    this.useDirectSource = useDirectSource;
   }
 
   /**
@@ -61,14 +54,8 @@ public abstract class AssociationsIterat
    * of the documents belonging to the association given to the constructor.
    */
   public final boolean setNextReader(AtomicReaderContext context) throws IOException {
-    DocValues dv = context.reader().docValues(dvField);
-    if (dv == null) {
-      current = null;
-      return false;
-    }
-    
-    current = useDirectSource ? dv.getDirectSource() : dv.getSource();
-    return true;
+    current = context.reader().getBinaryDocValues(dvField);
+    return current != null;
   }
   
   /**
@@ -78,7 +65,7 @@ public abstract class AssociationsIterat
    * extending classes.
    */
   protected final boolean setNextDoc(int docID) throws IOException {
-    current.getBytes(docID, bytes);
+    current.get(docID, bytes);
     if (bytes.length == 0) {
       return false; // no associations for the requested document
     }

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/associations/FloatAssociationsIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/associations/FloatAssociationsIterator.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/associations/FloatAssociationsIterator.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/associations/FloatAssociationsIterator.java Fri Feb  8 03:26:14 2013
@@ -2,8 +2,6 @@ package org.apache.lucene.facet.associat
 
 import java.io.IOException;
 
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.DocValues.Source;
 import org.apache.lucene.util.collections.IntToFloatMap;
 
 /*
@@ -31,22 +29,12 @@ import org.apache.lucene.util.collection
 public class FloatAssociationsIterator extends AssociationsIterator<CategoryFloatAssociation> {
 
   private final IntToFloatMap ordinalAssociations = new IntToFloatMap();
-
-  /**
-   * Constructs a new {@link FloatAssociationsIterator} which uses an
-   * in-memory {@link DocValues#getSource() DocValues source}.
-   */
-  public FloatAssociationsIterator(String field, CategoryFloatAssociation association) throws IOException {
-    this(field, association, false);
-  }
   
   /**
-   * Constructs a new {@link FloatAssociationsIterator} which uses a
-   * {@link DocValues} {@link Source} per {@code useDirectSource}.
+   * Constructs a new {@link FloatAssociationsIterator}.
    */
-  public FloatAssociationsIterator(String field, CategoryFloatAssociation association, boolean useDirectSource) 
-      throws IOException {
-    super(field, association, useDirectSource);
+  public FloatAssociationsIterator(String field, CategoryFloatAssociation association) throws IOException {
+    super(field, association);
   }
 
   @Override

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/associations/IntAssociationsIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/associations/IntAssociationsIterator.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/associations/IntAssociationsIterator.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/associations/IntAssociationsIterator.java Fri Feb  8 03:26:14 2013
@@ -2,8 +2,6 @@ package org.apache.lucene.facet.associat
 
 import java.io.IOException;
 
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.DocValues.Source;
 import org.apache.lucene.util.collections.IntToIntMap;
 
 /*
@@ -33,20 +31,10 @@ public class IntAssociationsIterator ext
   private final IntToIntMap ordinalAssociations = new IntToIntMap();
 
   /**
-   * Constructs a new {@link IntAssociationsIterator} which uses an
-   * in-memory {@link DocValues#getSource() DocValues source}.
+   * Constructs a new {@link IntAssociationsIterator}.
    */
   public IntAssociationsIterator(String field, CategoryIntAssociation association) throws IOException {
-    this(field, association, false);
-  }
-
-  /**
-   * Constructs a new {@link IntAssociationsIterator} which uses a
-   * {@link DocValues} {@link Source} per {@code useDirectSource}.
-   */
-  public IntAssociationsIterator(String field, CategoryIntAssociation association, boolean useDirectSource)
-      throws IOException {
-    super(field, association, useDirectSource);
+    super(field, association);
   }
 
   @Override

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/index/FacetFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/index/FacetFields.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/index/FacetFields.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/index/FacetFields.java Fri Feb  8 03:26:14 2013
@@ -8,10 +8,10 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
+import org.apache.lucene.document.BinaryDocValuesField;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
-import org.apache.lucene.document.StraightBytesDocValuesField;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.facet.index.params.CategoryListParams;
 import org.apache.lucene.facet.index.params.FacetIndexingParams;
@@ -154,7 +154,7 @@ public class FacetFields {
    */
   protected void addCountingListData(Document doc, Map<String,BytesRef> categoriesData, String field) {
     for (Entry<String,BytesRef> entry : categoriesData.entrySet()) {
-      doc.add(new StraightBytesDocValuesField(field + entry.getKey(), entry.getValue()));
+      doc.add(new BinaryDocValuesField(field + entry.getKey(), entry.getValue()));
     }
   }
   

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/index/FacetsPayloadMigrationReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/index/FacetsPayloadMigrationReader.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/index/FacetsPayloadMigrationReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/index/FacetsPayloadMigrationReader.java Fri Feb  8 03:26:14 2013
@@ -27,11 +27,10 @@ import org.apache.lucene.facet.index.par
 import org.apache.lucene.facet.index.params.FacetIndexingParams;
 import org.apache.lucene.index.AtomicReader;
 import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.DocValues.Source;
-import org.apache.lucene.index.DocValues.Type;
 import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.FieldInfo.DocValuesType;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.Fields;
@@ -45,9 +44,9 @@ import org.apache.lucene.util.BytesRef;
 
 /**
  * A {@link FilterAtomicReader} for migrating a facets index which encodes
- * category ordinals in a payload to {@link DocValues}. To migrate the index,
+ * category ordinals in a payload to {@link BinaryDocValues}. To migrate the index,
  * you should build a mapping from a field (String) to term ({@link Term}),
- * which denotes under which DocValues field to put the data encoded in the
+ * which denotes under which BinaryDocValues field to put the data encoded in the
  * matching term's payload. You can follow the code example below to migrate an
  * existing index:
  * 
@@ -76,39 +75,37 @@ import org.apache.lucene.util.BytesRef;
  */
 public class FacetsPayloadMigrationReader extends FilterAtomicReader {  
 
-  private class PayloadMigratingDocValues extends DocValues {
+  private class PayloadMigratingBinaryDocValues extends BinaryDocValues {
 
-    private final DocsAndPositionsEnum dpe;
-    
-    public PayloadMigratingDocValues(DocsAndPositionsEnum dpe) {
-      this.dpe = dpe;
-    }
-
-    @Override
-    protected Source loadDirectSource() throws IOException {
-      return new PayloadMigratingSource(getType(), dpe);
-    }
-
-    @Override
-    protected Source loadSource() throws IOException {
-      throw new UnsupportedOperationException("in-memory Source is not supported by this reader");
-    }
+    private Fields fields;
+    private Term term;
+    private DocsAndPositionsEnum dpe;
+    private int curDocID = -1;
+    private int lastRequestedDocID;
 
-    @Override
-    public Type getType() {
-      return Type.BYTES_VAR_STRAIGHT;
+    private DocsAndPositionsEnum getDPE() {
+      try {
+        DocsAndPositionsEnum dpe = null;
+        if (fields != null) {
+          Terms terms = fields.terms(term.field());
+          if (terms != null) {
+            TermsEnum te = terms.iterator(null); // no use for reusing
+            if (te.seekExact(term.bytes(), true)) {
+              // we're not expected to be called for deleted documents
+              dpe = te.docsAndPositions(null, null, DocsAndPositionsEnum.FLAG_PAYLOADS);
+            }
+          }
+        }
+        return dpe;
+      } catch (IOException ioe) {
+        throw new RuntimeException(ioe);
+      }
     }
     
-  }
-  
-  private class PayloadMigratingSource extends Source {
-
-    private final DocsAndPositionsEnum dpe;
-    private int curDocID;
-    
-    protected PayloadMigratingSource(Type type, DocsAndPositionsEnum dpe) {
-      super(type);
-      this.dpe = dpe;
+    protected PayloadMigratingBinaryDocValues(Fields fields, Term term) {
+      this.fields = fields;
+      this.term = term;
+      this.dpe = getDPE();
       if (dpe == null) {
         curDocID = DocIdSetIterator.NO_MORE_DOCS;
       } else {
@@ -121,31 +118,41 @@ public class FacetsPayloadMigrationReade
     }
     
     @Override
-    public BytesRef getBytes(int docID, BytesRef ref) {
-      if (curDocID > docID) {
-        // document does not exist
-        ref.length = 0;
-        return ref;
-      }
-      
+    public void get(int docID, BytesRef result) {
       try {
+        // If caller is moving backwards (eg, during merge,
+        // the consuming DV format is free to iterate over
+        // our values as many times as it wants), we must
+        // re-init the dpe:
+        if (docID <= lastRequestedDocID) {
+          dpe = getDPE();
+          if (dpe == null) {
+            curDocID = DocIdSetIterator.NO_MORE_DOCS;
+          } else{
+            curDocID = dpe.nextDoc();
+          }
+        }
+        lastRequestedDocID = docID;
+        if (curDocID > docID) {
+          // document does not exist
+          result.length = 0;
+          return;
+        }
+      
         if (curDocID < docID) {
           curDocID = dpe.advance(docID);
           if (curDocID != docID) { // requested document does not have a payload
-            ref.length = 0;
-            return ref;
+            result.length = 0;
+            return;
           }
         }
         
-        // we're on the document
         dpe.nextPosition();
-        ref.copyBytes(dpe.getPayload());
-        return ref;
+        result.copyBytes(dpe.getPayload());
       } catch (IOException e) {
         throw new RuntimeException(e);
       }
     }
-    
   }
   
   /** The {@link Term} text of the ordinals payload. */
@@ -195,7 +202,7 @@ public class FacetsPayloadMigrationReade
   private final Map<String,Term> fieldTerms;
   
   /**
-   * Wraps an {@link AtomicReader} and migrates the payload to {@link DocValues}
+   * Wraps an {@link AtomicReader} and migrates the payload to {@link BinaryDocValues}
    * fields by using the given mapping.
    */
   public FacetsPayloadMigrationReader(AtomicReader in, Map<String,Term> fieldTerms) {
@@ -204,26 +211,14 @@ public class FacetsPayloadMigrationReade
   }
   
   @Override
-  public DocValues docValues(String field) throws IOException {
+  public BinaryDocValues getBinaryDocValues(String field) throws IOException {
     Term term = fieldTerms.get(field);
     if (term == null) {
-      return super.docValues(field);
+      return super.getBinaryDocValues(field);
     } else {
-      DocsAndPositionsEnum dpe = null;
-      Fields fields = fields();
-      if (fields != null) {
-        Terms terms = fields.terms(term.field());
-        if (terms != null) {
-          TermsEnum te = terms.iterator(null); // no use for reusing
-          if (te.seekExact(term.bytes(), true)) {
-            // we're not expected to be called for deleted documents
-            dpe = te.docsAndPositions(null, null, DocsAndPositionsEnum.FLAG_PAYLOADS);
-          }
-        }
-      }
       // we shouldn't return null, even if the term does not exist or has no
       // payloads, since we already marked the field as having DocValues.
-      return new PayloadMigratingDocValues(dpe);
+      return new PayloadMigratingBinaryDocValues(fields(), term);
     }
   }
 
@@ -240,7 +235,7 @@ public class FacetsPayloadMigrationReade
         // mark this field as having a DocValues
         infos.add(new FieldInfo(info.name, true, info.number,
             info.hasVectors(), info.omitsNorms(), info.hasPayloads(),
-            info.getIndexOptions(), Type.BYTES_VAR_STRAIGHT,
+            info.getIndexOptions(), DocValuesType.BINARY,
             info.getNormType(), info.attributes()));
         leftoverFields.remove(info.name);
       } else {
@@ -250,9 +245,8 @@ public class FacetsPayloadMigrationReade
     }
     for (String field : leftoverFields) {
       infos.add(new FieldInfo(field, false, ++number, false, false, false,
-          null, Type.BYTES_VAR_STRAIGHT, null, null));
+          null, DocValuesType.BINARY, null, null));
     }
     return new FieldInfos(infos.toArray(new FieldInfo[infos.size()]));
   }
-  
 }

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/index/OrdinalMappingAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/index/OrdinalMappingAtomicReader.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/index/OrdinalMappingAtomicReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/index/OrdinalMappingAtomicReader.java Fri Feb  8 03:26:14 2013
@@ -25,9 +25,7 @@ import org.apache.lucene.facet.index.par
 import org.apache.lucene.facet.index.params.FacetIndexingParams;
 import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter.OrdinalMap;
 import org.apache.lucene.index.AtomicReader;
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.DocValues.Source;
-import org.apache.lucene.index.DocValues.Type;
+import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.FilterAtomicReader;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IntsRef;
@@ -92,8 +90,8 @@ public class OrdinalMappingAtomicReader 
   }
 
   @Override
-  public DocValues docValues(String field) throws IOException {
-    DocValues inner = super.docValues(field);
+  public BinaryDocValues getBinaryDocValues(String field) throws IOException {
+    BinaryDocValues inner = super.getBinaryDocValues(field);
     if (inner == null) {
       return inner;
     }
@@ -102,46 +100,19 @@ public class OrdinalMappingAtomicReader 
     if (clp == null) {
       return inner;
     } else {
-      return new OrdinalMappingDocValues(inner, clp);
+      return new OrdinalMappingBinaryDocValues(clp, inner);
     }
   }
   
-  private class OrdinalMappingDocValues extends DocValues {
-
-    private final CategoryListParams clp;
-    private final DocValues delegate;
-    
-    public OrdinalMappingDocValues(DocValues delegate, CategoryListParams clp) {
-      this.delegate = delegate;
-      this.clp = clp;
-    }
-
-    @Override
-    protected Source loadSource() throws IOException {
-      return new OrdinalMappingSource(getType(), clp, delegate.getSource());
-    }
-
-    @Override
-    protected Source loadDirectSource() throws IOException {
-      return new OrdinalMappingSource(getType(), clp, delegate.getDirectSource());
-    }
-
-    @Override
-    public Type getType() {
-      return Type.BYTES_VAR_STRAIGHT;
-    }
-    
-  }
-  
-  private class OrdinalMappingSource extends Source {
+  private class OrdinalMappingBinaryDocValues extends BinaryDocValues {
 
     private final IntEncoder encoder;
     private final IntDecoder decoder;
     private final IntsRef ordinals = new IntsRef(32);
-    private final Source delegate;
+    private final BinaryDocValues delegate;
+    private final BytesRef scratch = new BytesRef();
     
-    protected OrdinalMappingSource(Type type, CategoryListParams clp, Source delegate) {
-      super(type);
+    protected OrdinalMappingBinaryDocValues(CategoryListParams clp, BinaryDocValues delegate) {
       this.delegate = delegate;
       encoder = clp.createEncoder();
       decoder = encoder.createMatchingDecoder();
@@ -149,23 +120,26 @@ public class OrdinalMappingAtomicReader 
     
     @SuppressWarnings("synthetic-access")
     @Override
-    public BytesRef getBytes(int docID, BytesRef ref) {
-      ref = delegate.getBytes(docID, ref);
-      if (ref == null || ref.length == 0) {
-        return ref;
-      } else {
-        decoder.decode(ref, ordinals);
+    public void get(int docID, BytesRef result) {
+      // NOTE: this isn't quite koscher, because in general
+      // multiple threads can call BinaryDV.get which would
+      // then conflict on the single scratch instance, but
+      // because this impl is only used for merging, we know
+      // only 1 thread calls us:
+      delegate.get(docID, scratch);
+      if (scratch.length > 0) {
+        // We must use scratch (and not re-use result) here,
+        // else encoder may overwrite the DV provider's
+        // private byte[]:
+        decoder.decode(scratch, ordinals);
         
         // map the ordinals
         for (int i = 0; i < ordinals.length; i++) {
           ordinals.ints[i] = ordinalMap[ordinals.ints[i]];
         }
         
-        encoder.encode(ordinals, ref);
-        return ref;
+        encoder.encode(ordinals, result);
       }
     }
-    
   }
-  
 }

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/CountingFacetsCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/CountingFacetsCollector.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/CountingFacetsCollector.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/CountingFacetsCollector.java Fri Feb  8 03:26:14 2013
@@ -13,17 +13,16 @@ import org.apache.lucene.facet.index.par
 import org.apache.lucene.facet.index.params.CategoryListParams.OrdinalPolicy;
 import org.apache.lucene.facet.index.params.FacetIndexingParams;
 import org.apache.lucene.facet.search.params.CountFacetRequest;
-import org.apache.lucene.facet.search.params.FacetRequest;
 import org.apache.lucene.facet.search.params.FacetRequest.SortBy;
 import org.apache.lucene.facet.search.params.FacetRequest.SortOrder;
+import org.apache.lucene.facet.search.params.FacetRequest;
 import org.apache.lucene.facet.search.params.FacetSearchParams;
 import org.apache.lucene.facet.search.results.FacetResult;
 import org.apache.lucene.facet.search.results.FacetResultNode;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 import org.apache.lucene.facet.taxonomy.directory.ParallelTaxonomyArrays;
 import org.apache.lucene.index.AtomicReaderContext;
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.DocValues.Source;
+import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.search.Collector;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.util.BytesRef;
@@ -66,12 +65,9 @@ import org.apache.lucene.util.encoding.D
  * </ul>
  * 
  * <p>
- * <b>NOTE:</b> this colletro uses {@link DocValues#getSource()} by default,
+ * <b>NOTE:</b> this collector uses {@link BinaryDocValues} by default,
  * which pre-loads the values into memory. If your application cannot afford the
- * RAM, you should use
- * {@link #CountingFacetsCollector(FacetSearchParams, TaxonomyReader, FacetArrays, boolean)}
- * and specify to use a direct source (corresponds to
- * {@link DocValues#getDirectSource()}).
+ * RAM, you should pick a codec which keeps the values (or parts of them) on disk.
  * 
  * <p>
  * <b>NOTE:</b> this collector supports category lists that were indexed with
@@ -91,18 +87,16 @@ public class CountingFacetsCollector ext
   private final FacetArrays facetArrays;
   private final int[] counts;
   private final String facetsField;
-  private final boolean useDirectSource;
-  private final HashMap<Source,FixedBitSet> matchingDocs = new HashMap<Source,FixedBitSet>();
+  private final HashMap<BinaryDocValues,FixedBitSet> matchingDocs = new HashMap<BinaryDocValues,FixedBitSet>();
   
-  private DocValues facetsValues;
+  private BinaryDocValues facetsValues;
   private FixedBitSet bits;
   
   public CountingFacetsCollector(FacetSearchParams fsp, TaxonomyReader taxoReader) {
-    this(fsp, taxoReader, new FacetArrays(taxoReader.getSize()), false);
+    this(fsp, taxoReader, new FacetArrays(taxoReader.getSize()));
   }
   
-  public CountingFacetsCollector(FacetSearchParams fsp, TaxonomyReader taxoReader, FacetArrays facetArrays, 
-      boolean useDirectSource) {
+  public CountingFacetsCollector(FacetSearchParams fsp, TaxonomyReader taxoReader, FacetArrays facetArrays) {
     assert facetArrays.arrayLength >= taxoReader.getSize() : "too small facet array";
     assert assertParams(fsp) == null : assertParams(fsp);
     
@@ -112,7 +106,6 @@ public class CountingFacetsCollector ext
     this.taxoReader = taxoReader;
     this.facetArrays = facetArrays;
     this.counts = facetArrays.getIntArray();
-    this.useDirectSource = useDirectSource;
   }
   
   /**
@@ -169,11 +162,10 @@ public class CountingFacetsCollector ext
   
   @Override
   public void setNextReader(AtomicReaderContext context) throws IOException {
-    facetsValues = context.reader().docValues(facetsField);
+    facetsValues = context.reader().getBinaryDocValues(facetsField);
     if (facetsValues != null) {
-      Source facetSource = useDirectSource ? facetsValues.getDirectSource() : facetsValues.getSource();
       bits = new FixedBitSet(context.reader().maxDoc());
-      matchingDocs.put(facetSource, bits);
+      matchingDocs.put(facetsValues, bits);
     }
   }
   
@@ -187,13 +179,13 @@ public class CountingFacetsCollector ext
   }
   
   private void countFacets() {
-    for (Entry<Source,FixedBitSet> entry : matchingDocs.entrySet()) {
-      Source facetsSource = entry.getKey();
+    for (Entry<BinaryDocValues,FixedBitSet> entry : matchingDocs.entrySet()) {
+      BinaryDocValues facetsSource = entry.getKey();
       FixedBitSet bits = entry.getValue();
       int doc = 0;
       int length = bits.length();
       while (doc < length && (doc = bits.nextSetBit(doc)) != -1) {
-        facetsSource .getBytes(doc, buf);
+        facetsSource.get(doc, buf);
         if (buf.length > 0) {
           // this document has facets
           int upto = buf.offset + buf.length;

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/DocValuesCategoryListIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/DocValuesCategoryListIterator.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/DocValuesCategoryListIterator.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/DocValuesCategoryListIterator.java Fri Feb  8 03:26:14 2013
@@ -3,8 +3,7 @@ package org.apache.lucene.facet.search;
 import java.io.IOException;
 
 import org.apache.lucene.index.AtomicReaderContext;
-import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.DocValues.Source;
+import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.encoding.IntDecoder;
@@ -26,35 +25,23 @@ import org.apache.lucene.util.encoding.I
  * limitations under the License.
  */
 
-/** A {@link CategoryListIterator} which reads the ordinals from a {@link DocValues}. */
+/** A {@link CategoryListIterator} which reads the ordinals from a {@link BinaryDocValues}. */
 public class DocValuesCategoryListIterator implements CategoryListIterator {
   
   private final IntDecoder decoder;
   private final String field;
   private final int hashCode;
-  private final boolean useDirectSource;
   private final BytesRef bytes = new BytesRef(32);
   
-  private DocValues.Source current;
+  private BinaryDocValues current;
   
   /**
-   * Constructs a new {@link DocValuesCategoryListIterator} which uses an
-   * in-memory {@link Source}.
+   * Constructs a new {@link DocValuesCategoryListIterator}.
    */
   public DocValuesCategoryListIterator(String field, IntDecoder decoder) {
-    this(field, decoder, false);
-  }
-  
-  /**
-   * Constructs a new {@link DocValuesCategoryListIterator} which uses either a
-   * {@link DocValues#getDirectSource() direct source} or
-   * {@link DocValues#getSource() in-memory} one.
-   */
-  public DocValuesCategoryListIterator(String field, IntDecoder decoder, boolean useDirectSource) {
     this.field = field;
     this.decoder = decoder;
     this.hashCode = field.hashCode();
-    this.useDirectSource = useDirectSource;
   }
   
   @Override
@@ -78,19 +65,13 @@ public class DocValuesCategoryListIterat
   
   @Override
   public boolean setNextReader(AtomicReaderContext context) throws IOException {
-    DocValues dv = context.reader().docValues(field);
-    if (dv == null) {
-      current = null;
-      return false;
-    }
-    
-    current = useDirectSource ? dv.getDirectSource() : dv.getSource();
-    return true;
+    current = context.reader().getBinaryDocValues(field);
+    return current != null;
   }
   
   @Override
   public void getOrdinals(int docID, IntsRef ints) throws IOException {
-    current.getBytes(docID, bytes);
+    current.get(docID, bytes);
     ints.length = 0;
     if (bytes.length > 0) {
       decoder.decode(bytes, ints);

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/CategoryListIteratorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/CategoryListIteratorTest.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/CategoryListIteratorTest.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/CategoryListIteratorTest.java Fri Feb  8 03:26:14 2013
@@ -5,8 +5,8 @@ import java.util.Set;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockTokenizer;
+import org.apache.lucene.document.BinaryDocValuesField;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.StraightBytesDocValuesField;
 import org.apache.lucene.facet.FacetTestCase;
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.IndexReader;
@@ -57,7 +57,7 @@ public class CategoryListIteratorTest ex
     for (int i = 0; i < data.length; i++) {
       Document doc = new Document();
       encoder.encode(IntsRef.deepCopyOf(data[i]), buf);
-      doc.add(new StraightBytesDocValuesField("f", buf));
+      doc.add(new BinaryDocValuesField("f", buf));
       writer.addDocument(doc);
     }
     IndexReader reader = writer.getReader();
@@ -100,9 +100,9 @@ public class CategoryListIteratorTest ex
       if (i == 0) {
         BytesRef buf = new BytesRef();
         encoder.encode(IntsRef.deepCopyOf(data[i]), buf );
-        doc.add(new StraightBytesDocValuesField("f", buf));
+        doc.add(new BinaryDocValuesField("f", buf));
       } else {
-        doc.add(new StraightBytesDocValuesField("f", new BytesRef()));
+        doc.add(new BinaryDocValuesField("f", new BytesRef()));
       }
       writer.addDocument(doc);
       writer.commit();

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/CountingFacetsCollectorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/CountingFacetsCollectorTest.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/CountingFacetsCollectorTest.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/CountingFacetsCollectorTest.java Fri Feb  8 03:26:14 2013
@@ -433,7 +433,7 @@ public class CountingFacetsCollectorTest
     
     FacetSearchParams fsp = new FacetSearchParams(new CountFacetRequest(CP_A, NUM_CHILDREN_CP_A), 
         new CountFacetRequest(CP_B, NUM_CHILDREN_CP_B));
-    FacetsCollector fc = new CountingFacetsCollector(fsp , taxoReader, new FacetArrays(taxoReader.getSize()), true);
+    FacetsCollector fc = new CountingFacetsCollector(fsp , taxoReader, new FacetArrays(taxoReader.getSize()));
     searcher.search(new MatchAllDocsQuery(), fc);
     
     List<FacetResult> facetResults = fc.getFacetResults();

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/TestMultipleCategoryLists.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/TestMultipleCategoryLists.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/TestMultipleCategoryLists.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/search/TestMultipleCategoryLists.java Fri Feb  8 03:26:14 2013
@@ -187,7 +187,7 @@ public class TestMultipleCategoryLists e
   private void assertOrdinalsExist(String field, IndexReader ir) throws IOException {
     for (AtomicReaderContext context : ir.leaves()) {
       AtomicReader r = context.reader();
-      if (r.docValues(field) != null) {
+      if (r.getBinaryDocValues(field) != null) {
         return; // not all segments must have this DocValues
       }
     }

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/util/SlowRAMDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/util/SlowRAMDirectory.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/util/SlowRAMDirectory.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/util/SlowRAMDirectory.java Fri Feb  8 03:26:14 2013
@@ -33,7 +33,7 @@ public class SlowRAMDirectory extends RA
 
   private static final int IO_SLEEP_THRESHOLD = 50;
   
-  private Random random;
+  Random random;
   private int sleepMillis;
 
   public void setSleepMillis(int sleepMillis) {
@@ -62,7 +62,7 @@ public class SlowRAMDirectory extends RA
     return super.openInput(name, context);
   }
 
-  void doSleep(int length) {
+  void doSleep(Random random, int length) {
     int sTime = length<10 ? sleepMillis : (int) (sleepMillis * Math.log(length));
     if (random!=null) {
       sTime = random.nextInt(sTime);
@@ -74,6 +74,14 @@ public class SlowRAMDirectory extends RA
     }
   }
 
+  /** Make a private random. */
+  Random forkRandom() {
+    if (random == null) {
+      return null;
+    }
+    return new Random(random.nextLong());
+  }
+
   /**
    * Delegate class to wrap an IndexInput and delay reading bytes by some
    * specified time.
@@ -81,16 +89,18 @@ public class SlowRAMDirectory extends RA
   private class SlowIndexInput extends IndexInput {
     private IndexInput ii;
     private int numRead = 0;
+    private Random random;
     
     public SlowIndexInput(IndexInput ii) {
       super("SlowIndexInput(" + ii + ")");
+      this.random = forkRandom();
       this.ii = ii;
     }
     
     @Override
     public byte readByte() throws IOException {
       if (numRead >= IO_SLEEP_THRESHOLD) {
-        doSleep(0);
+        doSleep(random, 0);
         numRead = 0;
       }
       ++numRead;
@@ -100,7 +110,7 @@ public class SlowRAMDirectory extends RA
     @Override
     public void readBytes(byte[] b, int offset, int len) throws IOException {
       if (numRead >= IO_SLEEP_THRESHOLD) {
-        doSleep(len);
+        doSleep(random, len);
         numRead = 0;
       }
       numRead += len;
@@ -125,15 +135,17 @@ public class SlowRAMDirectory extends RA
     
     private IndexOutput io;
     private int numWrote;
+    private final Random random;
     
     public SlowIndexOutput(IndexOutput io) {
       this.io = io;
+      this.random = forkRandom();
     }
     
     @Override
     public void writeByte(byte b) throws IOException {
       if (numWrote >= IO_SLEEP_THRESHOLD) {
-        doSleep(0);
+        doSleep(random, 0);
         numWrote = 0;
       }
       ++numWrote;
@@ -143,7 +155,7 @@ public class SlowRAMDirectory extends RA
     @Override
     public void writeBytes(byte[] b, int offset, int length) throws IOException {
       if (numWrote >= IO_SLEEP_THRESHOLD) {
-        doSleep(length);
+        doSleep(random, length);
         numWrote = 0;
       }
       numWrote += length;

Modified: lucene/dev/branches/branch_4x/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupingSearch.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupingSearch.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupingSearch.java (original)
+++ lucene/dev/branches/branch_4x/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupingSearch.java Fri Feb  8 03:26:14 2013
@@ -20,14 +20,8 @@ package org.apache.lucene.search.groupin
 import java.io.IOException;
 import java.util.*;
 
-import org.apache.lucene.document.DerefBytesDocValuesField;
-import org.apache.lucene.index.DocValues;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.*;
-import org.apache.lucene.search.grouping.dv.DVAllGroupHeadsCollector;
-import org.apache.lucene.search.grouping.dv.DVAllGroupsCollector;
-import org.apache.lucene.search.grouping.dv.DVFirstPassGroupingCollector;
-import org.apache.lucene.search.grouping.dv.DVSecondPassGroupingCollector;
 import org.apache.lucene.search.grouping.function.FunctionAllGroupHeadsCollector;
 import org.apache.lucene.search.grouping.function.FunctionAllGroupsCollector;
 import org.apache.lucene.search.grouping.function.FunctionFirstPassGroupingCollector;
@@ -51,8 +45,6 @@ public class GroupingSearch {
   private final ValueSource groupFunction;
   private final Map<?, ?> valueSourceContext;
   private final Filter groupEndDocs;
-  private final DocValues.Type docValuesType;
-  private final boolean diskResidentDocValues;
 
   private Sort groupSort = Sort.RELEVANCE;
   private Sort sortWithinGroup;
@@ -80,20 +72,7 @@ public class GroupingSearch {
    * @param groupField The name of the field to group by.
    */
   public GroupingSearch(String groupField) {
-    this(groupField, null, null, null, null, false);
-  }
-
-  /**
-   * Constructs a <code>GroupingSearch</code> instance that groups documents by doc values.
-   * This constructor can only be used when the groupField
-   * is a <code>*DocValuesField</code> (eg, {@link DerefBytesDocValuesField}.
-   *
-   * @param groupField            The name of the field to group by that contains doc values
-   * @param docValuesType         The doc values type of the specified groupField
-   * @param diskResidentDocValues Whether the values to group by should be disk resident
-   */
-  public GroupingSearch(String groupField, DocValues.Type docValuesType, boolean diskResidentDocValues) {
-    this(groupField, null, null, null, docValuesType, diskResidentDocValues);
+    this(groupField, null, null, null);
   }
 
   /**
@@ -104,7 +83,7 @@ public class GroupingSearch {
    * @param valueSourceContext The context of the specified groupFunction
    */
   public GroupingSearch(ValueSource groupFunction, Map<?, ?> valueSourceContext) {
-    this(null, groupFunction, valueSourceContext, null, null, false);
+    this(null, groupFunction, valueSourceContext, null);
   }
 
   /**
@@ -114,16 +93,14 @@ public class GroupingSearch {
    * @param groupEndDocs The filter that marks the last document in all doc blocks
    */
   public GroupingSearch(Filter groupEndDocs) {
-    this(null, null, null, groupEndDocs, null, false);
+    this(null, null, null, groupEndDocs);
   }
 
-  private GroupingSearch(String groupField, ValueSource groupFunction, Map<?, ?> valueSourceContext, Filter groupEndDocs, DocValues.Type docValuesType, boolean diskResidentDocValues) {
+  private GroupingSearch(String groupField, ValueSource groupFunction, Map<?, ?> valueSourceContext, Filter groupEndDocs) {
     this.groupField = groupField;
     this.groupFunction = groupFunction;
     this.valueSourceContext = valueSourceContext;
     this.groupEndDocs = groupEndDocs;
-    this.docValuesType = docValuesType;
-    this.diskResidentDocValues = diskResidentDocValues;
   }
 
   /**
@@ -180,18 +157,6 @@ public class GroupingSearch {
       } else {
         allGroupHeadsCollector = null;
       }
-    } else if (docValuesType != null) {
-      firstPassCollector = DVFirstPassGroupingCollector.create(groupSort, topN, groupField, docValuesType, diskResidentDocValues);
-      if (allGroups) {
-        allGroupsCollector = DVAllGroupsCollector.create(groupField, docValuesType, diskResidentDocValues, initialSize);
-      } else {
-        allGroupsCollector = null;
-      }
-      if (allGroupHeads) {
-        allGroupHeadsCollector = DVAllGroupHeadsCollector.create(groupField, sortWithinGroup, docValuesType, diskResidentDocValues);
-      } else {
-        allGroupHeadsCollector = null;
-      }
     } else {
       firstPassCollector = new TermFirstPassGroupingCollector(groupField, groupSort, topN);
       if (allGroups) {
@@ -253,8 +218,6 @@ public class GroupingSearch {
     AbstractSecondPassGroupingCollector secondPassCollector;
     if (groupFunction != null) {
       secondPassCollector = new FunctionSecondPassGroupingCollector((Collection) topSearchGroups, groupSort, sortWithinGroup, topNInsideGroup, includeScores, includeMaxScore, fillSortFields, groupFunction, valueSourceContext);
-    } else if (docValuesType != null) {
-      secondPassCollector = DVSecondPassGroupingCollector.create(groupField, diskResidentDocValues, docValuesType, (Collection) topSearchGroups, groupSort, sortWithinGroup, topNInsideGroup, includeScores, includeMaxScore, fillSortFields);
     } else {
       secondPassCollector = new TermSecondPassGroupingCollector(groupField, (Collection) topSearchGroups, groupSort, sortWithinGroup, topNInsideGroup, includeScores, includeMaxScore, fillSortFields);
     }

Modified: lucene/dev/branches/branch_4x/lucene/grouping/src/java/org/apache/lucene/search/grouping/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/grouping/src/java/org/apache/lucene/search/grouping/package.html?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/grouping/src/java/org/apache/lucene/search/grouping/package.html (original)
+++ lucene/dev/branches/branch_4x/lucene/grouping/src/java/org/apache/lucene/search/grouping/package.html Fri Feb  8 03:26:14 2013
@@ -80,8 +80,8 @@ field fall into a single group.</p>
 <p>Known limitations:</p>
 <ul>
   <li> For the two-pass grouping search, the group field must be a
-    single-valued indexed field.
-    {@link org.apache.lucene.search.FieldCache} is used to load the {@link org.apache.lucene.search.FieldCache.DocTermsIndex} for this field.
+    single-valued indexed field (or indexed as a {@link org.apache.lucene.document.SortedDocValuesField}).
+    {@link org.apache.lucene.search.FieldCache} is used to load the {@link org.apache.lucene.index.SortedDocValues} for this field.
   <li> Although Solr support grouping by function and this module has abstraction of what a group is, there are currently only
     implementations for grouping based on terms.
   <li> Sharding is not directly supported, though is not too
@@ -196,25 +196,5 @@ fields, <code>FieldCache</code>, etc.).
   <code>GroupingSearch</code> convenience utility
 </p>
 
-<p>
-  There are also DocValues based implementations available for the group collectors. There are factory methods
-  available for creating dv based instances. A typical example using dv based grouping with the
-  <code>GroupingSearch</code> convenience utility:
-</p>
-
-<pre class="prettyprint">
-  boolean diskResident = true; // Whether values should fetched directly from disk by passing the Java heap space.
-  DocValues.Type docValuesType = DocValues.Type.BYTES_VAR_SORTED;
-  GroupingSearch groupingSearch = new GroupingSearch("author", docValuesType, diskResident);
-  groupingSearch.setGroupSort(groupSort);
-  groupingSearch.setFillSortFields(fillFields);
-
-  TermQuery query = new TermQuery(new Term("content", searchTerm));
-  // The docValuesType variable decides the generic type. When float is used this Double and in case of int this is Long
-  TopGroups&lt;BytesRef&gt; result = groupingSearch.search(indexSearcher, query, groupOffset, groupLimit);
-
-  // Render groupsResult...
-</pre>
-
 </body>
 </html>

Modified: lucene/dev/branches/branch_4x/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java (original)
+++ lucene/dev/branches/branch_4x/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java Fri Feb  8 03:26:14 2013
@@ -17,19 +17,20 @@ package org.apache.lucene.search.groupin
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.util.*;
+
 import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.search.*;
 import org.apache.lucene.search.grouping.AbstractAllGroupHeadsCollector;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.SentinelIntSet;
 
-import java.io.IOException;
-import java.util.*;
-
 /**
  * A base implementation of {@link org.apache.lucene.search.grouping.AbstractAllGroupHeadsCollector} for retrieving the most relevant groups when grouping
  * on a string based group field. More specifically this all concrete implementations of this base implementation
- * use {@link org.apache.lucene.search.FieldCache.DocTermsIndex}.
+ * use {@link org.apache.lucene.index.SortedDocValues}.
  *
  * @lucene.experimental
  */
@@ -40,7 +41,7 @@ public abstract class TermAllGroupHeadsC
   final String groupField;
   final BytesRef scratchBytesRef = new BytesRef();
 
-  FieldCache.DocTermsIndex groupIndex;
+  SortedDocValues groupIndex;
   AtomicReaderContext readerContext;
 
   protected TermAllGroupHeadsCollector(String groupField, int numberOfSorts) {
@@ -125,7 +126,13 @@ public abstract class TermAllGroupHeadsC
     @Override
     protected void retrieveGroupHeadAndAddIfNotExist(int doc) throws IOException {
       final int ord = groupIndex.getOrd(doc);
-      final BytesRef groupValue = ord == 0 ? null : groupIndex.lookup(ord, scratchBytesRef);
+      final BytesRef groupValue;
+      if (ord == -1) {
+        groupValue = null;
+      } else {
+        groupIndex.lookupOrd(ord, scratchBytesRef);
+        groupValue = scratchBytesRef;
+      }
       GroupHead groupHead = groups.get(groupValue);
       if (groupHead == null) {
         groupHead = new GroupHead(groupValue, sortWithinGroup, doc);
@@ -205,18 +212,18 @@ public abstract class TermAllGroupHeadsC
     private final List<GroupHead> collectedGroups;
     private final SortField[] fields;
 
-    private FieldCache.DocTermsIndex[] sortsIndex;
+    private SortedDocValues[] sortsIndex;
     private Scorer scorer;
     private GroupHead[] segmentGroupHeads;
 
     OrdScoreAllGroupHeadsCollector(String groupField, Sort sortWithinGroup, int initialSize) {
       super(groupField, sortWithinGroup.getSort().length);
-      ordSet = new SentinelIntSet(initialSize, -1);
+      ordSet = new SentinelIntSet(initialSize, -2);
       collectedGroups = new ArrayList<GroupHead>(initialSize);
 
       final SortField[] sortFields = sortWithinGroup.getSort();
       fields = new SortField[sortFields.length];
-      sortsIndex = new FieldCache.DocTermsIndex[sortFields.length];
+      sortsIndex = new SortedDocValues[sortFields.length];
       for (int i = 0; i < sortFields.length; i++) {
         reversed[i] = sortFields[i].getReverse() ? -1 : 1;
         fields[i] = sortFields[i];
@@ -239,14 +246,20 @@ public abstract class TermAllGroupHeadsC
       GroupHead groupHead;
       if (!ordSet.exists(key)) {
         ordSet.put(key);
-        BytesRef term = key == 0 ? null : groupIndex.getTerm(doc, new BytesRef());
+        BytesRef term;
+        if (key == -1) {
+          term = null;
+        } else {
+          term = new BytesRef();
+          groupIndex.lookupOrd(key, term);
+        }
         groupHead = new GroupHead(doc, term);
         collectedGroups.add(groupHead);
-        segmentGroupHeads[key] = groupHead;
+        segmentGroupHeads[key+1] = groupHead;
         temporalResult.stop = true;
       } else {
         temporalResult.stop = false;
-        groupHead = segmentGroupHeads[key];
+        groupHead = segmentGroupHeads[key+1];
       }
       temporalResult.groupHead = groupHead;
     }
@@ -265,19 +278,29 @@ public abstract class TermAllGroupHeadsC
 
       // Clear ordSet and fill it with previous encountered groups that can occur in the current segment.
       ordSet.clear();
-      segmentGroupHeads = new GroupHead[groupIndex.numOrd()];
+      segmentGroupHeads = new GroupHead[groupIndex.getValueCount()+1];
       for (GroupHead collectedGroup : collectedGroups) {
-        int ord = groupIndex.binarySearchLookup(collectedGroup.groupValue, scratchBytesRef);
-        if (ord >= 0) {
+        int ord;
+        if (collectedGroup.groupValue == null) {
+          ord = -1;
+        } else {
+          ord = groupIndex.lookupTerm(collectedGroup.groupValue);
+        }
+        if (collectedGroup.groupValue == null || ord >= 0) {
           ordSet.put(ord);
-          segmentGroupHeads[ord] = collectedGroup;
+          segmentGroupHeads[ord+1] = collectedGroup;
 
           for (int i = 0; i < sortsIndex.length; i++) {
             if (fields[i].getType() == SortField.Type.SCORE) {
               continue;
             }
-
-            collectedGroup.sortOrds[i] = sortsIndex[i].binarySearchLookup(collectedGroup.sortValues[i], scratchBytesRef);
+            int sortOrd;
+            if (collectedGroup.sortValues[i] == null) {
+              sortOrd = -1;
+            } else {
+              sortOrd = sortsIndex[i].lookupTerm(collectedGroup.sortValues[i]);
+            }
+            collectedGroup.sortOrds[i] = sortOrd;
           }
         }
       }
@@ -298,11 +321,13 @@ public abstract class TermAllGroupHeadsC
           if (fields[i].getType() == SortField.Type.SCORE) {
             scores[i] = scorer.score();
           } else {
-            sortValues[i] = sortsIndex[i].getTerm(doc, new BytesRef());
             sortOrds[i] = sortsIndex[i].getOrd(doc);
+            sortValues[i] = new BytesRef();
+            if (sortOrds[i] != -1) {
+              sortsIndex[i].get(doc, sortValues[i]);
+            }
           }
         }
-
       }
 
       @Override
@@ -318,7 +343,12 @@ public abstract class TermAllGroupHeadsC
         } else {
           if (sortOrds[compIDX] < 0) {
             // The current segment doesn't contain the sort value we encountered before. Therefore the ord is negative.
-            return sortValues[compIDX].compareTo(sortsIndex[compIDX].getTerm(doc, scratchBytesRef));
+            if (sortsIndex[compIDX].getOrd(doc) == -1) {
+              scratchBytesRef.length = 0;
+            } else {
+              sortsIndex[compIDX].get(doc, scratchBytesRef);
+            }
+            return sortValues[compIDX].compareTo(scratchBytesRef);
           } else {
             return sortOrds[compIDX] - sortsIndex[compIDX].getOrd(doc);
           }
@@ -331,15 +361,17 @@ public abstract class TermAllGroupHeadsC
           if (fields[i].getType() == SortField.Type.SCORE) {
             scores[i] = scorer.score();
           } else {
-            sortValues[i] = sortsIndex[i].getTerm(doc, sortValues[i]);
             sortOrds[i] = sortsIndex[i].getOrd(doc);
+            if (sortOrds[i] == -1) {
+              sortValues[i].length = 0;
+            } else {
+              sortsIndex[i].get(doc, sortValues[i]);
+            }
           }
         }
         this.doc = doc + readerContext.docBase;
       }
-
     }
-
   }
 
 
@@ -350,17 +382,17 @@ public abstract class TermAllGroupHeadsC
     private final List<GroupHead> collectedGroups;
     private final SortField[] fields;
 
-    private FieldCache.DocTermsIndex[] sortsIndex;
+    private SortedDocValues[] sortsIndex;
     private GroupHead[] segmentGroupHeads;
 
     OrdAllGroupHeadsCollector(String groupField, Sort sortWithinGroup, int initialSize) {
       super(groupField, sortWithinGroup.getSort().length);
-      ordSet = new SentinelIntSet(initialSize, -1);
+      ordSet = new SentinelIntSet(initialSize, -2);
       collectedGroups = new ArrayList<GroupHead>(initialSize);
 
       final SortField[] sortFields = sortWithinGroup.getSort();
       fields = new SortField[sortFields.length];
-      sortsIndex = new FieldCache.DocTermsIndex[sortFields.length];
+      sortsIndex = new SortedDocValues[sortFields.length];
       for (int i = 0; i < sortFields.length; i++) {
         reversed[i] = sortFields[i].getReverse() ? -1 : 1;
         fields[i] = sortFields[i];
@@ -382,14 +414,20 @@ public abstract class TermAllGroupHeadsC
       GroupHead groupHead;
       if (!ordSet.exists(key)) {
         ordSet.put(key);
-        BytesRef term = key == 0 ? null : groupIndex.getTerm(doc, new BytesRef());
+        BytesRef term;
+        if (key == -1) {
+          term = null;
+        } else {
+          term = new BytesRef();
+          groupIndex.lookupOrd(key, term);
+        }
         groupHead = new GroupHead(doc, term);
         collectedGroups.add(groupHead);
-        segmentGroupHeads[key] = groupHead;
+        segmentGroupHeads[key+1] = groupHead;
         temporalResult.stop = true;
       } else {
         temporalResult.stop = false;
-        groupHead = segmentGroupHeads[key];
+        groupHead = segmentGroupHeads[key+1];
       }
       temporalResult.groupHead = groupHead;
     }
@@ -404,15 +442,26 @@ public abstract class TermAllGroupHeadsC
 
       // Clear ordSet and fill it with previous encountered groups that can occur in the current segment.
       ordSet.clear();
-      segmentGroupHeads = new GroupHead[groupIndex.numOrd()];
+      segmentGroupHeads = new GroupHead[groupIndex.getValueCount()+1];
       for (GroupHead collectedGroup : collectedGroups) {
-        int groupOrd = groupIndex.binarySearchLookup(collectedGroup.groupValue, scratchBytesRef);
-        if (groupOrd >= 0) {
+        int groupOrd;
+        if (collectedGroup.groupValue == null) {
+          groupOrd = -1;
+        } else {
+          groupOrd = groupIndex.lookupTerm(collectedGroup.groupValue);
+        }
+        if (collectedGroup.groupValue == null || groupOrd >= 0) {
           ordSet.put(groupOrd);
-          segmentGroupHeads[groupOrd] = collectedGroup;
+          segmentGroupHeads[groupOrd+1] = collectedGroup;
 
           for (int i = 0; i < sortsIndex.length; i++) {
-            collectedGroup.sortOrds[i] = sortsIndex[i].binarySearchLookup(collectedGroup.sortValues[i], scratchBytesRef);
+            int sortOrd;
+            if (collectedGroup.sortOrds[i] == -1) {
+              sortOrd = -1;
+            } else {
+              sortOrd = sortsIndex[i].lookupTerm(collectedGroup.sortValues[i]);
+            }
+            collectedGroup.sortOrds[i] = sortOrd;
           }
         }
       }
@@ -428,8 +477,11 @@ public abstract class TermAllGroupHeadsC
         sortValues = new BytesRef[sortsIndex.length];
         sortOrds = new int[sortsIndex.length];
         for (int i = 0; i < sortsIndex.length; i++) {
-          sortValues[i] = sortsIndex[i].getTerm(doc, new BytesRef());
           sortOrds[i] = sortsIndex[i].getOrd(doc);
+          sortValues[i] = new BytesRef();
+          if (sortOrds[i] != -1) {
+            sortsIndex[i].get(doc, sortValues[i]);
+          }
         }
       }
 
@@ -437,7 +489,12 @@ public abstract class TermAllGroupHeadsC
       public int compare(int compIDX, int doc) throws IOException {
         if (sortOrds[compIDX] < 0) {
           // The current segment doesn't contain the sort value we encountered before. Therefore the ord is negative.
-          return sortValues[compIDX].compareTo(sortsIndex[compIDX].getTerm(doc, scratchBytesRef));
+          if (sortsIndex[compIDX].getOrd(doc) == -1) {
+            scratchBytesRef.length = 0;
+          } else {
+            sortsIndex[compIDX].get(doc, scratchBytesRef);
+          }
+          return sortValues[compIDX].compareTo(scratchBytesRef);
         } else {
           return sortOrds[compIDX] - sortsIndex[compIDX].getOrd(doc);
         }
@@ -446,8 +503,12 @@ public abstract class TermAllGroupHeadsC
       @Override
       public void updateDocHead(int doc) throws IOException {
         for (int i = 0; i < sortsIndex.length; i++) {
-          sortValues[i] = sortsIndex[i].getTerm(doc, sortValues[i]);
           sortOrds[i] = sortsIndex[i].getOrd(doc);
+          if (sortOrds[i] == -1) {
+            sortValues[i].length = 0;
+          } else {
+            sortsIndex[i].lookupOrd(sortOrds[i], sortValues[i]);
+          }
         }
         this.doc = doc + readerContext.docBase;
       }
@@ -469,7 +530,7 @@ public abstract class TermAllGroupHeadsC
 
     ScoreAllGroupHeadsCollector(String groupField, Sort sortWithinGroup, int initialSize) {
       super(groupField, sortWithinGroup.getSort().length);
-      ordSet = new SentinelIntSet(initialSize, -1);
+      ordSet = new SentinelIntSet(initialSize, -2);
       collectedGroups = new ArrayList<GroupHead>(initialSize);
 
       final SortField[] sortFields = sortWithinGroup.getSort();
@@ -496,14 +557,20 @@ public abstract class TermAllGroupHeadsC
       GroupHead groupHead;
       if (!ordSet.exists(key)) {
         ordSet.put(key);
-        BytesRef term = key == 0 ? null : groupIndex.getTerm(doc, new BytesRef());
+        BytesRef term;
+        if (key == -1) {
+          term = null;
+        } else {
+          term = new BytesRef();
+          groupIndex.lookupOrd(key, term);
+        }
         groupHead = new GroupHead(doc, term);
         collectedGroups.add(groupHead);
-        segmentGroupHeads[key] = groupHead;
+        segmentGroupHeads[key+1] = groupHead;
         temporalResult.stop = true;
       } else {
         temporalResult.stop = false;
-        groupHead = segmentGroupHeads[key];
+        groupHead = segmentGroupHeads[key+1];
       }
       temporalResult.groupHead = groupHead;
     }
@@ -515,12 +582,17 @@ public abstract class TermAllGroupHeadsC
 
       // Clear ordSet and fill it with previous encountered groups that can occur in the current segment.
       ordSet.clear();
-      segmentGroupHeads = new GroupHead[groupIndex.numOrd()];
+      segmentGroupHeads = new GroupHead[groupIndex.getValueCount()+1];
       for (GroupHead collectedGroup : collectedGroups) {
-        int ord = groupIndex.binarySearchLookup(collectedGroup.groupValue, scratchBytesRef);
-        if (ord >= 0) {
+        int ord;
+        if (collectedGroup.groupValue == null) {
+          ord = -1;
+        } else {
+          ord = groupIndex.lookupTerm(collectedGroup.groupValue);
+        }
+        if (collectedGroup.groupValue == null || ord >= 0) {
           ordSet.put(ord);
-          segmentGroupHeads[ord] = collectedGroup;
+          segmentGroupHeads[ord+1] = collectedGroup;
         }
       }
     }
@@ -561,5 +633,4 @@ public abstract class TermAllGroupHeadsC
     }
 
   }
-
 }

Modified: lucene/dev/branches/branch_4x/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupsCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupsCollector.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupsCollector.java (original)
+++ lucene/dev/branches/branch_4x/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupsCollector.java Fri Feb  8 03:26:14 2013
@@ -17,18 +17,18 @@ package org.apache.lucene.search.groupin
  * limitations under the License.
  */
 
-import org.apache.lucene.index.AtomicReaderContext;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.search.FieldCache;
-import org.apache.lucene.search.grouping.AbstractAllGroupsCollector;
-import org.apache.lucene.util.SentinelIntSet;
-import org.apache.lucene.util.BytesRef;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 
+import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.search.FieldCache;
+import org.apache.lucene.search.grouping.AbstractAllGroupsCollector;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.SentinelIntSet;
+
 /**
  * A collector that collects all groups that match the
  * query. Only the group value is collected, and the order
@@ -52,8 +52,7 @@ public class TermAllGroupsCollector exte
   private final SentinelIntSet ordSet;
   private final List<BytesRef> groups;
 
-  private FieldCache.DocTermsIndex index;
-  private final BytesRef spareBytesRef = new BytesRef();
+  private SortedDocValues index;
 
   /**
    * Expert: Constructs a {@link AbstractAllGroupsCollector}
@@ -66,7 +65,7 @@ public class TermAllGroupsCollector exte
    *                    heap usage is 4 bytes * initialSize.
    */
   public TermAllGroupsCollector(String groupField, int initialSize) {
-    ordSet = new SentinelIntSet(initialSize, -1);
+    ordSet = new SentinelIntSet(initialSize, -2);
     groups = new ArrayList<BytesRef>(initialSize);
     this.groupField = groupField;
   }
@@ -87,7 +86,13 @@ public class TermAllGroupsCollector exte
     int key = index.getOrd(doc);
     if (!ordSet.exists(key)) {
       ordSet.put(key);
-      BytesRef term = key == 0 ? null : index.lookup(key, new BytesRef());
+      BytesRef term;
+      if (key == -1) {
+        term = null;
+      } else {
+        term =  new BytesRef();
+        index.lookupOrd(key, term);
+      }
       groups.add(term);
     }
   }
@@ -104,11 +109,14 @@ public class TermAllGroupsCollector exte
     // Clear ordSet and fill it with previous encountered groups that can occur in the current segment.
     ordSet.clear();
     for (BytesRef countedGroup : groups) {
-      int ord = index.binarySearchLookup(countedGroup, spareBytesRef);
-      if (ord >= 0) {
-        ordSet.put(ord);
+      if (countedGroup == null) {
+        ordSet.put(-1);
+      } else {
+        int ord = index.lookupTerm(countedGroup);
+        if (ord >= 0) {
+          ordSet.put(ord);
+        }
       }
     }
   }
-
 }

Modified: lucene/dev/branches/branch_4x/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermDistinctValuesCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermDistinctValuesCollector.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermDistinctValuesCollector.java (original)
+++ lucene/dev/branches/branch_4x/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermDistinctValuesCollector.java Fri Feb  8 03:26:14 2013
@@ -17,20 +17,20 @@ package org.apache.lucene.search.groupin
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.util.*;
+
 import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.search.FieldCache;
-import org.apache.lucene.search.FieldCache.DocTermsIndex; // javadocs
 import org.apache.lucene.search.grouping.AbstractDistinctValuesCollector;
 import org.apache.lucene.search.grouping.SearchGroup;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.SentinelIntSet;
 
-import java.io.IOException;
-import java.util.*;
-
 /**
  * A term based implementation of {@link org.apache.lucene.search.grouping.AbstractDistinctValuesCollector} that relies
- * on {@link DocTermsIndex} to count the distinct values per group.
+ * on {@link SortedDocValues} to count the distinct values per group.
  *
  * @lucene.experimental
  */
@@ -41,10 +41,9 @@ public class TermDistinctValuesCollector
   private final List<GroupCount> groups;
   private final SentinelIntSet ordSet;
   private final GroupCount groupCounts[];
-  private final BytesRef spare = new BytesRef();
 
-  private FieldCache.DocTermsIndex groupFieldTermIndex;
-  private FieldCache.DocTermsIndex countFieldTermIndex;
+  private SortedDocValues groupFieldTermIndex;
+  private SortedDocValues countFieldTermIndex;
 
   /**
    * Constructs {@link TermDistinctValuesCollector} instance.
@@ -60,7 +59,7 @@ public class TermDistinctValuesCollector
     for (SearchGroup<BytesRef> group : groups) {
       this.groups.add(new GroupCount(group.groupValue));
     }
-    ordSet = new SentinelIntSet(groups.size(), -1);
+    ordSet = new SentinelIntSet(groups.size(), -2);
     groupCounts = new GroupCount[ordSet.keys.length];
   }
 
@@ -73,11 +72,13 @@ public class TermDistinctValuesCollector
 
     GroupCount gc = groupCounts[slot];
     int countOrd = countFieldTermIndex.getOrd(doc);
-    if (doesNotContainsOrd(countOrd, gc.ords)) {
-      if (countOrd == 0) {
+    if (doesNotContainOrd(countOrd, gc.ords)) {
+      if (countOrd == -1) {
         gc.uniqueValues.add(null);
       } else {
-        gc.uniqueValues.add(countFieldTermIndex.lookup(countOrd, new BytesRef()));
+        BytesRef br = new BytesRef();
+        countFieldTermIndex.lookupOrd(countOrd, br);
+        gc.uniqueValues.add(br);
       }
 
       gc.ords = Arrays.copyOf(gc.ords, gc.ords.length + 1);
@@ -88,7 +89,7 @@ public class TermDistinctValuesCollector
     }
   }
 
-  private boolean doesNotContainsOrd(int ord, int[] ords) {
+  private boolean doesNotContainOrd(int ord, int[] ords) {
     if (ords.length == 0) {
       return true;
     } else if (ords.length == 1) {
@@ -106,21 +107,20 @@ public class TermDistinctValuesCollector
   public void setNextReader(AtomicReaderContext context) throws IOException {
     groupFieldTermIndex = FieldCache.DEFAULT.getTermsIndex(context.reader(), groupField);
     countFieldTermIndex = FieldCache.DEFAULT.getTermsIndex(context.reader(), countField);
-
     ordSet.clear();
     for (GroupCount group : groups) {
-      int groupOrd = group.groupValue == null ? 0 : groupFieldTermIndex.binarySearchLookup(group.groupValue, spare);
-      if (groupOrd < 0) {
+      int groupOrd = group.groupValue == null ? -1 : groupFieldTermIndex.lookupTerm(group.groupValue);
+      if (group.groupValue != null && groupOrd < 0) {
         continue;
       }
 
       groupCounts[ordSet.put(groupOrd)] = group;
       group.ords = new int[group.uniqueValues.size()];
-      Arrays.fill(group.ords, -1);
+      Arrays.fill(group.ords, -2);
       int i = 0;
       for (BytesRef value : group.uniqueValues) {
-        int countOrd = value == null ? 0 : countFieldTermIndex.binarySearchLookup(value, new BytesRef());
-        if (countOrd >= 0) {
+        int countOrd = value == null ? -1 : countFieldTermIndex.lookupTerm(value);
+        if (value == null || countOrd >= 0) {
           group.ords[i++] = countOrd;
         }
       }

Modified: lucene/dev/branches/branch_4x/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermFirstPassGroupingCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermFirstPassGroupingCollector.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermFirstPassGroupingCollector.java (original)
+++ lucene/dev/branches/branch_4x/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermFirstPassGroupingCollector.java Fri Feb  8 03:26:14 2013
@@ -17,17 +17,18 @@ package org.apache.lucene.search.groupin
  * limitations under the License.
  */
 
+import java.io.IOException;
+
 import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.search.FieldCache;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.grouping.AbstractFirstPassGroupingCollector;
 import org.apache.lucene.util.BytesRef;
 
-import java.io.IOException;
-
 /**
  * Concrete implementation of {@link org.apache.lucene.search.grouping.AbstractFirstPassGroupingCollector} that groups based on
- * field values and more specifically uses {@link org.apache.lucene.search.FieldCache.DocTermsIndex}
+ * field values and more specifically uses {@link org.apache.lucene.index.SortedDocValues}
  * to collect groups.
  *
  * @lucene.experimental
@@ -35,7 +36,7 @@ import java.io.IOException;
 public class TermFirstPassGroupingCollector extends AbstractFirstPassGroupingCollector<BytesRef> {
 
   private final BytesRef scratchBytesRef = new BytesRef();
-  private FieldCache.DocTermsIndex index;
+  private SortedDocValues index;
 
   private String groupField;
 
@@ -63,7 +64,12 @@ public class TermFirstPassGroupingCollec
   @Override
   protected BytesRef getDocGroupValue(int doc) {
     final int ord = index.getOrd(doc);
-    return ord == 0 ? null : index.lookup(ord, scratchBytesRef);
+    if (ord == -1) {
+      return null;
+    } else {
+      index.lookupOrd(ord, scratchBytesRef);
+      return scratchBytesRef;
+    }
   }
 
   @Override

Modified: lucene/dev/branches/branch_4x/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java (original)
+++ lucene/dev/branches/branch_4x/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java Fri Feb  8 03:26:14 2013
@@ -17,17 +17,19 @@ package org.apache.lucene.search.groupin
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.DocTermOrds;
+import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedDocValuesTermsEnum;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.FieldCache;
 import org.apache.lucene.search.grouping.AbstractGroupFacetCollector;
 import org.apache.lucene.util.*;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
 /**
  * An implementation of {@link AbstractGroupFacetCollector} that computes grouped facets based on the indexed terms
  * from the {@link FieldCache}.
@@ -38,9 +40,8 @@ public abstract class TermGroupFacetColl
 
   final List<GroupedFacetHit> groupedFacetHits;
   final SentinelIntSet segmentGroupedFacetHits;
-  final BytesRef spare = new BytesRef();
 
-  FieldCache.DocTermsIndex groupFieldTermsIndex;
+  SortedDocValues groupFieldTermsIndex;
 
   /**
    * Factory method for creating the right implementation based on the fact whether the facet field contains
@@ -70,13 +71,13 @@ public abstract class TermGroupFacetColl
   TermGroupFacetCollector(String groupField, String facetField, BytesRef facetPrefix, int initialSize) {
     super(groupField, facetField, facetPrefix);
     groupedFacetHits = new ArrayList<GroupedFacetHit>(initialSize);
-    segmentGroupedFacetHits = new SentinelIntSet(initialSize, -1);
+    segmentGroupedFacetHits = new SentinelIntSet(initialSize, Integer.MIN_VALUE);
   }
 
   // Implementation for single valued facet fields.
   static class SV extends TermGroupFacetCollector {
 
-    private FieldCache.DocTermsIndex facetFieldTermsIndex;
+    private SortedDocValues facetFieldTermsIndex;
 
     SV(String groupField, String facetField, BytesRef facetPrefix, int initialSize) {
       super(groupField, facetField, facetPrefix, initialSize);
@@ -90,21 +91,33 @@ public abstract class TermGroupFacetColl
       }
 
       int groupOrd = groupFieldTermsIndex.getOrd(doc);
-      int segmentGroupedFacetsIndex = (groupOrd * facetFieldTermsIndex.numOrd()) + facetOrd;
+      int segmentGroupedFacetsIndex = groupOrd * (facetFieldTermsIndex.getValueCount()+1) + facetOrd;
       if (segmentGroupedFacetHits.exists(segmentGroupedFacetsIndex)) {
         return;
       }
 
       segmentTotalCount++;
-      segmentFacetCounts[facetOrd]++;
+      segmentFacetCounts[facetOrd+1]++;
 
       segmentGroupedFacetHits.put(segmentGroupedFacetsIndex);
-      groupedFacetHits.add(
-          new GroupedFacetHit(
-              groupOrd == 0 ? null : groupFieldTermsIndex.lookup(groupOrd, new BytesRef()),
-              facetOrd == 0 ? null : facetFieldTermsIndex.lookup(facetOrd, new BytesRef())
-          )
-      );
+
+      BytesRef groupKey;
+      if (groupOrd == -1) {
+        groupKey = null;
+      } else {
+        groupKey = new BytesRef();
+        groupFieldTermsIndex.lookupOrd(groupOrd, groupKey);
+      }
+
+      BytesRef facetKey;
+      if (facetOrd == -1) {
+        facetKey = null;
+      } else {
+        facetKey = new BytesRef();
+        facetFieldTermsIndex.lookupOrd(facetOrd, facetKey);
+      }
+
+      groupedFacetHits.add(new GroupedFacetHit(groupKey, facetKey));
     }
 
     @Override
@@ -115,44 +128,47 @@ public abstract class TermGroupFacetColl
 
       groupFieldTermsIndex = FieldCache.DEFAULT.getTermsIndex(context.reader(), groupField);
       facetFieldTermsIndex = FieldCache.DEFAULT.getTermsIndex(context.reader(), facetField);
-      segmentFacetCounts = new int[facetFieldTermsIndex.numOrd()];
+
+      // 1+ to allow for the -1 "not set":
+      segmentFacetCounts = new int[facetFieldTermsIndex.getValueCount()+1];
       segmentTotalCount = 0;
 
       segmentGroupedFacetHits.clear();
       for (GroupedFacetHit groupedFacetHit : groupedFacetHits) {
-        int facetOrd = facetFieldTermsIndex.binarySearchLookup(groupedFacetHit.facetValue, spare);
-        if (facetOrd < 0) {
+        int facetOrd = groupedFacetHit.facetValue == null ? -1 : facetFieldTermsIndex.lookupTerm(groupedFacetHit.facetValue);
+        if (groupedFacetHit.facetValue != null && facetOrd < 0) {
           continue;
         }
 
-        int groupOrd = groupFieldTermsIndex.binarySearchLookup(groupedFacetHit.groupValue, spare);
-        if (groupOrd < 0) {
+        int groupOrd = groupedFacetHit.groupValue == null ? -1 : groupFieldTermsIndex.lookupTerm(groupedFacetHit.groupValue);
+        if (groupedFacetHit.groupValue != null && groupOrd < 0) {
           continue;
         }
 
-        int segmentGroupedFacetsIndex = (groupOrd * facetFieldTermsIndex.numOrd()) + facetOrd;
+        int segmentGroupedFacetsIndex = groupOrd * (facetFieldTermsIndex.getValueCount()+1) + facetOrd;
         segmentGroupedFacetHits.put(segmentGroupedFacetsIndex);
       }
 
       if (facetPrefix != null) {
-        startFacetOrd = facetFieldTermsIndex.binarySearchLookup(facetPrefix, spare);
+        startFacetOrd = facetFieldTermsIndex.lookupTerm(facetPrefix);
         if (startFacetOrd < 0) {
           // Points to the ord one higher than facetPrefix
           startFacetOrd = -startFacetOrd - 1;
         }
         BytesRef facetEndPrefix = BytesRef.deepCopyOf(facetPrefix);
         facetEndPrefix.append(UnicodeUtil.BIG_TERM);
-        endFacetOrd = facetFieldTermsIndex.binarySearchLookup(facetEndPrefix, spare);
+        endFacetOrd = facetFieldTermsIndex.lookupTerm(facetEndPrefix);
+        assert endFacetOrd < 0;
         endFacetOrd = -endFacetOrd - 1; // Points to the ord one higher than facetEndPrefix
       } else {
-        startFacetOrd = 0;
-        endFacetOrd = facetFieldTermsIndex.numOrd();
+        startFacetOrd = -1;
+        endFacetOrd = facetFieldTermsIndex.getValueCount();
       }
     }
 
     @Override
     protected SegmentResult createSegmentResult() throws IOException {
-      return new SegmentResult(segmentFacetCounts, segmentTotalCount, facetFieldTermsIndex.getTermsEnum(), startFacetOrd, endFacetOrd);
+      return new SegmentResult(segmentFacetCounts, segmentTotalCount, new SortedDocValuesTermsEnum(facetFieldTermsIndex), startFacetOrd, endFacetOrd);
     }
 
     private static class SegmentResult extends AbstractGroupFacetCollector.SegmentResult {
@@ -160,11 +176,12 @@ public abstract class TermGroupFacetColl
       final TermsEnum tenum;
 
       SegmentResult(int[] counts, int total, TermsEnum tenum, int startFacetOrd, int endFacetOrd) throws IOException {
-        super(counts, total - counts[0], counts[0], endFacetOrd);
+        super(counts, total - counts[0], counts[0], endFacetOrd+1);
         this.tenum = tenum;
-        this.mergePos = startFacetOrd == 0 ? 1 : startFacetOrd;
+        this.mergePos = startFacetOrd == -1 ? 1 : startFacetOrd+1;
         if (mergePos < maxTermPos) {
-          tenum.seekExact(mergePos);
+          assert tenum != null;
+          tenum.seekExact(startFacetOrd == -1 ? 0 : startFacetOrd);
           mergeTerm = tenum.term();
         }
       }
@@ -173,9 +190,7 @@ public abstract class TermGroupFacetColl
       protected void nextTerm() throws IOException {
         mergeTerm = tenum.next();
       }
-
     }
-
   }
 
   // Implementation for multi valued facet fields.
@@ -202,9 +217,14 @@ public abstract class TermGroupFacetColl
         segmentFacetCounts[facetFieldDocTermOrds.numTerms()]++;
 
         segmentGroupedFacetHits.put(segmentGroupedFacetsIndex);
-        groupedFacetHits.add(
-            new GroupedFacetHit(groupOrd == 0 ? null : groupFieldTermsIndex.lookup(groupOrd, new BytesRef()), null)
-        );
+        BytesRef groupKey;
+        if (groupOrd == -1) {
+          groupKey = null;
+        } else {
+          groupKey = new BytesRef();
+          groupFieldTermsIndex.lookupOrd(groupOrd, groupKey);
+        }
+        groupedFacetHits.add(new GroupedFacetHit(groupKey, null));
         return;
       }
 
@@ -228,7 +248,7 @@ public abstract class TermGroupFacetColl
             continue;
           }
 
-          int segmentGroupedFacetsIndex = (groupOrd * (facetFieldDocTermOrds.numTerms() + 1)) + facetOrd;
+          int segmentGroupedFacetsIndex = groupOrd * (facetFieldDocTermOrds.numTerms() + 1) + facetOrd;
           if (segmentGroupedFacetHits.exists(segmentGroupedFacetsIndex)) {
             continue;
           }
@@ -237,9 +257,17 @@ public abstract class TermGroupFacetColl
           segmentFacetCounts[facetOrd]++;
 
           segmentGroupedFacetHits.put(segmentGroupedFacetsIndex);
+
+          BytesRef groupKey;
+          if (groupOrd == -1) {
+            groupKey = null;
+          } else {
+            groupKey = new BytesRef();
+            groupFieldTermsIndex.lookupOrd(groupOrd, groupKey);
+          }
+
           groupedFacetHits.add(
-              new GroupedFacetHit(
-                  groupOrd == 0 ? null : groupFieldTermsIndex.lookup(groupOrd, new BytesRef()),
+              new GroupedFacetHit(groupKey,
                   facetOrd == facetFieldDocTermOrds.numTerms() ? null : BytesRef.deepCopyOf(facetFieldDocTermOrds.lookupTerm(facetOrdTermsEnum, facetOrd))
               )
           );
@@ -263,8 +291,8 @@ public abstract class TermGroupFacetColl
 
       segmentGroupedFacetHits.clear();
       for (GroupedFacetHit groupedFacetHit : groupedFacetHits) {
-        int groupOrd = groupFieldTermsIndex.binarySearchLookup(groupedFacetHit.groupValue, spare);
-        if (groupOrd < 0) {
+        int groupOrd = groupedFacetHit.groupValue == null ? -1 : groupFieldTermsIndex.lookupTerm(groupedFacetHit.groupValue);
+        if (groupedFacetHit.groupValue != null && groupOrd < 0) {
           continue;
         }
 
@@ -279,7 +307,7 @@ public abstract class TermGroupFacetColl
         }
 
         // (facetFieldDocTermOrds.numTerms() + 1) for all possible facet values and docs not containing facet field
-        int segmentGroupedFacetsIndex = (groupOrd * (facetFieldDocTermOrds.numTerms() + 1)) + facetOrd;
+        int segmentGroupedFacetsIndex = groupOrd * (facetFieldDocTermOrds.numTerms() + 1) + facetOrd;
         segmentGroupedFacetHits.put(segmentGroupedFacetsIndex);
       }
 
@@ -337,10 +365,8 @@ public abstract class TermGroupFacetColl
       protected void nextTerm() throws IOException {
         mergeTerm = tenum.next();
       }
-
     }
   }
-
 }
 
 class GroupedFacetHit {

Modified: lucene/dev/branches/branch_4x/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java (original)
+++ lucene/dev/branches/branch_4x/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java Fri Feb  8 03:26:14 2013
@@ -17,20 +17,21 @@ package org.apache.lucene.search.groupin
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.util.Collection;
+
 import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.search.FieldCache;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.grouping.AbstractSecondPassGroupingCollector;
 import org.apache.lucene.search.grouping.SearchGroup;
-import org.apache.lucene.util.SentinelIntSet;
 import org.apache.lucene.util.BytesRef;
-
-import java.io.IOException;
-import java.util.Collection;
+import org.apache.lucene.util.SentinelIntSet;
 
 /**
  * Concrete implementation of {@link org.apache.lucene.search.grouping.AbstractSecondPassGroupingCollector} that groups based on
- * field values and more specifically uses {@link org.apache.lucene.search.FieldCache.DocTermsIndex}
+ * field values and more specifically uses {@link org.apache.lucene.index.SortedDocValues}
  * to collect grouped docs.
  *
  * @lucene.experimental
@@ -38,8 +39,7 @@ import java.util.Collection;
 public class TermSecondPassGroupingCollector extends AbstractSecondPassGroupingCollector<BytesRef> {
 
   private final SentinelIntSet ordSet;
-  private FieldCache.DocTermsIndex index;
-  private final BytesRef spareBytesRef = new BytesRef();
+  private SortedDocValues index;
   private final String groupField;
 
   @SuppressWarnings({"unchecked", "rawtypes"})
@@ -47,7 +47,7 @@ public class TermSecondPassGroupingColle
                                          int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields)
       throws IOException {
     super(groups, groupSort, withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields);
-    ordSet = new SentinelIntSet(groupMap.size(), -1);
+    ordSet = new SentinelIntSet(groupMap.size(), -2);
     this.groupField = groupField;
     groupDocs = (SearchGroupDocs<BytesRef>[]) new SearchGroupDocs[ordSet.keys.length];
   }
@@ -61,8 +61,8 @@ public class TermSecondPassGroupingColle
     ordSet.clear();
     for (SearchGroupDocs<BytesRef> group : groupMap.values()) {
 //      System.out.println("  group=" + (group.groupValue == null ? "null" : group.groupValue.utf8ToString()));
-      int ord = group.groupValue == null ? 0 : index.binarySearchLookup(group.groupValue, spareBytesRef);
-      if (ord >= 0) {
+      int ord = group.groupValue == null ? -1 : index.lookupTerm(group.groupValue);
+      if (group.groupValue == null || ord >= 0) {
         groupDocs[ordSet.put(ord)] = group;
       }
     }

Modified: lucene/dev/branches/branch_4x/lucene/grouping/src/test/org/apache/lucene/search/grouping/AbstractGroupingTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/grouping/src/test/org/apache/lucene/search/grouping/AbstractGroupingTestCase.java?rev=1443834&r1=1443833&r2=1443834&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/grouping/src/test/org/apache/lucene/search/grouping/AbstractGroupingTestCase.java (original)
+++ lucene/dev/branches/branch_4x/lucene/grouping/src/test/org/apache/lucene/search/grouping/AbstractGroupingTestCase.java Fri Feb  8 03:26:14 2013
@@ -30,10 +30,11 @@ public abstract class AbstractGroupingTe
     String randomValue;
     do {
       // B/c of DV based impl we can't see the difference between an empty string and a null value.
-      // For that reason we don't generate empty string groups.
+      // For that reason we don't generate empty string
+      // groups.
       randomValue = _TestUtil.randomRealisticUnicodeString(random());
+      //randomValue = _TestUtil.randomSimpleString(random());
     } while ("".equals(randomValue));
     return randomValue;
   }
-
 }