You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2013/10/24 09:48:52 UTC

svn commit: r1535296 - in /lucene/dev/trunk/lucene: ./ core/src/java/org/apache/lucene/codecs/lucene45/ test-framework/src/java/org/apache/lucene/index/

Author: jpountz
Date: Thu Oct 24 07:48:51 2013
New Revision: 1535296

URL: http://svn.apache.org/r1535296
Log:
LUCENE-5300: Optimized SORTED_SET storage for fields which are single-valued.

Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesConsumer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesFormat.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1535296&r1=1535295&r2=1535296&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Thu Oct 24 07:48:51 2013
@@ -191,6 +191,9 @@ Optimizations
 
 * LUCENE-5266: Improved number of read calls and branches in DirectPackedReader. (Ryan Ernst)
 
+* LUCENE-5300: Optimized SORTED_SET storage for fields which are single-valued.
+  (Adrien Grand)
+
 Documentation
 
 * LUCENE-5211: Better javadocs and error checking of 'format' option in 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesConsumer.java?rev=1535296&r1=1535295&r2=1535296&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesConsumer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesConsumer.java Thu Oct 24 07:48:51 2013
@@ -21,6 +21,7 @@ import java.io.Closeable; // javadocs
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Iterator;
 
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.DocValuesConsumer;
@@ -42,6 +43,7 @@ public class Lucene45DocValuesConsumer e
 
   static final int BLOCK_SIZE = 16384;
   static final int ADDRESS_INTERVAL = 16;
+  static final Number MISSING_ORD = Long.valueOf(-1);
 
   /** Compressed using packed blocks of ints. */
   public static final int DELTA_COMPRESSED = 0;
@@ -57,6 +59,13 @@ public class Lucene45DocValuesConsumer e
   /** Compressed binary with shared prefixes */
   public static final int BINARY_PREFIX_COMPRESSED = 2;
 
+  /** Standard storage for sorted set values with 1 level of indirection:
+   *  docId -> address -> ord. */
+  public static final int SORTED_SET_WITH_ADDRESSES = 0;
+  /** Single-valued sorted set values, encoded as sorted values, so no level
+   *  of indirection: docId -> ord. */
+  public static final int SORTED_SET_SINGLE_VALUED_SORTED = 1;
+
   final IndexOutput data, meta;
   final int maxDoc;
   
@@ -340,17 +349,70 @@ public class Lucene45DocValuesConsumer e
     addTermsDict(field, values);
     addNumericField(field, docToOrd, false);
   }
-  
+
+  private static boolean isSingleValued(Iterable<Number> docToOrdCount) {
+    for (Number ordCount : docToOrdCount) {
+      if (ordCount.longValue() > 1) {
+        return false;
+      }
+    }
+    return true;
+  }
+
   @Override
-  public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrdCount, Iterable<Number> ords) throws IOException {
+  public void addSortedSetField(FieldInfo field, Iterable<BytesRef> values, final Iterable<Number> docToOrdCount, final Iterable<Number> ords) throws IOException {
     meta.writeVInt(field.number);
     meta.writeByte(Lucene45DocValuesFormat.SORTED_SET);
+
+    if (isSingleValued(docToOrdCount)) {
+      meta.writeVInt(SORTED_SET_SINGLE_VALUED_SORTED);
+      // The field is single-valued, we can encode it as SORTED
+      addSortedField(field, values, new Iterable<Number>() {
+
+        @Override
+        public Iterator<Number> iterator() {
+          final Iterator<Number> docToOrdCountIt = docToOrdCount.iterator();
+          final Iterator<Number> ordsIt = ords.iterator();
+          return new Iterator<Number>() {
+
+            @Override
+            public boolean hasNext() {
+              assert ordsIt.hasNext() ? docToOrdCountIt.hasNext() : true;
+              return docToOrdCountIt.hasNext();
+            }
+
+            @Override
+            public Number next() {
+              final Number ordCount = docToOrdCountIt.next();
+              if (ordCount.longValue() == 0) {
+                return MISSING_ORD;
+              } else {
+                assert ordCount.longValue() == 1;
+                return ordsIt.next();
+              }
+            }
+
+            @Override
+            public void remove() {
+              throw new UnsupportedOperationException();
+            }
+
+          };
+        }
+
+      });
+      return;
+    }
+
+    meta.writeVInt(SORTED_SET_WITH_ADDRESSES);
+
     // write the ord -> byte[] as a binary field
     addTermsDict(field, values);
+
     // write the stream of ords as a numeric field
     // NOTE: we could return an iterator that delta-encodes these within a doc
     addNumericField(field, ords, false);
-    
+
     // write the doc -> ord count as a absolute index to the stream
     meta.writeVInt(field.number);
     meta.writeByte(Lucene45DocValuesFormat.NUMERIC);

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesFormat.java?rev=1535296&r1=1535295&r2=1535296&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesFormat.java Thu Oct 24 07:48:51 2013
@@ -178,7 +178,8 @@ public final class Lucene45DocValuesForm
   static final String META_CODEC = "Lucene45ValuesMetadata";
   static final String META_EXTENSION = "dvm";
   static final int VERSION_START = 0;
-  static final int VERSION_CURRENT = VERSION_START;
+  static final int VERSION_SORTED_SET_SINGLE_VALUE_OPTIMIZED = 1;
+  static final int VERSION_CURRENT = VERSION_SORTED_SET_SINGLE_VALUE_OPTIMIZED;
   static final byte NUMERIC = 0;
   static final byte BINARY = 1;
   static final byte SORTED = 2;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java?rev=1535296&r1=1535295&r2=1535296&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene45/Lucene45DocValuesProducer.java Thu Oct 24 07:48:51 2013
@@ -17,12 +17,15 @@ package org.apache.lucene.codecs.lucene4
  * limitations under the License.
  */
 
+import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.BINARY_FIXED_UNCOMPRESSED;
+import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.BINARY_PREFIX_COMPRESSED;
+import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.BINARY_VARIABLE_UNCOMPRESSED;
 import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.DELTA_COMPRESSED;
 import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.GCD_COMPRESSED;
+import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.SORTED_SET_SINGLE_VALUED_SORTED;
+import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.SORTED_SET_WITH_ADDRESSES;
 import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.TABLE_COMPRESSED;
-import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.BINARY_FIXED_UNCOMPRESSED;
-import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.BINARY_VARIABLE_UNCOMPRESSED;
-import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesConsumer.BINARY_PREFIX_COMPRESSED;
+import static org.apache.lucene.codecs.lucene45.Lucene45DocValuesFormat.VERSION_SORTED_SET_SINGLE_VALUE_OPTIMIZED;
 
 import java.io.Closeable; // javadocs
 import java.io.IOException;
@@ -40,6 +43,7 @@ import org.apache.lucene.index.FieldInfo
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SingletonSortedSetDocValues;
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.index.TermsEnum;
@@ -56,10 +60,12 @@ import org.apache.lucene.util.packed.Pac
 public class Lucene45DocValuesProducer extends DocValuesProducer implements Closeable {
   private final Map<Integer,NumericEntry> numerics;
   private final Map<Integer,BinaryEntry> binaries;
+  private final Map<Integer,SortedSetEntry> sortedSets;
   private final Map<Integer,NumericEntry> ords;
   private final Map<Integer,NumericEntry> ordIndexes;
   private final IndexInput data;
   private final int maxDoc;
+  private final int version;
 
   // memory-resident structures
   private final Map<Integer,MonotonicBlockPackedReader> addressInstances = new HashMap<Integer,MonotonicBlockPackedReader>();
@@ -72,7 +78,6 @@ public class Lucene45DocValuesProducer e
     IndexInput in = state.directory.openInput(metaName, state.context);
     this.maxDoc = state.segmentInfo.getDocCount();
     boolean success = false;
-    final int version;
     try {
       version = CodecUtil.checkHeader(in, metaCodec, 
                                       Lucene45DocValuesFormat.VERSION_CURRENT,
@@ -81,6 +86,7 @@ public class Lucene45DocValuesProducer e
       ords = new HashMap<Integer,NumericEntry>();
       ordIndexes = new HashMap<Integer,NumericEntry>();
       binaries = new HashMap<Integer,BinaryEntry>();
+      sortedSets = new HashMap<Integer,SortedSetEntry>();
       readFields(in, state.fieldInfos);
 
       success = true;
@@ -110,7 +116,58 @@ public class Lucene45DocValuesProducer e
       }
     }
   }
-  
+
+  private void readSortedField(int fieldNumber, IndexInput meta, FieldInfos infos) throws IOException {
+    // sorted = binary + numeric
+    if (meta.readVInt() != fieldNumber) {
+      throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+    }
+    if (meta.readByte() != Lucene45DocValuesFormat.BINARY) {
+      throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+    }
+    BinaryEntry b = readBinaryEntry(meta);
+    binaries.put(fieldNumber, b);
+    
+    if (meta.readVInt() != fieldNumber) {
+      throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+    }
+    if (meta.readByte() != Lucene45DocValuesFormat.NUMERIC) {
+      throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+    }
+    NumericEntry n = readNumericEntry(meta);
+    ords.put(fieldNumber, n);
+  }
+
+  private void readSortedSetFieldWithAddresses(int fieldNumber, IndexInput meta, FieldInfos infos) throws IOException {
+    // sortedset = binary + numeric (addresses) + ordIndex
+    if (meta.readVInt() != fieldNumber) {
+      throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+    }
+    if (meta.readByte() != Lucene45DocValuesFormat.BINARY) {
+      throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+    }
+    BinaryEntry b = readBinaryEntry(meta);
+    binaries.put(fieldNumber, b);
+
+    if (meta.readVInt() != fieldNumber) {
+      throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+    }
+    if (meta.readByte() != Lucene45DocValuesFormat.NUMERIC) {
+      throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+    }
+    NumericEntry n1 = readNumericEntry(meta);
+    ords.put(fieldNumber, n1);
+
+    if (meta.readVInt() != fieldNumber) {
+      throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+    }
+    if (meta.readByte() != Lucene45DocValuesFormat.NUMERIC) {
+      throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+    }
+    NumericEntry n2 = readNumericEntry(meta);
+    ordIndexes.put(fieldNumber, n2);
+  }
+
   private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
     int fieldNumber = meta.readVInt();
     while (fieldNumber != -1) {
@@ -121,52 +178,23 @@ public class Lucene45DocValuesProducer e
         BinaryEntry b = readBinaryEntry(meta);
         binaries.put(fieldNumber, b);
       } else if (type == Lucene45DocValuesFormat.SORTED) {
-        // sorted = binary + numeric
-        if (meta.readVInt() != fieldNumber) {
-          throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
-        }
-        if (meta.readByte() != Lucene45DocValuesFormat.BINARY) {
-          throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
-        }
-        BinaryEntry b = readBinaryEntry(meta);
-        binaries.put(fieldNumber, b);
-        
-        if (meta.readVInt() != fieldNumber) {
-          throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
-        }
-        if (meta.readByte() != Lucene45DocValuesFormat.NUMERIC) {
-          throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
-        }
-        NumericEntry n = readNumericEntry(meta);
-        ords.put(fieldNumber, n);
+        readSortedField(fieldNumber, meta, infos);
       } else if (type == Lucene45DocValuesFormat.SORTED_SET) {
-        // sortedset = binary + numeric + ordIndex
-        if (meta.readVInt() != fieldNumber) {
-          throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
-        }
-        if (meta.readByte() != Lucene45DocValuesFormat.BINARY) {
-          throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
-        }
-        BinaryEntry b = readBinaryEntry(meta);
-        binaries.put(fieldNumber, b);
-        
-        if (meta.readVInt() != fieldNumber) {
-          throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
-        }
-        if (meta.readByte() != Lucene45DocValuesFormat.NUMERIC) {
-          throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
-        }
-        NumericEntry n1 = readNumericEntry(meta);
-        ords.put(fieldNumber, n1);
-        
-        if (meta.readVInt() != fieldNumber) {
-          throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
-        }
-        if (meta.readByte() != Lucene45DocValuesFormat.NUMERIC) {
-          throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+        SortedSetEntry ss = readSortedSetEntry(meta);
+        sortedSets.put(fieldNumber, ss);
+        if (ss.format == SORTED_SET_WITH_ADDRESSES) {
+          readSortedSetFieldWithAddresses(fieldNumber, meta, infos);
+        } else if (ss.format == SORTED_SET_SINGLE_VALUED_SORTED) {
+          if (meta.readVInt() != fieldNumber) {
+            throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+          }
+          if (meta.readByte() != Lucene45DocValuesFormat.SORTED) {
+            throw new CorruptIndexException("sortedset entry for field: " + fieldNumber + " is corrupt (resource=" + meta + ")");
+          }
+          readSortedField(fieldNumber, meta, infos);
+        } else {
+          throw new AssertionError();
         }
-        NumericEntry n2 = readNumericEntry(meta);
-        ordIndexes.put(fieldNumber, n2);
       } else {
         throw new CorruptIndexException("invalid type: " + type + ", resource=" + meta);
       }
@@ -236,6 +264,19 @@ public class Lucene45DocValuesProducer e
     return entry;
   }
 
+  SortedSetEntry readSortedSetEntry(IndexInput meta) throws IOException {
+    SortedSetEntry entry = new SortedSetEntry();
+    if (version >= VERSION_SORTED_SET_SINGLE_VALUE_OPTIMIZED) {
+      entry.format = meta.readVInt();
+    } else {
+      entry.format = SORTED_SET_WITH_ADDRESSES;
+    }
+    if (entry.format != SORTED_SET_SINGLE_VALUED_SORTED && entry.format != SORTED_SET_WITH_ADDRESSES) {
+      throw new CorruptIndexException("Unknown format: " + entry.format + ", input=" + meta);
+    }
+    return entry;
+  }
+
   @Override
   public NumericDocValues getNumeric(FieldInfo field) throws IOException {
     NumericEntry entry = numerics.get(field.number);
@@ -469,6 +510,14 @@ public class Lucene45DocValuesProducer e
 
   @Override
   public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException {
+    SortedSetEntry ss = sortedSets.get(field.number);
+    if (ss.format == SORTED_SET_SINGLE_VALUED_SORTED) {
+      final SortedDocValues values = getSorted(field);
+      return new SingletonSortedSetDocValues(values);
+    } else if (ss.format != SORTED_SET_WITH_ADDRESSES) {
+      throw new AssertionError();
+    }
+
     final IndexInput data = this.data.clone();
     final long valueCount = binaries.get(field.number).count;
     // we keep the byte[]s and list of ords on disk, these could be large
@@ -619,7 +668,13 @@ public class Lucene45DocValuesProducer e
     /** packed ints blocksize */
     public int blockSize;
   }
-  
+
+  /** metadata entry for a sorted-set docvalues field */
+  protected static class SortedSetEntry {
+    private SortedSetEntry() {}
+    int format;
+  }
+
   // internally we compose complex dv (sorted/sortedset) from other ones
   static abstract class LongNumericDocValues extends NumericDocValues {
     @Override

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java?rev=1535296&r1=1535295&r2=1535296&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java Thu Oct 24 07:48:51 2013
@@ -2028,8 +2028,8 @@ public abstract class BaseDocValuesForma
     ireader.close();
     directory.close();
   }
-  
-  private void doTestSortedSetVsStoredFields(int minLength, int maxLength) throws Exception {
+
+  private void doTestSortedSetVsStoredFields(int minLength, int maxLength, int maxValuesPerDoc) throws Exception {
     Directory dir = newDirectory();
     IndexWriterConfig conf = new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
     RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
@@ -2046,7 +2046,7 @@ public abstract class BaseDocValuesForma
       } else {
         length = _TestUtil.nextInt(random(), minLength, maxLength);
       }
-      int numValues = random().nextInt(17);
+      int numValues = _TestUtil.nextInt(random(), 0, maxValuesPerDoc);
       // create a random set of strings
       Set<String> values = new TreeSet<String>();
       for (int v = 0; v < numValues; v++) {
@@ -2109,7 +2109,7 @@ public abstract class BaseDocValuesForma
     int numIterations = atLeast(1);
     for (int i = 0; i < numIterations; i++) {
       int fixedLength = _TestUtil.nextInt(random(), 1, 10);
-      doTestSortedSetVsStoredFields(fixedLength, fixedLength);
+      doTestSortedSetVsStoredFields(fixedLength, fixedLength, 16);
     }
   }
   
@@ -2117,7 +2117,24 @@ public abstract class BaseDocValuesForma
     assumeTrue("Codec does not support SORTED_SET", defaultCodecSupportsSortedSet());
     int numIterations = atLeast(1);
     for (int i = 0; i < numIterations; i++) {
-      doTestSortedSetVsStoredFields(1, 10);
+      doTestSortedSetVsStoredFields(1, 10, 16);
+    }
+  }
+
+  public void testSortedSetFixedLengthSingleValuedVsStoredFields() throws Exception {
+    assumeTrue("Codec does not support SORTED_SET", defaultCodecSupportsSortedSet());
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      int fixedLength = _TestUtil.nextInt(random(), 1, 10);
+      doTestSortedSetVsStoredFields(fixedLength, fixedLength, 1);
+    }
+  }
+  
+  public void testSortedSetVariableLengthSingleValuedVsStoredFields() throws Exception {
+    assumeTrue("Codec does not support SORTED_SET", defaultCodecSupportsSortedSet());
+    int numIterations = atLeast(1);
+    for (int i = 0; i < numIterations; i++) {
+      doTestSortedSetVsStoredFields(1, 10, 1);
     }
   }