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 2016/12/15 16:24:31 UTC

[1/2] lucene-solr:master: LUCENE-7589: Prevent outliers from raising the bpv for everyone.

Repository: lucene-solr
Updated Branches:
  refs/heads/master 268d4ace3 -> 3b182aa2f


LUCENE-7589: Prevent outliers from raising the bpv for everyone.


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/3b182aa2
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/3b182aa2
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/3b182aa2

Branch: refs/heads/master
Commit: 3b182aa2fb3e4062f6ec5be819f3aa70aa2e523d
Parents: ea1569e
Author: Adrien Grand <jp...@gmail.com>
Authored: Thu Dec 15 16:33:36 2016 +0100
Committer: Adrien Grand <jp...@gmail.com>
Committed: Thu Dec 15 17:17:54 2016 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   4 +
 .../lucene70/Lucene70DocValuesConsumer.java     | 163 +++++++++++---
 .../lucene70/Lucene70DocValuesFormat.java       |   5 +-
 .../lucene70/Lucene70DocValuesProducer.java     | 220 +++++++++++++++----
 .../java/org/apache/lucene/util/LongValues.java |   9 +
 .../apache/lucene/util/packed/DirectWriter.java |   8 +-
 .../lucene70/TestLucene70DocValuesFormat.java   | 152 +++++++++++++
 7 files changed, 479 insertions(+), 82 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b182aa2/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index bacc270..7e61469 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -48,6 +48,10 @@ Optimizations
 * LUCENE-7519: Add optimized APIs to compute browse-only top level
   facets (Mike McCandless)
 
+* LUCENE-7589: Numeric doc values now have the ability to encode blocks of
+  values using different numbers of bits per value if this proves to save
+  storage. (Adrien Grand)
+
 Other
 
 * LUCENE-7328: Remove LegacyNumericEncoding from GeoPointField. (Nick Knize)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b182aa2/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesConsumer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesConsumer.java
index e1b66e1..2dd68e9 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesConsumer.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesConsumer.java
@@ -18,6 +18,8 @@ package org.apache.lucene.codecs.lucene70;
 
 
 import static org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
+import static org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat.NUMERIC_BLOCK_SHIFT;
+import static org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE;
 
 import java.io.Closeable; // javadocs
 import java.io.IOException;
@@ -42,6 +44,7 @@ import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.SortedSetSelector;
+import org.apache.lucene.store.GrowableByteArrayDataOutput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.BytesRef;
@@ -112,12 +115,46 @@ final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Close
     });
   }
 
+  private static class MinMaxTracker {
+    long min, max, numValues, spaceInBits;
+
+    MinMaxTracker() {
+      reset();
+      spaceInBits = 0;
+    }
+
+    private void reset() {
+      min = Long.MAX_VALUE;
+      max = Long.MIN_VALUE;
+      numValues = 0;
+    }
+
+    /** Accumulate a new value. */
+    void update(long v) {
+      min = Math.min(min, v);
+      max = Math.max(max, v);
+      ++numValues;
+    }
+
+    /** Update the required space. */
+    void finish() {
+      if (max > min) {
+        spaceInBits += DirectWriter.unsignedBitsRequired(max - min) * numValues;
+      }
+    }
+
+    /** Update space usage and get ready for accumulating values for the next block. */
+    void nextBlock() {
+      finish();
+      reset();
+    }
+  }
+
   private long[] writeValues(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
     SortedNumericDocValues values = valuesProducer.getSortedNumeric(field);
     int numDocsWithValue = 0;
-    long numValues = 0;
-    long min = Long.MAX_VALUE;
-    long max = Long.MIN_VALUE;
+    MinMaxTracker minMax = new MinMaxTracker();
+    MinMaxTracker blockMinMax = new MinMaxTracker();
     long gcd = 0;
     Set<Long> uniqueValues = new HashSet<>();
     for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
@@ -130,26 +167,35 @@ final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Close
             // wrong results. Since these extreme values are unlikely, we just discard
             // GCD computation for them
             gcd = 1;
-          } else if (numValues != 0) { // minValue needs to be set first
-            gcd = MathUtil.gcd(gcd, v - min);
+          } else if (minMax.numValues != 0) { // minValue needs to be set first
+            gcd = MathUtil.gcd(gcd, v - minMax.min);
           }
         }
 
-        min = Math.min(min, v);
-        max = Math.max(max, v);
+        minMax.update(v);
+        blockMinMax.update(v);
+        if (blockMinMax.numValues == NUMERIC_BLOCK_SIZE) {
+          blockMinMax.nextBlock();
+        }
 
         if (uniqueValues != null
             && uniqueValues.add(v)
             && uniqueValues.size() > 256) {
           uniqueValues = null;
         }
-
-        numValues++;
       }
 
       numDocsWithValue++;
     }
 
+    minMax.finish();
+    blockMinMax.finish();
+
+    final long numValues = minMax.numValues;
+    long min = minMax.min;
+    final long max = minMax.max;
+    assert blockMinMax.spaceInBits <= minMax.spaceInBits;
+
     if (numDocsWithValue == 0) {
       meta.writeLong(-2);
       meta.writeLong(0L);
@@ -166,6 +212,7 @@ final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Close
 
     meta.writeLong(numValues);
     final int numBitsPerValue;
+    boolean doBlocks = false;
     Map<Long, Integer> encode = null;
     if (min >= max) {
       numBitsPerValue = 0;
@@ -189,12 +236,19 @@ final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Close
         gcd = 1;
       } else {
         uniqueValues = null;
-        numBitsPerValue = DirectWriter.unsignedBitsRequired((max - min) / gcd);
-        if (gcd == 1 && min > 0
-            && DirectWriter.unsignedBitsRequired(max) == DirectWriter.unsignedBitsRequired(max - min)) {
-          min = 0;
+        // we do blocks if that appears to save 10+% storage
+        doBlocks = minMax.spaceInBits > 0 && (double) blockMinMax.spaceInBits / minMax.spaceInBits <= 0.9;
+        if (doBlocks) {
+          numBitsPerValue = 0xFF;
+          meta.writeInt(-2 - NUMERIC_BLOCK_SHIFT);
+        } else {
+          numBitsPerValue = DirectWriter.unsignedBitsRequired((max - min) / gcd);
+          if (gcd == 1 && min > 0
+              && DirectWriter.unsignedBitsRequired(max) == DirectWriter.unsignedBitsRequired(max - min)) {
+            min = 0;
+          }
+          meta.writeInt(-1);
         }
-        meta.writeInt(-1);
       }
     }
 
@@ -203,26 +257,79 @@ final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Close
     meta.writeLong(gcd);
     long startOffset = data.getFilePointer();
     meta.writeLong(startOffset);
-    if (numBitsPerValue != 0) {
-      values = valuesProducer.getSortedNumeric(field);
-      DirectWriter writer = DirectWriter.getInstance(data, numValues, numBitsPerValue);
-      for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
-        for (int i = 0, count = values.docValueCount(); i < count; ++i) {
-          long v = values.nextValue();
-          if (encode == null) {
-            writer.add((v - min) / gcd);
-          } else {
-            writer.add(encode.get(v));
-          }
-        }
-      }
-      writer.finish();
+    if (doBlocks) {
+      writeValuesMultipleBlocks(valuesProducer.getSortedNumeric(field), gcd);
+    } else if (numBitsPerValue != 0) {
+      writeValuesSingleBlock(valuesProducer.getSortedNumeric(field), numValues, numBitsPerValue, min, gcd, encode);
     }
     meta.writeLong(data.getFilePointer() - startOffset);
 
     return new long[] {numDocsWithValue, numValues};
   }
 
+  private void writeValuesSingleBlock(SortedNumericDocValues values, long numValues, int numBitsPerValue,
+      long min, long gcd, Map<Long, Integer> encode) throws IOException {
+    DirectWriter writer = DirectWriter.getInstance(data, numValues, numBitsPerValue);
+    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
+      for (int i = 0, count = values.docValueCount(); i < count; ++i) {
+        long v = values.nextValue();
+        if (encode == null) {
+          writer.add((v - min) / gcd);
+        } else {
+          writer.add(encode.get(v));
+        }
+      }
+    }
+    writer.finish();
+  }
+ 
+  private void writeValuesMultipleBlocks(SortedNumericDocValues values, long gcd) throws IOException {
+    final long[] buffer = new long[NUMERIC_BLOCK_SIZE];
+    final GrowableByteArrayDataOutput encodeBuffer = new GrowableByteArrayDataOutput(NUMERIC_BLOCK_SIZE);
+    int upTo = 0;
+    for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
+      for (int i = 0, count = values.docValueCount(); i < count; ++i) {
+        buffer[upTo++] = values.nextValue();
+        if (upTo == NUMERIC_BLOCK_SIZE) {
+          writeBlock(buffer, NUMERIC_BLOCK_SIZE, gcd, encodeBuffer);
+          upTo = 0;
+        }
+      }
+    }
+    if (upTo > 0) {
+      writeBlock(buffer, upTo, gcd, encodeBuffer);
+    }
+  }
+
+  private void writeBlock(long[] values, int length, long gcd, GrowableByteArrayDataOutput buffer) throws IOException {
+    assert length > 0;
+    long min = values[0];
+    long max = values[0];
+    for (int i = 1; i < length; ++i) {
+      final long v = values[i];
+      assert Math.floorMod(values[i] - min, gcd) == 0;
+      min = Math.min(min, v);
+      max = Math.max(max, v);
+    }
+    if (min == max) {
+      data.writeByte((byte) 0);
+      data.writeLong(min);
+    } else {
+      final int bitsPerValue = DirectWriter.unsignedBitsRequired(max - min);
+      buffer.reset();
+      assert buffer.getPosition() == 0;
+      final DirectWriter w = DirectWriter.getInstance(buffer, length, bitsPerValue);
+      for (int i = 0; i < length; ++i) {
+        w.add((values[i] - min) / gcd);
+      }
+      w.finish();
+      data.writeByte((byte) bitsPerValue);
+      data.writeLong(min);
+      data.writeInt(buffer.getPosition());
+      data.writeBytes(buffer.getBytes(), buffer.getPosition());
+    }
+  }
+
   @Override
   public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
     meta.writeInt(field.number);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b182aa2/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesFormat.java
index ee477d6..2ce2124 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesFormat.java
@@ -146,10 +146,11 @@ public final class Lucene70DocValuesFormat extends DocValuesFormat {
   static final byte SORTED_SET = 3;
   static final byte SORTED_NUMERIC = 4;
 
-  // addressing uses 16k blocks
-  static final int MONOTONIC_BLOCK_SIZE = 16384;
   static final int DIRECT_MONOTONIC_BLOCK_SHIFT = 16;
 
+  static final int NUMERIC_BLOCK_SHIFT = 14;
+  static final int NUMERIC_BLOCK_SIZE = 1 << NUMERIC_BLOCK_SHIFT;
+
   static final int TERMS_DICT_BLOCK_SHIFT = 4;
   static final int TERMS_DICT_BLOCK_SIZE = 1 << TERMS_DICT_BLOCK_SHIFT;
   static final int TERMS_DICT_BLOCK_MASK = TERMS_DICT_BLOCK_SIZE - 1;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b182aa2/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesProducer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesProducer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesProducer.java
index 3f3e73f..386655e 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesProducer.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesProducer.java
@@ -144,7 +144,7 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
     entry.docsWithFieldLength = meta.readLong();
     entry.numValues = meta.readLong();
     int tableSize = meta.readInt();
-    if (tableSize < -1 || tableSize > 256) {
+    if (tableSize > 256) {
       throw new CorruptIndexException("invalid table size: " + tableSize, meta);
     }
     if (tableSize >= 0) {
@@ -154,6 +154,11 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
         entry.table[i] = meta.readLong();
       }
     }
+    if (tableSize < -1) {
+      entry.blockShift = -2 - tableSize;
+    } else {
+      entry.blockShift = -1;
+    }
     entry.bitsPerValue = meta.readByte();
     entry.minValue = meta.readLong();
     entry.gcd = meta.readLong();
@@ -260,6 +265,7 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
 
   private static class NumericEntry {
     long[] table;
+    int blockShift;
     byte bitsPerValue;
     long docsWithFieldOffset;
     long docsWithFieldLength;
@@ -429,24 +435,62 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
         };
       } else {
         final RandomAccessInput slice = data.randomAccessSlice(entry.valuesOffset, entry.valuesLength);
-        final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue);
-        if (entry.table != null) {
-          final long[] table = entry.table;
-          return new DenseNumericDocValues(maxDoc) {
-            @Override
-            public long longValue() throws IOException {
-              return table[(int) values.get(doc)];
-            }
-          };
-        } else {
+        if (entry.blockShift >= 0) {
+          // dense but split into blocks of different bits per value
+          final int shift = entry.blockShift;
           final long mul = entry.gcd;
-          final long delta = entry.minValue;
+          final int mask = (1 << shift) - 1;
           return new DenseNumericDocValues(maxDoc) {
+            int block = -1;
+            long delta;
+            long offset;
+            long blockEndOffset;
+            LongValues values;
+
             @Override
             public long longValue() throws IOException {
-              return mul * values.get(doc) + delta;
+              final int block = doc >>> shift;
+              if (this.block != block) {
+                int bitsPerValue;
+                do {
+                  offset = blockEndOffset;
+                  bitsPerValue = slice.readByte(offset++);
+                  delta = slice.readLong(offset);
+                  offset += Long.BYTES;
+                  if (bitsPerValue == 0) {
+                    blockEndOffset = offset;
+                  } else {
+                    final int length = slice.readInt(offset);
+                    offset += Integer.BYTES;
+                    blockEndOffset = offset + length;
+                  }
+                  this.block ++;
+                } while (this.block != block);
+                values = bitsPerValue == 0 ? LongValues.ZEROES : DirectReader.getInstance(slice, bitsPerValue, offset);
+              }
+              return mul * values.get(doc & mask) + delta;
             }
           };
+        } else {
+          final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue);
+          if (entry.table != null) {
+            final long[] table = entry.table;
+            return new DenseNumericDocValues(maxDoc) {
+              @Override
+              public long longValue() throws IOException {
+                return table[(int) values.get(doc)];
+              }
+            };
+          } else {
+            final long mul = entry.gcd;
+            final long delta = entry.minValue;
+            return new DenseNumericDocValues(maxDoc) {
+              @Override
+              public long longValue() throws IOException {
+                return mul * values.get(doc) + delta;
+              }
+            };
+          }
         }
       }
     } else {
@@ -461,24 +505,63 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
         };
       } else {
         final RandomAccessInput slice = data.randomAccessSlice(entry.valuesOffset, entry.valuesLength);
-        final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue);
-        if (entry.table != null) {
-          final long[] table = entry.table;
-          return new SparseNumericDocValues(disi) {
-            @Override
-            public long longValue() throws IOException {
-              return table[(int) values.get(disi.index())];
-            }
-          };
-        } else {
+        if (entry.blockShift >= 0) {
+          // sparse and split into blocks of different bits per value
+          final int shift = entry.blockShift;
           final long mul = entry.gcd;
-          final long delta = entry.minValue;
+          final int mask = (1 << shift) - 1;
           return new SparseNumericDocValues(disi) {
+            int block = -1;
+            long delta;
+            long offset;
+            long blockEndOffset;
+            LongValues values;
+
             @Override
             public long longValue() throws IOException {
-              return mul * values.get(disi.index()) + delta;
+              final int index = disi.index();
+              final int block = index >>> shift;
+              if (this.block != block) {
+                int bitsPerValue;
+                do {
+                  offset = blockEndOffset;
+                  bitsPerValue = slice.readByte(offset++);
+                  delta = slice.readLong(offset);
+                  offset += Long.BYTES;
+                  if (bitsPerValue == 0) {
+                    blockEndOffset = offset;
+                  } else {
+                    final int length = slice.readInt(offset);
+                    offset += Integer.BYTES;
+                    blockEndOffset = offset + length;
+                  }
+                  this.block ++;
+                } while (this.block != block);
+                values = bitsPerValue == 0 ? LongValues.ZEROES : DirectReader.getInstance(slice, bitsPerValue, offset);
+              }
+              return mul * values.get(index & mask) + delta;
             }
           };
+        } else {
+          final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue);
+          if (entry.table != null) {
+            final long[] table = entry.table;
+            return new SparseNumericDocValues(disi) {
+              @Override
+              public long longValue() throws IOException {
+                return table[(int) values.get(disi.index())];
+              }
+            };
+          } else {
+            final long mul = entry.gcd;
+            final long delta = entry.minValue;
+            return new SparseNumericDocValues(disi) {
+              @Override
+              public long longValue() throws IOException {
+                return mul * values.get(disi.index()) + delta;
+              }
+            };
+          }
         }
       }
     }
@@ -494,34 +577,75 @@ final class Lucene70DocValuesProducer extends DocValuesProducer implements Close
       };
     } else {
       final RandomAccessInput slice = data.randomAccessSlice(entry.valuesOffset, entry.valuesLength);
-      final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue);
-      if (entry.table != null) {
-        final long[] table = entry.table;
+      if (entry.blockShift >= 0) {
+        final int shift = entry.blockShift;
+        final long mul = entry.gcd;
+        final long mask = (1L << shift) - 1;
         return new LongValues() {
-          @Override
-          public long get(long index) {
-            return table[(int) values.get(index)];
-          }
-        };
-      } else if (entry.gcd != 1) {
-        final long gcd = entry.gcd;
-        final long minValue = entry.minValue;
-        return new LongValues() {
-          @Override
-          public long get(long index) {
-            return values.get(index) * gcd + minValue;
-          }
-        };
-      } else if (entry.minValue != 0) {
-        final long minValue = entry.minValue;
-        return new LongValues() {
-          @Override
+          long block = -1;
+          long delta;
+          long offset;
+          long blockEndOffset;
+          LongValues values;
+
           public long get(long index) {
-            return values.get(index) + minValue;
+            final long block = index >>> shift;
+            if (this.block != block) {
+              assert block > this.block : "Reading backwards is illegal: " + this.block + " < " + block;
+              int bitsPerValue;
+              do {
+                offset = blockEndOffset;
+                try {
+                  bitsPerValue = slice.readByte(offset++);
+                  delta = slice.readLong(offset);
+                  offset += Long.BYTES;
+                  if (bitsPerValue == 0) {
+                    blockEndOffset = offset;
+                  } else {
+                    final int length = slice.readInt(offset);
+                    offset += Integer.BYTES;
+                    blockEndOffset = offset + length;
+                  }
+                } catch (IOException e) {
+                  throw new RuntimeException(e);
+                }
+                this.block ++;
+              } while (this.block != block);
+              values = bitsPerValue == 0 ? LongValues.ZEROES : DirectReader.getInstance(slice, bitsPerValue, offset);
+            }
+            return mul * values.get(index & mask) + delta;
           }
         };
       } else {
-        return values;
+        final LongValues values = DirectReader.getInstance(slice, entry.bitsPerValue);
+        if (entry.table != null) {
+          final long[] table = entry.table;
+          return new LongValues() {
+            @Override
+            public long get(long index) {
+              return table[(int) values.get(index)];
+            }
+          };
+        } else if (entry.gcd != 1) {
+          final long gcd = entry.gcd;
+          final long minValue = entry.minValue;
+          return new LongValues() {
+            @Override
+            public long get(long index) {
+              return values.get(index) * gcd + minValue;
+            }
+          };
+        } else if (entry.minValue != 0) {
+          final long minValue = entry.minValue;
+          return new LongValues() {
+            @Override
+            public long get(long index) {
+              return values.get(index) + minValue;
+            }
+          };
+        } else {
+          return values;
+        }
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b182aa2/lucene/core/src/java/org/apache/lucene/util/LongValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/LongValues.java b/lucene/core/src/java/org/apache/lucene/util/LongValues.java
index 23f4d32..04fbf81 100644
--- a/lucene/core/src/java/org/apache/lucene/util/LongValues.java
+++ b/lucene/core/src/java/org/apache/lucene/util/LongValues.java
@@ -30,6 +30,15 @@ public abstract class LongValues  {
 
   };
 
+  public static final LongValues ZEROES = new LongValues() {
+
+    @Override
+    public long get(long index) {
+      return 0;
+    }
+
+  };
+
   /** Get value at <code>index</code>. */
   public abstract long get(long index);
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b182aa2/lucene/core/src/java/org/apache/lucene/util/packed/DirectWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/packed/DirectWriter.java b/lucene/core/src/java/org/apache/lucene/util/packed/DirectWriter.java
index 9a7f18e..5a38445 100644
--- a/lucene/core/src/java/org/apache/lucene/util/packed/DirectWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/util/packed/DirectWriter.java
@@ -21,7 +21,7 @@ import java.io.EOFException;
 import java.io.IOException;
 import java.util.Arrays;
 
-import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.DataOutput;
 
 /** 
  * Class for writing packed integers to be directly read from Directory.
@@ -44,7 +44,7 @@ import org.apache.lucene.store.IndexOutput;
 public final class DirectWriter {
   final int bitsPerValue;
   final long numValues;
-  final IndexOutput output;
+  final DataOutput output;
   
   long count;
   boolean finished;
@@ -56,7 +56,7 @@ public final class DirectWriter {
   final BulkOperation encoder;
   final int iterations;
   
-  DirectWriter(IndexOutput output, long numValues, int bitsPerValue) {
+  DirectWriter(DataOutput output, long numValues, int bitsPerValue) {
     this.output = output;
     this.numValues = numValues;
     this.bitsPerValue = bitsPerValue;
@@ -103,7 +103,7 @@ public final class DirectWriter {
   }
   
   /** Returns an instance suitable for encoding {@code numValues} using {@code bitsPerValue} */
-  public static DirectWriter getInstance(IndexOutput output, long numValues, int bitsPerValue) {
+  public static DirectWriter getInstance(DataOutput output, long numValues, int bitsPerValue) {
     if (Arrays.binarySearch(SUPPORTED_BITS_PER_VALUE, bitsPerValue) < 0) {
       throw new IllegalArgumentException("Unsupported bitsPerValue " + bitsPerValue + ". Did you use bitsRequired?");
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3b182aa2/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70DocValuesFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70DocValuesFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70DocValuesFormat.java
index 8661298..6cca55e 100644
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70DocValuesFormat.java
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70DocValuesFormat.java
@@ -25,6 +25,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 import java.util.TreeSet;
+import java.util.function.LongSupplier;
 import java.util.function.Supplier;
 
 import org.apache.lucene.analysis.MockAnalyzer;
@@ -61,6 +62,7 @@ import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum.SeekStatus;
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.RAMFile;
@@ -534,4 +536,154 @@ public class TestLucene70DocValuesFormat extends BaseCompressingDocValuesFormatT
       dir.close();
     }
   }
+
+  @Slow
+  public void testSortedNumericBlocksOfVariousBitsPerValue() throws Exception {
+    doTestSortedNumericBlocksOfVariousBitsPerValue(() -> TestUtil.nextInt(random(), 1, 3));
+  }
+
+  @Slow
+  public void testSparseSortedNumericBlocksOfVariousBitsPerValue() throws Exception {
+    doTestSortedNumericBlocksOfVariousBitsPerValue(() -> TestUtil.nextInt(random(), 0, 2));
+  }
+
+  @Slow
+  public void testNumericBlocksOfVariousBitsPerValue() throws Exception {
+    doTestSparseNumericBlocksOfVariousBitsPerValue(1);
+  }
+
+  @Slow
+  public void testSparseNumericBlocksOfVariousBitsPerValue() throws Exception {
+    doTestSparseNumericBlocksOfVariousBitsPerValue(random().nextDouble());
+  }
+
+  private static LongSupplier blocksOfVariousBPV() {
+    final long mul = TestUtil.nextInt(random(), 1, 100);
+    final long min = random().nextInt();
+    return new LongSupplier() {
+      int i = Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE;
+      int maxDelta;
+      @Override
+      public long getAsLong() {
+        if (i == Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE) {
+          maxDelta = 1 << random().nextInt(5);
+          i = 0;
+        }
+        i++;
+        return min + mul * random().nextInt(maxDelta);
+      }
+    };
+  }
+
+  private void doTestSortedNumericBlocksOfVariousBitsPerValue(LongSupplier counts) throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
+    conf.setMaxBufferedDocs(atLeast(Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE));
+    conf.setRAMBufferSizeMB(-1);
+    conf.setMergePolicy(newLogMergePolicy(random().nextBoolean()));
+    IndexWriter writer = new IndexWriter(dir, conf);
+    
+    final int numDocs = atLeast(Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE*3);
+    final LongSupplier values = blocksOfVariousBPV();
+    for (int i = 0; i < numDocs; i++) {
+      Document doc = new Document();
+      
+      int valueCount = (int) counts.getAsLong();
+      long valueArray[] = new long[valueCount];
+      for (int j = 0; j < valueCount; j++) {
+        long value = values.getAsLong();
+        valueArray[j] = value;
+        doc.add(new SortedNumericDocValuesField("dv", value));
+      }
+      Arrays.sort(valueArray);
+      for (int j = 0; j < valueCount; j++) {
+        doc.add(new StoredField("stored", Long.toString(valueArray[j])));
+      }
+      writer.addDocument(doc);
+      if (random().nextInt(31) == 0) {
+        writer.commit();
+      }
+    }
+    writer.forceMerge(1);
+
+    writer.close();
+    
+    // compare
+    DirectoryReader ir = DirectoryReader.open(dir);
+    TestUtil.checkReader(ir);
+    for (LeafReaderContext context : ir.leaves()) {
+      LeafReader r = context.reader();
+      SortedNumericDocValues docValues = DocValues.getSortedNumeric(r, "dv");
+      for (int i = 0; i < r.maxDoc(); i++) {
+        if (i > docValues.docID()) {
+          docValues.nextDoc();
+        }
+        String expected[] = r.document(i).getValues("stored");
+        if (i < docValues.docID()) {
+          assertEquals(0, expected.length);
+        } else {
+          String actual[] = new String[docValues.docValueCount()];
+          for (int j = 0; j < actual.length; j++) {
+            actual[j] = Long.toString(docValues.nextValue());
+          }
+          assertArrayEquals(expected, actual);
+        }
+      }
+    }
+    ir.close();
+    dir.close();
+  }
+
+  private void doTestSparseNumericBlocksOfVariousBitsPerValue(double density) throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
+    conf.setMaxBufferedDocs(atLeast(Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE));
+    conf.setRAMBufferSizeMB(-1);
+    conf.setMergePolicy(newLogMergePolicy(random().nextBoolean()));
+    IndexWriter writer = new IndexWriter(dir, conf);
+    Document doc = new Document();
+    Field storedField = newStringField("stored", "", Field.Store.YES);
+    Field dvField = new NumericDocValuesField("dv", 0);
+    doc.add(storedField);
+    doc.add(dvField);
+
+    final int numDocs = atLeast(Lucene70DocValuesFormat.NUMERIC_BLOCK_SIZE*3);
+    final LongSupplier longs = blocksOfVariousBPV();
+    for (int i = 0; i < numDocs; i++) {
+      if (random().nextDouble() > density) {
+        writer.addDocument(new Document());
+        continue;
+      }
+      long value = longs.getAsLong();
+      storedField.setStringValue(Long.toString(value));
+      dvField.setLongValue(value);
+      writer.addDocument(doc);
+    }
+
+    writer.forceMerge(1);
+
+    writer.close();
+    
+    // compare
+    DirectoryReader ir = DirectoryReader.open(dir);
+    TestUtil.checkReader(ir);
+    for (LeafReaderContext context : ir.leaves()) {
+      LeafReader r = context.reader();
+      NumericDocValues docValues = DocValues.getNumeric(r, "dv");
+      docValues.nextDoc();
+      for (int i = 0; i < r.maxDoc(); i++) {
+        String storedValue = r.document(i).get("stored");
+        if (storedValue == null) {
+          assertTrue(docValues.docID() > i);
+        } else {
+          assertEquals(i, docValues.docID());
+          assertEquals(Long.parseLong(storedValue), docValues.longValue());
+          docValues.nextDoc();
+        }
+      }
+      assertEquals(DocIdSetIterator.NO_MORE_DOCS, docValues.docID());
+    }
+    ir.close();
+    dir.close();
+  }
 }


[2/2] lucene-solr:master: LUCENE-7572: Cache the hash code of doc values queries.

Posted by jp...@apache.org.
LUCENE-7572: Cache the hash code of doc values queries.


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/ea1569e2
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/ea1569e2
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/ea1569e2

Branch: refs/heads/master
Commit: ea1569e2914f9ba914b582a0801d6cb83a29529b
Parents: 268d4ac
Author: Adrien Grand <jp...@gmail.com>
Authored: Thu Dec 15 16:30:15 2016 +0100
Committer: Adrien Grand <jp...@gmail.com>
Committed: Thu Dec 15 17:17:54 2016 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   2 +
 .../apache/lucene/index/PrefixCodedTerms.java   |   4 +-
 .../lucene/search/DocValuesNumbersQuery.java    |  26 ++--
 .../lucene/search/DocValuesTermsQuery.java      |  49 ++++--
 .../org/apache/lucene/search/LongHashSet.java   | 156 +++++++++++++++++++
 .../apache/lucene/search/LongHashSetTests.java  | 100 ++++++++++++
 .../lucene/search/TestDocValuesTermsQuery.java  |   1 +
 7 files changed, 310 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ea1569e2/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 0e327d2..bacc270 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -158,6 +158,8 @@ Optimizations
   writing to disk, giving a small speedup in points-heavy use cases.
   (Mike McCandless)
 
+* LUCENE-7572: Doc values queries now cache their hash code. (Adrien Grand)
+
 Other
 
 * LUCENE-7546: Fixed references to benchmark wikipedia data and the Jenkins line-docs file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ea1569e2/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java b/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java
index 3dca3db..df1653b 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java
@@ -28,7 +28,9 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 
 /**
- * Prefix codes term instances (prefixes are shared)
+ * Prefix codes term instances (prefixes are shared). This is expected to be
+ * faster to build than a FST and might also be more compact if there are no
+ * common suffixes.
  * @lucene.internal
  */
 public class PrefixCodedTerms implements Accountable {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ea1569e2/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesNumbersQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesNumbersQuery.java b/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesNumbersQuery.java
index 0fd2244..7725703 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesNumbersQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesNumbersQuery.java
@@ -18,6 +18,7 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.HashSet;
 import java.util.Objects;
 import java.util.Set;
@@ -45,11 +46,16 @@ import org.apache.lucene.index.SortedNumericDocValues;
 public class DocValuesNumbersQuery extends Query {
 
   private final String field;
-  private final Set<Long> numbers;
+  private final LongHashSet numbers;
 
-  public DocValuesNumbersQuery(String field, Set<Long> numbers) {
+  public DocValuesNumbersQuery(String field, long[] numbers) {
     this.field = Objects.requireNonNull(field);
-    this.numbers = Objects.requireNonNull(numbers, "Set of numbers must not be null");
+    this.numbers = new LongHashSet(numbers);
+  }
+
+  public DocValuesNumbersQuery(String field, Collection<Long> numbers) {
+    this.field = Objects.requireNonNull(field);
+    this.numbers = new LongHashSet(numbers.stream().mapToLong(Long::longValue).toArray());
   }
 
   public DocValuesNumbersQuery(String field, Long... numbers) {
@@ -82,15 +88,11 @@ public class DocValuesNumbersQuery extends Query {
 
   @Override
   public String toString(String defaultField) {
-    StringBuilder sb = new StringBuilder();
-    sb.append(field).append(": [");
-    for (Long number : numbers) {
-      sb.append(number).append(", ");
-    }
-    if (numbers.size() > 0) {
-      sb.setLength(sb.length() - 2);
-    }
-    return sb.append(']').toString();
+    return new StringBuilder()
+        .append(field)
+        .append(": ")
+        .append(numbers.toString())
+        .toString();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ea1569e2/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesTermsQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesTermsQuery.java b/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesTermsQuery.java
index 6d852a8..6e30bae 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesTermsQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesTermsQuery.java
@@ -25,7 +25,10 @@ import java.util.Objects;
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.PrefixCodedTerms;
 import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.PrefixCodedTerms.TermIterator;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FixedBitSet;
@@ -91,13 +94,24 @@ import org.apache.lucene.util.LongBitSet;
 public class DocValuesTermsQuery extends Query {
 
   private final String field;
-  private final BytesRef[] terms;
+  private final PrefixCodedTerms termData;
+  private final int termDataHashCode; // cached hashcode of termData
 
   public DocValuesTermsQuery(String field, Collection<BytesRef> terms) {
     this.field = Objects.requireNonNull(field);
     Objects.requireNonNull(terms, "Collection of terms must not be null");
-    this.terms = terms.toArray(new BytesRef[terms.size()]);
-    ArrayUtil.timSort(this.terms);
+    BytesRef[] sortedTerms = terms.toArray(new BytesRef[terms.size()]);
+    ArrayUtil.timSort(sortedTerms);
+    PrefixCodedTerms.Builder builder = new PrefixCodedTerms.Builder();
+    BytesRef previous = null;
+    for (BytesRef term : sortedTerms) {
+      if (term.equals(previous) == false) {
+        builder.add(field, term);
+      }
+      previous = term;
+    }
+    termData = builder.finish();
+    termDataHashCode = termData.hashCode();
   }
 
   public DocValuesTermsQuery(String field, BytesRef... terms) {
@@ -124,26 +138,30 @@ public class DocValuesTermsQuery extends Query {
   }
 
   private boolean equalsTo(DocValuesTermsQuery other) {
-    return field.equals(other.field) &&
-           Arrays.equals(terms, other.terms);
+    // termData might be heavy to compare so check the hash code first
+    return termDataHashCode == other.termDataHashCode && 
+           termData.equals(other.termData);
   }
 
   @Override
   public int hashCode() {
-    return 31 * classHash() + Objects.hash(field, Arrays.asList(terms));
+    return 31 * classHash() + termDataHashCode;
   }
 
   @Override
   public String toString(String defaultField) {
-    StringBuilder sb = new StringBuilder();
-    sb.append(field).append(": [");
-    for (BytesRef term : terms) {
-      sb.append(term).append(", ");
-    }
-    if (terms.length > 0) {
-      sb.setLength(sb.length() - 2);
+    StringBuilder builder = new StringBuilder();
+    boolean first = true;
+    TermIterator iterator = termData.iterator();
+    for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
+      if (!first) {
+        builder.append(' ');
+      }
+      first = false;
+      builder.append(new Term(iterator.field(), term).toString());
     }
-    return sb.append(']').toString();
+
+    return builder.toString();
   }
 
   @Override
@@ -155,7 +173,8 @@ public class DocValuesTermsQuery extends Query {
         final SortedSetDocValues values = DocValues.getSortedSet(context.reader(), field);
         final LongBitSet bits = new LongBitSet(values.getValueCount());
         boolean matchesAtLeastOneTerm = false;
-        for (BytesRef term : terms) {
+        TermIterator iterator = termData.iterator();
+        for (BytesRef term = iterator.next(); term != null; term = iterator.next()) {
           final long ord = values.lookupTerm(term);
           if (ord >= 0) {
             matchesAtLeastOneTerm = true;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ea1569e2/lucene/sandbox/src/java/org/apache/lucene/search/LongHashSet.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/LongHashSet.java b/lucene/sandbox/src/java/org/apache/lucene/search/LongHashSet.java
new file mode 100644
index 0000000..3a6af5f
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/LongHashSet.java
@@ -0,0 +1,156 @@
+/*
+ * 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.
+ */
+package org.apache.lucene.search;
+
+import java.util.AbstractSet;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+import org.apache.lucene.util.packed.PackedInts;
+
+final class LongHashSet extends AbstractSet<Long> {
+
+  private static final long MISSING = Long.MIN_VALUE;
+
+  final long[] table;
+  final int mask;
+  final boolean hasMissingValue;
+  final int size;
+  final int hashCode;
+
+  LongHashSet(long... values) {
+    int tableSize = Math.toIntExact(values.length * 3L / 2);
+    tableSize = 1 << PackedInts.bitsRequired(tableSize); // make it a power of 2
+    assert tableSize >= values.length * 3L / 2;
+    table = new long[tableSize];
+    Arrays.fill(table, MISSING);
+    mask = tableSize - 1;
+    boolean hasMissingValue = false;
+    int size = 0;
+    int hashCode = 0;
+    for (long value : values) {
+      if (value == MISSING || add(value)) {
+        if (value == MISSING) {
+          hasMissingValue = true;
+        }
+        ++size;
+        hashCode += Long.hashCode(value);
+      }
+    }
+    this.hasMissingValue = hasMissingValue;
+    this.size = size;
+    this.hashCode = hashCode;
+  }
+
+  private boolean add(long l) {
+    assert l != MISSING;
+    final int slot = Long.hashCode(l) & mask;
+    for (int i = slot; ; i = (i + 1) & mask) {
+      if (table[i] == MISSING) {
+        table[i] = l;
+        return true;
+      } else if (table[i] == l) {
+        // already added
+        return false;
+      }
+    }
+  }
+
+  boolean contains(long l) {
+    if (l == MISSING) {
+      return hasMissingValue;
+    }
+    final int slot = Long.hashCode(l) & mask;
+    for (int i = slot; ; i = (i + 1) & mask) {
+      if (table[i] == MISSING) {
+        return false;
+      } else if (table[i] == l) {
+        return true;
+      }
+    }
+  }
+
+  @Override
+  public int size() {
+    return size;
+  }
+
+  @Override
+  public int hashCode() {
+    return hashCode;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj != null && obj.getClass() == LongHashSet.class) {
+      LongHashSet that = (LongHashSet) obj;
+      if (hashCode != that.hashCode
+          || size != that.size
+          || hasMissingValue != that.hasMissingValue) {
+        return false;
+      }
+      for (long v : table) {
+        if (v != MISSING && that.contains(v) == false) {
+          return false;
+        }
+      }
+      return true;
+    }
+    return super.equals(obj);
+  }
+
+  @Override
+  public boolean contains(Object o) {
+    return o instanceof Long && contains(((Long) o).longValue());
+  }
+
+  @Override
+  public Iterator<Long> iterator() {
+    return new Iterator<Long>() {
+
+      private boolean hasNext = hasMissingValue;
+      private int i = -1;
+      private long value = MISSING;
+
+      @Override
+      public boolean hasNext() {
+        if (hasNext) {
+          return true;
+        }
+        while (++i < table.length) {
+          value = table[i];
+          if (value != MISSING) {
+            return hasNext = true;
+          }
+        }
+        return false;
+      }
+
+      @Override
+      public Long next() {
+        if (hasNext() == false) {
+          throw new NoSuchElementException();
+        }
+        hasNext = false;
+        return value;
+      }
+
+    };
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ea1569e2/lucene/sandbox/src/test/org/apache/lucene/search/LongHashSetTests.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/LongHashSetTests.java b/lucene/sandbox/src/test/org/apache/lucene/search/LongHashSetTests.java
new file mode 100644
index 0000000..25d94a6
--- /dev/null
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/LongHashSetTests.java
@@ -0,0 +1,100 @@
+/*
+ * 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.
+ */
+package org.apache.lucene.search;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.LongStream;
+
+import org.apache.lucene.util.LuceneTestCase;
+
+public class LongHashSetTests extends LuceneTestCase {
+
+  private void assertEquals(Set<Long> set1, LongHashSet set2) {
+    LuceneTestCase.assertEquals(set1, set2);
+    LuceneTestCase.assertEquals(set2, set1);
+    LuceneTestCase.assertEquals(set2, set2);
+    assertEquals(set1.hashCode(), set2.hashCode());
+
+    if (set1.isEmpty() == false) {
+      Set<Long> set3 = new HashSet<>(set1);
+      long removed = set3.iterator().next();
+      while (true) {
+        long next = random().nextLong();
+        if (next != removed && set3.add(next)) {
+          break;
+        }
+      }
+      assertNotEquals(set3, set2);
+    }
+  }
+
+  private void assertNotEquals(Set<Long> set1, LongHashSet set2) {
+    assertFalse(set1.equals(set2));
+    assertFalse(set2.equals(set1));
+    LongHashSet set3 = new LongHashSet(set1.stream().mapToLong(Long::longValue).toArray());
+    assertFalse(set2.equals(set3));
+  }
+
+  public void testEmpty() {
+    Set<Long> set1 = new HashSet<>();
+    LongHashSet set2 = new LongHashSet();
+    assertEquals(set1, set2);
+  }
+
+  public void testOneValue() {
+    Set<Long> set1 = new HashSet<>(Arrays.asList(42L));
+    LongHashSet set2 = new LongHashSet(42);
+    assertEquals(set1, set2);
+
+    set1 = new HashSet<>(Arrays.asList(Long.MIN_VALUE));
+    set2 = new LongHashSet(Long.MIN_VALUE);
+    assertEquals(set1, set2);
+  }
+
+  public void testTwoValues() {
+    Set<Long> set1 = new HashSet<>(Arrays.asList(42L, Long.MAX_VALUE));
+    LongHashSet set2 = new LongHashSet(42, Long.MAX_VALUE);
+    assertEquals(set1, set2);
+
+    set1 = new HashSet<>(Arrays.asList(Long.MIN_VALUE, 42L));
+    set2 = new LongHashSet(Long.MIN_VALUE, 42L);
+    assertEquals(set1, set2);
+  }
+
+  public void testRandom() {
+    final int iters = atLeast(10);
+    for (int iter = 0; iter < iters; ++iter) {
+      long[] values = new long[random().nextInt(1 << random().nextInt(16))];
+      for (int i = 0; i < values.length; ++i) {
+        if (i == 0 || random().nextInt(10) < 9) {
+          values[i] = random().nextLong();
+        } else {
+          values[i] = values[random().nextInt(i)];
+        }
+      }
+      if (values.length > 0 && random().nextBoolean()) {
+        values[values.length/2] = Long.MIN_VALUE;
+      }
+      Set<Long> set1 = LongStream.of(values).mapToObj(Long::valueOf).collect(Collectors.toCollection(HashSet::new));
+      LongHashSet set2 = new LongHashSet(values);
+      assertEquals(set1, set2);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ea1569e2/lucene/sandbox/src/test/org/apache/lucene/search/TestDocValuesTermsQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/TestDocValuesTermsQuery.java b/lucene/sandbox/src/test/org/apache/lucene/search/TestDocValuesTermsQuery.java
index 6e99492..187f172 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/search/TestDocValuesTermsQuery.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/TestDocValuesTermsQuery.java
@@ -38,6 +38,7 @@ public class TestDocValuesTermsQuery extends LuceneTestCase {
 
   public void testEquals() {
     assertEquals(new DocValuesTermsQuery("foo", "bar"), new DocValuesTermsQuery("foo", "bar"));
+    assertEquals(new DocValuesTermsQuery("foo", "bar"), new DocValuesTermsQuery("foo", "bar", "bar"));
     assertEquals(new DocValuesTermsQuery("foo", "bar", "baz"), new DocValuesTermsQuery("foo", "baz", "bar"));
     assertFalse(new DocValuesTermsQuery("foo", "bar").equals(new DocValuesTermsQuery("foo2", "bar")));
     assertFalse(new DocValuesTermsQuery("foo", "bar").equals(new DocValuesTermsQuery("foo", "baz")));