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

svn commit: r1498804 [6/8] - in /lucene/dev/branches/lucene4258: lucene/ lucene/analysis/common/src/java/org/apache/lucene/analysis/compound/ lucene/analysis/common/src/java/org/apache/lucene/analysis/ngram/ lucene/analysis/common/src/java/org/apache/l...

Modified: lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java?rev=1498804&r1=1498803&r2=1498804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java (original)
+++ lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java Tue Jul  2 07:12:00 2013
@@ -126,7 +126,7 @@ public class TestFSTs extends LuceneTest
 
       // FST ord pos int
       {
-        final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
+        final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
         final List<FSTTester.InputOutput<Long>> pairs = new ArrayList<FSTTester.InputOutput<Long>>(terms2.length);
         for(int idx=0;idx<terms2.length;idx++) {
           pairs.add(new FSTTester.InputOutput<Long>(terms2[idx], (long) idx));
@@ -171,7 +171,7 @@ public class TestFSTs extends LuceneTest
 
     // PositiveIntOutput (ord)
     {
-      final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
+      final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
       final List<FSTTester.InputOutput<Long>> pairs = new ArrayList<FSTTester.InputOutput<Long>>(terms.length);
       for(int idx=0;idx<terms.length;idx++) {
         pairs.add(new FSTTester.InputOutput<Long>(terms[idx], (long) idx));
@@ -181,8 +181,7 @@ public class TestFSTs extends LuceneTest
 
     // PositiveIntOutput (random monotonically increasing positive number)
     {
-      final boolean doShare = random().nextBoolean();
-      final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(doShare);
+      final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
       final List<FSTTester.InputOutput<Long>> pairs = new ArrayList<FSTTester.InputOutput<Long>>(terms.length);
       long lastOutput = 0;
       for(int idx=0;idx<terms.length;idx++) {
@@ -190,12 +189,12 @@ public class TestFSTs extends LuceneTest
         lastOutput = value;
         pairs.add(new FSTTester.InputOutput<Long>(terms[idx], value));
       }
-      new FSTTester<Long>(random(), dir, inputMode, pairs, outputs, doShare).doTest(true);
+      new FSTTester<Long>(random(), dir, inputMode, pairs, outputs, true).doTest(true);
     }
 
     // PositiveIntOutput (random positive number)
     {
-      final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(random().nextBoolean());
+      final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
       final List<FSTTester.InputOutput<Long>> pairs = new ArrayList<FSTTester.InputOutput<Long>>(terms.length);
       for(int idx=0;idx<terms.length;idx++) {
         pairs.add(new FSTTester.InputOutput<Long>(terms[idx], _TestUtil.nextLong(random(), 0, Long.MAX_VALUE)));
@@ -205,8 +204,8 @@ public class TestFSTs extends LuceneTest
 
     // Pair<ord, (random monotonically increasing positive number>
     {
-      final PositiveIntOutputs o1 = PositiveIntOutputs.getSingleton(random().nextBoolean());
-      final PositiveIntOutputs o2 = PositiveIntOutputs.getSingleton(random().nextBoolean());
+      final PositiveIntOutputs o1 = PositiveIntOutputs.getSingleton();
+      final PositiveIntOutputs o2 = PositiveIntOutputs.getSingleton();
       final PairOutputs<Long,Long> outputs = new PairOutputs<Long,Long>(o1, o2);
       final List<FSTTester.InputOutput<PairOutputs.Pair<Long,Long>>> pairs = new ArrayList<FSTTester.InputOutput<PairOutputs.Pair<Long,Long>>>(terms.length);
       long lastOutput = 0;
@@ -306,7 +305,7 @@ public class TestFSTs extends LuceneTest
     }
     IndexReader r = DirectoryReader.open(writer, true);
     writer.close();
-    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(random().nextBoolean());
+    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
 
     final boolean doRewrite = random().nextBoolean();
 
@@ -653,8 +652,8 @@ public class TestFSTs extends LuceneTest
 
     if (storeOrds && storeDocFreqs) {
       // Store both ord & docFreq:
-      final PositiveIntOutputs o1 = PositiveIntOutputs.getSingleton(true);
-      final PositiveIntOutputs o2 = PositiveIntOutputs.getSingleton(false);
+      final PositiveIntOutputs o1 = PositiveIntOutputs.getSingleton();
+      final PositiveIntOutputs o2 = PositiveIntOutputs.getSingleton();
       final PairOutputs<Long,Long> outputs = new PairOutputs<Long,Long>(o1, o2);
       new VisitTerms<PairOutputs.Pair<Long,Long>>(dirOut, wordsFileIn, inputMode, prune, outputs, doPack, noArcArrays) {
         Random rand;
@@ -669,7 +668,7 @@ public class TestFSTs extends LuceneTest
       }.run(limit, verify, false);
     } else if (storeOrds) {
       // Store only ords
-      final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
+      final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
       new VisitTerms<Long>(dirOut, wordsFileIn, inputMode, prune, outputs, doPack, noArcArrays) {
         @Override
         public Long getOutput(IntsRef input, int ord) {
@@ -678,7 +677,7 @@ public class TestFSTs extends LuceneTest
       }.run(limit, verify, true);
     } else if (storeDocFreqs) {
       // Store only docFreq
-      final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(false);
+      final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
       new VisitTerms<Long>(dirOut, wordsFileIn, inputMode, prune, outputs, doPack, noArcArrays) {
         Random rand;
         @Override
@@ -781,7 +780,7 @@ public class TestFSTs extends LuceneTest
     // smaller FST if the outputs grow monotonically.  But
     // if numbers are "random", false should give smaller
     // final size:
-    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
+    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
 
     // Build an FST mapping BytesRef -> Long
     final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, outputs);
@@ -1100,7 +1099,7 @@ public class TestFSTs extends LuceneTest
   }
 
   public void testFinalOutputOnEndState() throws Exception {
-    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
+    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
 
     final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE4, 2, 0, true, true, Integer.MAX_VALUE, outputs, null, random().nextBoolean(), PackedInts.DEFAULT, true, 15);
     builder.add(Util.toUTF32("stat", new IntsRef()), 17L);
@@ -1115,7 +1114,7 @@ public class TestFSTs extends LuceneTest
   }
 
   public void testInternalFinalState() throws Exception {
-    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
+    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
     final boolean willRewrite = random().nextBoolean();
     final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, willRewrite, PackedInts.DEFAULT, true, 15);
     builder.add(Util.toIntsRef(new BytesRef("stat"), new IntsRef()), outputs.getNoOutput());
@@ -1136,7 +1135,7 @@ public class TestFSTs extends LuceneTest
   // Make sure raw FST can differentiate between final vs
   // non-final end nodes
   public void testNonFinalStopNode() throws Exception {
-    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
+    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
     final Long nothing = outputs.getNoOutput();
     final Builder<Long> b = new Builder<Long>(FST.INPUT_TYPE.BYTE1, outputs);
 
@@ -1216,7 +1215,7 @@ public class TestFSTs extends LuceneTest
   };
 
   public void testShortestPaths() throws Exception {
-    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
+    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
     final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, outputs);
 
     final IntsRef scratch = new IntsRef();
@@ -1258,8 +1257,8 @@ public class TestFSTs extends LuceneTest
   public void testShortestPathsWFST() throws Exception {
 
     PairOutputs<Long,Long> outputs = new PairOutputs<Long,Long>(
-        PositiveIntOutputs.getSingleton(true), // weight
-        PositiveIntOutputs.getSingleton(true)  // output
+        PositiveIntOutputs.getSingleton(), // weight
+        PositiveIntOutputs.getSingleton()  // output
     );
     
     final Builder<Pair<Long,Long>> builder = new Builder<Pair<Long,Long>>(FST.INPUT_TYPE.BYTE1, outputs);
@@ -1301,7 +1300,7 @@ public class TestFSTs extends LuceneTest
     final TreeMap<String,Long> slowCompletor = new TreeMap<String,Long>();
     final TreeSet<String> allPrefixes = new TreeSet<String>();
     
-    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
+    final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
     final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, outputs);
     final IntsRef scratch = new IntsRef();
     
@@ -1416,8 +1415,8 @@ public class TestFSTs extends LuceneTest
     final TreeSet<String> allPrefixes = new TreeSet<String>();
     
     PairOutputs<Long,Long> outputs = new PairOutputs<Long,Long>(
-        PositiveIntOutputs.getSingleton(true), // weight
-        PositiveIntOutputs.getSingleton(true)  // output
+        PositiveIntOutputs.getSingleton(), // weight
+        PositiveIntOutputs.getSingleton()  // output
     );
     final Builder<Pair<Long,Long>> builder = new Builder<Pair<Long,Long>>(FST.INPUT_TYPE.BYTE1, outputs);
     final IntsRef scratch = new IntsRef();

Modified: lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/util/junitcompat/TestFailOnFieldCacheInsanity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/util/junitcompat/TestFailOnFieldCacheInsanity.java?rev=1498804&r1=1498803&r2=1498804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/util/junitcompat/TestFailOnFieldCacheInsanity.java (original)
+++ lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/util/junitcompat/TestFailOnFieldCacheInsanity.java Tue Jul  2 07:12:00 2013
@@ -69,6 +69,7 @@ public class TestFailOnFieldCacheInsanit
     for(Failure f : r.getFailures()) {
       if (f.getMessage().indexOf("Insane") != -1) {
         insane = true;
+        break;
       }
     }
     Assert.assertTrue(insane);

Modified: lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/util/packed/TestPackedInts.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/util/packed/TestPackedInts.java?rev=1498804&r1=1498803&r2=1498804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/util/packed/TestPackedInts.java (original)
+++ lucene/dev/branches/lucene4258/lucene/core/src/test/org/apache/lucene/util/packed/TestPackedInts.java Tue Jul  2 07:12:00 2013
@@ -659,6 +659,61 @@ public class TestPackedInts extends Luce
     assertEquals(1 << 10, wrt.get(valueCount - 1));
   }
 
+  public void testPagedGrowableWriter() {
+    int pageSize = 1 << (_TestUtil.nextInt(random(), 6, 30));
+    // supports 0 values?
+    PagedGrowableWriter writer = new PagedGrowableWriter(0, pageSize, _TestUtil.nextInt(random(), 1, 64), random().nextFloat());
+    assertEquals(0, writer.size());
+
+    // compare against AppendingLongBuffer
+    AppendingLongBuffer buf = new AppendingLongBuffer();
+    int size = random().nextInt(1000000);
+    long max = 5;
+    for (int i = 0; i < size; ++i) {
+      buf.add(_TestUtil.nextLong(random(), 0, max));
+      if (rarely()) {
+        max = PackedInts.maxValue(rarely() ? _TestUtil.nextInt(random(), 0, 63) : _TestUtil.nextInt(random(), 0, 31));
+      }
+    }
+    writer = new PagedGrowableWriter(size, pageSize, _TestUtil.nextInt(random(), 1, 64), random().nextFloat());
+    assertEquals(size, writer.size());
+    for (int i = size - 1; i >= 0; --i) {
+      writer.set(i, buf.get(i));
+    }
+    for (int i = 0; i < size; ++i) {
+      assertEquals(buf.get(i), writer.get(i));
+    }
+
+    // test copy
+    PagedGrowableWriter copy = writer.resize(_TestUtil.nextLong(random(), writer.size() / 2, writer.size() * 3 / 2));
+    for (long i = 0; i < copy.size(); ++i) {
+      if (i < writer.size()) {
+        assertEquals(writer.get(i), copy.get(i));
+      } else {
+        assertEquals(0, copy.get(i));
+      }
+    }
+  }
+
+  // memory hole
+  @Ignore
+  public void testPagedGrowableWriterOverflow() {
+    final long size = _TestUtil.nextLong(random(), 2 * (long) Integer.MAX_VALUE, 3 * (long) Integer.MAX_VALUE);
+    final int pageSize = 1 << (_TestUtil.nextInt(random(), 16, 30));
+    final PagedGrowableWriter writer = new PagedGrowableWriter(size, pageSize, 1, random().nextFloat());
+    final long index = _TestUtil.nextLong(random(), (long) Integer.MAX_VALUE, size - 1);
+    writer.set(index, 2);
+    assertEquals(2, writer.get(index));
+    for (int i = 0; i < 1000000; ++i) {
+      final long idx = _TestUtil.nextLong(random(), 0, size);
+      if (idx == index) {
+        assertEquals(2, writer.get(idx));
+      } else {
+        assertEquals(0, writer.get(idx));
+      }
+    }
+  }
+
   public void testSave() throws IOException {
     final int valueCount = _TestUtil.nextInt(random(), 1, 2048);
     for (int bpv = 1; bpv <= 64; ++bpv) {
@@ -808,13 +863,15 @@ public class TestPackedInts extends Luce
     final long[] arr = new long[RandomInts.randomIntBetween(random(), 1, 1000000)];
     for (int bpv : new int[] {0, 1, 63, 64, RandomInts.randomIntBetween(random(), 2, 62)}) {
       for (boolean monotonic : new boolean[] {true, false}) {
+        final int pageSize = 1 << _TestUtil.nextInt(random(), 6, 20);
+        final int initialPageCount = _TestUtil.nextInt(random(), 0, 16);
         AbstractAppendingLongBuffer buf;
         final int inc;
         if (monotonic) {
-          buf = new MonotonicAppendingLongBuffer();
+          buf = new MonotonicAppendingLongBuffer(initialPageCount, pageSize);
           inc = _TestUtil.nextInt(random(), -1000, 1000);
         } else {
-          buf = new AppendingLongBuffer();
+          buf = new AppendingLongBuffer(initialPageCount, pageSize);
           inc = 0;
         }
         if (bpv == 0) {

Modified: lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/range/RangeAccumulator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/range/RangeAccumulator.java?rev=1498804&r1=1498803&r2=1498804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/range/RangeAccumulator.java (original)
+++ lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/range/RangeAccumulator.java Tue Jul  2 07:12:00 2013
@@ -64,7 +64,7 @@ public class RangeAccumulator extends Fa
         throw new IllegalArgumentException("only flat (dimension only) CategoryPath is allowed");
       }
 
-      RangeFacetRequest<?> rfr = (RangeFacetRequest) fr;
+      RangeFacetRequest<?> rfr = (RangeFacetRequest<?>) fr;
 
       requests.add(new RangeSet(rfr.ranges, rfr.categoryPath.components[0]));
     }
@@ -86,8 +86,11 @@ public class RangeAccumulator extends Fa
       RangeSet ranges = requests.get(i);
 
       int[] counts = new int[ranges.ranges.length];
-      for(MatchingDocs hits : matchingDocs) {
+      for (MatchingDocs hits : matchingDocs) {
         NumericDocValues ndv = hits.context.reader().getNumericDocValues(ranges.field);
+        if (ndv == null) {
+          continue; // no numeric values for this field in this reader
+        }
         final int length = hits.bits.length();
         int doc = 0;
         while (doc < length && (doc = hits.bits.nextSetBit(doc)) != -1) {

Modified: lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/sampling/SampleFixer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/sampling/SampleFixer.java?rev=1498804&r1=1498803&r2=1498804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/sampling/SampleFixer.java (original)
+++ lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/sampling/SampleFixer.java Tue Jul  2 07:12:00 2013
@@ -3,6 +3,7 @@ package org.apache.lucene.facet.sampling
 import java.io.IOException;
 
 import org.apache.lucene.facet.search.FacetResult;
+import org.apache.lucene.facet.search.FacetResultNode;
 import org.apache.lucene.facet.search.ScoredDocIDs;
 
 /*
@@ -23,22 +24,50 @@ import org.apache.lucene.facet.search.Sc
  */
 
 /**
- * Fixer of sample facet accumulation results
+ * Fixer of sample facet accumulation results.
  * 
  * @lucene.experimental
  */
-public interface SampleFixer {
+public abstract class SampleFixer {
   
   /**
    * Alter the input result, fixing it to account for the sampling. This
-   * implementation can compute accurate or estimated counts for the sampled facets. 
-   * For example, a faster correction could just multiply by a compensating factor.
+   * implementation can compute accurate or estimated counts for the sampled
+   * facets. For example, a faster correction could just multiply by a
+   * compensating factor.
    * 
    * @param origDocIds
    *          full set of matching documents.
    * @param fres
    *          sample result to be fixed.
-   * @throws IOException If there is a low-level I/O error.
+   * @throws IOException
+   *           If there is a low-level I/O error.
    */
-  public void fixResult(ScoredDocIDs origDocIds, FacetResult fres) throws IOException; 
+  public void fixResult(ScoredDocIDs origDocIds, FacetResult fres, double samplingRatio) throws IOException {
+    FacetResultNode topRes = fres.getFacetResultNode();
+    fixResultNode(topRes, origDocIds, samplingRatio);
+  }
+  
+  /**
+   * Fix result node count, and, recursively, fix all its children
+   * 
+   * @param facetResNode
+   *          result node to be fixed
+   * @param docIds
+   *          docids in effect
+   * @throws IOException
+   *           If there is a low-level I/O error.
+   */
+  protected void fixResultNode(FacetResultNode facetResNode, ScoredDocIDs docIds, double samplingRatio) 
+      throws IOException {
+    singleNodeFix(facetResNode, docIds, samplingRatio);
+    for (FacetResultNode frn : facetResNode.subResults) {
+      fixResultNode(frn, docIds, samplingRatio);
+    }
+  }
+  
+  /** Fix the given node's value. */
+  protected abstract void singleNodeFix(FacetResultNode facetResNode, ScoredDocIDs docIds, double samplingRatio) 
+      throws IOException;
+  
 }
\ No newline at end of file

Modified: lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/sampling/Sampler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/sampling/Sampler.java?rev=1498804&r1=1498803&r2=1498804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/sampling/Sampler.java (original)
+++ lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/sampling/Sampler.java Tue Jul  2 07:12:00 2013
@@ -12,7 +12,6 @@ import org.apache.lucene.facet.search.Fa
 import org.apache.lucene.facet.search.FacetResultNode;
 import org.apache.lucene.facet.search.ScoredDocIDs;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
-import org.apache.lucene.index.IndexReader;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -111,16 +110,6 @@ public abstract class Sampler {
       throws IOException;
 
   /**
-   * Get a fixer of sample facet accumulation results. Default implementation
-   * returns a <code>TakmiSampleFixer</code> which is adequate only for
-   * counting. For any other accumulator, provide a different fixer.
-   */
-  public SampleFixer getSampleFixer(IndexReader indexReader, TaxonomyReader taxonomyReader,
-      FacetSearchParams searchParams) {
-    return new TakmiSampleFixer(indexReader, taxonomyReader, searchParams);
-  }
-  
-  /**
    * Result of sample computation
    */
   public final static class SampleResult {
@@ -220,7 +209,7 @@ public abstract class Sampler {
       super(orig.categoryPath, num);
       this.orig = orig;
       setDepth(orig.getDepth());
-      setNumLabel(orig.getNumLabel());
+      setNumLabel(0); // don't label anything as we're over-sampling
       setResultMode(orig.getResultMode());
       setSortOrder(orig.getSortOrder());
     }

Modified: lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/sampling/SamplingAccumulator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/sampling/SamplingAccumulator.java?rev=1498804&r1=1498803&r2=1498804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/sampling/SamplingAccumulator.java (original)
+++ lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/sampling/SamplingAccumulator.java Tue Jul  2 07:12:00 2013
@@ -79,30 +79,43 @@ public class SamplingAccumulator extends
   public List<FacetResult> accumulate(ScoredDocIDs docids) throws IOException {
     // Replacing the original searchParams with the over-sampled
     FacetSearchParams original = searchParams;
-    searchParams = sampler.overSampledSearchParams(original);
+    SampleFixer samplerFixer = sampler.samplingParams.getSampleFixer();
+    final boolean shouldOversample = sampler.samplingParams.shouldOverSample();
+    if (shouldOversample) {
+      searchParams = sampler.overSampledSearchParams(original);
+    }
     
     List<FacetResult> sampleRes = super.accumulate(docids);
     
-    List<FacetResult> fixedRes = new ArrayList<FacetResult>();
+    List<FacetResult> results = new ArrayList<FacetResult>();
     for (FacetResult fres : sampleRes) {
       // for sure fres is not null because this is guaranteed by the delegee.
       PartitionsFacetResultsHandler frh = createFacetResultsHandler(fres.getFacetRequest());
-      // fix the result of current request
-      sampler.getSampleFixer(indexReader, taxonomyReader, searchParams).fixResult(docids, fres);
-      
-      fres = frh.rearrangeFacetResult(fres); // let delegee's handler do any arranging it needs to
-
-      // Using the sampler to trim the extra (over-sampled) results
-      fres = sampler.trimResult(fres);
+      if (samplerFixer != null) {
+        // fix the result of current request
+        samplerFixer.fixResult(docids, fres, samplingRatio);
+        
+        fres = frh.rearrangeFacetResult(fres); // let delegee's handler do any arranging it needs to
 
+        if (shouldOversample) {
+          // Using the sampler to trim the extra (over-sampled) results
+          fres = sampler.trimResult(fres);
+        }
+      }
+      
       // final labeling if allowed (because labeling is a costly operation)
-      frh.labelResult(fres);
-      fixedRes.add(fres); // add to final results
+      if (fres.getFacetResultNode().ordinal == TaxonomyReader.INVALID_ORDINAL) {
+        // category does not exist, add an empty result
+        results.add(emptyResult(fres.getFacetResultNode().ordinal, fres.getFacetRequest()));
+      } else {
+        frh.labelResult(fres);
+        results.add(fres);
+      }
     }
     
     searchParams = original; // Back to original params
     
-    return fixedRes; 
+    return results; 
   }
 
   @Override

Modified: lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/sampling/SamplingParams.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/sampling/SamplingParams.java?rev=1498804&r1=1498803&r2=1498804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/sampling/SamplingParams.java (original)
+++ lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/sampling/SamplingParams.java Tue Jul  2 07:12:00 2013
@@ -28,7 +28,7 @@ public class SamplingParams {
    * Default factor by which more results are requested over the sample set.
    * @see SamplingParams#getOversampleFactor()
    */
-  public static final double DEFAULT_OVERSAMPLE_FACTOR = 2d;
+  public static final double DEFAULT_OVERSAMPLE_FACTOR = 1d;
   
   /**
    * Default ratio between size of sample to original size of document set.
@@ -59,6 +59,8 @@ public class SamplingParams {
   private double sampleRatio = DEFAULT_SAMPLE_RATIO;
   private int samplingThreshold = DEFAULT_SAMPLING_THRESHOLD;
   private double oversampleFactor = DEFAULT_OVERSAMPLE_FACTOR;
+
+  private SampleFixer sampleFixer = null;
   
   /**
    * Return the maxSampleSize.
@@ -166,4 +168,29 @@ public class SamplingParams {
     this.oversampleFactor = oversampleFactor;
   }
 
-}
\ No newline at end of file
+  /**
+   * @return {@link SampleFixer} to be used while fixing the sampled results, if
+   *         <code>null</code> no fixing will be performed
+   */
+  public SampleFixer getSampleFixer() {
+    return sampleFixer;
+  }
+
+  /**
+   * Set a {@link SampleFixer} to be used while fixing the sampled results.
+   * {@code null} means no fixing will be performed
+   */
+  public void setSampleFixer(SampleFixer sampleFixer) {
+    this.sampleFixer = sampleFixer;
+  }
+
+  /**
+   * Returns whether over-sampling should be done. By default returns
+   * {@code true} when {@link #getSampleFixer()} is not {@code null} and
+   * {@link #getOversampleFactor()} &gt; 1, {@code false} otherwise.
+   */
+  public boolean shouldOverSample() {
+    return sampleFixer != null && oversampleFactor > 1d;
+  }
+  
+}

Modified: lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/sampling/SamplingWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/sampling/SamplingWrapper.java?rev=1498804&r1=1498803&r2=1498804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/sampling/SamplingWrapper.java (original)
+++ lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/sampling/SamplingWrapper.java Tue Jul  2 07:12:00 2013
@@ -10,6 +10,7 @@ import org.apache.lucene.facet.sampling.
 import org.apache.lucene.facet.search.FacetResult;
 import org.apache.lucene.facet.search.ScoredDocIDs;
 import org.apache.lucene.facet.search.StandardFacetsAccumulator;
+import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -52,31 +53,48 @@ public class SamplingWrapper extends Sta
   public List<FacetResult> accumulate(ScoredDocIDs docids) throws IOException {
     // Replacing the original searchParams with the over-sampled (and without statistics-compute)
     FacetSearchParams original = delegee.searchParams;
-    delegee.searchParams = sampler.overSampledSearchParams(original);
+    boolean shouldOversample = sampler.samplingParams.shouldOverSample();
+   
+    if (shouldOversample) {
+      delegee.searchParams = sampler.overSampledSearchParams(original);
+    }
     
     SampleResult sampleSet = sampler.getSampleSet(docids);
 
     List<FacetResult> sampleRes = delegee.accumulate(sampleSet.docids);
 
-    List<FacetResult> fixedRes = new ArrayList<FacetResult>();
+    List<FacetResult> results = new ArrayList<FacetResult>();
+    SampleFixer sampleFixer = sampler.samplingParams.getSampleFixer();
+    
     for (FacetResult fres : sampleRes) {
       // for sure fres is not null because this is guaranteed by the delegee.
       PartitionsFacetResultsHandler frh = createFacetResultsHandler(fres.getFacetRequest());
-      // fix the result of current request
-      sampler.getSampleFixer(indexReader, taxonomyReader, searchParams).fixResult(docids, fres); 
-      fres = frh.rearrangeFacetResult(fres); // let delegee's handler do any
+      if (sampleFixer != null) {
+        // fix the result of current request
+        sampleFixer.fixResult(docids, fres, sampleSet.actualSampleRatio); 
+        fres = frh.rearrangeFacetResult(fres); // let delegee's handler do any
+      }
       
-      // Using the sampler to trim the extra (over-sampled) results
-      fres = sampler.trimResult(fres);
+      if (shouldOversample) {
+        // Using the sampler to trim the extra (over-sampled) results
+        fres = sampler.trimResult(fres);
+      }
       
       // final labeling if allowed (because labeling is a costly operation)
-      frh.labelResult(fres);
-      fixedRes.add(fres); // add to final results
+      if (fres.getFacetResultNode().ordinal == TaxonomyReader.INVALID_ORDINAL) {
+        // category does not exist, add an empty result
+        results.add(emptyResult(fres.getFacetResultNode().ordinal, fres.getFacetRequest()));
+      } else {
+        frh.labelResult(fres);
+        results.add(fres);
+      }
     }
 
-    delegee.searchParams = original; // Back to original params
+    if (shouldOversample) {
+      delegee.searchParams = original; // Back to original params
+    }
     
-    return fixedRes; 
+    return results; 
   }
 
   @Override

Modified: lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/sampling/TakmiSampleFixer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/sampling/TakmiSampleFixer.java?rev=1498804&r1=1498803&r2=1498804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/sampling/TakmiSampleFixer.java (original)
+++ lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/sampling/TakmiSampleFixer.java Tue Jul  2 07:12:00 2013
@@ -2,21 +2,19 @@ package org.apache.lucene.facet.sampling
 
 import java.io.IOException;
 
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.MultiFields;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.util.Bits;
-
 import org.apache.lucene.facet.params.FacetSearchParams;
 import org.apache.lucene.facet.search.DrillDownQuery;
-import org.apache.lucene.facet.search.FacetResult;
 import org.apache.lucene.facet.search.FacetResultNode;
 import org.apache.lucene.facet.search.ScoredDocIDs;
 import org.apache.lucene.facet.search.ScoredDocIDsIterator;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.MultiFields;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.Bits;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -36,16 +34,21 @@ import org.apache.lucene.facet.taxonomy.
  */
 
 /**
- * Fix sampling results by counting the intersection between two lists: a
- * TermDocs (list of documents in a certain category) and a DocIdSetIterator
- * (list of documents matching the query).
- * 
+ * Fix sampling results by correct results, by counting the intersection between
+ * two lists: a TermDocs (list of documents in a certain category) and a
+ * DocIdSetIterator (list of documents matching the query).
+ * <p>
+ * This fixer is suitable for scenarios which prioritize accuracy over
+ * performance. 
+ * <p>
+ * <b>Note:</b> for statistically more accurate top-k selection, set
+ * {@link SamplingParams#setOversampleFactor(double) oversampleFactor} to at
+ * least 2, so that the top-k categories would have better chance of showing up
+ * in the sampled top-cK results (see {@link SamplingParams#getOversampleFactor}
  * 
  * @lucene.experimental
  */
-// TODO (Facet): implement also an estimated fixing by ratio (taking into
-// account "translation" of counts!)
-class TakmiSampleFixer implements SampleFixer {
+public class TakmiSampleFixer extends SampleFixer {
   
   private TaxonomyReader taxonomyReader;
   private IndexReader indexReader;
@@ -59,28 +62,10 @@ class TakmiSampleFixer implements Sample
   }
 
   @Override
-  public void fixResult(ScoredDocIDs origDocIds, FacetResult fres)
-      throws IOException {
-    FacetResultNode topRes = fres.getFacetResultNode();
-    fixResultNode(topRes, origDocIds);
-  }
-  
-  /**
-   * Fix result node count, and, recursively, fix all its children
-   * 
-   * @param facetResNode
-   *          result node to be fixed
-   * @param docIds
-   *          docids in effect
-   * @throws IOException If there is a low-level I/O error.
-   */
-  private void fixResultNode(FacetResultNode facetResNode, ScoredDocIDs docIds) throws IOException {
+  public void singleNodeFix(FacetResultNode facetResNode, ScoredDocIDs docIds, double samplingRatio) throws IOException {
     recount(facetResNode, docIds);
-    for (FacetResultNode frn : facetResNode.subResults) {
-      fixResultNode(frn, docIds);
-    }
   }
-
+  
   /**
    * Internal utility: recount for a facet result node
    * 
@@ -179,4 +164,5 @@ class TakmiSampleFixer implements Sample
     }
     return false; // exhausted
   }
+
 }
\ No newline at end of file

Modified: lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSideways.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSideways.java?rev=1498804&r1=1498803&r2=1498804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSideways.java (original)
+++ lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSideways.java Tue Jul  2 07:12:00 2013
@@ -402,16 +402,20 @@ public class DrillSideways {
       query = new DrillDownQuery(filter, query);
     }
     if (sort != null) {
+      int limit = searcher.getIndexReader().maxDoc();
+      if (limit == 0) {
+        limit = 1; // the collector does not alow numHits = 0
+      }
+      topN = Math.min(topN, limit);
       final TopFieldCollector hitCollector = TopFieldCollector.create(sort,
-                                                                      Math.min(topN, searcher.getIndexReader().maxDoc()),
+                                                                      topN,
                                                                       after,
                                                                       true,
                                                                       doDocScores,
                                                                       doMaxScore,
                                                                       true);
       DrillSidewaysResult r = search(query, hitCollector, fsp);
-      r.hits = hitCollector.topDocs();
-      return r;
+      return new DrillSidewaysResult(r.facetResults, hitCollector.topDocs());
     } else {
       return search(after, query, topN, fsp);
     }
@@ -423,10 +427,14 @@ public class DrillSideways {
    */
   public DrillSidewaysResult search(ScoreDoc after,
                                     DrillDownQuery query, int topN, FacetSearchParams fsp) throws IOException {
-    TopScoreDocCollector hitCollector = TopScoreDocCollector.create(Math.min(topN, searcher.getIndexReader().maxDoc()), after, true);
+    int limit = searcher.getIndexReader().maxDoc();
+    if (limit == 0) {
+      limit = 1; // the collector does not alow numHits = 0
+    }
+    topN = Math.min(topN, limit);
+    TopScoreDocCollector hitCollector = TopScoreDocCollector.create(topN, after, true);
     DrillSidewaysResult r = search(query, hitCollector, fsp);
-    r.hits = hitCollector.topDocs();
-    return r;
+    return new DrillSidewaysResult(r.facetResults, hitCollector.topDocs());
   }
 
   /** Override this to use a custom drill-down {@link
@@ -454,16 +462,20 @@ public class DrillSideways {
     return false;
   }
 
-  /** Represents the returned result from a drill sideways
-   *  search. */
+  /**
+   * Represents the returned result from a drill sideways search. Note that if
+   * you called
+   * {@link DrillSideways#search(DrillDownQuery, Collector, FacetSearchParams)},
+   * then {@link #hits} will be {@code null}.
+   */
   public static class DrillSidewaysResult {
     /** Combined drill down & sideways results. */
     public final List<FacetResult> facetResults;
 
     /** Hits. */
-    public TopDocs hits;
+    public final TopDocs hits;
 
-    DrillSidewaysResult(List<FacetResult> facetResults, TopDocs hits) {
+    public DrillSidewaysResult(List<FacetResult> facetResults, TopDocs hits) {
       this.facetResults = facetResults;
       this.hits = hits;
     }

Modified: lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResult.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResult.java?rev=1498804&r1=1498803&r2=1498804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResult.java (original)
+++ lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResult.java Tue Jul  2 07:12:00 2013
@@ -1,5 +1,16 @@
 package org.apache.lucene.facet.search;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.facet.taxonomy.CategoryPath;
+import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+import org.apache.lucene.util.CollectionUtil;
+
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -24,6 +35,140 @@ package org.apache.lucene.facet.search;
  */
 public class FacetResult {
   
+  private static FacetResultNode addIfNotExist(Map<CategoryPath, FacetResultNode> nodes, FacetResultNode node) {
+    FacetResultNode n = nodes.get(node.label);
+    if (n == null) {
+      nodes.put(node.label, node);
+      n = node;
+    }
+    return n;
+  }
+
+  /**
+   * A utility for merging multiple {@link FacetResult} of the same
+   * (hierarchical) dimension into a single {@link FacetResult}, to reconstruct
+   * the hierarchy. The results are merged according to the following rules:
+   * <ul>
+   * <li>If two results share the same dimension (first component in their
+   * {@link CategoryPath}), they are merged.
+   * <li>If a result is missing ancestors in the other results, e.g. A/B/C but
+   * no corresponding A or A/B, these nodes are 'filled' with their label,
+   * ordinal and value (obtained from the respective {@link FacetArrays}).
+   * <li>If a result does not share a dimension with other results, it is
+   * returned as is.
+   * </ul>
+   * <p>
+   * <b>NOTE:</b> the returned results are not guaranteed to be in the same
+   * order of the input ones.
+   * 
+   * @param results
+   *          the results to merge
+   * @param taxoReader
+   *          the {@link TaxonomyReader} to use when creating missing ancestor
+   *          nodes
+   * @param dimArrays
+   *          a mapping from a dimension to the respective {@link FacetArrays}
+   *          from which to pull the nodes values
+   */
+  public static List<FacetResult> mergeHierarchies(List<FacetResult> results, TaxonomyReader taxoReader,
+      Map<String, FacetArrays> dimArrays) throws IOException {
+    final Map<String, List<FacetResult>> dims = new HashMap<String,List<FacetResult>>();
+    for (FacetResult fr : results) {
+      String dim = fr.getFacetRequest().categoryPath.components[0];
+      List<FacetResult> frs = dims.get(dim);
+      if (frs == null) {
+        frs = new ArrayList<FacetResult>();
+        dims.put(dim, frs);
+      }
+      frs.add(fr);
+    }
+
+    final List<FacetResult> res = new ArrayList<FacetResult>();
+    for (List<FacetResult> frs : dims.values()) {
+      FacetResult mergedResult = frs.get(0);
+      if (frs.size() > 1) {
+        CollectionUtil.introSort(frs, new Comparator<FacetResult>() {
+          @Override
+          public int compare(FacetResult fr1, FacetResult fr2) {
+            return fr1.getFacetRequest().categoryPath.compareTo(fr2.getFacetRequest().categoryPath);
+          }
+        });
+        Map<CategoryPath, FacetResultNode> mergedNodes = new HashMap<CategoryPath,FacetResultNode>();
+        FacetArrays arrays = dimArrays != null ? dimArrays.get(frs.get(0).getFacetRequest().categoryPath.components[0]) : null;
+        for (FacetResult fr : frs) {
+          FacetResultNode frn = fr.getFacetResultNode();
+          FacetResultNode merged = mergedNodes.get(frn.label);
+          if (merged == null) {
+            CategoryPath parent = frn.label.subpath(frn.label.length - 1);
+            FacetResultNode childNode = frn;
+            FacetResultNode parentNode = null;
+            while (parent.length > 0 && (parentNode = mergedNodes.get(parent)) == null) {
+              int parentOrd = taxoReader.getOrdinal(parent);
+              double parentValue = arrays != null ? fr.getFacetRequest().getValueOf(arrays, parentOrd) : -1;
+              parentNode = new FacetResultNode(parentOrd, parentValue);
+              parentNode.label = parent;
+              parentNode.subResults = new ArrayList<FacetResultNode>();
+              parentNode.subResults.add(childNode);
+              mergedNodes.put(parent, parentNode);
+              childNode = parentNode;
+              parent = parent.subpath(parent.length - 1);
+            }
+
+            // at least one parent was added, so link the final (existing)
+            // parent with the child
+            if (parent.length > 0) {
+              if (!(parentNode.subResults instanceof ArrayList)) {
+                parentNode.subResults = new ArrayList<FacetResultNode>(parentNode.subResults);
+              }
+              parentNode.subResults.add(childNode);
+            }
+
+            // for missing FRNs, add new ones with label and value=-1
+            // first time encountered this label, add it and all its children to
+            // the map.
+            mergedNodes.put(frn.label, frn);
+            for (FacetResultNode child : frn.subResults) {
+              addIfNotExist(mergedNodes, child);
+            }
+          } else {
+            if (!(merged.subResults instanceof ArrayList)) {
+              merged.subResults = new ArrayList<FacetResultNode>(merged.subResults);
+            }
+            for (FacetResultNode sub : frn.subResults) {
+              // make sure sub wasn't already added
+              sub = addIfNotExist(mergedNodes, sub);
+              if (!merged.subResults.contains(sub)) {
+                merged.subResults.add(sub);
+              }
+            }
+          }
+        }
+        
+        // find the 'first' node to put on the FacetResult root
+        CategoryPath min = null;
+        for (CategoryPath cp : mergedNodes.keySet()) {
+          if (min == null || cp.compareTo(min) < 0) {
+            min = cp;
+          }
+        }
+        FacetRequest dummy = new FacetRequest(min, frs.get(0).getFacetRequest().numResults) {
+          @Override
+          public double getValueOf(FacetArrays arrays, int idx) {
+            throw new UnsupportedOperationException("not supported by this request");
+          }
+          
+          @Override
+          public FacetArraysSource getFacetArraysSource() {
+            throw new UnsupportedOperationException("not supported by this request");
+          }
+        };
+        mergedResult = new FacetResult(dummy, mergedNodes.get(min), -1);
+      }
+      res.add(mergedResult);
+    }
+    return res;
+  }
+
   private final FacetRequest facetRequest;
   private final FacetResultNode rootNode;
   private final int numValidDescendants;

Modified: lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/search/StandardFacetsAccumulator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/search/StandardFacetsAccumulator.java?rev=1498804&r1=1498803&r2=1498804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/search/StandardFacetsAccumulator.java (original)
+++ lucene/dev/branches/lucene4258/lucene/facet/src/java/org/apache/lucene/facet/search/StandardFacetsAccumulator.java Tue Jul  2 07:12:00 2013
@@ -94,7 +94,7 @@ public class StandardFacetsAccumulator e
 
   private Object accumulateGuard;
 
-  private double complementThreshold;
+  private double complementThreshold = DEFAULT_COMPLEMENT_THRESHOLD;
   
   public StandardFacetsAccumulator(FacetSearchParams searchParams, IndexReader indexReader, 
       TaxonomyReader taxonomyReader) {

Modified: lucene/dev/branches/lucene4258/lucene/facet/src/test/org/apache/lucene/facet/complements/TestFacetsAccumulatorWithComplement.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/facet/src/test/org/apache/lucene/facet/complements/TestFacetsAccumulatorWithComplement.java?rev=1498804&r1=1498803&r2=1498804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/facet/src/test/org/apache/lucene/facet/complements/TestFacetsAccumulatorWithComplement.java (original)
+++ lucene/dev/branches/lucene4258/lucene/facet/src/test/org/apache/lucene/facet/complements/TestFacetsAccumulatorWithComplement.java Tue Jul  2 07:12:00 2013
@@ -121,8 +121,8 @@ public class TestFacetsAccumulatorWithCo
     
     // Results are ready, printing them...
     int i = 0;
-    for (FacetResult facetResult : res) {
-      if (VERBOSE) {
+    if (VERBOSE) {
+      for (FacetResult facetResult : res) {
         System.out.println("Res "+(i++)+": "+facetResult);
       }
     }

Modified: lucene/dev/branches/lucene4258/lucene/facet/src/test/org/apache/lucene/facet/sampling/BaseSampleTestTopK.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/facet/src/test/org/apache/lucene/facet/sampling/BaseSampleTestTopK.java?rev=1498804&r1=1498803&r2=1498804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/facet/src/test/org/apache/lucene/facet/sampling/BaseSampleTestTopK.java (original)
+++ lucene/dev/branches/lucene4258/lucene/facet/src/test/org/apache/lucene/facet/sampling/BaseSampleTestTopK.java Tue Jul  2 07:12:00 2013
@@ -94,7 +94,7 @@ public abstract class BaseSampleTestTopK
         for (int nTrial = 0; nTrial < RETRIES; nTrial++) {
           try {
             // complement with sampling!
-            final Sampler sampler = createSampler(nTrial, useRandomSampler);
+            final Sampler sampler = createSampler(nTrial, useRandomSampler, samplingSearchParams);
             
             assertSampling(expectedResults, q, sampler, samplingSearchParams, false);
             assertSampling(expectedResults, q, sampler, samplingSearchParams, true);
@@ -128,14 +128,20 @@ public abstract class BaseSampleTestTopK
     return FacetsCollector.create(sfa);
   }
   
-  private Sampler createSampler(int nTrial, boolean useRandomSampler) {
+  private Sampler createSampler(int nTrial, boolean useRandomSampler, FacetSearchParams sParams) {
     SamplingParams samplingParams = new SamplingParams();
     
+    /*
+     * Set sampling to Exact fixing with TakmiSampleFixer as it is not easy to
+     * validate results with amortized results. 
+     */
+    samplingParams.setSampleFixer(new TakmiSampleFixer(indexReader, taxoReader, sParams));
+        
     final double retryFactor = Math.pow(1.01, nTrial);
+    samplingParams.setOversampleFactor(5.0 * retryFactor); // Oversampling 
     samplingParams.setSampleRatio(0.8 * retryFactor);
     samplingParams.setMinSampleSize((int) (100 * retryFactor));
     samplingParams.setMaxSampleSize((int) (10000 * retryFactor));
-    samplingParams.setOversampleFactor(5.0 * retryFactor);
     samplingParams.setSamplingThreshold(11000); //force sampling
 
     Sampler sampler = useRandomSampler ? 

Added: lucene/dev/branches/lucene4258/lucene/facet/src/test/org/apache/lucene/facet/sampling/SamplerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/facet/src/test/org/apache/lucene/facet/sampling/SamplerTest.java?rev=1498804&view=auto
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/facet/src/test/org/apache/lucene/facet/sampling/SamplerTest.java (added)
+++ lucene/dev/branches/lucene4258/lucene/facet/src/test/org/apache/lucene/facet/sampling/SamplerTest.java Tue Jul  2 07:12:00 2013
@@ -0,0 +1,111 @@
+package org.apache.lucene.facet.sampling;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.facet.FacetTestBase;
+import org.apache.lucene.facet.params.FacetIndexingParams;
+import org.apache.lucene.facet.params.FacetSearchParams;
+import org.apache.lucene.facet.search.CountFacetRequest;
+import org.apache.lucene.facet.search.FacetResultNode;
+import org.apache.lucene.facet.search.FacetsCollector;
+import org.apache.lucene.facet.search.StandardFacetsAccumulator;
+import org.apache.lucene.facet.taxonomy.CategoryPath;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.junit.After;
+import org.junit.Before;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+public class SamplerTest extends FacetTestBase {
+  
+  private FacetIndexingParams fip;
+  
+  @Override
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+    fip = getFacetIndexingParams(Integer.MAX_VALUE);
+    initIndex(fip);
+  }
+  
+  @Override
+  protected int numDocsToIndex() {
+    return 100;
+  }
+  
+  @Override
+  protected List<CategoryPath> getCategories(final int doc) {
+    return new ArrayList<CategoryPath>() {
+      {
+        add(new CategoryPath("root", "a", Integer.toString(doc % 10)));
+      }
+    };
+  }
+  
+  @Override
+  protected String getContent(int doc) {
+    return "";
+  }
+  
+  @Override
+  @After
+  public void tearDown() throws Exception {
+    closeAll();
+    super.tearDown();
+  }
+  
+  public void testDefaultFixer() throws Exception {
+    RandomSampler randomSampler = new RandomSampler();
+    SampleFixer fixer = randomSampler.samplingParams.getSampleFixer();
+    assertEquals(null, fixer);
+  }
+  
+  public void testCustomFixer() throws Exception {
+    SamplingParams sp = new SamplingParams();
+    sp.setSampleFixer(new TakmiSampleFixer(null, null, null));
+    assertEquals(TakmiSampleFixer.class, sp.getSampleFixer().getClass());
+  }
+  
+  public void testNoFixing() throws Exception {
+    SamplingParams sp = new SamplingParams();
+    sp.setMaxSampleSize(10);
+    sp.setMinSampleSize(5);
+    sp.setSampleRatio(0.01d);
+    sp.setSamplingThreshold(50);
+    sp.setOversampleFactor(5d);
+    
+    assertNull("Fixer should be null as the test is for no-fixing",
+        sp.getSampleFixer());
+    FacetSearchParams fsp = new FacetSearchParams(fip, new CountFacetRequest(
+        new CategoryPath("root", "a"), 1));
+    SamplingAccumulator accumulator = new SamplingAccumulator(
+        new RandomSampler(sp, random()), fsp, indexReader, taxoReader);
+    
+    // Make sure no complements are in action
+    accumulator
+        .setComplementThreshold(StandardFacetsAccumulator.DISABLE_COMPLEMENT);
+    
+    FacetsCollector fc = FacetsCollector.create(accumulator);
+    
+    searcher.search(new MatchAllDocsQuery(), fc);
+    FacetResultNode node = fc.getFacetResults().get(0).getFacetResultNode();
+    
+    assertTrue(node.value < numDocsToIndex());
+  }
+}

Modified: lucene/dev/branches/lucene4258/lucene/facet/src/test/org/apache/lucene/facet/search/FacetRequestTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/facet/src/test/org/apache/lucene/facet/search/FacetRequestTest.java?rev=1498804&r1=1498803&r2=1498804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/facet/src/test/org/apache/lucene/facet/search/FacetRequestTest.java (original)
+++ lucene/dev/branches/lucene4258/lucene/facet/src/test/org/apache/lucene/facet/search/FacetRequestTest.java Tue Jul  2 07:12:00 2013
@@ -23,7 +23,7 @@ import org.junit.Test;
  */
 
 public class FacetRequestTest extends FacetTestCase {
-
+  
   @Test(expected=IllegalArgumentException.class)
   public void testIllegalNumResults() throws Exception {
     assertNotNull(new CountFacetRequest(new CategoryPath("a", "b"), 0));
@@ -33,7 +33,7 @@ public class FacetRequestTest extends Fa
   public void testIllegalCategoryPath() throws Exception {
     assertNotNull(new CountFacetRequest(null, 1));
   }
-
+  
   @Test
   public void testHashAndEquals() {
     CountFacetRequest fr1 = new CountFacetRequest(new CategoryPath("a"), 8);

Added: lucene/dev/branches/lucene4258/lucene/facet/src/test/org/apache/lucene/facet/search/FacetResultTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/facet/src/test/org/apache/lucene/facet/search/FacetResultTest.java?rev=1498804&view=auto
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/facet/src/test/org/apache/lucene/facet/search/FacetResultTest.java (added)
+++ lucene/dev/branches/lucene4258/lucene/facet/src/test/org/apache/lucene/facet/search/FacetResultTest.java Tue Jul  2 07:12:00 2013
@@ -0,0 +1,204 @@
+package org.apache.lucene.facet.search;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.facet.FacetTestCase;
+import org.apache.lucene.facet.FacetTestUtils;
+import org.apache.lucene.facet.index.FacetFields;
+import org.apache.lucene.facet.params.FacetIndexingParams;
+import org.apache.lucene.facet.params.FacetSearchParams;
+import org.apache.lucene.facet.search.DrillSideways.DrillSidewaysResult;
+import org.apache.lucene.facet.taxonomy.CategoryPath;
+import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader;
+import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.RAMDirectory;
+import org.apache.lucene.util.CollectionUtil;
+import org.apache.lucene.util.IOUtils;
+import org.junit.Test;
+
+public class FacetResultTest extends FacetTestCase {
+  
+  private Document newDocument(FacetFields facetFields, String... categories) throws IOException {
+    Document doc = new Document();
+    List<CategoryPath> cats = new ArrayList<CategoryPath>();
+    for (String cat : categories) {
+      cats.add(new CategoryPath(cat, '/'));
+    }
+    facetFields.addFields(doc, cats);
+    return doc;
+  }
+  
+  private void initIndex(Directory indexDir, Directory taxoDir) throws IOException {
+    IndexWriterConfig conf = new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+    IndexWriter indexWriter = new IndexWriter(indexDir, conf);
+    DirectoryTaxonomyWriter taxoWriter = new DirectoryTaxonomyWriter(taxoDir);
+    FacetFields facetFields = new FacetFields(taxoWriter);
+    indexWriter.addDocument(newDocument(facetFields, "Date/2010/March/12", "A/1"));
+    indexWriter.addDocument(newDocument(facetFields, "Date/2010/March/23", "A/2"));
+    indexWriter.addDocument(newDocument(facetFields, "Date/2010/April/17", "A/3"));
+    indexWriter.addDocument(newDocument(facetFields, "Date/2010/May/18", "A/1"));
+    indexWriter.addDocument(newDocument(facetFields, "Date/2011/January/1", "A/3"));
+    indexWriter.addDocument(newDocument(facetFields, "Date/2011/February/12", "A/1"));
+    indexWriter.addDocument(newDocument(facetFields, "Date/2011/February/18", "A/4"));
+    indexWriter.addDocument(newDocument(facetFields, "Date/2012/August/15", "A/1"));
+    indexWriter.addDocument(newDocument(facetFields, "Date/2012/July/5", "A/2"));
+    indexWriter.addDocument(newDocument(facetFields, "Date/2013/September/13", "A/1"));
+    indexWriter.addDocument(newDocument(facetFields, "Date/2013/September/25", "A/4"));
+    IOUtils.close(indexWriter, taxoWriter);
+  }
+  
+  private void searchIndex(TaxonomyReader taxoReader, IndexSearcher searcher, boolean fillMissingCounts, String[] exp,
+      String[][] drillDowns, int[] numResults) throws IOException {
+    CategoryPath[][] cps = new CategoryPath[drillDowns.length][];
+    for (int i = 0; i < cps.length; i++) {
+      cps[i] = new CategoryPath[drillDowns[i].length];
+      for (int j = 0; j < cps[i].length; j++) {
+        cps[i][j] = new CategoryPath(drillDowns[i][j], '/');
+      }
+    }
+    DrillDownQuery ddq = new DrillDownQuery(FacetIndexingParams.DEFAULT, new MatchAllDocsQuery());
+    for (CategoryPath[] cats : cps) {
+      ddq.add(cats);
+    }
+    
+    List<FacetRequest> facetRequests = new ArrayList<FacetRequest>();
+    for (CategoryPath[] cats : cps) {
+      for (int i = 0; i < cats.length; i++) {
+        CategoryPath cp = cats[i];
+        int numres = numResults == null ? 2 : numResults[i];
+        // for each drill-down, add itself as well as its parent as requests, so
+        // we get the drill-sideways
+        facetRequests.add(new CountFacetRequest(cp, numres));
+        CountFacetRequest parent = new CountFacetRequest(cp.subpath(cp.length - 1), numres);
+        if (!facetRequests.contains(parent) && parent.categoryPath.length > 0) {
+          facetRequests.add(parent);
+        }
+      }
+    }
+    
+    FacetSearchParams fsp = new FacetSearchParams(facetRequests);
+    final DrillSideways ds;
+    final Map<String,FacetArrays> dimArrays;
+    if (fillMissingCounts) {
+      dimArrays = new HashMap<String,FacetArrays>();
+      ds = new DrillSideways(searcher, taxoReader) {
+        @Override
+        protected FacetsAccumulator getDrillSidewaysAccumulator(String dim, FacetSearchParams fsp) throws IOException {
+          FacetsAccumulator fa = super.getDrillSidewaysAccumulator(dim, fsp);
+          dimArrays.put(dim, fa.facetArrays);
+          return fa;
+        }
+      };
+    } else {
+      ds = new DrillSideways(searcher, taxoReader);
+      dimArrays = null;
+    }
+    
+    final DrillSidewaysResult sidewaysRes = ds.search(null, ddq, 5, fsp);
+    List<FacetResult> facetResults = FacetResult.mergeHierarchies(sidewaysRes.facetResults, taxoReader, dimArrays);
+    CollectionUtil.introSort(facetResults, new Comparator<FacetResult>() {
+      @Override
+      public int compare(FacetResult o1, FacetResult o2) {
+        return o1.getFacetRequest().categoryPath.compareTo(o2.getFacetRequest().categoryPath);
+      }
+    });
+    assertEquals(exp.length, facetResults.size()); // A + single one for date
+    for (int i = 0; i < facetResults.size(); i++) {
+      assertEquals(exp[i], FacetTestUtils.toSimpleString(facetResults.get(i)));
+    }
+  }
+  
+  @Test
+  public void testMergeHierarchies() throws Exception {
+    Directory indexDir = new RAMDirectory(), taxoDir = new RAMDirectory();
+    initIndex(indexDir, taxoDir);
+    
+    DirectoryReader indexReader = DirectoryReader.open(indexDir);
+    TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoDir);
+    IndexSearcher searcher = new IndexSearcher(indexReader);
+    
+    String[] exp = new String[] { "Date (0)\n  2010 (4)\n  2011 (3)\n" };
+    searchIndex(taxoReader, searcher, false, exp, new String[][] { new String[] { "Date" } }, null);
+    
+    // two dimensions
+    exp = new String[] { "A (0)\n  1 (5)\n  4 (2)\n", "Date (0)\n  2010 (4)\n  2011 (3)\n" };
+    searchIndex(taxoReader, searcher, false, exp, new String[][] { new String[] { "Date" }, new String[] { "A" } }, null);
+    
+    // both parent and child are OR'd
+    exp = new String[] { "Date (-1)\n  2010 (4)\n    March (2)\n      23 (1)\n      12 (1)\n    May (1)\n" };
+    searchIndex(taxoReader, searcher, false, exp, new String[][] { new String[] { "Date/2010/March", "Date/2010/March/23" }}, null);
+    
+    // both parent and child are OR'd (fill counts)
+    exp = new String[] { "Date (0)\n  2010 (4)\n    March (2)\n      23 (1)\n      12 (1)\n    May (1)\n" };
+    searchIndex(taxoReader, searcher, true, exp, new String[][] { new String[] { "Date/2010/March", "Date/2010/March/23" }}, null);
+    
+    // same DD twice
+    exp = new String[] { "Date (0)\n  2010 (4)\n    March (2)\n    May (1)\n  2011 (3)\n" };
+    searchIndex(taxoReader, searcher, false, exp, new String[][] { new String[] { "Date/2010", "Date/2010" }}, null);
+    
+    exp = new String[] { "Date (0)\n  2010 (4)\n    March (2)\n    May (1)\n  2011 (3)\n" };
+    searchIndex(taxoReader, searcher, false, exp, new String[][] { new String[] { "Date/2010" }}, null);
+    
+    exp = new String[] { "Date (0)\n  2010 (4)\n    March (2)\n    May (1)\n  2011 (3)\n    February (2)\n    January (1)\n" };
+    searchIndex(taxoReader, searcher, false, exp, new String[][] { new String[] { "Date/2010", "Date/2011" }}, null);
+    
+    exp = new String[] { "Date (0)\n  2010 (4)\n    March (2)\n      23 (1)\n      12 (1)\n    May (1)\n  2011 (3)\n    February (2)\n    January (1)\n" };
+    searchIndex(taxoReader, searcher, false, exp, new String[][] { new String[] { "Date/2010/March", "Date/2011" }}, null);
+    
+    // Date/2010/April not in top-2 of Date/2010
+    exp = new String[] { "Date (0)\n  2010 (4)\n    March (2)\n      23 (1)\n      12 (1)\n    May (1)\n    April (1)\n      17 (1)\n  2011 (3)\n    February (2)\n    January (1)\n" };
+    searchIndex(taxoReader, searcher, false, exp, new String[][] { new String[] { "Date/2010/March", "Date/2010/April", "Date/2011" }}, null);
+    
+    // missing ancestors
+    exp = new String[] { "Date (-1)\n  2010 (4)\n    March (2)\n    May (1)\n    April (1)\n      17 (1)\n  2011 (-1)\n    January (1)\n      1 (1)\n" };
+    searchIndex(taxoReader, searcher, false, exp, new String[][] { new String[] { "Date/2011/January/1", "Date/2010/April" }}, null);
+    
+    // missing ancestors (fill counts)
+    exp = new String[] { "Date (0)\n  2010 (4)\n    March (2)\n    May (1)\n    April (1)\n      17 (1)\n  2011 (3)\n    January (1)\n      1 (1)\n" };
+    searchIndex(taxoReader, searcher, true, exp, new String[][] { new String[] { "Date/2011/January/1", "Date/2010/April" }}, null);
+    
+    // non-hierarchical dimension with both parent and child
+    exp = new String[] { "A (0)\n  1 (5)\n  4 (2)\n  3 (2)\n" };
+    searchIndex(taxoReader, searcher, INFOSTREAM, exp, new String[][] { new String[] { "A", "A/3" }}, null);
+    
+    // non-hierarchical dimension with same request but different numResults
+    exp = new String[] { "A (0)\n  1 (5)\n  4 (2)\n  3 (2)\n  2 (2)\n" };
+    searchIndex(taxoReader, searcher, INFOSTREAM, exp, new String[][] { new String[] { "A", "A" }}, new int[] { 2, 4 });
+    
+    IOUtils.close(indexReader, taxoReader);
+    
+    IOUtils.close(indexDir, taxoDir);
+  }
+  
+}

Modified: lucene/dev/branches/lucene4258/lucene/facet/src/test/org/apache/lucene/facet/search/TestDrillSideways.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/facet/src/test/org/apache/lucene/facet/search/TestDrillSideways.java?rev=1498804&r1=1498803&r2=1498804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/facet/src/test/org/apache/lucene/facet/search/TestDrillSideways.java (original)
+++ lucene/dev/branches/lucene4258/lucene/facet/src/test/org/apache/lucene/facet/search/TestDrillSideways.java Tue Jul  2 07:12:00 2013
@@ -59,15 +59,18 @@ import org.apache.lucene.search.ScoreDoc
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
+import org.apache.lucene.search.SortField.Type;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.InPlaceMergeSorter;
 import org.apache.lucene.util.InfoStream;
 import org.apache.lucene.util._TestUtil;
+import org.junit.Test;
 
 public class TestDrillSideways extends FacetTestCase {
 
@@ -1144,5 +1147,34 @@ public class TestDrillSideways extends F
     }
     return b.toString();
   }
+  
+  @Test
+  public void testEmptyIndex() throws Exception {
+    // LUCENE-5045: make sure DrillSideways works with an empty index
+    Directory dir = newDirectory();
+    Directory taxoDir = newDirectory();
+    writer = new RandomIndexWriter(random(), dir);
+    taxoWriter = new DirectoryTaxonomyWriter(taxoDir, IndexWriterConfig.OpenMode.CREATE);
+    IndexSearcher searcher = newSearcher(writer.getReader());
+    writer.close();
+    TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoWriter);
+    taxoWriter.close();
+
+    // Count "Author"
+    FacetSearchParams fsp = new FacetSearchParams(new CountFacetRequest(new CategoryPath("Author"), 10));
+
+    DrillSideways ds = new DrillSideways(searcher, taxoReader);
+    DrillDownQuery ddq = new DrillDownQuery(fsp.indexingParams, new MatchAllDocsQuery());
+    ddq.add(new CategoryPath("Author", "Lisa"));
+    
+    DrillSidewaysResult r = ds.search(null, ddq, 10, fsp); // this used to fail on IllegalArgEx
+    assertEquals(0, r.hits.totalHits);
+
+    r = ds.search(ddq, null, null, 10, new Sort(new SortField("foo", Type.INT)), false, false, fsp); // this used to fail on IllegalArgEx
+    assertEquals(0, r.hits.totalHits);
+    
+    IOUtils.close(searcher.getIndexReader(), taxoReader, dir, taxoDir);
+  }
+  
 }
 

Modified: lucene/dev/branches/lucene4258/lucene/facet/src/test/org/apache/lucene/facet/search/TestFacetsCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/facet/src/test/org/apache/lucene/facet/search/TestFacetsCollector.java?rev=1498804&r1=1498803&r2=1498804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/facet/src/test/org/apache/lucene/facet/search/TestFacetsCollector.java (original)
+++ lucene/dev/branches/lucene4258/lucene/facet/src/test/org/apache/lucene/facet/search/TestFacetsCollector.java Tue Jul  2 07:12:00 2013
@@ -17,8 +17,20 @@ import org.apache.lucene.facet.params.Ca
 import org.apache.lucene.facet.params.FacetIndexingParams;
 import org.apache.lucene.facet.params.FacetSearchParams;
 import org.apache.lucene.facet.params.PerDimensionIndexingParams;
+import org.apache.lucene.facet.range.LongRange;
+import org.apache.lucene.facet.range.RangeAccumulator;
+import org.apache.lucene.facet.range.RangeFacetRequest;
+import org.apache.lucene.facet.sampling.RandomSampler;
+import org.apache.lucene.facet.sampling.Sampler;
+import org.apache.lucene.facet.sampling.SamplingAccumulator;
+import org.apache.lucene.facet.sampling.SamplingParams;
+import org.apache.lucene.facet.sampling.SamplingWrapper;
+import org.apache.lucene.facet.sampling.TakmiSampleFixer;
 import org.apache.lucene.facet.search.FacetRequest.ResultMode;
+import org.apache.lucene.facet.sortedset.SortedSetDocValuesAccumulator;
+import org.apache.lucene.facet.sortedset.SortedSetDocValuesReaderState;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
+import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 import org.apache.lucene.facet.taxonomy.TaxonomyWriter;
 import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader;
 import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter;
@@ -384,5 +396,72 @@ public class TestFacetsCollector extends
     
     IOUtils.close(taxo, taxoDir, r, indexDir);
   }
-  
+
+  @Test
+  public void testLabeling() throws Exception {
+    Directory indexDir = newDirectory(), taxoDir = newDirectory();
+
+    // create the index
+    IndexWriter indexWriter = new IndexWriter(indexDir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())));
+    DirectoryTaxonomyWriter taxoWriter = new DirectoryTaxonomyWriter(taxoDir);
+    FacetFields facetFields = new FacetFields(taxoWriter);
+    Document doc = new Document();
+    facetFields.addFields(doc, Arrays.asList(new CategoryPath("A/1", '/')));
+    indexWriter.addDocument(doc);
+    IOUtils.close(indexWriter, taxoWriter);
+    
+    DirectoryReader indexReader = DirectoryReader.open(indexDir);
+    TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoDir);
+    IndexSearcher searcher = new IndexSearcher(indexReader);
+    // ask to count a non-existing category to test labeling
+    FacetSearchParams fsp = new FacetSearchParams(new CountFacetRequest(new CategoryPath("B"), 5));
+    
+    final SamplingParams sampleParams = new SamplingParams();
+    sampleParams.setMaxSampleSize(100);
+    sampleParams.setMinSampleSize(100);
+    sampleParams.setSamplingThreshold(100);
+    sampleParams.setOversampleFactor(1.0d);
+    if (random().nextBoolean()) {
+      sampleParams.setSampleFixer(new TakmiSampleFixer(indexReader, taxoReader, fsp));
+    }
+    final Sampler sampler = new RandomSampler(sampleParams, random());
+    
+    FacetsAccumulator[] accumulators = new FacetsAccumulator[] {
+      new FacetsAccumulator(fsp, indexReader, taxoReader),
+      new StandardFacetsAccumulator(fsp, indexReader, taxoReader),
+      new SamplingAccumulator(sampler, fsp, indexReader, taxoReader),
+      new AdaptiveFacetsAccumulator(fsp, indexReader, taxoReader),
+      new SamplingWrapper(new StandardFacetsAccumulator(fsp, indexReader, taxoReader), sampler)
+    };
+    
+    for (FacetsAccumulator fa : accumulators) {
+      FacetsCollector fc = FacetsCollector.create(fa);
+      searcher.search(new MatchAllDocsQuery(), fc);
+      List<FacetResult> facetResults = fc.getFacetResults();
+      assertNotNull(facetResults);
+      assertEquals("incorrect label returned for " + fa, fsp.facetRequests.get(0).categoryPath, facetResults.get(0).getFacetResultNode().label);
+    }
+    
+    try {
+      // SortedSetDocValuesAccumulator cannot even be created in such state
+      assertNull(new SortedSetDocValuesAccumulator(fsp, new SortedSetDocValuesReaderState(indexReader)));
+      // if this ever changes, make sure FacetResultNode is labeled correctly 
+      fail("should not have succeeded to execute a request over a category which wasn't indexed as SortedSetDVField");
+    } catch (IllegalArgumentException e) {
+      // expected
+    }
+
+    fsp = new FacetSearchParams(new RangeFacetRequest<LongRange>("f", new LongRange("grr", 0, true, 1, true)));
+    RangeAccumulator ra = new RangeAccumulator(fsp, indexReader);
+    FacetsCollector fc = FacetsCollector.create(ra);
+    searcher.search(new MatchAllDocsQuery(), fc);
+    List<FacetResult> facetResults = fc.getFacetResults();
+    assertNotNull(facetResults);
+    assertEquals("incorrect label returned for RangeAccumulator", fsp.facetRequests.get(0).categoryPath, facetResults.get(0).getFacetResultNode().label);
+
+    IOUtils.close(indexReader, taxoReader);
+
+    IOUtils.close(indexDir, taxoDir);
+  }
+
 }

Modified: lucene/dev/branches/lucene4258/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java?rev=1498804&r1=1498803&r2=1498804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java (original)
+++ lucene/dev/branches/lucene4258/lucene/grouping/src/test/org/apache/lucene/search/grouping/AllGroupHeadsCollectorTest.java Tue Jul  2 07:12:00 2013
@@ -72,74 +72,78 @@ public class AllGroupHeadsCollectorTest 
         dir,
         newIndexWriterConfig(TEST_VERSION_CURRENT,
             new MockAnalyzer(random())).setMergePolicy(newLogMergePolicy()));
-    boolean canUseIDV = true;
     DocValuesType valueType = vts[random().nextInt(vts.length)];
 
     // 0
     Document doc = new Document();
-    addGroupField(doc, groupField, "author1", canUseIDV, valueType);
-    doc.add(newTextField("content", "random text", Field.Store.YES));
-    doc.add(newStringField("id", "1", Field.Store.YES));
+    addGroupField(doc, groupField, "author1", valueType);
+    doc.add(newTextField("content", "random text", Field.Store.NO));
+    doc.add(newStringField("id_1", "1", Field.Store.NO));
+    doc.add(newStringField("id_2", "1", Field.Store.NO));
     w.addDocument(doc);
 
     // 1
     doc = new Document();
-    addGroupField(doc, groupField, "author1", canUseIDV, valueType);
-    doc.add(newTextField("content", "some more random text blob", Field.Store.YES));
-    doc.add(newStringField("id", "2", Field.Store.YES));
+    addGroupField(doc, groupField, "author1", valueType);
+    doc.add(newTextField("content", "some more random text blob", Field.Store.NO));
+    doc.add(newStringField("id_1", "2", Field.Store.NO));
+    doc.add(newStringField("id_2", "2", Field.Store.NO));
     w.addDocument(doc);
 
     // 2
     doc = new Document();
-    addGroupField(doc, groupField, "author1", canUseIDV, valueType);
-    doc.add(newTextField("content", "some more random textual data", Field.Store.YES));
-    doc.add(newStringField("id", "3", Field.Store.YES));
+    addGroupField(doc, groupField, "author1", valueType);
+    doc.add(newTextField("content", "some more random textual data", Field.Store.NO));
+    doc.add(newStringField("id_1", "3", Field.Store.NO));
+    doc.add(newStringField("id_2", "3", Field.Store.NO));
     w.addDocument(doc);
     w.commit(); // To ensure a second segment
 
     // 3
     doc = new Document();
-    addGroupField(doc, groupField, "author2", canUseIDV, valueType);
-    doc.add(newTextField("content", "some random text", Field.Store.YES));
-    doc.add(newStringField("id", "4", Field.Store.YES));
+    addGroupField(doc, groupField, "author2", valueType);
+    doc.add(newTextField("content", "some random text", Field.Store.NO));
+    doc.add(newStringField("id_1", "4", Field.Store.NO));
+    doc.add(newStringField("id_2", "4", Field.Store.NO));
     w.addDocument(doc);
 
     // 4
     doc = new Document();
-    addGroupField(doc, groupField, "author3", canUseIDV, valueType);
-    doc.add(newTextField("content", "some more random text", Field.Store.YES));
-    doc.add(newStringField("id", "5", Field.Store.YES));
+    addGroupField(doc, groupField, "author3", valueType);
+    doc.add(newTextField("content", "some more random text", Field.Store.NO));
+    doc.add(newStringField("id_1", "5", Field.Store.NO));
+    doc.add(newStringField("id_2", "5", Field.Store.NO));
     w.addDocument(doc);
 
     // 5
     doc = new Document();
-    addGroupField(doc, groupField, "author3", canUseIDV, valueType);
-    doc.add(newTextField("content", "random blob", Field.Store.YES));
-    doc.add(newStringField("id", "6", Field.Store.YES));
+    addGroupField(doc, groupField, "author3", valueType);
+    doc.add(newTextField("content", "random blob", Field.Store.NO));
+    doc.add(newStringField("id_1", "6", Field.Store.NO));
+    doc.add(newStringField("id_2", "6", Field.Store.NO));
     w.addDocument(doc);
 
     // 6 -- no author field
     doc = new Document();
-    doc.add(newTextField("content", "random word stuck in alot of other text", Field.Store.YES));
-    doc.add(newStringField("id", "6", Field.Store.YES));
+    doc.add(newTextField("content", "random word stuck in alot of other text", Field.Store.NO));
+    doc.add(newStringField("id_1", "6", Field.Store.NO));
+    doc.add(newStringField("id_2", "6", Field.Store.NO));
     w.addDocument(doc);
 
     // 7 -- no author field
     doc = new Document();
-    doc.add(newTextField("content", "random word stuck in alot of other text", Field.Store.YES));
-    doc.add(newStringField("id", "7", Field.Store.YES));
+    doc.add(newTextField("content", "random word stuck in alot of other text", Field.Store.NO));
+    doc.add(newStringField("id_1", "7", Field.Store.NO));
+    doc.add(newStringField("id_2", "7", Field.Store.NO));
     w.addDocument(doc);
 
     IndexReader reader = w.getReader();
     IndexSearcher indexSearcher = newSearcher(reader);
-    if (SlowCompositeReaderWrapper.class.isAssignableFrom(reader.getClass())) {
-      canUseIDV = false;
-    }
 
     w.close();
     int maxDoc = reader.maxDoc();
 
-    Sort sortWithinGroup = new Sort(new SortField("id", SortField.Type.INT, true));
+    Sort sortWithinGroup = new Sort(new SortField("id_1", SortField.Type.INT, true));
     AbstractAllGroupHeadsCollector<?> allGroupHeadsCollector = createRandomCollector(groupField, sortWithinGroup);
     indexSearcher.search(new TermQuery(new Term("content", "random")), allGroupHeadsCollector);
     assertTrue(arrayContains(new int[]{2, 3, 5, 7}, allGroupHeadsCollector.retrieveGroupHeads()));
@@ -156,13 +160,13 @@ public class AllGroupHeadsCollectorTest 
     assertTrue(openBitSetContains(new int[]{1, 5}, allGroupHeadsCollector.retrieveGroupHeads(maxDoc), maxDoc));
 
     // STRING sort type triggers different implementation
-    Sort sortWithinGroup2 = new Sort(new SortField("id", SortField.Type.STRING, true));
+    Sort sortWithinGroup2 = new Sort(new SortField("id_2", SortField.Type.STRING, true));
     allGroupHeadsCollector = createRandomCollector(groupField, sortWithinGroup2);
     indexSearcher.search(new TermQuery(new Term("content", "random")), allGroupHeadsCollector);
     assertTrue(arrayContains(new int[]{2, 3, 5, 7}, allGroupHeadsCollector.retrieveGroupHeads()));
     assertTrue(openBitSetContains(new int[]{2, 3, 5, 7}, allGroupHeadsCollector.retrieveGroupHeads(maxDoc), maxDoc));
 
-    Sort sortWithinGroup3 = new Sort(new SortField("id", SortField.Type.STRING, false));
+    Sort sortWithinGroup3 = new Sort(new SortField("id_2", SortField.Type.STRING, false));
     allGroupHeadsCollector = createRandomCollector(groupField, sortWithinGroup3);
     indexSearcher.search(new TermQuery(new Term("content", "random")), allGroupHeadsCollector);
     // 7 b/c higher doc id wins, even if order of field is in not in reverse.
@@ -402,6 +406,7 @@ public class AllGroupHeadsCollectorTest 
       for (int a : actual) {
         if (e == a) {
           found = true;
+          break;
         }
       }
 
@@ -539,11 +544,10 @@ public class AllGroupHeadsCollectorTest 
     return collector;
   }
 
-  private void addGroupField(Document doc, String groupField, String value, boolean canUseIDV, DocValuesType valueType) {
-    doc.add(new TextField(groupField, value, Field.Store.YES));
-    if (canUseIDV) {
-      Field valuesField = null;
-      switch(valueType) {
+  private void addGroupField(Document doc, String groupField, String value, DocValuesType valueType) {
+    doc.add(new TextField(groupField, value, Field.Store.NO));
+    Field valuesField = null;
+    switch(valueType) {
       case BINARY:
         valuesField = new BinaryDocValuesField(groupField + "_dv", new BytesRef(value));
         break;
@@ -552,9 +556,8 @@ public class AllGroupHeadsCollectorTest 
         break;
       default:
         fail("unhandled type");
-      }
-      doc.add(valuesField);
     }
+    doc.add(valuesField);
   }
 
   private static class GroupDoc {

Modified: lucene/dev/branches/lucene4258/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java?rev=1498804&r1=1498803&r2=1498804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java (original)
+++ lucene/dev/branches/lucene4258/lucene/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java Tue Jul  2 07:12:00 2013
@@ -827,12 +827,14 @@ public class TestGrouping extends Lucene
           for(SortField sf : docSort.getSort()) {
             if (sf.getType() == SortField.Type.SCORE) {
               getScores = true;
+              break;
             }
           }
 
           for(SortField sf : groupSort.getSort()) {
             if (sf.getType() == SortField.Type.SCORE) {
               getScores = true;
+              break;
             }
           }
 

Modified: lucene/dev/branches/lucene4258/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/PostingsHighlighter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/PostingsHighlighter.java?rev=1498804&r1=1498803&r2=1498804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/PostingsHighlighter.java (original)
+++ lucene/dev/branches/lucene4258/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/PostingsHighlighter.java Tue Jul  2 07:12:00 2013
@@ -669,7 +669,7 @@ public class PostingsHighlighter {
     public void stringField(FieldInfo fieldInfo, String value) throws IOException {
       assert currentField >= 0;
       StringBuilder builder = builders[currentField];
-      if (builder.length() > 0) {
+      if (builder.length() > 0 && builder.length() < maxLength) {
         builder.append(' '); // for the offset gap, TODO: make this configurable
       }
       if (builder.length() + value.length() > maxLength) {

Modified: lucene/dev/branches/lucene4258/lucene/highlighter/src/test/org/apache/lucene/search/postingshighlight/TestPostingsHighlighter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4258/lucene/highlighter/src/test/org/apache/lucene/search/postingshighlight/TestPostingsHighlighter.java?rev=1498804&r1=1498803&r2=1498804&view=diff
==============================================================================
--- lucene/dev/branches/lucene4258/lucene/highlighter/src/test/org/apache/lucene/search/postingshighlight/TestPostingsHighlighter.java (original)
+++ lucene/dev/branches/lucene4258/lucene/highlighter/src/test/org/apache/lucene/search/postingshighlight/TestPostingsHighlighter.java Tue Jul  2 07:12:00 2013
@@ -123,6 +123,43 @@ public class TestPostingsHighlighter ext
     dir.close();
   }
   
+  // simple test with multiple values that make a result longer than maxLength.
+  public void testMaxLengthWithMultivalue() throws Exception {
+    Directory dir = newDirectory();
+    // use simpleanalyzer for more natural tokenization (else "test." is a token)
+    IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random(), MockTokenizer.SIMPLE, true));
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
+    
+    FieldType offsetsType = new FieldType(TextField.TYPE_STORED);
+    offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
+    Document doc = new Document();
+    
+    for(int i = 0; i < 3 ; i++) {
+      Field body = new Field("body", "", offsetsType);
+      body.setStringValue("This is a multivalued field");
+      doc.add(body);
+    }
+    
+    iw.addDocument(doc);
+    
+    IndexReader ir = iw.getReader();
+    iw.close();
+    
+    IndexSearcher searcher = newSearcher(ir);
+    PostingsHighlighter highlighter = new PostingsHighlighter(40);
+    Query query = new TermQuery(new Term("body", "field"));
+    TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER);
+    assertEquals(1, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, searcher, topDocs);
+    assertEquals(1, snippets.length);
+    assertTrue("Snippet should have maximum 40 characters plus the pre and post tags",
+        snippets[0].length() == (40 + "<b></b>".length()));
+    
+    ir.close();
+    dir.close();
+  }
+  
   public void testMultipleFields() throws Exception {
     Directory dir = newDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random(), MockTokenizer.SIMPLE, true));