lucene-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From mikemcc...@apache.org
Subject svn commit: r1131158 - in /lucene/dev/trunk: lucene/src/java/org/apache/lucene/index/ modules/grouping/src/java/org/apache/lucene/search/grouping/ modules/grouping/src/test/org/apache/lucene/search/grouping/
Date Fri, 03 Jun 2011 19:31:52 GMT
Author: mikemccand
Date: Fri Jun  3 19:31:51 2011
New Revision: 1131158

URL: http://svn.apache.org/viewvc?rev=1131158&view=rev
Log:
LUCENE-3129: BlockGroupingCollector wasn't tracking scores correctly; fang'd up TestGrouping
to reveal the bug

Modified:
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java
    lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java
    lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/TermFirstPassGroupingCollector.java
    lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/TermSecondPassGroupingCollector.java
    lucene/dev/trunk/modules/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java?rev=1131158&r1=1131157&r2=1131158&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java Fri Jun
 3 19:31:51 2011
@@ -181,9 +181,9 @@ final class TermsHashPerField extends In
     // term text into textStart address
     // Get the text & hash of this term.
     int termID;
-    try{
-       termID = bytesHash.add(termBytesRef, termAtt.fillBytesRef());
-    }catch (MaxBytesLengthExceededException e) {
+    try {
+      termID = bytesHash.add(termBytesRef, termAtt.fillBytesRef());
+    } catch (MaxBytesLengthExceededException e) {
       // Not enough room in current block
       // Just skip this term, to remain as robust as
       // possible during indexing.  A TokenFilter

Modified: lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java?rev=1131158&r1=1131157&r2=1131158&view=diff
==============================================================================
--- lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java
(original)
+++ lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java
Fri Jun  3 19:31:51 2011
@@ -212,7 +212,7 @@ public class BlockGroupingCollector exte
           // Swap pending scores
           final float[] savScores = og.scores;
           og.scores = pendingSubScores;
-          pendingSubScores = og.scores;
+          pendingSubScores = savScores;
         }
         og.readerContext = currentReaderContext;
         //og.groupOrd = lastGroupOrd;

Modified: lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/TermFirstPassGroupingCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/TermFirstPassGroupingCollector.java?rev=1131158&r1=1131157&r2=1131158&view=diff
==============================================================================
--- lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/TermFirstPassGroupingCollector.java
(original)
+++ lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/TermFirstPassGroupingCollector.java
Fri Jun  3 19:31:51 2011
@@ -26,7 +26,7 @@ import java.io.IOException;
 
 /**
  * Concrete implementation of {@link AbstractFirstPassGroupingCollector} that groups based
on
- * field values and more specifically uses {@link org.apache.lucene.search.FieldCache.DocTerms}
+ * field values and more specifically uses {@link org.apache.lucene.search.FieldCache.DocTermsIndex}
  * to collect groups.
  *
  * @lucene.experimental

Modified: lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/TermSecondPassGroupingCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/TermSecondPassGroupingCollector.java?rev=1131158&r1=1131157&r2=1131158&view=diff
==============================================================================
--- lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/TermSecondPassGroupingCollector.java
(original)
+++ lucene/dev/trunk/modules/grouping/src/java/org/apache/lucene/search/grouping/TermSecondPassGroupingCollector.java
Fri Jun  3 19:31:51 2011
@@ -27,7 +27,7 @@ import java.util.Collection;
 
 /**
  * Concrete implementation of {@link AbstractSecondPassGroupingCollector} that groups based
on
- * field values and more specifically uses {@link org.apache.lucene.search.FieldCache.DocTerms}
+ * field values and more specifically uses {@link org.apache.lucene.search.FieldCache.DocTermsIndex}
  * to collect grouped docs.
  *
  * @lucene.experimental

Modified: lucene/dev/trunk/modules/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java?rev=1131158&r1=1131157&r2=1131158&view=diff
==============================================================================
--- lucene/dev/trunk/modules/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java
(original)
+++ lucene/dev/trunk/modules/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java
Fri Jun  3 19:31:51 2011
@@ -154,7 +154,10 @@ public class TestGrouping extends Lucene
     final BytesRef group;
     final BytesRef sort1;
     final BytesRef sort2;
+    // content must be "realN ..."
     final String content;
+    float score;
+    float score2;
 
     public GroupDoc(int id, BytesRef group, BytesRef sort1, BytesRef sort2, String content)
{
       this.id = id;
@@ -167,16 +170,21 @@ public class TestGrouping extends Lucene
 
   private Sort getRandomSort() {
     final List<SortField> sortFields = new ArrayList<SortField>();
-    if (random.nextBoolean()) {
+    if (random.nextInt(7) == 2) {
+      sortFields.add(SortField.FIELD_SCORE);
+    } else {
       if (random.nextBoolean()) {
+        if (random.nextBoolean()) {
+          sortFields.add(new SortField("sort1", SortField.STRING, random.nextBoolean()));
+        } else {
+          sortFields.add(new SortField("sort2", SortField.STRING, random.nextBoolean()));
+        }
+      } else if (random.nextBoolean()) {
         sortFields.add(new SortField("sort1", SortField.STRING, random.nextBoolean()));
-      } else {
         sortFields.add(new SortField("sort2", SortField.STRING, random.nextBoolean()));
       }
-    } else if (random.nextBoolean()) {
-      sortFields.add(new SortField("sort1", SortField.STRING, random.nextBoolean()));
-      sortFields.add(new SortField("sort2", SortField.STRING, random.nextBoolean()));
     }
+    // Break ties:
     sortFields.add(new SortField("id", SortField.INT));
     return new Sort(sortFields.toArray(new SortField[sortFields.size()]));
   }
@@ -188,7 +196,15 @@ public class TestGrouping extends Lucene
       public int compare(GroupDoc d1, GroupDoc d2) {
         for(SortField sf : sortFields) {
           final int cmp;
-          if (sf.getField().equals("sort1")) {
+          if (sf.getType() == SortField.SCORE) {
+            if (d1.score > d2.score) {
+              cmp = -1;
+            } else if (d1.score < d2.score) {
+              cmp = 1;
+            } else {
+              cmp = 0;
+            }
+          } else if (sf.getField().equals("sort1")) {
             cmp = d1.sort1.compareTo(d2.sort1);
           } else if (sf.getField().equals("sort2")) {
             cmp = d1.sort2.compareTo(d2.sort2);
@@ -213,7 +229,9 @@ public class TestGrouping extends Lucene
     for(int fieldIDX=0;fieldIDX<sortFields.length;fieldIDX++) {
       final Comparable<?> c;
       final SortField sf = sortFields[fieldIDX];
-      if (sf.getField().equals("sort1")) {
+      if (sf.getType() == SortField.SCORE) {
+        c = new Float(d.score);
+      } else if (sf.getField().equals("sort1")) {
         c = d.sort1;
       } else if (sf.getField().equals("sort2")) {
         c = d.sort2;
@@ -237,17 +255,17 @@ public class TestGrouping extends Lucene
   */
 
   private TopGroups<BytesRef> slowGrouping(GroupDoc[] groupDocs,
-                                 String searchTerm,
-                                 boolean fillFields,
-                                 boolean getScores,
-                                 boolean getMaxScores,
-                                 boolean doAllGroups,
-                                 Sort groupSort,
-                                 Sort docSort,
-                                 int topNGroups,
-                                 int docsPerGroup,
-                                 int groupOffset,
-                                 int docOffset) {
+                                           String searchTerm,
+                                           boolean fillFields,
+                                           boolean getScores,
+                                           boolean getMaxScores,
+                                           boolean doAllGroups,
+                                           Sort groupSort,
+                                           Sort docSort,
+                                           int topNGroups,
+                                           int docsPerGroup,
+                                           int groupOffset,
+                                           int docOffset) {
 
     final Comparator<GroupDoc> groupSortComp = getComparator(groupSort);
 
@@ -262,11 +280,11 @@ public class TestGrouping extends Lucene
     //System.out.println("TEST: slowGrouping");
     for(GroupDoc d : groupDocs) {
       // TODO: would be better to filter by searchTerm before sorting!
-      if (!d.content.equals(searchTerm)) {
+      if (!d.content.startsWith(searchTerm)) {
         continue;
       }
       totalHitCount++;
-      //System.out.println("  match id=" + d.id);
+      //System.out.println("  match id=" + d.id + " score=" + d.score);
 
       if (doAllGroups) {
         if (!knownGroups.contains(d.group)) {
@@ -312,9 +330,9 @@ public class TestGrouping extends Lucene
           final GroupDoc d = docs.get(docIDX);
           final FieldDoc fd;
           if (fillFields) {
-            fd = new FieldDoc(d.id, 0.0f, fillFields(d, docSort));
+            fd = new FieldDoc(d.id, getScores ? d.score : Float.NaN, fillFields(d, docSort));
           } else {
-            fd = new FieldDoc(d.id, 0.0f);
+            fd = new FieldDoc(d.id, getScores ? d.score : Float.NaN);
           }
           hits[docIDX-docOffset] = fd;
         }
@@ -373,7 +391,7 @@ public class TestGrouping extends Lucene
         doc.add(newField("sort1", groupValue.sort1.utf8ToString(), Field.Index.NOT_ANALYZED));
         doc.add(newField("sort2", groupValue.sort2.utf8ToString(), Field.Index.NOT_ANALYZED));
         doc.add(new NumericField("id").setIntValue(groupValue.id));
-        doc.add(newField("content", groupValue.content, Field.Index.NOT_ANALYZED));
+        doc.add(newField("content", groupValue.content, Field.Index.ANALYZED));
         //System.out.println("TEST:     doc content=" + groupValue.content + " group=" +
(groupValue.group == null ? "null" : groupValue.group.utf8ToString()) + " sort1=" + groupValue.sort1.utf8ToString()
+ " id=" + groupValue.id);
       }
       // So we can pull filter marking last doc in block:
@@ -421,7 +439,22 @@ public class TestGrouping extends Lucene
         groups.add(new BytesRef(_TestUtil.randomRealisticUnicodeString(random)));
         //groups.add(new BytesRef(_TestUtil.randomSimpleString(random)));
       }
-      final String[] contentStrings = new String[] {"a", "b", "c", "d"};
+      final String[] contentStrings = new String[_TestUtil.nextInt(random, 2, 20)];
+      if (VERBOSE) {
+        System.out.println("TEST: create fake content");
+      }
+      for(int contentIDX=0;contentIDX<contentStrings.length;contentIDX++) {
+        final StringBuilder sb = new StringBuilder();
+        sb.append("real" + random.nextInt(3)).append(' ');
+        final int fakeCount = random.nextInt(10);
+        for(int fakeIDX=0;fakeIDX<fakeCount;fakeIDX++) {
+          sb.append("fake ");
+        }
+        contentStrings[contentIDX] = sb.toString();
+        if (VERBOSE) {
+          System.out.println("  content=" + sb.toString());
+        }
+      }
 
       Directory dir = newDirectory();
       RandomIndexWriter w = new RandomIndexWriter(
@@ -440,7 +473,7 @@ public class TestGrouping extends Lucene
       Field sort2 = newField("sort2", "", Field.Index.NOT_ANALYZED);
       doc.add(sort2);
       docNoGroup.add(sort2);
-      Field content = newField("content", "", Field.Index.NOT_ANALYZED);
+      Field content = newField("content", "", Field.Index.ANALYZED);
       doc.add(content);
       docNoGroup.add(content);
       NumericField id = new NumericField("id");
@@ -480,40 +513,96 @@ public class TestGrouping extends Lucene
         }
       }
 
+      final GroupDoc[] groupDocsByID = new GroupDoc[groupDocs.length];
+      System.arraycopy(groupDocs, 0, groupDocsByID, 0, groupDocs.length);
+
       final IndexReader r = w.getReader();
       w.close();
 
-      // Build 2nd index, where docs are added in blocks by
-      // group, so we can use single pass collector
-      final Directory dir2 = newDirectory();
-      final IndexReader r2 = getDocBlockReader(dir2, groupDocs);
-      final Filter lastDocInBlock = new CachingWrapperFilter(new QueryWrapperFilter(new TermQuery(new
Term("groupend", "x"))));
-
-      final IndexSearcher s = new IndexSearcher(r);
-      final IndexSearcher s2 = new IndexSearcher(r2);
-
+      // NOTE: intentional but temporary field cache insanity!
       final int[] docIDToID = FieldCache.DEFAULT.getInts(r, "id");
-      final int[] docIDToID2 = FieldCache.DEFAULT.getInts(r2, "id");
+      IndexReader r2 = null;
+      Directory dir2 = null;
 
       try {
+        final IndexSearcher s = new IndexSearcher(r);
+
+        for(int contentID=0;contentID<3;contentID++) {
+          final ScoreDoc[] hits = s.search(new TermQuery(new Term("content", "real"+contentID)),
numDocs).scoreDocs;
+          for(ScoreDoc hit : hits) {
+            final GroupDoc gd = groupDocs[docIDToID[hit.doc]];
+            assertTrue(gd.score == 0.0);
+            gd.score = hit.score;
+            assertEquals(gd.id, docIDToID[hit.doc]);
+            //System.out.println("  score=" + hit.score + " id=" + docIDToID[hit.doc]);
+          }
+        }
+
+        for(GroupDoc gd : groupDocs) {
+          assertTrue(gd.score != 0.0);
+        }
+
+        // Build 2nd index, where docs are added in blocks by
+        // group, so we can use single pass collector
+        dir2 = newDirectory();
+        r2 = getDocBlockReader(dir2, groupDocs);
+        final Filter lastDocInBlock = new CachingWrapperFilter(new QueryWrapperFilter(new
TermQuery(new Term("groupend", "x"))));
+        final int[] docIDToID2 = FieldCache.DEFAULT.getInts(r2, "id");
+
+        final IndexSearcher s2 = new IndexSearcher(r2);
+
+        // Reader2 only increases maxDoc() vs reader, which
+        // means a monotonic shift in scores, so we can
+        // reliably remap them w/ Map:
+        final Map<Float,Float> scoreMap = new HashMap<Float,Float>();
+
+        // Tricky: must separately set .score2, because the doc
+        // block index was created with possible deletions!
+        for(int contentID=0;contentID<3;contentID++) {
+          //System.out.println("term=real" + contentID + " dfold=" + s.docFreq(new Term("content",
"real"+contentID)) +
+          //" dfnew=" + s2.docFreq(new Term("content", "real"+contentID)));
+          final ScoreDoc[] hits = s2.search(new TermQuery(new Term("content", "real"+contentID)),
numDocs).scoreDocs;
+          for(ScoreDoc hit : hits) {
+            final GroupDoc gd = groupDocsByID[docIDToID2[hit.doc]];
+            assertTrue(gd.score2 == 0.0);
+            gd.score2 = hit.score;
+            assertEquals(gd.id, docIDToID2[hit.doc]);
+            //System.out.println("  score=" + hit.score + " id=" + docIDToID2[hit.doc]);
+            scoreMap.put(gd.score, gd.score2);
+          }
+        }
+
         for(int searchIter=0;searchIter<100;searchIter++) {
 
           if (VERBOSE) {
             System.out.println("TEST: searchIter=" + searchIter);
           }
 
-          final String searchTerm = contentStrings[random.nextInt(contentStrings.length)];
+          final String searchTerm = "real" + random.nextInt(3);
           final boolean fillFields = random.nextBoolean();
-          final boolean getScores = random.nextBoolean();
+          boolean getScores = random.nextBoolean();
           final boolean getMaxScores = random.nextBoolean();
           final Sort groupSort = getRandomSort();
           //final Sort groupSort = new Sort(new SortField[] {new SortField("sort1", SortField.STRING),
new SortField("id", SortField.INT)});
           // TODO: also test null (= sort by relevance)
           final Sort docSort = getRandomSort();
 
+          for(SortField sf : docSort.getSort()) {
+            if (sf.getType() == SortField.SCORE) {
+              getScores = true;
+            }
+          }
+
+          for(SortField sf : groupSort.getSort()) {
+            if (sf.getType() == SortField.SCORE) {
+              getScores = true;
+            }
+          }
+
           final int topNGroups = _TestUtil.nextInt(random, 1, 30);
           //final int topNGroups = 4;
           final int docsPerGroup = _TestUtil.nextInt(random, 1, 50);
+
           final int groupOffset = _TestUtil.nextInt(random, 0, (topNGroups-1)/2);
           //final int groupOffset = 0;
 
@@ -523,7 +612,7 @@ public class TestGrouping extends Lucene
           final boolean doCache = random.nextBoolean();
           final boolean doAllGroups = random.nextBoolean();
           if (VERBOSE) {
-            System.out.println("TEST: groupSort=" + groupSort + " docSort=" + docSort + "
searchTerm=" + searchTerm + " topNGroups=" + topNGroups + " groupOffset=" + groupOffset +
" docOffset=" + docOffset + " doCache=" + doCache + " docsPerGroup=" + docsPerGroup + " doAllGroups="
+ doAllGroups);
+            System.out.println("TEST: groupSort=" + groupSort + " docSort=" + docSort + "
searchTerm=" + searchTerm + " topNGroups=" + topNGroups + " groupOffset=" + groupOffset +
" docOffset=" + docOffset + " doCache=" + doCache + " docsPerGroup=" + docsPerGroup + " doAllGroups="
+ doAllGroups + " getScores=" + getScores + " getMaxScores=" + getMaxScores);
           }
 
           final TermAllGroupsCollector allGroupsCollector;
@@ -636,13 +725,12 @@ public class TestGrouping extends Lucene
               for(GroupDocs<BytesRef> gd : expectedGroups.groups) {
                 System.out.println("  group=" + (gd.groupValue == null ? "null" : gd.groupValue.utf8ToString()));
                 for(ScoreDoc sd : gd.scoreDocs) {
-                  System.out.println("    id=" + sd.doc);
+                  System.out.println("    id=" + sd.doc + " score=" + sd.score);
                 }
               }
             }
           }
-          // NOTE: intentional but temporary field cache insanity!
-          assertEquals(docIDToID, expectedGroups, groupsResult, true);
+          assertEquals(docIDToID, expectedGroups, groupsResult, true, getScores);
 
           final boolean needsScores = getScores || getMaxScores || docSort == null;
           final BlockGroupingCollector c3 = new BlockGroupingCollector(groupSort, groupOffset+topNGroups,
needsScores, lastDocInBlock);
@@ -665,11 +753,53 @@ public class TestGrouping extends Lucene
           } else {
             groupsResult2 = tempTopGroups2;
           }
-          assertEquals(docIDToID2, expectedGroups, groupsResult2, false);
+
+          if (expectedGroups != null) {
+            // Fixup scores for reader2
+            for (GroupDocs groupDocsHits : expectedGroups.groups) {
+              for(ScoreDoc hit : groupDocsHits.scoreDocs) {
+                final GroupDoc gd = groupDocsByID[hit.doc];
+                assertEquals(gd.id, hit.doc);
+                //System.out.println("fixup score " + hit.score + " to " + gd.score2 + "
vs " + gd.score);
+                hit.score = gd.score2;
+              }
+            }
+
+            final SortField[] sortFields = groupSort.getSort();
+            for(int groupSortIDX=0;groupSortIDX<sortFields.length;groupSortIDX++) {
+              if (sortFields[groupSortIDX].getType() == SortField.SCORE) {
+                for (GroupDocs groupDocsHits : expectedGroups.groups) {
+                  if (groupDocsHits.groupSortValues != null) {
+                    groupDocsHits.groupSortValues[groupSortIDX] = scoreMap.get(groupDocsHits.groupSortValues[groupSortIDX]);
+                    assertNotNull(groupDocsHits.groupSortValues[groupSortIDX]);
+                  }
+                }
+              }
+            }
+
+            final SortField[] docSortFields = docSort.getSort();
+            for(int docSortIDX=0;docSortIDX<docSortFields.length;docSortIDX++) {
+              if (docSortFields[docSortIDX].getType() == SortField.SCORE) {
+                for (GroupDocs groupDocsHits : expectedGroups.groups) {
+                  for(ScoreDoc _hit : groupDocsHits.scoreDocs) {
+                    FieldDoc hit = (FieldDoc) _hit;
+                    if (hit.fields != null) {
+                      hit.fields[docSortIDX] = scoreMap.get(hit.fields[docSortIDX]);
+                      assertNotNull(hit.fields[docSortIDX]);
+                    }
+                  }
+                }
+              }
+            }
+          }
+
+          assertEquals(docIDToID2, expectedGroups, groupsResult2, false, getScores);
         }
       } finally {
         FieldCache.DEFAULT.purge(r);
-        FieldCache.DEFAULT.purge(r2);
+        if (r2 != null) {
+          FieldCache.DEFAULT.purge(r2);
+        }
       }
 
       r.close();
@@ -680,7 +810,7 @@ public class TestGrouping extends Lucene
     }
   }
 
-  private void assertEquals(int[] docIDtoID, TopGroups expected, TopGroups actual, boolean
verifyGroupValues) {
+  private void assertEquals(int[] docIDtoID, TopGroups expected, TopGroups actual, boolean
verifyGroupValues, boolean testScores) {
     if (expected == null) {
       assertNull(actual);
       return;
@@ -716,9 +846,14 @@ public class TestGrouping extends Lucene
       for(int docIDX=0;docIDX<expectedFDs.length;docIDX++) {
         final FieldDoc expectedFD = (FieldDoc) expectedFDs[docIDX];
         final FieldDoc actualFD = (FieldDoc) actualFDs[docIDX];
+        //System.out.println("  actual doc=" + docIDtoID[actualFD.doc] + " score=" + actualFD.score);
         assertEquals(expectedFD.doc, docIDtoID[actualFD.doc]);
-        // TODO
-        // assertEquals(expectedFD.score, actualFD.score);
+        if (testScores) {
+          assertEquals(expectedFD.score, actualFD.score);
+        } else {
+          // TODO: too anal for now
+          //assertEquals(Float.NaN, actualFD.score);
+        }
         assertArrayEquals(expectedFD.fields, actualFD.fields);
       }
     }



Mime
View raw message