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

svn commit: r1442321 - in /lucene/dev/branches/lucene4547/lucene/core/src: java/org/apache/lucene/index/MultiDocValues.java test/org/apache/lucene/index/TestMultiDocValues.java

Author: rmuir
Date: Mon Feb  4 19:57:04 2013
New Revision: 1442321

URL: http://svn.apache.org/viewvc?rev=1442321&view=rev
Log:
add test, cleanup a bit

Added:
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestMultiDocValues.java   (with props)
Modified:
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java?rev=1442321&r1=1442320&r2=1442321&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/index/MultiDocValues.java Mon Feb  4 19:57:04 2013
@@ -214,7 +214,7 @@ public class MultiDocValues {
     if (!anyReal) {
       return null;
     } else {
-      OrdinalMapping mapping = new OrdinalMapping(values);
+      OrdinalMap mapping = new OrdinalMap(values);
       return new MultiSortedDocValues(values, starts, mapping);
     }
   }
@@ -222,7 +222,7 @@ public class MultiDocValues {
   /** maps per-segment ordinals to/from global ordinal space */
   // TODO: use more efficient packed ints structures (these are all positive values!)
   // nocommit: cache this in SlowWrapper, it can create MultiSortedDV with it directly.
-  static class OrdinalMapping {
+  static class OrdinalMap {
     // globalOrd -> (globalOrd - segmentOrd)
     final AppendingLongBuffer globalOrdDeltas;
     // globalOrd -> sub index
@@ -230,7 +230,7 @@ public class MultiDocValues {
     // segmentOrd -> (globalOrd - segmentOrd)
     final AppendingLongBuffer ordDeltas[];
     
-    OrdinalMapping(SortedDocValues subs[]) throws IOException {
+    OrdinalMap(SortedDocValues subs[]) throws IOException {
       // create the ordinal mappings by pulling a termsenum over each sub's 
       // unique terms, and walking a multitermsenum over those
       globalOrdDeltas = new AppendingLongBuffer();
@@ -253,13 +253,15 @@ public class MultiDocValues {
         TermsEnumWithSlice matches[] = mte.getMatchArray();
         for (int i = 0; i < mte.getMatchCount(); i++) {
           int subIndex = matches[i].index;
+          int delta = globalOrd - segmentOrds[subIndex];
+          assert delta >= 0;
           // for each unique term, just mark the first subindex/delta where it occurs
           if (i == 0) {
             subIndexes.add(subIndex);
-            globalOrdDeltas.add(globalOrd - segmentOrds[subIndex]);
+            globalOrdDeltas.add(delta);
           }
           // for each per-segment ord, map it back to the global term.
-          ordDeltas[subIndex].add(globalOrd - segmentOrds[subIndex]);
+          ordDeltas[subIndex].add(delta);
           segmentOrds[subIndex]++;
         }
         globalOrd++;
@@ -267,13 +269,13 @@ public class MultiDocValues {
     }
   }
   
-  /** implements SortedDocValues over n subs, using a SortedBytesMapping */
+  /** implements SortedDocValues over n subs, using an OrdinalMap */
   static class MultiSortedDocValues extends SortedDocValues {
     final int docStarts[];
     final SortedDocValues values[];
-    final OrdinalMapping mapping;
+    final OrdinalMap mapping;
   
-    MultiSortedDocValues(SortedDocValues values[], int docStarts[], OrdinalMapping mapping) throws IOException {
+    MultiSortedDocValues(SortedDocValues values[], int docStarts[], OrdinalMap mapping) throws IOException {
       this.values = values;
       this.docStarts = docStarts;
       this.mapping = mapping;

Added: lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestMultiDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestMultiDocValues.java?rev=1442321&view=auto
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestMultiDocValues.java (added)
+++ lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/index/TestMultiDocValues.java Mon Feb  4 19:57:04 2013
@@ -0,0 +1,192 @@
+package org.apache.lucene.index;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.document.BinaryDocValuesField;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.SortedDocValuesField;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
+
+/** Tests MultiDocValues versus ordinary segment merging */
+public class TestMultiDocValues extends LuceneTestCase {
+  
+  public void testNumerics() throws Exception {
+    Directory dir = newDirectory();
+    Document doc = new Document();
+    Field field = new NumericDocValuesField("numbers", 0);
+    doc.add(field);
+    
+    IndexWriterConfig iwc = newIndexWriterConfig(random(), TEST_VERSION_CURRENT, null);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
+
+    int numDocs = atLeast(500);
+    for (int i = 0; i < numDocs; i++) {
+      field.setLongValue(random().nextLong());
+      iw.addDocument(doc);
+      if (random().nextInt(17) == 0) {
+        iw.commit();
+      }
+    }
+    DirectoryReader ir = iw.getReader();
+    iw.forceMerge(1);
+    DirectoryReader ir2 = iw.getReader();
+    AtomicReader merged = getOnlySegmentReader(ir2);
+    iw.close();
+    
+    NumericDocValues multi = MultiDocValues.getNumericValues(ir, "numbers");
+    NumericDocValues single = merged.getNumericDocValues("numbers");
+    for (int i = 0; i < numDocs; i++) {
+      assertEquals(single.get(i), multi.get(i));
+    }
+    ir.close();
+    ir2.close();
+    dir.close();
+  }
+  
+  public void testBinary() throws Exception {
+    Directory dir = newDirectory();
+    Document doc = new Document();
+    BytesRef ref = new BytesRef();
+    Field field = new BinaryDocValuesField("bytes", ref);
+    doc.add(field);
+    
+    IndexWriterConfig iwc = newIndexWriterConfig(random(), TEST_VERSION_CURRENT, null);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
+
+    int numDocs = atLeast(500);
+    for (int i = 0; i < numDocs; i++) {
+      ref.copyChars(_TestUtil.randomUnicodeString(random()));
+      iw.addDocument(doc);
+      if (random().nextInt(17) == 0) {
+        iw.commit();
+      }
+    }
+    DirectoryReader ir = iw.getReader();
+    iw.forceMerge(1);
+    DirectoryReader ir2 = iw.getReader();
+    AtomicReader merged = getOnlySegmentReader(ir2);
+    iw.close();
+    
+    BinaryDocValues multi = MultiDocValues.getBinaryValues(ir, "bytes");
+    BinaryDocValues single = merged.getBinaryDocValues("bytes");
+    BytesRef actual = new BytesRef();
+    BytesRef expected = new BytesRef();
+    for (int i = 0; i < numDocs; i++) {
+      single.get(i, expected);
+      multi.get(i, actual);
+      assertEquals(expected, actual);
+    }
+    ir.close();
+    ir2.close();
+    dir.close();
+  }
+  
+  public void testSorted() throws Exception {
+    Directory dir = newDirectory();
+    Document doc = new Document();
+    BytesRef ref = new BytesRef();
+    Field field = new SortedDocValuesField("bytes", ref);
+    doc.add(field);
+    
+    IndexWriterConfig iwc = newIndexWriterConfig(random(), TEST_VERSION_CURRENT, null);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
+
+    int numDocs = atLeast(500);
+    for (int i = 0; i < numDocs; i++) {
+      ref.copyChars(_TestUtil.randomUnicodeString(random()));
+      iw.addDocument(doc);
+      if (random().nextInt(17) == 0) {
+        iw.commit();
+      }
+    }
+    DirectoryReader ir = iw.getReader();
+    iw.forceMerge(1);
+    DirectoryReader ir2 = iw.getReader();
+    AtomicReader merged = getOnlySegmentReader(ir2);
+    iw.close();
+    
+    SortedDocValues multi = MultiDocValues.getSortedValues(ir, "bytes");
+    SortedDocValues single = merged.getSortedDocValues("bytes");
+    assertEquals(single.getValueCount(), multi.getValueCount());
+    BytesRef actual = new BytesRef();
+    BytesRef expected = new BytesRef();
+    for (int i = 0; i < numDocs; i++) {
+      // check ord
+      assertEquals(single.getOrd(i), multi.getOrd(i));
+      // check ord value
+      single.get(i, expected);
+      multi.get(i, actual);
+      assertEquals(expected, actual);
+    }
+    ir.close();
+    ir2.close();
+    dir.close();
+  }
+  
+  // tries to make more dups than testSorted
+  public void testSortedWithLotsOfDups() throws Exception {
+    Directory dir = newDirectory();
+    Document doc = new Document();
+    BytesRef ref = new BytesRef();
+    Field field = new SortedDocValuesField("bytes", ref);
+    doc.add(field);
+    
+    IndexWriterConfig iwc = newIndexWriterConfig(random(), TEST_VERSION_CURRENT, null);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
+
+    int numDocs = atLeast(500);
+    for (int i = 0; i < numDocs; i++) {
+      ref.copyChars(_TestUtil.randomSimpleString(random(), 2));
+      iw.addDocument(doc);
+      if (random().nextInt(17) == 0) {
+        iw.commit();
+      }
+    }
+    DirectoryReader ir = iw.getReader();
+    iw.forceMerge(1);
+    DirectoryReader ir2 = iw.getReader();
+    AtomicReader merged = getOnlySegmentReader(ir2);
+    iw.close();
+    
+    SortedDocValues multi = MultiDocValues.getSortedValues(ir, "bytes");
+    SortedDocValues single = merged.getSortedDocValues("bytes");
+    assertEquals(single.getValueCount(), multi.getValueCount());
+    BytesRef actual = new BytesRef();
+    BytesRef expected = new BytesRef();
+    for (int i = 0; i < numDocs; i++) {
+      // check ord
+      assertEquals(single.getOrd(i), multi.getOrd(i));
+      // check ord value
+      single.get(i, expected);
+      multi.get(i, actual);
+      assertEquals(expected, actual);
+    }
+    ir.close();
+    ir2.close();
+    dir.close();
+  }
+}