You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2016/12/19 12:51:45 UTC

[01/23] lucene-solr:feature/metrics: fix RangeFieldQuery.scorer to return null if no docs in a segment indexed the field

Repository: lucene-solr
Updated Branches:
  refs/heads/feature/metrics 92ef10dbd -> 67dc7604e


fix RangeFieldQuery.scorer to return null if no docs in a segment indexed the field


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

Branch: refs/heads/feature/metrics
Commit: 9aa5b734c38ed0b9327577bd2b1413d448230eab
Parents: 8c79ab2
Author: Nicholas Knize <nk...@gmail.com>
Authored: Tue Dec 13 15:07:06 2016 -0600
Committer: Nicholas Knize <nk...@gmail.com>
Committed: Tue Dec 13 15:07:06 2016 -0600

----------------------------------------------------------------------
 .../src/java/org/apache/lucene/document/RangeFieldQuery.java       | 1 +
 .../test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java | 2 +-
 2 files changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9aa5b734/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java b/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java
index 7ebdec4..5249191 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/RangeFieldQuery.java
@@ -165,6 +165,7 @@ abstract class RangeFieldQuery extends Query {
         FieldInfo fieldInfo = reader.getFieldInfos().fieldInfo(field);
         if (fieldInfo == null) {
           // no docs in this segment indexed this field
+          return null;
         }
         checkFieldInfo(fieldInfo);
         boolean allDocsMatch = true;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9aa5b734/lucene/sandbox/src/test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java b/lucene/sandbox/src/test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java
index ff61ff6..ceafd53 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/BaseRangeFieldQueryTestCase.java
@@ -93,7 +93,7 @@ public abstract class BaseRangeFieldQueryTestCase extends LuceneTestCase {
         ranges[id] = new Range[] {nextRange(dimensions)};
       }
       if (x == 17) {
-        // dome docs don't have a box:
+        // some docs don't have a box:
         ranges[id][0].isMissing = true;
         if (VERBOSE) {
           System.out.println("  id=" + id + " is missing");


[04/23] lucene-solr:feature/metrics: LUCENE-7590: fix typo in method parameter

Posted by ab...@apache.org.
LUCENE-7590: fix typo in method parameter


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

Branch: refs/heads/feature/metrics
Commit: 85582dabe4372085e1af5d01ebbfcfd0303b9f12
Parents: 770f1eb
Author: Shai Erera <sh...@apache.org>
Authored: Wed Dec 14 13:28:02 2016 +0200
Committer: Shai Erera <sh...@apache.org>
Committed: Wed Dec 14 13:28:02 2016 +0200

----------------------------------------------------------------------
 .../misc/src/java/org/apache/lucene/search/DocValuesStats.java | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/85582dab/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java b/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
index fad9f97..38158cf 100644
--- a/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
+++ b/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
@@ -51,7 +51,7 @@ public abstract class DocValuesStats<T> {
    * Initializes this object with the given reader context. Returns whether stats can be computed for this segment (i.e.
    * it does have the requested DocValues field).
    */
-  protected abstract boolean init(LeafReaderContext contxt) throws IOException;
+  protected abstract boolean init(LeafReaderContext context) throws IOException;
 
   /** Returns whether the given document has a value for the requested DocValues field. */
   protected abstract boolean hasValue(int doc) throws IOException;
@@ -106,8 +106,8 @@ public abstract class DocValuesStats<T> {
     }
 
     @Override
-    protected final boolean init(LeafReaderContext contxt) throws IOException {
-      ndv = contxt.reader().getNumericDocValues(field);
+    protected final boolean init(LeafReaderContext context) throws IOException {
+      ndv = context.reader().getNumericDocValues(field);
       return ndv != null;
     }
 


[13/23] lucene-solr:feature/metrics: LUCENE-7589: Prevent outliers from raising the bpv for everyone.

Posted by ab...@apache.org.
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/feature/metrics
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();
+  }
 }


[21/23] lucene-solr:feature/metrics: LUCENE-7590: make (Sorted)NumericDocValuesStats public

Posted by ab...@apache.org.
LUCENE-7590: make (Sorted)NumericDocValuesStats public


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

Branch: refs/heads/feature/metrics
Commit: 321c6f090f04463a8798d090e5426efeabbdc418
Parents: 23206ca
Author: Shai Erera <sh...@apache.org>
Authored: Mon Dec 19 10:14:58 2016 +0200
Committer: Shai Erera <sh...@apache.org>
Committed: Mon Dec 19 10:14:58 2016 +0200

----------------------------------------------------------------------
 .../src/java/org/apache/lucene/search/DocValuesStats.java | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/321c6f09/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java b/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
index b6449cc..f3319ee 100644
--- a/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
+++ b/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
@@ -44,7 +44,7 @@ public abstract class DocValuesStats<T> {
   }
 
   /**
-   * Called after #{@link DocValuesStats#accumulate(int)} was processed and verified that the document has a value for
+   * Called after {@link #accumulate(int)} was processed and verified that the document has a value for
    * the field. Implementations should update the statistics based on the value of the current document.
    *
    * @param count
@@ -89,18 +89,18 @@ public abstract class DocValuesStats<T> {
     return missing;
   }
 
-  /** The minimum value of the field. Undefined when {@link #count} is zero. */
+  /** The minimum value of the field. Undefined when {@link #count()} is zero. */
   public final T min() {
     return min;
   }
 
-  /** The maximum value of the field. Undefined when {@link #count} is zero. */
+  /** The maximum value of the field. Undefined when {@link #count()} is zero. */
   public final T max() {
     return max;
   }
 
   /** Holds statistics for a numeric DocValues field. */
-  static abstract class NumericDocValuesStats<T extends Number> extends DocValuesStats<T> {
+  public static abstract class NumericDocValuesStats<T extends Number> extends DocValuesStats<T> {
 
     protected double mean = 0.0;
     protected double variance = 0.0;
@@ -205,7 +205,7 @@ public abstract class DocValuesStats<T> {
   }
 
   /** Holds statistics for a sorted-numeric DocValues field. */
-  static abstract class SortedNumericDocValuesStats<T extends Number> extends DocValuesStats<T> {
+  public static abstract class SortedNumericDocValuesStats<T extends Number> extends DocValuesStats<T> {
 
     protected long valuesCount = 0;
     protected double mean = 0.0;


[07/23] lucene-solr:feature/metrics: SOLR-9846: OverseerAutoReplicaFailoverThread can take too long to stop and leak out of unit tests.

Posted by ab...@apache.org.
SOLR-9846: OverseerAutoReplicaFailoverThread can take too long to stop and leak out of unit tests.


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

Branch: refs/heads/feature/metrics
Commit: 7dec783b287ab554cc781622b4d6127e553fd2ae
Parents: e82399d
Author: markrmiller <ma...@apache.org>
Authored: Sun Dec 11 22:02:48 2016 -0500
Committer: markrmiller <ma...@apache.org>
Committed: Wed Dec 14 12:57:21 2016 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                            | 2 ++
 .../solr/cloud/OverseerAutoReplicaFailoverThread.java       | 9 +++++++--
 2 files changed, 9 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7dec783b/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 946a04e..5f0357b 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -288,6 +288,8 @@ Other Changes
   response (instead of a SolrException) and includes the remote error message as part of the exception message
   (Tom�s Fern�ndez L�bbe)
 
+* SOLR-9846: OverseerAutoReplicaFailoverThread can take too long to stop and leak out of unit tests. (Mark Miller)
+
 ==================  6.3.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7dec783b/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java b/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java
index 83679a5..10b4bf3 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java
@@ -89,6 +89,8 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable {
 
   private final int workLoopDelay;
   private final int waitAfterExpiration;
+
+  private volatile Thread thread;
   
   public OverseerAutoReplicaFailoverThread(CloudConfig config, ZkStateReader zkStateReader,
       UpdateShardHandler updateShardHandler) {
@@ -118,7 +120,7 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable {
   
   @Override
   public void run() {
-    
+    this.thread = Thread.currentThread();
     while (!this.isClosed) {
       // work loop
       log.debug("do " + this.getClass().getSimpleName() + " work loop");
@@ -136,7 +138,6 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable {
         try {
           Thread.sleep(workLoopDelay);
         } catch (InterruptedException e) {
-          Thread.currentThread().interrupt();
           return;
         }
       }
@@ -480,6 +481,10 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable {
   @Override
   public void close() {
     isClosed = true;
+    Thread lThread = thread;
+    if (lThread != null) {
+      lThread.interrupt();
+    }
   }
   
   public boolean isClosed() {


[14/23] lucene-solr:feature/metrics: LUCENE-7572: Cache the hash code of doc values queries.

Posted by ab...@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/feature/metrics
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")));


[18/23] lucene-solr:feature/metrics: SOLR-9873: tests - fix SolrTestCaseJ4.compareSolrDocument num fields comparison

Posted by ab...@apache.org.
SOLR-9873: tests - fix SolrTestCaseJ4.compareSolrDocument num fields comparison


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

Branch: refs/heads/feature/metrics
Commit: dcf202a95813d72b1fd56daa7e30cbf413b891b9
Parents: 5f4521d
Author: yonik <yo...@apache.org>
Authored: Fri Dec 16 18:41:02 2016 -0500
Committer: yonik <yo...@apache.org>
Committed: Fri Dec 16 18:41:02 2016 -0500

----------------------------------------------------------------------
 solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/dcf202a9/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
index 19bf601..437679d 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java
@@ -1948,7 +1948,7 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
     SolrDocument solrDocument1 = (SolrDocument) expected;
     SolrDocument solrDocument2 = (SolrDocument) actual;
 
-    if(solrDocument1.getFieldNames().size() != solrDocument1.getFieldNames().size()) {
+    if(solrDocument1.getFieldNames().size() != solrDocument2.getFieldNames().size()) {
       return false;
     }
 


[02/23] lucene-solr:feature/metrics: LUCENE-7590: add DocValuesStatsCollector

Posted by ab...@apache.org.
LUCENE-7590: add DocValuesStatsCollector


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

Branch: refs/heads/feature/metrics
Commit: ad7152ad4739a47aa2b45405ba1682b3dda18923
Parents: 9aa5b73
Author: Shai Erera <sh...@apache.org>
Authored: Sun Dec 11 12:49:50 2016 +0200
Committer: Shai Erera <sh...@apache.org>
Committed: Wed Dec 14 13:04:52 2016 +0200

----------------------------------------------------------------------
 .../apache/lucene/search/DocValuesStats.java    | 165 ++++++++++++++++++
 .../lucene/search/DocValuesStatsCollector.java  |  64 +++++++
 .../search/TestDocValuesStatsCollector.java     | 166 +++++++++++++++++++
 3 files changed, 395 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ad7152ad/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java b/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
new file mode 100644
index 0000000..fad9f97
--- /dev/null
+++ b/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
@@ -0,0 +1,165 @@
+/*
+ * 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.io.IOException;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+
+/** Holds statistics for a DocValues field. */
+public abstract class DocValuesStats<T> {
+
+  private int missing = 0;
+  private int count = 0;
+
+  protected final String field;
+
+  protected T min;
+  protected T max;
+
+  protected DocValuesStats(String field, T initialMin, T initialMax) {
+    this.field = field;
+    this.min = initialMin;
+    this.max = initialMax;
+  }
+
+  /**
+   * Called after #{@link DocValuesStats#accumulate(int)} was processed and verified that the document has a value for
+   * the field. Implementations should update the statistics based on the value of the current document.
+   *
+   * @param count
+   *          the updated number of documents with value for this field.
+   */
+  protected abstract void doAccumulate(int count) throws IOException;
+
+  /**
+   * Initializes this object with the given reader context. Returns whether stats can be computed for this segment (i.e.
+   * it does have the requested DocValues field).
+   */
+  protected abstract boolean init(LeafReaderContext contxt) throws IOException;
+
+  /** Returns whether the given document has a value for the requested DocValues field. */
+  protected abstract boolean hasValue(int doc) throws IOException;
+
+  final void accumulate(int doc) throws IOException {
+    if (hasValue(doc)) {
+      ++count;
+      doAccumulate(count);
+    } else {
+      ++missing;
+    }
+  }
+
+  final void addMissing() {
+    ++missing;
+  }
+
+  /** The field for which these stats were computed. */
+  public final String field() {
+    return field;
+  }
+
+  /** The number of documents which have a value of the field. */
+  public final int count() {
+    return count;
+  }
+
+  /** The number of documents which do not have a value of the field. */
+  public final int missing() {
+    return missing;
+  }
+
+  /** The minimum value of the field. Undefined when {@link #count} is zero. */
+  public final T min() {
+    return min;
+  }
+
+  /** The maximum value of the field. Undefined when {@link #count} is zero. */
+  public final T max() {
+    return max;
+  }
+
+  /** Holds statistics for a numeric DocValues field. */
+  public static abstract class NumericDocValuesStats<T extends Number> extends DocValuesStats<T> {
+
+    protected double mean = 0.0;
+
+    protected NumericDocValues ndv;
+
+    protected NumericDocValuesStats(String field, T initialMin, T initialMax) {
+      super(field, initialMin, initialMax);
+    }
+
+    @Override
+    protected final boolean init(LeafReaderContext contxt) throws IOException {
+      ndv = contxt.reader().getNumericDocValues(field);
+      return ndv != null;
+    }
+
+    @Override
+    protected boolean hasValue(int doc) throws IOException {
+      return ndv.advanceExact(doc);
+    }
+
+    /** The mean of all values of the field. Undefined when {@link #count} is zero. */
+    public final double mean() {
+      return mean;
+    }
+  }
+
+  /** Holds DocValues statistics for a numeric field storing {@code long} values. */
+  public static final class LongDocValuesStats extends NumericDocValuesStats<Long> {
+
+    public LongDocValuesStats(String description) {
+      super(description, Long.MAX_VALUE, Long.MIN_VALUE);
+    }
+
+    @Override
+    protected void doAccumulate(int count) throws IOException {
+      long val = ndv.longValue();
+      if (val > max) {
+        max = val;
+      }
+      if (val < min) {
+        min = val;
+      }
+      mean += (val - mean) / count;
+    }
+  }
+
+  /** Holds DocValues statistics for a numeric field storing {@code double} values. */
+  public static final class DoubleDocValuesStats extends NumericDocValuesStats<Double> {
+
+    public DoubleDocValuesStats(String description) {
+      super(description, Double.MAX_VALUE, Double.MIN_VALUE);
+    }
+
+    @Override
+    protected void doAccumulate(int count) throws IOException {
+      double val = Double.longBitsToDouble(ndv.longValue());
+      if (Double.compare(val, max) > 0) {
+        max = val;
+      }
+      if (Double.compare(val, min) < 0) {
+        min = val;
+      }
+      mean += (val - mean) / count;
+    }
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ad7152ad/lucene/misc/src/java/org/apache/lucene/search/DocValuesStatsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/search/DocValuesStatsCollector.java b/lucene/misc/src/java/org/apache/lucene/search/DocValuesStatsCollector.java
new file mode 100644
index 0000000..2b1fa4f
--- /dev/null
+++ b/lucene/misc/src/java/org/apache/lucene/search/DocValuesStatsCollector.java
@@ -0,0 +1,64 @@
+/*
+ * 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.io.IOException;
+
+import org.apache.lucene.index.LeafReaderContext;
+
+/** A {@link Collector} which computes statistics for a DocValues field. */
+public class DocValuesStatsCollector implements Collector {
+
+  private final DocValuesStats<?> stats;
+
+  /** Creates a collector to compute statistics for a DocValues field using the given {@code stats}. */
+  public DocValuesStatsCollector(DocValuesStats<?> stats) {
+    this.stats = stats;
+  }
+
+  @Override
+  public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
+    boolean shouldProcess = stats.init(context);
+    if (!shouldProcess) {
+      // Stats cannot be computed for this segment, therefore consider all matching documents as a 'miss'. 
+      return new LeafCollector() {
+        @Override public void setScorer(Scorer scorer) throws IOException {}
+
+        @Override
+        public void collect(int doc) throws IOException {
+          // All matching documents in this reader are missing a value
+          stats.addMissing();
+        }
+      };
+    }
+
+    return new LeafCollector() {
+      @Override public void setScorer(Scorer scorer) throws IOException {}
+
+      @Override
+      public void collect(int doc) throws IOException {
+        stats.accumulate(doc);
+      }
+    };
+  }
+
+  @Override
+  public boolean needsScores() {
+    return false;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ad7152ad/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java b/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
new file mode 100644
index 0000000..65f82e6
--- /dev/null
+++ b/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
@@ -0,0 +1,166 @@
+/*
+ * 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.io.IOException;
+import java.util.Arrays;
+import java.util.stream.DoubleStream;
+import java.util.stream.LongStream;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.DoubleDocValuesField;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.DocValuesStats.DoubleDocValuesStats;
+import org.apache.lucene.search.DocValuesStats.LongDocValuesStats;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+/** Unit tests for {@link DocValuesStatsCollector}. */
+public class TestDocValuesStatsCollector extends LuceneTestCase {
+
+  public void testNoDocsWithField() throws IOException {
+    try (Directory dir = newDirectory();
+        IndexWriter indexWriter = new IndexWriter(dir, newIndexWriterConfig())) {
+      int numDocs = TestUtil.nextInt(random(), 1, 100);
+      for (int i = 0; i < numDocs; i++) {
+        indexWriter.addDocument(new Document());
+      }
+
+      try (DirectoryReader reader = DirectoryReader.open(indexWriter)) {
+        IndexSearcher searcher = new IndexSearcher(reader);
+        LongDocValuesStats stats = new LongDocValuesStats("foo");
+        searcher.search(new MatchAllDocsQuery(), new DocValuesStatsCollector(stats));
+
+        assertEquals(0, stats.count());
+        assertEquals(numDocs, stats.missing());
+      }
+    }
+  }
+
+  public void testRandomDocsWithLongValues() throws IOException {
+    try (Directory dir = newDirectory();
+        IndexWriter indexWriter = new IndexWriter(dir, newIndexWriterConfig())) {
+      String field = "numeric";
+      int numDocs = TestUtil.nextInt(random(), 1, 100);
+      long[] docValues = new long[numDocs];
+      int nextVal = 1;
+      for (int i = 0; i < numDocs; i++) {
+        Document doc = new Document();
+        if (random().nextBoolean()) { // not all documents have a value
+          doc.add(new NumericDocValuesField(field, nextVal));
+          doc.add(new StringField("id", "doc" + i, Store.NO));
+          docValues[i] = nextVal;
+          ++nextVal;
+        }
+        indexWriter.addDocument(doc);
+      }
+
+      // 20% of cases delete some docs
+      if (random().nextDouble() < 0.2) {
+        for (int i = 0; i < numDocs; i++) {
+          if (random().nextBoolean()) {
+            indexWriter.deleteDocuments(new Term("id", "doc" + i));
+            docValues[i] = 0;
+          }
+        }
+      }
+
+      try (DirectoryReader reader = DirectoryReader.open(indexWriter)) {
+        IndexSearcher searcher = new IndexSearcher(reader);
+        LongDocValuesStats stats = new LongDocValuesStats(field);
+        searcher.search(new MatchAllDocsQuery(), new DocValuesStatsCollector(stats));
+
+        int expCount = (int) Arrays.stream(docValues).filter(v -> v > 0).count();
+        assertEquals(expCount, stats.count());
+        assertEquals(getZeroValues(docValues).count() - reader.numDeletedDocs(), stats.missing());
+        if (stats.count() > 0) {
+          assertEquals(getPositiveValues(docValues).max().getAsLong(), stats.max().longValue());
+          assertEquals(getPositiveValues(docValues).min().getAsLong(), stats.min().longValue());
+          assertEquals(getPositiveValues(docValues).average().getAsDouble(), stats.mean(), 0.00001);
+        }
+      }
+    }
+  }
+
+  public void testRandomDocsWithDoubleValues() throws IOException {
+    try (Directory dir = newDirectory();
+        IndexWriter indexWriter = new IndexWriter(dir, newIndexWriterConfig())) {
+      String field = "numeric";
+      int numDocs = TestUtil.nextInt(random(), 1, 100);
+      double[] docValues = new double[numDocs];
+      double nextVal = 1.0;
+      for (int i = 0; i < numDocs; i++) {
+        Document doc = new Document();
+        if (random().nextBoolean()) { // not all documents have a value
+          doc.add(new DoubleDocValuesField(field, nextVal));
+          doc.add(new StringField("id", "doc" + i, Store.NO));
+          docValues[i] = nextVal;
+          ++nextVal;
+        }
+        indexWriter.addDocument(doc);
+      }
+
+      // 20% of cases delete some docs
+      if (random().nextDouble() < 0.2) {
+        for (int i = 0; i < numDocs; i++) {
+          if (random().nextBoolean()) {
+            indexWriter.deleteDocuments(new Term("id", "doc" + i));
+            docValues[i] = 0;
+          }
+        }
+      }
+
+      try (DirectoryReader reader = DirectoryReader.open(indexWriter)) {
+        IndexSearcher searcher = new IndexSearcher(reader);
+        DoubleDocValuesStats stats = new DoubleDocValuesStats(field);
+        searcher.search(new MatchAllDocsQuery(), new DocValuesStatsCollector(stats));
+
+        int expCount = (int) Arrays.stream(docValues).filter(v -> v > 0).count();
+        assertEquals(expCount, stats.count());
+        assertEquals(getZeroValues(docValues).count() - reader.numDeletedDocs(), stats.missing());
+        if (stats.count() > 0) {
+          assertEquals(getPositiveValues(docValues).max().getAsDouble(), stats.max().doubleValue(), 0.00001);
+          assertEquals(getPositiveValues(docValues).min().getAsDouble(), stats.min().doubleValue(), 0.00001);
+          assertEquals(getPositiveValues(docValues).average().getAsDouble(), stats.mean(), 0.00001);
+        }
+      }
+    }
+  }
+
+  private static LongStream getPositiveValues(long[] docValues) {
+    return Arrays.stream(docValues).filter(v -> v > 0);
+  }
+
+  private static DoubleStream getPositiveValues(double[] docValues) {
+    return Arrays.stream(docValues).filter(v -> v > 0);
+  }
+
+  private static LongStream getZeroValues(long[] docValues) {
+    return Arrays.stream(docValues).filter(v -> v == 0);
+  }
+
+  private static DoubleStream getZeroValues(double[] docValues) {
+    return Arrays.stream(docValues).filter(v -> v == 0);
+  }
+
+}


[11/23] lucene-solr:feature/metrics: LUCENE-7590: add sum, variance and stdev stats to NumericDVStats

Posted by ab...@apache.org.
LUCENE-7590: add sum, variance and stdev stats to NumericDVStats


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

Branch: refs/heads/feature/metrics
Commit: 295cab7216ca76debaf4d354409741058a8641a1
Parents: e4f31fa
Author: Shai Erera <sh...@apache.org>
Authored: Thu Dec 15 12:52:37 2016 +0200
Committer: Shai Erera <sh...@apache.org>
Committed: Thu Dec 15 14:42:08 2016 +0200

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  3 +
 .../apache/lucene/search/DocValuesStats.java    | 39 +++++++++++-
 .../search/TestDocValuesStatsCollector.java     | 62 +++++++++++++++++---
 3 files changed, 95 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/295cab72/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index f38c0d5..0e327d2 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -67,6 +67,9 @@ New features
 
 * LUCENE-7466: Added AxiomaticSimilarity. (Peilin Yang via Tommaso Teofili)
 
+* LUCENE-7590: Added DocValuesStatsCollector to compute statistics on DocValues
+  fields. (Shai Erera)
+
 Bug Fixes
 
 * LUCENE-7547: JapaneseTokenizerFactory was failing to close the

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/295cab72/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java b/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
index 998bef4..c8b7752 100644
--- a/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
+++ b/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
@@ -98,6 +98,7 @@ public abstract class DocValuesStats<T> {
   public static abstract class NumericDocValuesStats<T extends Number> extends DocValuesStats<T> {
 
     protected double mean = 0.0;
+    protected double variance = 0.0;
 
     protected NumericDocValues ndv;
 
@@ -116,15 +117,32 @@ public abstract class DocValuesStats<T> {
       return ndv.advanceExact(doc);
     }
 
-    /** The mean of all values of the field. Undefined when {@link #count} is zero. */
+    /** The mean of all values of the field. */
     public final double mean() {
       return mean;
     }
+
+    /** Returns the variance of all values of the field. */
+    public final double variance() {
+      int count = count();
+      return count > 0 ? variance / count : 0;
+    }
+
+    /** Returns the stdev of all values of the field. */
+    public final double stdev() {
+      return Math.sqrt(variance());
+    }
+
+    /** Returns the sum of values of the field. Note that if the values are large, the {@code sum} might overflow. */
+    public abstract T sum();
   }
 
   /** Holds DocValues statistics for a numeric field storing {@code long} values. */
   public static final class LongDocValuesStats extends NumericDocValuesStats<Long> {
 
+    // To avoid boxing 'long' to 'Long' while the sum is computed, declare it as private variable.
+    private long sum = 0;
+
     public LongDocValuesStats(String field) {
       super(field, Long.MAX_VALUE, Long.MIN_VALUE);
     }
@@ -138,13 +156,24 @@ public abstract class DocValuesStats<T> {
       if (val < min) {
         min = val;
       }
+      sum += val;
+      double oldMean = mean;
       mean += (val - mean) / count;
+      variance += (val - mean) * (val - oldMean);
+    }
+
+    @Override
+    public Long sum() {
+      return sum;
     }
   }
 
   /** Holds DocValues statistics for a numeric field storing {@code double} values. */
   public static final class DoubleDocValuesStats extends NumericDocValuesStats<Double> {
 
+    // To avoid boxing 'double' to 'Double' while the sum is computed, declare it as private variable.
+    private double sum = 0;
+
     public DoubleDocValuesStats(String field) {
       super(field, Double.MAX_VALUE, Double.MIN_VALUE);
     }
@@ -158,7 +187,15 @@ public abstract class DocValuesStats<T> {
       if (Double.compare(val, min) < 0) {
         min = val;
       }
+      sum += val;
+      double oldMean = mean;
       mean += (val - mean) / count;
+      variance += (val - mean) * (val - oldMean);
+    }
+
+    @Override
+    public Double sum() {
+      return sum;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/295cab72/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java b/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
index 65f82e6..8f8b09e 100644
--- a/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
+++ b/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
@@ -18,6 +18,8 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.DoubleSummaryStatistics;
+import java.util.LongSummaryStatistics;
 import java.util.stream.DoubleStream;
 import java.util.stream.LongStream;
 
@@ -57,7 +59,33 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
     }
   }
 
-  public void testRandomDocsWithLongValues() throws IOException {
+  public void testOneDoc() throws IOException {
+    try (Directory dir = newDirectory();
+        IndexWriter indexWriter = new IndexWriter(dir, newIndexWriterConfig())) {
+      String field = "numeric";
+      Document doc = new Document();
+      doc.add(new NumericDocValuesField(field, 1));
+      doc.add(new StringField("id", "doc1", Store.NO));
+      indexWriter.addDocument(doc);
+
+      try (DirectoryReader reader = DirectoryReader.open(indexWriter)) {
+        IndexSearcher searcher = new IndexSearcher(reader);
+        LongDocValuesStats stats = new LongDocValuesStats(field);
+        searcher.search(new MatchAllDocsQuery(), new DocValuesStatsCollector(stats));
+
+        assertEquals(1, stats.count());
+        assertEquals(0, stats.missing());
+        assertEquals(1, stats.max().longValue());
+        assertEquals(1, stats.min().longValue());
+        assertEquals(1, stats.sum().longValue());
+        assertEquals(1, stats.mean(), 0.0001);
+        assertEquals(0, stats.variance(), 0.0001);
+        assertEquals(0, stats.stdev(), 0.0001);
+      }
+    }
+  }
+
+  public void testDocsWithLongValues() throws IOException {
     try (Directory dir = newDirectory();
         IndexWriter indexWriter = new IndexWriter(dir, newIndexWriterConfig())) {
       String field = "numeric";
@@ -94,15 +122,20 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
         assertEquals(expCount, stats.count());
         assertEquals(getZeroValues(docValues).count() - reader.numDeletedDocs(), stats.missing());
         if (stats.count() > 0) {
-          assertEquals(getPositiveValues(docValues).max().getAsLong(), stats.max().longValue());
-          assertEquals(getPositiveValues(docValues).min().getAsLong(), stats.min().longValue());
-          assertEquals(getPositiveValues(docValues).average().getAsDouble(), stats.mean(), 0.00001);
+          LongSummaryStatistics sumStats = getPositiveValues(docValues).summaryStatistics();
+          assertEquals(sumStats.getMax(), stats.max().longValue());
+          assertEquals(sumStats.getMin(), stats.min().longValue());
+          assertEquals(sumStats.getAverage(), stats.mean(), 0.00001);
+          assertEquals(sumStats.getSum(), stats.sum().longValue());
+          double variance = computeVariance(docValues, stats.mean, stats.count());
+          assertEquals(variance, stats.variance(), 0.00001);
+          assertEquals(Math.sqrt(variance), stats.stdev(), 0.00001);
         }
       }
     }
   }
 
-  public void testRandomDocsWithDoubleValues() throws IOException {
+  public void testDocsWithDoubleValues() throws IOException {
     try (Directory dir = newDirectory();
         IndexWriter indexWriter = new IndexWriter(dir, newIndexWriterConfig())) {
       String field = "numeric";
@@ -139,9 +172,14 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
         assertEquals(expCount, stats.count());
         assertEquals(getZeroValues(docValues).count() - reader.numDeletedDocs(), stats.missing());
         if (stats.count() > 0) {
-          assertEquals(getPositiveValues(docValues).max().getAsDouble(), stats.max().doubleValue(), 0.00001);
-          assertEquals(getPositiveValues(docValues).min().getAsDouble(), stats.min().doubleValue(), 0.00001);
-          assertEquals(getPositiveValues(docValues).average().getAsDouble(), stats.mean(), 0.00001);
+          DoubleSummaryStatistics sumStats = getPositiveValues(docValues).summaryStatistics();
+          assertEquals(sumStats.getMax(), stats.max().doubleValue(), 0.00001);
+          assertEquals(sumStats.getMin(), stats.min().doubleValue(), 0.00001);
+          assertEquals(sumStats.getAverage(), stats.mean(), 0.00001);
+          assertEquals(sumStats.getSum(), stats.sum(), 0.00001);
+          double variance = computeVariance(docValues, stats.mean, stats.count());
+          assertEquals(variance, stats.variance(), 0.00001);
+          assertEquals(Math.sqrt(variance), stats.stdev(), 0.00001);
         }
       }
     }
@@ -163,4 +201,12 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
     return Arrays.stream(docValues).filter(v -> v == 0);
   }
 
+  private static double computeVariance(long[] values, double mean, int count) {
+    return getPositiveValues(values).mapToDouble(v -> (v - mean) * (v-mean)).sum() / count;
+  }
+
+  private static double computeVariance(double[] values, double mean, int count) {
+    return getPositiveValues(values).map(v -> (v - mean) * (v-mean)).sum() / count;
+  }
+
 }


[19/23] lucene-solr:feature/metrics: LUCENE-7590: add DocValuesStats for SortedNumeric DV fields

Posted by ab...@apache.org.
LUCENE-7590: add DocValuesStats for SortedNumeric DV fields


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

Branch: refs/heads/feature/metrics
Commit: 944b8e07f557b9320895998fe33d71cae5199eee
Parents: dcf202a
Author: Shai Erera <sh...@apache.org>
Authored: Sat Dec 17 21:17:14 2016 +0200
Committer: Shai Erera <sh...@apache.org>
Committed: Sun Dec 18 08:39:42 2016 +0200

----------------------------------------------------------------------
 .../apache/lucene/search/DocValuesStats.java    | 128 +++++++++++++++-
 .../search/TestDocValuesStatsCollector.java     | 153 ++++++++++++++++++-
 2 files changed, 271 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/944b8e07/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java b/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
index c8b7752..9dd97a6 100644
--- a/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
+++ b/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
@@ -20,6 +20,7 @@ import java.io.IOException;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.SortedNumericDocValues;
 
 /** Holds statistics for a DocValues field. */
 public abstract class DocValuesStats<T> {
@@ -95,7 +96,7 @@ public abstract class DocValuesStats<T> {
   }
 
   /** Holds statistics for a numeric DocValues field. */
-  public static abstract class NumericDocValuesStats<T extends Number> extends DocValuesStats<T> {
+  static abstract class NumericDocValuesStats<T extends Number> extends DocValuesStats<T> {
 
     protected double mean = 0.0;
     protected double variance = 0.0;
@@ -113,7 +114,7 @@ public abstract class DocValuesStats<T> {
     }
 
     @Override
-    protected boolean hasValue(int doc) throws IOException {
+    protected final boolean hasValue(int doc) throws IOException {
       return ndv.advanceExact(doc);
     }
 
@@ -199,4 +200,127 @@ public abstract class DocValuesStats<T> {
     }
   }
 
+  /** Holds statistics for a sorted-numeric DocValues field. */
+  static abstract class SortedNumericDocValuesStats<T extends Number> extends DocValuesStats<T> {
+
+    protected long valuesCount = 0;
+    protected double mean = 0.0;
+    protected double variance = 0.0;
+
+    protected SortedNumericDocValues sndv;
+
+    protected SortedNumericDocValuesStats(String field, T initialMin, T initialMax) {
+      super(field, initialMin, initialMax);
+    }
+
+    @Override
+    protected final boolean init(LeafReaderContext context) throws IOException {
+      sndv = context.reader().getSortedNumericDocValues(field);
+      return sndv != null;
+    }
+
+    @Override
+    protected final boolean hasValue(int doc) throws IOException {
+      return sndv.advanceExact(doc);
+    }
+
+    /** The mean of all values of the field. */
+    public final double mean() {
+      return mean;
+    }
+
+    /** Returns the variance of all values of the field. */
+    public final double variance() {
+      int count = count();
+      return count > 0 ? variance / count : 0;
+    }
+
+    /** Returns the stdev of all values of the field. */
+    public final double stdev() {
+      return Math.sqrt(variance());
+    }
+
+    /** Returns the total number of values for this field. */
+    public final long valuesCount() {
+      return valuesCount;
+    }
+
+    /** Returns the sum of values of the field. Note that if the values are large, the {@code sum} might overflow. */
+    public abstract T sum();
+  }
+
+  /** Holds DocValues statistics for a sorted-numeric field storing {@code long} values. */
+  public static final class SortedLongDocValuesStats extends SortedNumericDocValuesStats<Long> {
+
+    // To avoid boxing 'long' to 'Long' while the sum is computed, declare it as private variable.
+    private long sum = 0;
+
+    public SortedLongDocValuesStats(String field) {
+      super(field, Long.MAX_VALUE, Long.MIN_VALUE);
+    }
+
+    @Override
+    protected void doAccumulate(int count) throws IOException {
+      int numValues = sndv.docValueCount();
+      while (numValues-- > 0) {
+        long val = sndv.nextValue();
+        if (val > max) {
+          max = val;
+        }
+        if (val < min) {
+          min = val;
+        }
+        sum += val;
+        double oldMean = mean;
+        // for correct "running average computation", increase valuesCount with each value, rather than once before the
+        // loop stats.
+        ++valuesCount;
+        mean += (val - mean) / valuesCount;
+        variance += (val - mean) * (val - oldMean);
+      }
+    }
+
+    @Override
+    public Long sum() {
+      return sum;
+    }
+  }
+
+  /** Holds DocValues statistics for a sorted-numeric field storing {@code double} values. */
+  public static final class SortedDoubleDocValuesStats extends SortedNumericDocValuesStats<Double> {
+
+    // To avoid boxing 'double' to 'Double' while the sum is computed, declare it as private variable.
+    private double sum = 0;
+
+    public SortedDoubleDocValuesStats(String field) {
+      super(field, Double.MAX_VALUE, Double.MIN_VALUE);
+    }
+
+    @Override
+    protected void doAccumulate(int count) throws IOException {
+      int numValues = sndv.docValueCount();
+      while (numValues-- > 0) {
+        double val = Double.longBitsToDouble(sndv.nextValue());
+        if (Double.compare(val, max) > 0) {
+          max = val;
+        }
+        if (Double.compare(val, min) < 0) {
+          min = val;
+        }
+        sum += val;
+        double oldMean = mean;
+        // for correct "running average computation", increase valuesCount with each value, rather than once before the
+        // loop stats.
+        ++valuesCount;
+        mean += (val - mean) / valuesCount;
+        variance += (val - mean) * (val - oldMean);
+      }
+    }
+
+    @Override
+    public Double sum() {
+      return sum;
+    }
+  }
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/944b8e07/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java b/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
index 8f8b09e..5fa4b04 100644
--- a/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
+++ b/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
@@ -20,19 +20,24 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.DoubleSummaryStatistics;
 import java.util.LongSummaryStatistics;
+import java.util.function.Predicate;
 import java.util.stream.DoubleStream;
 import java.util.stream.LongStream;
+import java.util.stream.Stream;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.DoubleDocValuesField;
 import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.SortedNumericDocValuesField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.DocValuesStats.DoubleDocValuesStats;
 import org.apache.lucene.search.DocValuesStats.LongDocValuesStats;
+import org.apache.lucene.search.DocValuesStats.SortedDoubleDocValuesStats;
+import org.apache.lucene.search.DocValuesStats.SortedLongDocValuesStats;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
@@ -185,20 +190,136 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
     }
   }
 
-  private static LongStream getPositiveValues(long[] docValues) {
-    return Arrays.stream(docValues).filter(v -> v > 0);
+  public void testDocsWithMultipleLongValues() throws IOException {
+    try (Directory dir = newDirectory();
+        IndexWriter indexWriter = new IndexWriter(dir, newIndexWriterConfig())) {
+      String field = "numeric";
+      int numDocs = TestUtil.nextInt(random(), 1, 100);
+      long[][] docValues = new long[numDocs][];
+      long nextVal = 1;
+      for (int i = 0; i < numDocs; i++) {
+        Document doc = new Document();
+        if (random().nextBoolean()) { // not all documents have a value
+          int numValues = TestUtil.nextInt(random(), 1, 5);
+          docValues[i] = new long[numValues];
+          for (int j = 0; j < numValues; j++) {
+            doc.add(new SortedNumericDocValuesField(field, nextVal));
+            docValues[i][j] = nextVal;
+            ++nextVal;
+          }
+          doc.add(new StringField("id", "doc" + i, Store.NO));
+        }
+        indexWriter.addDocument(doc);
+      }
+
+      // 20% of cases delete some docs
+      if (random().nextDouble() < 0.2) {
+        for (int i = 0; i < numDocs; i++) {
+          if (random().nextBoolean()) {
+            indexWriter.deleteDocuments(new Term("id", "doc" + i));
+            docValues[i] = null;
+          }
+        }
+      }
+
+      try (DirectoryReader reader = DirectoryReader.open(indexWriter)) {
+        IndexSearcher searcher = new IndexSearcher(reader);
+        SortedLongDocValuesStats stats = new SortedLongDocValuesStats(field);
+        searcher.search(new MatchAllDocsQuery(), new DocValuesStatsCollector(stats));
+
+        assertEquals(filterValues(docValues, (v) -> v != null).count(), stats.count());
+        assertEquals(filterValues(docValues, (v) -> v == null).count() - reader.numDeletedDocs(), stats.missing());
+        if (stats.count() > 0) {
+          LongSummaryStatistics sumStats = filterAndFlatValues(docValues, (v) -> v != null).summaryStatistics();
+          assertEquals(sumStats.getMax(), stats.max().longValue());
+          assertEquals(sumStats.getMin(), stats.min().longValue());
+          assertEquals(sumStats.getAverage(), stats.mean(), 0.00001);
+          assertEquals(sumStats.getSum(), stats.sum().longValue());
+          assertEquals(sumStats.getCount(), stats.valuesCount());
+          double variance = computeVariance(filterAndFlatValues(docValues, (v) -> v != null), stats.mean, stats.count());
+          assertEquals(variance, stats.variance(), 0.00001);
+          assertEquals(Math.sqrt(variance), stats.stdev(), 0.00001);
+        }
+      }
+    }
   }
 
-  private static DoubleStream getPositiveValues(double[] docValues) {
-    return Arrays.stream(docValues).filter(v -> v > 0);
+  public void testDocsWithMultipleDoubleValues() throws IOException {
+    try (Directory dir = newDirectory();
+        IndexWriter indexWriter = new IndexWriter(dir, newIndexWriterConfig())) {
+      String field = "numeric";
+      int numDocs = TestUtil.nextInt(random(), 1, 100);
+      double[][] docValues = new double[numDocs][];
+      double nextVal = 1;
+      for (int i = 0; i < numDocs; i++) {
+        Document doc = new Document();
+        if (random().nextBoolean()) { // not all documents have a value
+          int numValues = TestUtil.nextInt(random(), 1, 5);
+          docValues[i] = new double[numValues];
+          for (int j = 0; j < numValues; j++) {
+            doc.add(new SortedNumericDocValuesField(field, Double.doubleToRawLongBits(nextVal)));
+            docValues[i][j] = nextVal;
+            ++nextVal;
+          }
+          doc.add(new StringField("id", "doc" + i, Store.NO));
+        }
+        indexWriter.addDocument(doc);
+      }
+
+      // 20% of cases delete some docs
+      if (random().nextDouble() < 0.2) {
+        for (int i = 0; i < numDocs; i++) {
+          if (random().nextBoolean()) {
+            indexWriter.deleteDocuments(new Term("id", "doc" + i));
+            docValues[i] = null;
+          }
+        }
+      }
+
+      try (DirectoryReader reader = DirectoryReader.open(indexWriter)) {
+        IndexSearcher searcher = new IndexSearcher(reader);
+        SortedDoubleDocValuesStats stats = new SortedDoubleDocValuesStats(field);
+        searcher.search(new MatchAllDocsQuery(), new DocValuesStatsCollector(stats));
+
+        assertEquals(filterValues(docValues, (v) -> v != null).count(), stats.count());
+        assertEquals(filterValues(docValues, (v) -> v == null).count() - reader.numDeletedDocs(), stats.missing());
+        if (stats.count() > 0) {
+          DoubleSummaryStatistics sumStats = filterAndFlatValues(docValues, (v) -> v != null).summaryStatistics();
+          assertEquals(sumStats.getMax(), stats.max().longValue(), 0.00001);
+          assertEquals(sumStats.getMin(), stats.min().longValue(), 0.00001);
+          assertEquals(sumStats.getAverage(), stats.mean(), 0.00001);
+          assertEquals(sumStats.getSum(), stats.sum().doubleValue(), 0.00001);
+          assertEquals(sumStats.getCount(), stats.valuesCount());
+          double variance = computeVariance(filterAndFlatValues(docValues, (v) -> v != null), stats.mean, stats.count());
+          assertEquals(variance, stats.variance(), 0.00001);
+          assertEquals(Math.sqrt(variance), stats.stdev(), 0.00001);
+        }
+      }
+    }
+  }
+
+  private static LongStream getPositiveValues(long[] values) {
+    return Arrays.stream(values).filter(v -> v > 0);
+  }
+
+  private static DoubleStream getPositiveValues(double[] values) {
+    return Arrays.stream(values).filter(v -> v > 0);
+  }
+
+  private static LongStream getZeroValues(long[] values) {
+    return Arrays.stream(values).filter(v -> v == 0);
   }
 
-  private static LongStream getZeroValues(long[] docValues) {
-    return Arrays.stream(docValues).filter(v -> v == 0);
+  private static DoubleStream getZeroValues(double[] values) {
+    return Arrays.stream(values).filter(v -> v == 0);
   }
 
-  private static DoubleStream getZeroValues(double[] docValues) {
-    return Arrays.stream(docValues).filter(v -> v == 0);
+  private static Stream<long[]> filterValues(long[][] values, Predicate<? super long[]> p) {
+    return Arrays.stream(values).filter(p);
+  }
+
+  private static Stream<double[]> filterValues(double[][] values, Predicate<? super double[]> p) {
+    return Arrays.stream(values).filter(p);
   }
 
   private static double computeVariance(long[] values, double mean, int count) {
@@ -209,4 +330,20 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
     return getPositiveValues(values).map(v -> (v - mean) * (v-mean)).sum() / count;
   }
 
+  private static LongStream filterAndFlatValues(long[][] values, Predicate<? super long[]> p) {
+    return filterValues(values, (v) -> v != null).flatMapToLong(Arrays::stream);
+  }
+
+  private static DoubleStream filterAndFlatValues(double[][] values, Predicate<? super double[]> p) {
+    return filterValues(values, (v) -> v != null).flatMapToDouble(Arrays::stream);
+  }
+
+  private static double computeVariance(LongStream values, double mean, int count) {
+    return values.mapToDouble(v -> (v - mean) * (v-mean)).sum() / count;
+  }
+
+  private static double computeVariance(DoubleStream values, double mean, int count) {
+    return values.map(v -> (v - mean) * (v-mean)).sum() / count;
+  }
+
 }


[06/23] lucene-solr:feature/metrics: SOLR-1953: It may be possible for temporary files to accumulate until the Solr process is shut down.

Posted by ab...@apache.org.
SOLR-1953: It may be possible for temporary files to accumulate until the Solr process is shut down.


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

Branch: refs/heads/feature/metrics
Commit: e82399d0677651ad4be1d8d2bdc4777b5d90b0fa
Parents: 22d9af4
Author: markrmiller <ma...@apache.org>
Authored: Mon Dec 12 11:10:58 2016 -0500
Committer: markrmiller <ma...@apache.org>
Committed: Wed Dec 14 11:16:50 2016 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../apache/solr/servlet/SolrDispatchFilter.java |  15 ++
 .../apache/solr/servlet/SolrRequestParsers.java |  31 ++--
 .../solr/util/SolrFileCleaningTracker.java      | 147 +++++++++++++++++++
 4 files changed, 182 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e82399d0/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 41af0ff..946a04e 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -244,6 +244,9 @@ Bug Fixes
 
 * SOLR-9823: CoreContainer incorrectly setting MDCLoggingContext for core (Jessica Cheng Mallet via Erick Erickson)
 
+* SOLR-1953: It may be possible for temporary files to accumulate until the Solr process is shut down.
+  (Karl Wright, Mark Miller)
+
 Other Changes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e82399d0/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
index 5a4cfb6..e8c4657 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
@@ -45,6 +45,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import org.apache.commons.io.FileCleaningTracker;
 import org.apache.commons.io.input.CloseShieldInputStream;
 import org.apache.commons.io.output.CloseShieldOutputStream;
 import org.apache.commons.lang.StringUtils;
@@ -62,6 +63,7 @@ import org.apache.solr.core.SolrXmlConfig;
 import org.apache.solr.request.SolrRequestInfo;
 import org.apache.solr.security.AuthenticationPlugin;
 import org.apache.solr.security.PKIAuthenticationPlugin;
+import org.apache.solr.util.SolrFileCleaningTracker;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -123,6 +125,8 @@ public class SolrDispatchFilter extends BaseSolrFilter {
   {
     log.trace("SolrDispatchFilter.init(): {}", this.getClass().getClassLoader());
 
+    SolrRequestParsers.fileCleaningTracker = new SolrFileCleaningTracker();
+    
     StartupLoggingUtils.checkLogDir();
     logWelcomeBanner();
     String muteConsole = System.getProperty(SOLR_LOG_MUTECONSOLE);
@@ -240,6 +244,17 @@ public class SolrDispatchFilter extends BaseSolrFilter {
   
   @Override
   public void destroy() {
+    try {
+      FileCleaningTracker fileCleaningTracker = SolrRequestParsers.fileCleaningTracker;
+      if (fileCleaningTracker != null) {
+        fileCleaningTracker.exitWhenFinished();
+      }
+    } catch (Exception e) {
+      log.warn("Exception closing FileCleaningTracker", e);
+    } finally {
+      SolrRequestParsers.fileCleaningTracker = null;
+    }
+
     if (cores != null) {
       try {
         cores.shutdown();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e82399d0/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java b/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java
index 9d7e7d9..968320e 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrRequestParsers.java
@@ -42,6 +42,7 @@ import java.util.Map;
 import org.apache.commons.fileupload.FileItem;
 import org.apache.commons.fileupload.disk.DiskFileItemFactory;
 import org.apache.commons.fileupload.servlet.ServletFileUpload;
+import org.apache.commons.io.FileCleaningTracker;
 import org.apache.commons.io.input.CloseShieldInputStream;
 import org.apache.lucene.util.IOUtils;
 import org.apache.solr.common.SolrException;
@@ -58,6 +59,7 @@ import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequestBase;
 import org.apache.solr.util.RTimerTree;
+import org.apache.solr.util.SolrFileCleaningTracker;
 
 import static org.apache.solr.common.params.CommonParams.PATH;
 
@@ -88,6 +90,8 @@ public class SolrRequestParsers
   /** Default instance for e.g. admin requests. Limits to 2 MB uploads and does not allow remote streams. */
   public static final SolrRequestParsers DEFAULT = new SolrRequestParsers();
   
+  public static volatile SolrFileCleaningTracker fileCleaningTracker;
+  
   /**
    * Pass in an xml configuration.  A null configuration will enable
    * everything with maximum values.
@@ -532,31 +536,30 @@ public class SolrRequestParsers
   /**
    * Extract Multipart streams
    */
-  static class MultipartRequestParser implements SolrRequestParser
-  {
+  static class MultipartRequestParser implements SolrRequestParser {
     private final int uploadLimitKB;
+    private DiskFileItemFactory factory = new DiskFileItemFactory();
     
-    public MultipartRequestParser( int limit )
-    {
+    public MultipartRequestParser(int limit) {
       uploadLimitKB = limit;
+
+      // Set factory constraints
+      FileCleaningTracker fct = fileCleaningTracker;
+      if (fct != null) {
+        factory.setFileCleaningTracker(fileCleaningTracker);
+      }
+      // TODO - configure factory.setSizeThreshold(yourMaxMemorySize);
+      // TODO - configure factory.setRepository(yourTempDirectory);
     }
     
     @Override
-    public SolrParams parseParamsAndFillStreams( 
-        final HttpServletRequest req, ArrayList<ContentStream> streams ) throws Exception
-    {
+    public SolrParams parseParamsAndFillStreams(
+        final HttpServletRequest req, ArrayList<ContentStream> streams) throws Exception {
       if( !ServletFileUpload.isMultipartContent(req) ) {
         throw new SolrException( ErrorCode.BAD_REQUEST, "Not multipart content! "+req.getContentType() );
       }
       
       MultiMapSolrParams params = parseQueryString( req.getQueryString() );
-      
-      // Create a factory for disk-based file items
-      DiskFileItemFactory factory = new DiskFileItemFactory();
-
-      // Set factory constraints
-      // TODO - configure factory.setSizeThreshold(yourMaxMemorySize);
-      // TODO - configure factory.setRepository(yourTempDirectory);
 
       // Create a new file upload handler
       ServletFileUpload upload = new ServletFileUpload(factory);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e82399d0/solr/core/src/java/org/apache/solr/util/SolrFileCleaningTracker.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/SolrFileCleaningTracker.java b/solr/core/src/java/org/apache/solr/util/SolrFileCleaningTracker.java
new file mode 100644
index 0000000..9c66f0f
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/util/SolrFileCleaningTracker.java
@@ -0,0 +1,147 @@
+/*
+ * 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.solr.util;
+
+import java.io.File;
+import java.lang.ref.PhantomReference;
+import java.lang.ref.ReferenceQueue;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+
+import org.apache.commons.io.FileCleaningTracker;
+import org.apache.commons.io.FileDeleteStrategy;
+
+public class SolrFileCleaningTracker extends FileCleaningTracker {
+
+  ReferenceQueue<Object> q = new ReferenceQueue<>();
+
+  final Collection<Tracker> trackers = Collections.synchronizedSet(new HashSet<Tracker>());
+
+  final List<String> deleteFailures = Collections.synchronizedList(new ArrayList<String>());
+
+  volatile boolean exitWhenFinished = false;
+
+  Thread reaper;
+
+  public void track(final File file, final Object marker) {
+    track(file, marker, null);
+  }
+
+  public void track(final File file, final Object marker, final FileDeleteStrategy deleteStrategy) {
+    if (file == null) {
+      throw new NullPointerException("The file must not be null");
+    }
+    addTracker(file.getPath(), marker, deleteStrategy);
+  }
+
+  public void track(final String path, final Object marker) {
+    track(path, marker, null);
+  }
+
+  public void track(final String path, final Object marker, final FileDeleteStrategy deleteStrategy) {
+    if (path == null) {
+      throw new NullPointerException("The path must not be null");
+    }
+    addTracker(path, marker, deleteStrategy);
+  }
+
+  private synchronized void addTracker(final String path, final Object marker,
+      final FileDeleteStrategy deleteStrategy) {
+    if (exitWhenFinished) {
+      throw new IllegalStateException("No new trackers can be added once exitWhenFinished() is called");
+    }
+    if (reaper == null) {
+      reaper = new Reaper();
+      reaper.start();
+    }
+    trackers.add(new Tracker(path, deleteStrategy, marker, q));
+  }
+
+  public int getTrackCount() {
+    return trackers.size();
+  }
+
+  public List<String> getDeleteFailures() {
+    return deleteFailures;
+  }
+
+  public synchronized void exitWhenFinished() {
+    // synchronized block protects reaper
+    exitWhenFinished = true;
+    if (reaper != null) {
+      synchronized (reaper) {
+        reaper.interrupt();
+        try {
+          reaper.join();
+        } catch (InterruptedException e) { 
+          Thread.currentThread().interrupt();
+        }
+      }
+    }
+  }
+
+  private final class Reaper extends Thread {
+    Reaper() {
+      super("MultiPart Upload Tmp File Reaper");
+      setDaemon(true);
+    }
+
+    @Override
+    public void run() {
+      while (exitWhenFinished == false || trackers.size() > 0) {
+        try {
+          // Wait for a tracker to remove.
+          final Tracker tracker = (Tracker) q.remove(); // cannot return null
+          trackers.remove(tracker);
+          if (!tracker.delete()) {
+            deleteFailures.add(tracker.getPath());
+          }
+          tracker.clear();
+        } catch (final InterruptedException e) {
+          Thread.currentThread().interrupt();
+          break;
+        }
+      }
+    }
+  }
+
+  private static final class Tracker extends PhantomReference<Object> {
+
+    private final String path;
+
+    private final FileDeleteStrategy deleteStrategy;
+
+    Tracker(final String path, final FileDeleteStrategy deleteStrategy, final Object marker,
+        final ReferenceQueue<? super Object> queue) {
+      super(marker, queue);
+      this.path = path;
+      this.deleteStrategy = deleteStrategy == null ? FileDeleteStrategy.NORMAL : deleteStrategy;
+    }
+
+    public String getPath() {
+      return path;
+    }
+
+    public boolean delete() {
+      return deleteStrategy.deleteQuietly(new File(path));
+    }
+  }
+
+}
\ No newline at end of file


[17/23] lucene-solr:feature/metrics: null the static fields after test finishes

Posted by ab...@apache.org.
null the static fields after test finishes


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

Branch: refs/heads/feature/metrics
Commit: 5f4521de628ced22804c3fd9bcd3de5fa815c27d
Parents: c9522a3
Author: Mike McCandless <mi...@apache.org>
Authored: Fri Dec 16 18:29:40 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Fri Dec 16 18:29:40 2016 -0500

----------------------------------------------------------------------
 .../facet/src/test/org/apache/lucene/facet/TestFacetQuery.java  | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5f4521de/lucene/facet/src/test/org/apache/lucene/facet/TestFacetQuery.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/TestFacetQuery.java b/lucene/facet/src/test/org/apache/lucene/facet/TestFacetQuery.java
index f3aa079..07edd6e 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/TestFacetQuery.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/TestFacetQuery.java
@@ -81,6 +81,11 @@ public class TestFacetQuery extends FacetTestCase {
   @AfterClass
   public static void closeTestIndex() throws IOException {
     IOUtils.close(indexReader, indexWriter, indexDirectory);
+    indexReader = null;
+    indexWriter = null;
+    indexDirectory = null;
+    searcher = null;
+    config = null;
   }
 
   @Test


[09/23] lucene-solr:feature/metrics: SOLR-8959: Refactored TestSegmentSorting out of TestMiniSolrCloudCluster

Posted by ab...@apache.org.
SOLR-8959: Refactored TestSegmentSorting out of TestMiniSolrCloudCluster


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

Branch: refs/heads/feature/metrics
Commit: 6525bb56f027655e5a01f028fa373305c0d01caa
Parents: 5123743
Author: Chris Hostetter <ho...@apache.org>
Authored: Wed Dec 14 13:18:56 2016 -0700
Committer: Chris Hostetter <ho...@apache.org>
Committed: Wed Dec 14 13:18:56 2016 -0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../cloud/SegmentTerminateEarlyTestState.java   |  12 +-
 .../solr/cloud/TestMiniSolrCloudCluster.java    |  50 -------
 .../apache/solr/cloud/TestSegmentSorting.java   | 133 +++++++++++++++++++
 4 files changed, 145 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6525bb56/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 5f0357b..73b0e9b 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -290,6 +290,9 @@ Other Changes
 
 * SOLR-9846: OverseerAutoReplicaFailoverThread can take too long to stop and leak out of unit tests. (Mark Miller)
 
+* SOLR-8959: Refactored TestSegmentSorting out of TestMiniSolrCloudCluster (hossman)
+
+
 ==================  6.3.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6525bb56/solr/core/src/test/org/apache/solr/cloud/SegmentTerminateEarlyTestState.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/SegmentTerminateEarlyTestState.java b/solr/core/src/test/org/apache/solr/cloud/SegmentTerminateEarlyTestState.java
index 199423b..b3df9e7 100644
--- a/solr/core/src/test/org/apache/solr/cloud/SegmentTerminateEarlyTestState.java
+++ b/solr/core/src/test/org/apache/solr/cloud/SegmentTerminateEarlyTestState.java
@@ -22,6 +22,7 @@ import java.time.ZonedDateTime;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
+import java.util.Random;
 
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
@@ -47,7 +48,12 @@ class SegmentTerminateEarlyTestState {
   Integer maxTimestampMM = null;
 
   int numDocs = 0;
+  final Random rand;
 
+  public SegmentTerminateEarlyTestState(Random rand) {
+    this.rand = rand;
+  }
+  
   void addDocuments(CloudSolrClient cloudSolrClient,
       int numCommits, int numDocsPerCommit, boolean optimize) throws Exception {
     for (int cc = 1; cc <= numCommits; ++cc) {
@@ -56,7 +62,7 @@ class SegmentTerminateEarlyTestState {
         final Integer docKey = new Integer(numDocs);
         SolrInputDocument doc = new SolrInputDocument();
         doc.setField(keyField, ""+docKey);
-        final int MM = TestMiniSolrCloudCluster.random().nextInt(60); // minutes
+        final int MM = rand.nextInt(60); // minutes
         if (minTimestampMM == null || MM <= minTimestampMM.intValue()) {
           if (minTimestampMM != null && MM < minTimestampMM.intValue()) {
             minTimestampDocKeys.clear();
@@ -116,7 +122,7 @@ class SegmentTerminateEarlyTestState {
     query.setFields(keyField, oddField, timestampField);
     final int rowsWanted = 1;
     query.setRows(rowsWanted);
-    final Boolean shardsInfoWanted = (TestMiniSolrCloudCluster.random().nextBoolean() ? null : new Boolean(TestMiniSolrCloudCluster.random().nextBoolean()));
+    final Boolean shardsInfoWanted = (rand.nextBoolean() ? null : new Boolean(rand.nextBoolean()));
     if (shardsInfoWanted != null) {
       query.set(ShardParams.SHARDS_INFO, shardsInfoWanted.booleanValue());
     }
@@ -163,7 +169,7 @@ class SegmentTerminateEarlyTestState {
     query.setSort(timestampField, SolrQuery.ORDER.desc);
     query.setFields(keyField, oddField, timestampField);
     query.setRows(1);
-    final Boolean shardsInfoWanted = (TestMiniSolrCloudCluster.random().nextBoolean() ? null : new Boolean(TestMiniSolrCloudCluster.random().nextBoolean()));
+    final Boolean shardsInfoWanted = (rand.nextBoolean() ? null : new Boolean(rand.nextBoolean()));
     if (shardsInfoWanted != null) {
       query.set(ShardParams.SHARDS_INFO, shardsInfoWanted.booleanValue());
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6525bb56/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudCluster.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudCluster.java b/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudCluster.java
index 97ecb67..de18875 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudCluster.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestMiniSolrCloudCluster.java
@@ -16,7 +16,6 @@
  */
 package org.apache.solr.cloud;
 
-import java.io.File;
 import java.lang.invoke.MethodHandles;
 import java.net.URL;
 import java.util.ArrayList;
@@ -384,53 +383,4 @@ public class TestMiniSolrCloudCluster extends LuceneTestCase {
     }
   }
 
-  @Test
-  public void testSegmentTerminateEarly() throws Exception {
-
-    final String collectionName = "testSegmentTerminateEarlyCollection";
-
-    final SegmentTerminateEarlyTestState tstes = new SegmentTerminateEarlyTestState();
-
-    File solrXml = new File(SolrTestCaseJ4.TEST_HOME(), "solr.xml");
-    Builder jettyConfig = JettyConfig.builder();
-    jettyConfig.waitForLoadingCoresToFinish(null);
-    final MiniSolrCloudCluster miniCluster = createMiniSolrCloudCluster();
-    final CloudSolrClient cloudSolrClient = miniCluster.getSolrClient();
-    cloudSolrClient.setDefaultCollection(collectionName);
-
-    try {
-      // create collection
-      {
-        final String asyncId = (random().nextBoolean() ? null : "asyncId("+collectionName+".create)="+random().nextInt());
-        final Map<String, String> collectionProperties = new HashMap<>();
-        collectionProperties.put(CoreDescriptor.CORE_CONFIG, "solrconfig-sortingmergepolicyfactory.xml");
-        createCollection(miniCluster, collectionName, null, asyncId, Boolean.TRUE, collectionProperties);
-      }
-
-      ZkStateReader zkStateReader = cloudSolrClient.getZkStateReader();
-      AbstractDistribZkTestBase.waitForRecoveriesToFinish(collectionName, zkStateReader, true, true, 330);
-
-      // add some documents, then optimize to get merged-sorted segments
-      tstes.addDocuments(cloudSolrClient, 10, 10, true);
-
-      // CommonParams.SEGMENT_TERMINATE_EARLY parameter intentionally absent
-      tstes.queryTimestampDescending(cloudSolrClient);
-
-      // add a few more documents, but don't optimize to have some not-merge-sorted segments
-      tstes.addDocuments(cloudSolrClient, 2, 10, false);
-
-      // CommonParams.SEGMENT_TERMINATE_EARLY parameter now present
-      tstes.queryTimestampDescendingSegmentTerminateEarlyYes(cloudSolrClient);
-      tstes.queryTimestampDescendingSegmentTerminateEarlyNo(cloudSolrClient);
-
-      // CommonParams.SEGMENT_TERMINATE_EARLY parameter present but it won't be used
-      tstes.queryTimestampDescendingSegmentTerminateEarlyYesGrouped(cloudSolrClient);
-      tstes.queryTimestampAscendingSegmentTerminateEarlyYes(cloudSolrClient); // uses a sort order that is _not_ compatible with the merge sort order
-
-    }
-    finally {
-      miniCluster.shutdown();
-    }
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6525bb56/solr/core/src/test/org/apache/solr/cloud/TestSegmentSorting.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestSegmentSorting.java b/solr/core/src/test/org/apache/solr/cloud/TestSegmentSorting.java
new file mode 100644
index 0000000..016b63e
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/TestSegmentSorting.java
@@ -0,0 +1,133 @@
+/*
+ * 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.solr.cloud;
+
+import java.lang.invoke.MethodHandles;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.index.TieredMergePolicy;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.core.CoreDescriptor;
+import org.apache.solr.index.TieredMergePolicyFactory;
+
+import org.junit.After;
+import org.junit.BeforeClass;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestSegmentSorting extends SolrCloudTestCase {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  private static final int NUM_SERVERS = 5;
+  private static final int NUM_SHARDS = 2;
+  private static final int REPLICATION_FACTOR = 2;
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(NUM_SERVERS).configure();
+  }
+
+  @After
+  public void ensureClusterEmpty() throws Exception {
+    cluster.deleteAllCollections();
+    cluster.getSolrClient().setDefaultCollection(null);
+  }
+  
+  private void createCollection(MiniSolrCloudCluster miniCluster, String collectionName, String createNodeSet, String asyncId,
+      Boolean indexToPersist, Map<String,String> collectionProperties) throws Exception {
+    String configName = "solrCloudCollectionConfig";
+    miniCluster.uploadConfigSet(SolrTestCaseJ4.TEST_PATH().resolve("collection1").resolve("conf"), configName);
+
+    final boolean persistIndex = (indexToPersist != null ? indexToPersist.booleanValue() : random().nextBoolean());
+    if (collectionProperties == null) {
+      collectionProperties = new HashMap<>();
+    }
+    collectionProperties.putIfAbsent(CoreDescriptor.CORE_CONFIG, "solrconfig-tlog.xml");
+    collectionProperties.putIfAbsent("solr.tests.maxBufferedDocs", "100000");
+    collectionProperties.putIfAbsent("solr.tests.ramBufferSizeMB", "100");
+    // use non-test classes so RandomizedRunner isn't necessary
+    if (random().nextBoolean()) {
+      collectionProperties.putIfAbsent(SolrTestCaseJ4.SYSTEM_PROPERTY_SOLR_TESTS_MERGEPOLICY, TieredMergePolicy.class.getName());
+      collectionProperties.putIfAbsent(SolrTestCaseJ4.SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICY, "true");
+      collectionProperties.putIfAbsent(SolrTestCaseJ4.SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICYFACTORY, "false");
+    } else {
+      collectionProperties.putIfAbsent(SolrTestCaseJ4.SYSTEM_PROPERTY_SOLR_TESTS_MERGEPOLICYFACTORY, TieredMergePolicyFactory.class.getName());
+      collectionProperties.putIfAbsent(SolrTestCaseJ4.SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICYFACTORY, "true");
+      collectionProperties.putIfAbsent(SolrTestCaseJ4.SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICY, "false");
+    }
+    collectionProperties.putIfAbsent("solr.tests.mergeScheduler", "org.apache.lucene.index.ConcurrentMergeScheduler");
+    collectionProperties.putIfAbsent("solr.directoryFactory", (persistIndex ? "solr.StandardDirectoryFactory" : "solr.RAMDirectoryFactory"));
+
+    if (asyncId == null) {
+      CollectionAdminRequest.createCollection(collectionName, configName, NUM_SHARDS, REPLICATION_FACTOR)
+          .setCreateNodeSet(createNodeSet)
+          .setProperties(collectionProperties)
+          .process(miniCluster.getSolrClient());
+    }
+    else {
+      CollectionAdminRequest.createCollection(collectionName, configName, NUM_SHARDS, REPLICATION_FACTOR)
+          .setCreateNodeSet(createNodeSet)
+          .setProperties(collectionProperties)
+          .processAndWait(miniCluster.getSolrClient(), 30);
+    }
+  }
+
+
+  public void testSegmentTerminateEarly() throws Exception {
+
+    final String collectionName = "testSegmentTerminateEarlyCollection";
+
+    final SegmentTerminateEarlyTestState tstes = new SegmentTerminateEarlyTestState(random());
+    
+    final CloudSolrClient cloudSolrClient = cluster.getSolrClient();
+    cloudSolrClient.setDefaultCollection(collectionName);
+
+    // create collection
+    {
+      final String asyncId = (random().nextBoolean() ? null : "asyncId("+collectionName+".create)="+random().nextInt());
+      final Map<String, String> collectionProperties = new HashMap<>();
+      collectionProperties.put(CoreDescriptor.CORE_CONFIG, "solrconfig-sortingmergepolicyfactory.xml");
+      createCollection(cluster, collectionName, null, asyncId, Boolean.TRUE, collectionProperties);
+    }
+    
+    ZkStateReader zkStateReader = cloudSolrClient.getZkStateReader();
+    AbstractDistribZkTestBase.waitForRecoveriesToFinish(collectionName, zkStateReader, true, true, 330);
+    
+    // add some documents, then optimize to get merged-sorted segments
+    tstes.addDocuments(cloudSolrClient, 10, 10, true);
+    
+    // CommonParams.SEGMENT_TERMINATE_EARLY parameter intentionally absent
+    tstes.queryTimestampDescending(cloudSolrClient);
+    
+    // add a few more documents, but don't optimize to have some not-merge-sorted segments
+    tstes.addDocuments(cloudSolrClient, 2, 10, false);
+    
+    // CommonParams.SEGMENT_TERMINATE_EARLY parameter now present
+    tstes.queryTimestampDescendingSegmentTerminateEarlyYes(cloudSolrClient);
+    tstes.queryTimestampDescendingSegmentTerminateEarlyNo(cloudSolrClient);
+    
+    // CommonParams.SEGMENT_TERMINATE_EARLY parameter present but it won't be used
+    tstes.queryTimestampDescendingSegmentTerminateEarlyYesGrouped(cloudSolrClient);
+    tstes.queryTimestampAscendingSegmentTerminateEarlyYes(cloudSolrClient); // uses a sort order that is _not_ compatible with the merge sort order
+    
+  }
+}


[12/23] lucene-solr:feature/metrics: remove bad assertion

Posted by ab...@apache.org.
remove bad assertion


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

Branch: refs/heads/feature/metrics
Commit: 268d4ace3695ad3738402d623400fa4775b113ef
Parents: 295cab7
Author: Mike McCandless <mi...@apache.org>
Authored: Thu Dec 15 09:23:48 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Thu Dec 15 09:30:25 2016 -0500

----------------------------------------------------------------------
 .../src/java/org/apache/lucene/search/QueryUtils.java            | 4 ----
 1 file changed, 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/268d4ace/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java b/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java
index a3eaa80..ae4c890 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java
@@ -93,10 +93,6 @@ public class QueryUtils {
   public static void checkUnequal(Query q1, Query q2) {
     assertFalse(q1 + " equal to " + q2, q1.equals(q2));
     assertFalse(q2 + " equal to " + q1, q2.equals(q1));
-
-    // possible this test can fail on a hash collision... if that
-    // happens, please change test to use a different example.
-    assertTrue(q1.hashCode() != q2.hashCode());
   }
 
   /** deep check that explanations of a query 'score' correctly */


[23/23] lucene-solr:feature/metrics: Merge branch 'master' into feature/metrics

Posted by ab...@apache.org.
Merge branch 'master' into feature/metrics


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

Branch: refs/heads/feature/metrics
Commit: 67dc7604ef7340cf94ad286f3693977aec12f26f
Parents: f56da1d 321c6f0
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Mon Dec 19 13:51:25 2016 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Mon Dec 19 13:51:25 2016 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  17 +
 .../lucene70/Lucene70DocValuesConsumer.java     | 163 +++++--
 .../lucene70/Lucene70DocValuesFormat.java       |   5 +-
 .../lucene70/Lucene70DocValuesProducer.java     | 220 +++++++--
 .../org/apache/lucene/index/LeafReader.java     |   2 +-
 .../apache/lucene/index/PrefixCodedTerms.java   |   4 +-
 .../org/apache/lucene/index/SegmentInfos.java   |   7 +-
 .../java/org/apache/lucene/util/LongValues.java |   9 +
 .../apache/lucene/util/packed/DirectWriter.java |   8 +-
 .../lucene70/TestLucene70DocValuesFormat.java   | 152 +++++++
 .../org/apache/lucene/facet/FacetQuery.java     |  52 +++
 .../apache/lucene/facet/MultiFacetQuery.java    |  61 +++
 .../org/apache/lucene/facet/TestFacetQuery.java | 103 +++++
 .../apache/lucene/search/DocValuesStats.java    | 409 +++++++++++++++++
 .../lucene/search/DocValuesStatsCollector.java  |  64 +++
 .../search/TestDocValuesStatsCollector.java     | 448 +++++++++++++++++++
 .../apache/lucene/document/RangeFieldQuery.java |   1 +
 .../lucene/search/DocValuesNumbersQuery.java    |  26 +-
 .../lucene/search/DocValuesTermsQuery.java      |  49 +-
 .../org/apache/lucene/search/LongHashSet.java   | 156 +++++++
 .../search/BaseRangeFieldQueryTestCase.java     |   2 +-
 .../apache/lucene/search/LongHashSetTests.java  | 100 +++++
 .../lucene/search/TestDocValuesTermsQuery.java  |   1 +
 .../org/apache/lucene/search/QueryUtils.java    |   4 -
 solr/CHANGES.txt                                |  14 +
 .../solr/collection1/conf/solrconfig.xml        |   5 -
 solr/contrib/ltr/example/solrconfig.xml         |  11 -
 .../solr/solrcloud/conf/solrconfig.xml          |  11 -
 .../OverseerAutoReplicaFailoverThread.java      |   9 +-
 .../java/org/apache/solr/core/SolrConfig.java   |   2 +-
 .../src/java/org/apache/solr/core/SolrCore.java |  92 ++--
 .../apache/solr/search/SolrFieldCacheMBean.java |   6 +-
 .../apache/solr/servlet/SolrDispatchFilter.java |  15 +
 .../apache/solr/servlet/SolrRequestParsers.java |  31 +-
 .../solr/uninverting/UninvertingReader.java     |  25 +-
 .../solr/util/SolrFileCleaningTracker.java      | 147 ++++++
 .../solr/collection1/conf/solrconfig_perf.xml   |   1 -
 .../cloud/SegmentTerminateEarlyTestState.java   |  12 +-
 .../solr/cloud/TestMiniSolrCloudCluster.java    |  50 ---
 .../apache/solr/cloud/TestSegmentSorting.java   | 133 ++++++
 .../org/apache/solr/search/TestRealTimeGet.java |   2 +-
 .../solr/search/TestSolrFieldCacheMBean.java    |   3 +-
 .../apache/solr/search/TestStressLucene.java    |   2 +-
 .../apache/solr/search/TestStressRecovery.java  |   2 +-
 .../apache/solr/search/TestStressReorder.java   |   5 +-
 .../solr/search/TestStressUserVersions.java     |   4 +-
 .../apache/solr/search/TestStressVersions.java  |   2 +-
 .../example-DIH/solr/db/conf/solrconfig.xml     |  11 -
 .../example-DIH/solr/mail/conf/solrconfig.xml   |  11 -
 .../example-DIH/solr/rss/conf/solrconfig.xml    |  11 -
 .../example-DIH/solr/solr/conf/solrconfig.xml   |  11 -
 .../example-DIH/solr/tika/conf/solrconfig.xml   |  11 -
 solr/example/files/conf/solrconfig.xml          |  11 -
 .../basic_configs/conf/solrconfig.xml           |  11 -
 .../conf/solrconfig.xml                         |  11 -
 .../conf/solrconfig.xml                         |  11 -
 .../solr/client/solrj/LargeVolumeTestBase.java  |   3 +-
 .../solr/client/solrj/SolrExampleTestBase.java  |   1 -
 .../java/org/apache/solr/SolrJettyTestBase.java |   2 -
 .../java/org/apache/solr/SolrTestCaseJ4.java    |   2 +-
 60 files changed, 2371 insertions(+), 383 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/67dc7604/solr/CHANGES.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/67dc7604/solr/core/src/java/org/apache/solr/core/SolrCore.java
----------------------------------------------------------------------
diff --cc solr/core/src/java/org/apache/solr/core/SolrCore.java
index cad6423,b3facef..4a63686
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@@ -206,12 -200,6 +206,12 @@@ public final class SolrCore implements 
    private final ReentrantLock ruleExpiryLock;
    private final ReentrantLock snapshotDelLock; // A lock instance to guard against concurrent deletions.
  
 +  private final Timer newSearcherTimer;
 +  private final Timer newSearcherWarmupTimer;
 +  private final Counter newSearcherCounter;
-   private final Counter newSearcherMaxErrorsCounter;
++  private final Counter newSearcherMaxReachedCounter;
 +  private final Counter newSearcherOtherErrorsCounter;
 +
    public Date getStartTimeStamp() { return startTime; }
  
    private final Map<Object, IndexFingerprint> perSegmentFingerprintCache = new MapMaker().weakKeys().makeMap();
@@@ -863,16 -838,6 +863,16 @@@
  
      checkVersionFieldExistsInSchema(schema, coreDescriptor);
  
 +    // Initialize the metrics manager
 +    this.metricManager = initMetricManager(config);
 +
 +    // initialize searcher-related metrics
 +    newSearcherCounter = SolrMetricManager.counter(metricManager.getRegistryName(), "newSearcher");
 +    newSearcherTimer = SolrMetricManager.timer(metricManager.getRegistryName(), "newSearcherTime");
 +    newSearcherWarmupTimer = SolrMetricManager.timer(metricManager.getRegistryName(), "newSearcherWarmup");
-     newSearcherMaxErrorsCounter = SolrMetricManager.counter(metricManager.getRegistryName(), "newSearcherMaxErrors");
++    newSearcherMaxReachedCounter = SolrMetricManager.counter(metricManager.getRegistryName(), "newSearcherMaxReached");
 +    newSearcherOtherErrorsCounter = SolrMetricManager.counter(metricManager.getRegistryName(), "newSearcherErrors");
 +
      // Initialize JMX
      this.infoRegistry = initInfoRegistry(name, config);
      infoRegistry.put("fieldCache", new SolrFieldCacheMBean());
@@@ -1943,53 -1886,57 +1943,59 @@@
      // if it isn't necessary.
  
      synchronized (searcherLock) {
-       // see if we can return the current searcher
-       if (_searcher!=null && !forceNew) {
-         if (returnSearcher) {
-           _searcher.incref();
-           return _searcher;
-         } else {
-           return null;
+       for(;;) { // this loop is so w can retry in the event that we exceed maxWarmingSearchers
+         // see if we can return the current searcher
+         if (_searcher != null && !forceNew) {
+           if (returnSearcher) {
+             _searcher.incref();
+             return _searcher;
+           } else {
+             return null;
+           }
          }
-       }
  
-       // check to see if we can wait for someone else's searcher to be set
-       if (onDeckSearchers>0 && !forceNew && _searcher==null) {
-         try {
-           searcherLock.wait();
-         } catch (InterruptedException e) {
-           log.info(SolrException.toStr(e));
+         // check to see if we can wait for someone else's searcher to be set
+         if (onDeckSearchers > 0 && !forceNew && _searcher == null) {
+           try {
+             searcherLock.wait();
+           } catch (InterruptedException e) {
+             log.info(SolrException.toStr(e));
+           }
          }
-       }
  
-       // check again: see if we can return right now
-       if (_searcher!=null && !forceNew) {
-         if (returnSearcher) {
-           _searcher.incref();
-           return _searcher;
-         } else {
-           return null;
+         // check again: see if we can return right now
+         if (_searcher != null && !forceNew) {
+           if (returnSearcher) {
+             _searcher.incref();
+             return _searcher;
+           } else {
+             return null;
+           }
+         }
+ 
+         // At this point, we know we need to open a new searcher...
+         // first: increment count to signal other threads that we are
+         //        opening a new searcher.
+         onDeckSearchers++;
++        newSearcherCounter.inc();
+         if (onDeckSearchers < 1) {
+           // should never happen... just a sanity check
+           log.error(logid + "ERROR!!! onDeckSearchers is " + onDeckSearchers);
+           onDeckSearchers = 1;  // reset
+         } else if (onDeckSearchers > maxWarmingSearchers) {
+           onDeckSearchers--;
++          newSearcherMaxReachedCounter.inc();
+           try {
+             searcherLock.wait();
+           } catch (InterruptedException e) {
+             log.info(SolrException.toStr(e));
+           }
+           continue;  // go back to the top of the loop and retry
+         } else if (onDeckSearchers > 1) {
+           log.warn(logid + "PERFORMANCE WARNING: Overlapping onDeckSearchers=" + onDeckSearchers);
          }
-       }
  
-       // At this point, we know we need to open a new searcher...
-       // first: increment count to signal other threads that we are
-       //        opening a new searcher.
-       onDeckSearchers++;
-       newSearcherCounter.inc();
-       if (onDeckSearchers < 1) {
-         // should never happen... just a sanity check
-         log.error(logid+"ERROR!!! onDeckSearchers is " + onDeckSearchers);
-         onDeckSearchers=1;  // reset
-       } else if (onDeckSearchers > maxWarmingSearchers) {
-         onDeckSearchers--;
-         String msg="Error opening new searcher. exceeded limit of maxWarmingSearchers="+maxWarmingSearchers + ", try again later.";
-         log.warn(logid+""+ msg);
-         // HTTP 503==service unavailable, or 409==Conflict
-         newSearcherMaxErrorsCounter.inc();
-         throw new SolrException(SolrException.ErrorCode.SERVICE_UNAVAILABLE,msg);
-       } else if (onDeckSearchers > 1) {
-         log.warn(logid+"PERFORMANCE WARNING: Overlapping onDeckSearchers=" + onDeckSearchers);
+         break; // I can now exit the loop and proceed to open a searcher
        }
      }
  

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/67dc7604/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
----------------------------------------------------------------------
diff --cc solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
index cfd90cb,e8c4657..2852041
--- a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
@@@ -47,12 -45,7 +47,13 @@@ import java.util.concurrent.atomic.Atom
  import java.util.regex.Matcher;
  import java.util.regex.Pattern;
  
 +import com.codahale.metrics.jvm.BufferPoolMetricSet;
 +import com.codahale.metrics.jvm.ClassLoadingGaugeSet;
 +import com.codahale.metrics.jvm.FileDescriptorRatioGauge;
 +import com.codahale.metrics.jvm.GarbageCollectorMetricSet;
 +import com.codahale.metrics.jvm.MemoryUsageGaugeSet;
 +import com.codahale.metrics.jvm.ThreadStatesGaugeSet;
+ import org.apache.commons.io.FileCleaningTracker;
  import org.apache.commons.io.input.CloseShieldInputStream;
  import org.apache.commons.io.output.CloseShieldOutputStream;
  import org.apache.commons.lang.StringUtils;


[05/23] lucene-solr:feature/metrics: Rename constructor parameter name

Posted by ab...@apache.org.
Rename constructor parameter name


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

Branch: refs/heads/feature/metrics
Commit: 22d9af41a435feaa3307880b7c7ed4f5860faa21
Parents: 85582da
Author: Shai Erera <sh...@apache.org>
Authored: Wed Dec 14 13:49:42 2016 +0200
Committer: Shai Erera <sh...@apache.org>
Committed: Wed Dec 14 13:49:42 2016 +0200

----------------------------------------------------------------------
 .../src/java/org/apache/lucene/search/DocValuesStats.java    | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/22d9af41/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java b/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
index 38158cf..998bef4 100644
--- a/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
+++ b/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
@@ -125,8 +125,8 @@ public abstract class DocValuesStats<T> {
   /** Holds DocValues statistics for a numeric field storing {@code long} values. */
   public static final class LongDocValuesStats extends NumericDocValuesStats<Long> {
 
-    public LongDocValuesStats(String description) {
-      super(description, Long.MAX_VALUE, Long.MIN_VALUE);
+    public LongDocValuesStats(String field) {
+      super(field, Long.MAX_VALUE, Long.MIN_VALUE);
     }
 
     @Override
@@ -145,8 +145,8 @@ public abstract class DocValuesStats<T> {
   /** Holds DocValues statistics for a numeric field storing {@code double} values. */
   public static final class DoubleDocValuesStats extends NumericDocValuesStats<Double> {
 
-    public DoubleDocValuesStats(String description) {
-      super(description, Double.MAX_VALUE, Double.MIN_VALUE);
+    public DoubleDocValuesStats(String field) {
+      super(field, Double.MAX_VALUE, Double.MIN_VALUE);
     }
 
     @Override


[10/23] lucene-solr:feature/metrics: LUCENE-7592: if segments file is truncated, throw CorruptIndexException

Posted by ab...@apache.org.
LUCENE-7592: if segments file is truncated, throw CorruptIndexException


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

Branch: refs/heads/feature/metrics
Commit: e4f31fab2f98b7af6d2ec12a2eb3456521b446df
Parents: 6525bb5
Author: Mike McCandless <mi...@apache.org>
Authored: Wed Dec 14 18:00:51 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Wed Dec 14 18:00:51 2016 -0500

----------------------------------------------------------------------
 lucene/CHANGES.txt                                            | 4 ++++
 .../core/src/java/org/apache/lucene/index/SegmentInfos.java   | 7 ++++++-
 2 files changed, 10 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e4f31fab/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 15b89f0..f38c0d5 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -138,6 +138,10 @@ Improvements
   necessarily refer to that field (AKA requireFieldMatch==false). Disabled by default.
   See UH get/setFieldMatcher. (Jim Ferenczi via David Smiley)
 
+* LUCENE-7592: If the segments file is truncated, we now throw
+  CorruptIndexException instead of the more confusing EOFException
+  (Mike Drob via Mike McCandless)
+
 Optimizations
 
 * LUCENE-7568: Optimize merging when index sorting is used but the

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e4f31fab/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
index 8f627cd..3e8b1f8 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
@@ -17,6 +17,7 @@
 package org.apache.lucene.index;
 
 
+import java.io.EOFException;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.util.ArrayList;
@@ -277,7 +278,11 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
     long generation = generationFromSegmentsFileName(segmentFileName);
     //System.out.println(Thread.currentThread() + ": SegmentInfos.readCommit " + segmentFileName);
     try (ChecksumIndexInput input = directory.openChecksumInput(segmentFileName, IOContext.READ)) {
-      return readCommit(directory, input, generation);
+      try {
+        return readCommit(directory, input, generation);
+      } catch (EOFException e) {
+        throw new CorruptIndexException("Unexpected end of file while reading index.", input, e);
+      }
     }
   }
 


[16/23] lucene-solr:feature/metrics: SOLR-9712: block when maxWarmingSearchers is exceeded instead of throwing exception, default to 1, remove from most configs

Posted by ab...@apache.org.
SOLR-9712: block when maxWarmingSearchers is exceeded instead of throwing exception, default to 1, remove from most configs


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

Branch: refs/heads/feature/metrics
Commit: c9522a393661c8878d488ad4475ac7e2cbb9c25c
Parents: 835296f
Author: yonik <yo...@apache.org>
Authored: Fri Dec 16 11:46:48 2016 -0500
Committer: yonik <yo...@apache.org>
Committed: Fri Dec 16 11:46:48 2016 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  6 ++
 .../solr/collection1/conf/solrconfig.xml        |  5 --
 solr/contrib/ltr/example/solrconfig.xml         | 11 ---
 .../solr/solrcloud/conf/solrconfig.xml          | 11 ---
 .../java/org/apache/solr/core/SolrConfig.java   |  2 +-
 .../src/java/org/apache/solr/core/SolrCore.java | 84 +++++++++++---------
 .../solr/collection1/conf/solrconfig_perf.xml   |  1 -
 .../org/apache/solr/search/TestRealTimeGet.java |  2 +-
 .../apache/solr/search/TestStressLucene.java    |  2 +-
 .../apache/solr/search/TestStressRecovery.java  |  2 +-
 .../apache/solr/search/TestStressReorder.java   |  5 +-
 .../solr/search/TestStressUserVersions.java     |  4 +-
 .../apache/solr/search/TestStressVersions.java  |  2 +-
 .../example-DIH/solr/db/conf/solrconfig.xml     | 11 ---
 .../example-DIH/solr/mail/conf/solrconfig.xml   | 11 ---
 .../example-DIH/solr/rss/conf/solrconfig.xml    | 11 ---
 .../example-DIH/solr/solr/conf/solrconfig.xml   | 11 ---
 .../example-DIH/solr/tika/conf/solrconfig.xml   | 11 ---
 solr/example/files/conf/solrconfig.xml          | 11 ---
 .../basic_configs/conf/solrconfig.xml           | 11 ---
 .../conf/solrconfig.xml                         | 11 ---
 .../conf/solrconfig.xml                         | 11 ---
 .../solr/client/solrj/LargeVolumeTestBase.java  |  3 +-
 .../solr/client/solrj/SolrExampleTestBase.java  |  1 -
 .../java/org/apache/solr/SolrJettyTestBase.java |  2 -
 25 files changed, 62 insertions(+), 180 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 73b0e9b..c920575 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -114,6 +114,12 @@ Upgrade Notes
   for a listing of highlight parameters annotated with which highlighters use them.
   hl.useFastVectorHighlighter is now considered deprecated in lieu of hl.method=fastVector.
 
+* SOLR-9712: maxWarmingSearchers now defaults to 1, and more importantly commits will now block if this
+  limit is exceeded instead of throwing an exception (a good thing). Consequently there is no longer a
+  risk in overlapping commits.  Nonetheless users should continue to avoid excessive committing. Users are
+  advised to remove any pre-existing maxWarmingSearchers entries from their solrconfig.xml files.
+
+
 New Features
 ----------------------
 * SOLR-9293: Solrj client support for hierarchical clusters and other topics 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/contrib/clustering/src/test-files/clustering/solr/collection1/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/contrib/clustering/src/test-files/clustering/solr/collection1/conf/solrconfig.xml b/solr/contrib/clustering/src/test-files/clustering/solr/collection1/conf/solrconfig.xml
index 5e0e66c..6482286 100644
--- a/solr/contrib/clustering/src/test-files/clustering/solr/collection1/conf/solrconfig.xml
+++ b/solr/contrib/clustering/src/test-files/clustering/solr/collection1/conf/solrconfig.xml
@@ -212,11 +212,6 @@
          warming. -->
     <useColdSearcher>false</useColdSearcher>
 
-    <!-- Maximum number of searchers that may be warming in the background
-      concurrently.  An error is returned if this limit is exceeded. Recommend
-      1-2 for read-only slaves, higher for masters w/o cache warming. -->
-    <maxWarmingSearchers>2</maxWarmingSearchers>
-
   </query>
 
   <!-- 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/contrib/ltr/example/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/contrib/ltr/example/solrconfig.xml b/solr/contrib/ltr/example/solrconfig.xml
index 55da6a4..1c66c49 100644
--- a/solr/contrib/ltr/example/solrconfig.xml
+++ b/solr/contrib/ltr/example/solrconfig.xml
@@ -609,17 +609,6 @@
       -->
     <useColdSearcher>false</useColdSearcher>
 
-    <!-- Max Warming Searchers
-
-         Maximum number of searchers that may be warming in the
-         background concurrently.  An error is returned if this limit
-         is exceeded.
-
-         Recommend values of 1-2 for read-only slaves, higher for
-         masters w/o cache warming.
-      -->
-    <maxWarmingSearchers>2</maxWarmingSearchers>
-
   </query>
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/contrib/morphlines-core/src/test-files/solr/solrcloud/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcloud/conf/solrconfig.xml b/solr/contrib/morphlines-core/src/test-files/solr/solrcloud/conf/solrconfig.xml
index 1eda53e..42ab056 100644
--- a/solr/contrib/morphlines-core/src/test-files/solr/solrcloud/conf/solrconfig.xml
+++ b/solr/contrib/morphlines-core/src/test-files/solr/solrcloud/conf/solrconfig.xml
@@ -606,17 +606,6 @@
       -->
     <useColdSearcher>false</useColdSearcher>
 
-    <!-- Max Warming Searchers
-         
-         Maximum number of searchers that may be warming in the
-         background concurrently.  An error is returned if this limit
-         is exceeded.
-
-         Recommend values of 1-2 for read-only slaves, higher for
-         masters w/o cache warming.
-      -->
-    <maxWarmingSearchers>4</maxWarmingSearchers>
-
   </query>
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/core/src/java/org/apache/solr/core/SolrConfig.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrConfig.java b/solr/core/src/java/org/apache/solr/core/SolrConfig.java
index e5e25a1..bd98075 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrConfig.java
@@ -287,7 +287,7 @@ public class SolrConfig extends Config implements MapSerializable {
     } else {
       jmxConfig = new JmxConfiguration(false, null, null, null);
     }
-    maxWarmingSearchers = getInt("query/maxWarmingSearchers", Integer.MAX_VALUE);
+    maxWarmingSearchers = getInt("query/maxWarmingSearchers", 1);
     slowQueryThresholdMillis = getInt("query/slowQueryThresholdMillis", -1);
     for (SolrPluginInfo plugin : plugins) loadPluginInfo(plugin);
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/core/src/java/org/apache/solr/core/SolrCore.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrCore.java b/solr/core/src/java/org/apache/solr/core/SolrCore.java
index a459bf2..b3facef 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -1886,51 +1886,57 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
     // if it isn't necessary.
 
     synchronized (searcherLock) {
-      // see if we can return the current searcher
-      if (_searcher!=null && !forceNew) {
-        if (returnSearcher) {
-          _searcher.incref();
-          return _searcher;
-        } else {
-          return null;
+      for(;;) { // this loop is so w can retry in the event that we exceed maxWarmingSearchers
+        // see if we can return the current searcher
+        if (_searcher != null && !forceNew) {
+          if (returnSearcher) {
+            _searcher.incref();
+            return _searcher;
+          } else {
+            return null;
+          }
         }
-      }
 
-      // check to see if we can wait for someone else's searcher to be set
-      if (onDeckSearchers>0 && !forceNew && _searcher==null) {
-        try {
-          searcherLock.wait();
-        } catch (InterruptedException e) {
-          log.info(SolrException.toStr(e));
+        // check to see if we can wait for someone else's searcher to be set
+        if (onDeckSearchers > 0 && !forceNew && _searcher == null) {
+          try {
+            searcherLock.wait();
+          } catch (InterruptedException e) {
+            log.info(SolrException.toStr(e));
+          }
         }
-      }
 
-      // check again: see if we can return right now
-      if (_searcher!=null && !forceNew) {
-        if (returnSearcher) {
-          _searcher.incref();
-          return _searcher;
-        } else {
-          return null;
+        // check again: see if we can return right now
+        if (_searcher != null && !forceNew) {
+          if (returnSearcher) {
+            _searcher.incref();
+            return _searcher;
+          } else {
+            return null;
+          }
         }
-      }
 
-      // At this point, we know we need to open a new searcher...
-      // first: increment count to signal other threads that we are
-      //        opening a new searcher.
-      onDeckSearchers++;
-      if (onDeckSearchers < 1) {
-        // should never happen... just a sanity check
-        log.error(logid+"ERROR!!! onDeckSearchers is " + onDeckSearchers);
-        onDeckSearchers=1;  // reset
-      } else if (onDeckSearchers > maxWarmingSearchers) {
-        onDeckSearchers--;
-        String msg="Error opening new searcher. exceeded limit of maxWarmingSearchers="+maxWarmingSearchers + ", try again later.";
-        log.warn(logid+""+ msg);
-        // HTTP 503==service unavailable, or 409==Conflict
-        throw new SolrException(SolrException.ErrorCode.SERVICE_UNAVAILABLE,msg);
-      } else if (onDeckSearchers > 1) {
-        log.warn(logid+"PERFORMANCE WARNING: Overlapping onDeckSearchers=" + onDeckSearchers);
+        // At this point, we know we need to open a new searcher...
+        // first: increment count to signal other threads that we are
+        //        opening a new searcher.
+        onDeckSearchers++;
+        if (onDeckSearchers < 1) {
+          // should never happen... just a sanity check
+          log.error(logid + "ERROR!!! onDeckSearchers is " + onDeckSearchers);
+          onDeckSearchers = 1;  // reset
+        } else if (onDeckSearchers > maxWarmingSearchers) {
+          onDeckSearchers--;
+          try {
+            searcherLock.wait();
+          } catch (InterruptedException e) {
+            log.info(SolrException.toStr(e));
+          }
+          continue;  // go back to the top of the loop and retry
+        } else if (onDeckSearchers > 1) {
+          log.warn(logid + "PERFORMANCE WARNING: Overlapping onDeckSearchers=" + onDeckSearchers);
+        }
+
+        break; // I can now exit the loop and proceed to open a searcher
       }
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/core/src/test-files/solr/collection1/conf/solrconfig_perf.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig_perf.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig_perf.xml
index 8bd2dbb..c0b297f 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig_perf.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig_perf.xml
@@ -57,7 +57,6 @@
     <queryResultWindowSize>20</queryResultWindowSize>
     <queryResultMaxDocsCached>200</queryResultMaxDocsCached>
     <useColdSearcher>false</useColdSearcher>
-    <maxWarmingSearchers>2</maxWarmingSearchers>
 
   </query>
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/core/src/test/org/apache/solr/search/TestRealTimeGet.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestRealTimeGet.java b/solr/core/src/test/org/apache/solr/search/TestRealTimeGet.java
index 51eaf34..28ecaa3 100644
--- a/solr/core/src/test/org/apache/solr/search/TestRealTimeGet.java
+++ b/solr/core/src/test/org/apache/solr/search/TestRealTimeGet.java
@@ -513,7 +513,7 @@ public class TestRealTimeGet extends TestRTGBase {
     final int ndocs = 5 + (random().nextBoolean() ? random().nextInt(25) : random().nextInt(200));
     int nWriteThreads = 5 + random().nextInt(25);
 
-    final int maxConcurrentCommits = nWriteThreads;   // number of committers at a time... it should be <= maxWarmingSearchers
+    final int maxConcurrentCommits = nWriteThreads;   // number of committers at a time...
 
         // query variables
     final int percentRealtimeQuery = 60;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/core/src/test/org/apache/solr/search/TestStressLucene.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestStressLucene.java b/solr/core/src/test/org/apache/solr/search/TestStressLucene.java
index 94cf300..779be43 100644
--- a/solr/core/src/test/org/apache/solr/search/TestStressLucene.java
+++ b/solr/core/src/test/org/apache/solr/search/TestStressLucene.java
@@ -55,7 +55,7 @@ public class TestStressLucene extends TestRTGBase {
     final int ndocs = 5 + (random().nextBoolean() ? random().nextInt(25) : random().nextInt(200));
     int nWriteThreads = 5 + random().nextInt(25);
 
-    final int maxConcurrentCommits = nWriteThreads;   // number of committers at a time... it should be <= maxWarmingSearchers
+    final int maxConcurrentCommits = nWriteThreads;
 
     final AtomicLong operations = new AtomicLong(100000);  // number of query operations to perform in total
     int nReadThreads = 5 + random().nextInt(25);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/core/src/test/org/apache/solr/search/TestStressRecovery.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestStressRecovery.java b/solr/core/src/test/org/apache/solr/search/TestStressRecovery.java
index 412908d..b6ecc2e 100644
--- a/solr/core/src/test/org/apache/solr/search/TestStressRecovery.java
+++ b/solr/core/src/test/org/apache/solr/search/TestStressRecovery.java
@@ -70,7 +70,7 @@ public class TestStressRecovery extends TestRTGBase {
     final int ndocs = 5 + (random().nextBoolean() ? random().nextInt(25) : random().nextInt(200));
     int nWriteThreads = 2 + random().nextInt(10);  // fewer write threads to give recovery thread more of a chance
 
-    final int maxConcurrentCommits = nWriteThreads;   // number of committers at a time... it should be <= maxWarmingSearchers
+    final int maxConcurrentCommits = nWriteThreads;
 
     // query variables
     final int percentRealtimeQuery = 75;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/core/src/test/org/apache/solr/search/TestStressReorder.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestStressReorder.java b/solr/core/src/test/org/apache/solr/search/TestStressReorder.java
index 4415946..bce1d75 100644
--- a/solr/core/src/test/org/apache/solr/search/TestStressReorder.java
+++ b/solr/core/src/test/org/apache/solr/search/TestStressReorder.java
@@ -68,8 +68,7 @@ public class TestStressReorder extends TestRTGBase {
     final int ndocs = 5 + (random().nextBoolean() ? random().nextInt(25) : random().nextInt(200));
     int nWriteThreads = 5 + random().nextInt(25);
 
-    final int maxConcurrentCommits = nWriteThreads;   // number of committers at a time... it should be <= maxWarmingSearchers
-
+    final int maxConcurrentCommits = nWriteThreads;
         // query variables
     final int percentRealtimeQuery = 75;
     final AtomicLong operations = new AtomicLong(50000);  // number of query operations to perform in total
@@ -84,7 +83,7 @@ public class TestStressReorder extends TestRTGBase {
     final int ndocs = 1;
     int nWriteThreads = 2;
 
-    final int maxConcurrentCommits = nWriteThreads;   // number of committers at a time... it should be <= maxWarmingSearchers
+    final int maxConcurrentCommits = nWriteThreads;
 
     // query variables
     final int percentRealtimeQuery = 101;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/core/src/test/org/apache/solr/search/TestStressUserVersions.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestStressUserVersions.java b/solr/core/src/test/org/apache/solr/search/TestStressUserVersions.java
index e098ecc..4eea434 100644
--- a/solr/core/src/test/org/apache/solr/search/TestStressUserVersions.java
+++ b/solr/core/src/test/org/apache/solr/search/TestStressUserVersions.java
@@ -74,7 +74,7 @@ public class TestStressUserVersions extends TestRTGBase {
     final int ndocs = 5 + (random().nextBoolean() ? random().nextInt(25) : random().nextInt(200));
     int nWriteThreads = 5 + random().nextInt(25);
 
-    final int maxConcurrentCommits = nWriteThreads;   // number of committers at a time... it should be <= maxWarmingSearchers
+    final int maxConcurrentCommits = nWriteThreads;
 
     // query variables
     final int percentRealtimeQuery = 75;
@@ -90,7 +90,7 @@ public class TestStressUserVersions extends TestRTGBase {
      final int ndocs = 1;
      int nWriteThreads = 2;
 
-     final int maxConcurrentCommits = nWriteThreads;   // number of committers at a time... it should be <= maxWarmingSearchers
+     final int maxConcurrentCommits = nWriteThreads;
 
      // query variables
      final int percentRealtimeQuery = 101;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/core/src/test/org/apache/solr/search/TestStressVersions.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestStressVersions.java b/solr/core/src/test/org/apache/solr/search/TestStressVersions.java
index d91a2cc..ed51ae5 100644
--- a/solr/core/src/test/org/apache/solr/search/TestStressVersions.java
+++ b/solr/core/src/test/org/apache/solr/search/TestStressVersions.java
@@ -56,7 +56,7 @@ public class TestStressVersions extends TestRTGBase {
     final int ndocs = 5 + (random().nextBoolean() ? random().nextInt(25) : random().nextInt(200));
     int nWriteThreads = 5 + random().nextInt(25);
 
-    final int maxConcurrentCommits = nWriteThreads;   // number of committers at a time... it should be <= maxWarmingSearchers
+    final int maxConcurrentCommits = nWriteThreads;
 
     // query variables
     final int percentRealtimeQuery = 75;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/example/example-DIH/solr/db/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/example/example-DIH/solr/db/conf/solrconfig.xml b/solr/example/example-DIH/solr/db/conf/solrconfig.xml
index 6a30473..71d8f5a 100644
--- a/solr/example/example-DIH/solr/db/conf/solrconfig.xml
+++ b/solr/example/example-DIH/solr/db/conf/solrconfig.xml
@@ -581,17 +581,6 @@
       -->
     <useColdSearcher>false</useColdSearcher>
 
-    <!-- Max Warming Searchers
-         
-         Maximum number of searchers that may be warming in the
-         background concurrently.  An error is returned if this limit
-         is exceeded.
-
-         Recommend values of 1-2 for read-only slaves, higher for
-         masters w/o cache warming.
-      -->
-    <maxWarmingSearchers>2</maxWarmingSearchers>
-
   </query>
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/example/example-DIH/solr/mail/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/example/example-DIH/solr/mail/conf/solrconfig.xml b/solr/example/example-DIH/solr/mail/conf/solrconfig.xml
index 24da135..8bc24f5 100644
--- a/solr/example/example-DIH/solr/mail/conf/solrconfig.xml
+++ b/solr/example/example-DIH/solr/mail/conf/solrconfig.xml
@@ -584,17 +584,6 @@
       -->
     <useColdSearcher>false</useColdSearcher>
 
-    <!-- Max Warming Searchers
-         
-         Maximum number of searchers that may be warming in the
-         background concurrently.  An error is returned if this limit
-         is exceeded.
-
-         Recommend values of 1-2 for read-only slaves, higher for
-         masters w/o cache warming.
-      -->
-    <maxWarmingSearchers>2</maxWarmingSearchers>
-
   </query>
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/example/example-DIH/solr/rss/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/example/example-DIH/solr/rss/conf/solrconfig.xml b/solr/example/example-DIH/solr/rss/conf/solrconfig.xml
index 224a94b..eae60a7 100644
--- a/solr/example/example-DIH/solr/rss/conf/solrconfig.xml
+++ b/solr/example/example-DIH/solr/rss/conf/solrconfig.xml
@@ -581,17 +581,6 @@
       -->
     <useColdSearcher>false</useColdSearcher>
 
-    <!-- Max Warming Searchers
-         
-         Maximum number of searchers that may be warming in the
-         background concurrently.  An error is returned if this limit
-         is exceeded.
-
-         Recommend values of 1-2 for read-only slaves, higher for
-         masters w/o cache warming.
-      -->
-    <maxWarmingSearchers>2</maxWarmingSearchers>
-
   </query>
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/example/example-DIH/solr/solr/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/example/example-DIH/solr/solr/conf/solrconfig.xml b/solr/example/example-DIH/solr/solr/conf/solrconfig.xml
index 2c67189..873391b 100644
--- a/solr/example/example-DIH/solr/solr/conf/solrconfig.xml
+++ b/solr/example/example-DIH/solr/solr/conf/solrconfig.xml
@@ -581,17 +581,6 @@
       -->
     <useColdSearcher>false</useColdSearcher>
 
-    <!-- Max Warming Searchers
-         
-         Maximum number of searchers that may be warming in the
-         background concurrently.  An error is returned if this limit
-         is exceeded.
-
-         Recommend values of 1-2 for read-only slaves, higher for
-         masters w/o cache warming.
-      -->
-    <maxWarmingSearchers>2</maxWarmingSearchers>
-
   </query>
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/example/example-DIH/solr/tika/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/example/example-DIH/solr/tika/conf/solrconfig.xml b/solr/example/example-DIH/solr/tika/conf/solrconfig.xml
index 7554e17..ac0c22a 100644
--- a/solr/example/example-DIH/solr/tika/conf/solrconfig.xml
+++ b/solr/example/example-DIH/solr/tika/conf/solrconfig.xml
@@ -584,17 +584,6 @@
       -->
     <useColdSearcher>false</useColdSearcher>
 
-    <!-- Max Warming Searchers
-         
-         Maximum number of searchers that may be warming in the
-         background concurrently.  An error is returned if this limit
-         is exceeded.
-
-         Recommend values of 1-2 for read-only slaves, higher for
-         masters w/o cache warming.
-      -->
-    <maxWarmingSearchers>2</maxWarmingSearchers>
-
   </query>
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/example/files/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/example/files/conf/solrconfig.xml b/solr/example/files/conf/solrconfig.xml
index fcbc4d6..3fd825e 100644
--- a/solr/example/files/conf/solrconfig.xml
+++ b/solr/example/files/conf/solrconfig.xml
@@ -579,17 +579,6 @@
       -->
     <useColdSearcher>false</useColdSearcher>
 
-    <!-- Max Warming Searchers
-         
-         Maximum number of searchers that may be warming in the
-         background concurrently.  An error is returned if this limit
-         is exceeded.
-
-         Recommend values of 1-2 for read-only slaves, higher for
-         masters w/o cache warming.
-      -->
-    <maxWarmingSearchers>2</maxWarmingSearchers>
-
   </query>
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/server/solr/configsets/basic_configs/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/server/solr/configsets/basic_configs/conf/solrconfig.xml b/solr/server/solr/configsets/basic_configs/conf/solrconfig.xml
index b0a8cdf..4ef902f 100644
--- a/solr/server/solr/configsets/basic_configs/conf/solrconfig.xml
+++ b/solr/server/solr/configsets/basic_configs/conf/solrconfig.xml
@@ -599,17 +599,6 @@
       -->
     <useColdSearcher>false</useColdSearcher>
 
-    <!-- Max Warming Searchers
-         
-         Maximum number of searchers that may be warming in the
-         background concurrently.  An error is returned if this limit
-         is exceeded.
-
-         Recommend values of 1-2 for read-only slaves, higher for
-         masters w/o cache warming.
-      -->
-    <maxWarmingSearchers>2</maxWarmingSearchers>
-
   </query>
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml b/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml
index ea50610..4b0899b 100644
--- a/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml
+++ b/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml
@@ -599,17 +599,6 @@
       -->
     <useColdSearcher>false</useColdSearcher>
 
-    <!-- Max Warming Searchers
-         
-         Maximum number of searchers that may be warming in the
-         background concurrently.  An error is returned if this limit
-         is exceeded.
-
-         Recommend values of 1-2 for read-only slaves, higher for
-         masters w/o cache warming.
-      -->
-    <maxWarmingSearchers>2</maxWarmingSearchers>
-
   </query>
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml b/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml
index 65b348a..e4b0526 100644
--- a/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml
+++ b/solr/server/solr/configsets/sample_techproducts_configs/conf/solrconfig.xml
@@ -611,17 +611,6 @@
       -->
     <useColdSearcher>false</useColdSearcher>
 
-    <!-- Max Warming Searchers
-         
-         Maximum number of searchers that may be warming in the
-         background concurrently.  An error is returned if this limit
-         is exceeded.
-
-         Recommend values of 1-2 for read-only slaves, higher for
-         masters w/o cache warming.
-      -->
-    <maxWarmingSearchers>2</maxWarmingSearchers>
-
   </query>
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/solrj/src/test/org/apache/solr/client/solrj/LargeVolumeTestBase.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/LargeVolumeTestBase.java b/solr/solrj/src/test/org/apache/solr/client/solrj/LargeVolumeTestBase.java
index b9d525c..8f43c33 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/LargeVolumeTestBase.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/LargeVolumeTestBase.java
@@ -60,7 +60,8 @@ public abstract class LargeVolumeTestBase extends SolrJettyTestBase
 
     // some of the commits could have failed because maxWarmingSearchers exceeded,
     // so do a final commit to make sure everything is visible.
-    client.commit();
+    // This should no longer be true as of SOLR-9712 (Solr 6.4)
+    // client.commit();
     
     query(threadCount * numdocs);
     log.info("done");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTestBase.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTestBase.java b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTestBase.java
index ce762a0..114d8ed 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTestBase.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTestBase.java
@@ -38,7 +38,6 @@ abstract public class SolrExampleTestBase extends AbstractSolrTestCase {
   
   @Override
   public void setUp() throws Exception {
-    ignoreException("maxWarmingSearchers");
     super.setUp();
     
     // this sets the property for jetty starting SolrDispatchFilter

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9522a39/solr/test-framework/src/java/org/apache/solr/SolrJettyTestBase.java
----------------------------------------------------------------------
diff --git a/solr/test-framework/src/java/org/apache/solr/SolrJettyTestBase.java b/solr/test-framework/src/java/org/apache/solr/SolrJettyTestBase.java
index 0badec1..a936ee5 100644
--- a/solr/test-framework/src/java/org/apache/solr/SolrJettyTestBase.java
+++ b/solr/test-framework/src/java/org/apache/solr/SolrJettyTestBase.java
@@ -110,8 +110,6 @@ abstract public class SolrJettyTestBase extends SolrTestCaseJ4
     nodeProps.setProperty("coreRootDirectory", coresDir.toString());
     nodeProps.setProperty("configSetBaseDir", solrHome);
 
-    ignoreException("maxWarmingSearchers");
-
     jetty = new JettySolrRunner(solrHome, nodeProps, jettyConfig);
     jetty.start();
     port = jetty.getLocalPort();


[03/23] lucene-solr:feature/metrics: Fix LeafReader.getNumericDocValues javadoc

Posted by ab...@apache.org.
Fix LeafReader.getNumericDocValues javadoc


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

Branch: refs/heads/feature/metrics
Commit: 770f1eb8ad6af5cce55d1bdf52f1288216c9691f
Parents: ad7152a
Author: Shai Erera <sh...@apache.org>
Authored: Wed Dec 14 13:07:19 2016 +0200
Committer: Shai Erera <sh...@apache.org>
Committed: Wed Dec 14 13:07:19 2016 +0200

----------------------------------------------------------------------
 lucene/core/src/java/org/apache/lucene/index/LeafReader.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/770f1eb8/lucene/core/src/java/org/apache/lucene/index/LeafReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/LeafReader.java b/lucene/core/src/java/org/apache/lucene/index/LeafReader.java
index acdd0d8..73394f2 100644
--- a/lucene/core/src/java/org/apache/lucene/index/LeafReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/LeafReader.java
@@ -242,7 +242,7 @@ public abstract class LeafReader extends IndexReader {
   /** Returns {@link NumericDocValues} for this field, or
    *  null if no numeric doc values were indexed for
    *  this field.  The returned instance should only be
-   *  used by a single thread.  This will never return null. */
+   *  used by a single thread. */
   public abstract NumericDocValues getNumericDocValues(String field) throws IOException;
 
   /** Returns {@link BinaryDocValues} for this field, or


[22/23] lucene-solr:feature/metrics: SOLR-4735 Update CHANGES.txt

Posted by ab...@apache.org.
SOLR-4735 Update CHANGES.txt


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

Branch: refs/heads/feature/metrics
Commit: f56da1df6e92da5f1ab524caf62d30ea3a3ceede
Parents: 92ef10d
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Mon Dec 19 13:45:05 2016 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Mon Dec 19 13:45:05 2016 +0100

----------------------------------------------------------------------
 solr/CHANGES.txt | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f56da1df/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index a0f11f1..0ab961c 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -167,6 +167,10 @@ New Features
 * SOLR-9844: FieldCache information fetched via the mbeans handler or seen via the UI now displays the total size used.
   The individual cache entries in the response are now formatted better as well. (Varun Thacker)
 
+* SOLR-4735: Improve metrics reporting. This uses the dropwizard metric library, adding an internal API
+  for registering and reporting metrics from Solr components. Several new metrics and an improved JMX
+  reporter have been added (Alan Woodward, Jeff Wartes, Christine Poerschke, Kelvin Wong, shalin, ab)
+
 * SOLR-9788: Use instrumented jetty classes provided by the dropwizard metric library. (shalin)
 
 * SOLR-9805: Use metrics-jvm library to instrument jvm internals such as GC, memory usage and others. (shalin)


[15/23] lucene-solr:feature/metrics: LUCENE-7587: add helper FacetQuery and MultiFacetQuery classes to simplify drill down implementation

Posted by ab...@apache.org.
LUCENE-7587: add helper FacetQuery and MultiFacetQuery classes to simplify drill down implementation


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

Branch: refs/heads/feature/metrics
Commit: 835296f20a17c12c66b4f043074c94e3ddd5c2b5
Parents: 3b182aa
Author: Mike McCandless <mi...@apache.org>
Authored: Fri Dec 16 09:56:51 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Fri Dec 16 09:56:51 2016 -0500

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  4 +
 .../org/apache/lucene/facet/FacetQuery.java     | 52 +++++++++++
 .../apache/lucene/facet/MultiFacetQuery.java    | 61 ++++++++++++
 .../org/apache/lucene/facet/TestFacetQuery.java | 98 ++++++++++++++++++++
 4 files changed, 215 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/835296f2/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 7e61469..47cd6e8 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -74,6 +74,10 @@ New features
 * LUCENE-7590: Added DocValuesStatsCollector to compute statistics on DocValues
   fields. (Shai Erera)
 
+* LUCENE-7587: The new FacetQuery and MultiFacetQuery helper classes
+  make it simpler to execute drill down when drill sideways counts are
+  not needed (Emmanuel Keller via Mike McCandless)
+
 Bug Fixes
 
 * LUCENE-7547: JapaneseTokenizerFactory was failing to close the

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/835296f2/lucene/facet/src/java/org/apache/lucene/facet/FacetQuery.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/FacetQuery.java b/lucene/facet/src/java/org/apache/lucene/facet/FacetQuery.java
new file mode 100644
index 0000000..ec20292
--- /dev/null
+++ b/lucene/facet/src/java/org/apache/lucene/facet/FacetQuery.java
@@ -0,0 +1,52 @@
+/*
+ * 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.facet;
+
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermQuery;
+
+/**
+ * A term {@link Query} over a {@link FacetField}.
+ * <p>
+ * <b>NOTE:</b>This helper class is an alternative to {@link DrillDownQuery}
+ * especially in cases where you don't intend to use {@link DrillSideways}
+ *
+ * @lucene.experimental
+ */
+public class FacetQuery extends TermQuery {
+
+  /**
+   * Creates a new {@code FacetQuery} filtering the query on the given dimension.
+   */
+  public FacetQuery(final FacetsConfig facetsConfig, final String dimension, final String... path) {
+    super(toTerm(facetsConfig.getDimConfig(dimension), dimension, path));
+  }
+
+  /**
+   * Creates a new {@code FacetQuery} filtering the query on the given dimension.
+   * <p>
+   * <b>NOTE:</b>Uses FacetsConfig.DEFAULT_DIM_CONFIG.
+   */
+  public FacetQuery(final String dimension, final String... path) {
+    super(toTerm(FacetsConfig.DEFAULT_DIM_CONFIG, dimension, path));
+  }
+
+  static Term toTerm(final FacetsConfig.DimConfig dimConfig, final String dimension, final String... path) {
+    return new Term(dimConfig.indexFieldName, FacetsConfig.pathToString(dimension, path));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/835296f2/lucene/facet/src/java/org/apache/lucene/facet/MultiFacetQuery.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/MultiFacetQuery.java b/lucene/facet/src/java/org/apache/lucene/facet/MultiFacetQuery.java
new file mode 100644
index 0000000..dd212c6
--- /dev/null
+++ b/lucene/facet/src/java/org/apache/lucene/facet/MultiFacetQuery.java
@@ -0,0 +1,61 @@
+/*
+ * 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.facet;
+
+import org.apache.lucene.index.Term;
+import org.apache.lucene.queries.TermsQuery;
+import org.apache.lucene.search.Query;
+
+import java.util.ArrayList;
+import java.util.Collection;
+
+/**
+ * A multi-terms {@link Query} over a {@link FacetField}.
+ * <p>
+ * <b>NOTE:</b>This helper class is an alternative to {@link DrillDownQuery}
+ * especially in cases where you don't intend to use {@link DrillSideways}
+ *
+ * @lucene.experimental
+ * @see org.apache.lucene.queries.TermsQuery
+ */
+public class MultiFacetQuery extends TermsQuery {
+
+  /**
+   * Creates a new {@code MultiFacetQuery} filtering the query on the given dimension.
+   */
+  public MultiFacetQuery(final FacetsConfig facetsConfig, final String dimension, final String[]... paths) {
+    super(toTerms(facetsConfig.getDimConfig(dimension), dimension, paths));
+  }
+
+  /**
+   * Creates a new {@code MultiFacetQuery} filtering the query on the given dimension.
+   * <p>
+   * <b>NOTE:</b>Uses FacetsConfig.DEFAULT_DIM_CONFIG.
+   */
+  public MultiFacetQuery(final String dimension, final String[]... paths) {
+    super(toTerms(FacetsConfig.DEFAULT_DIM_CONFIG, dimension, paths));
+  }
+
+  static Collection<Term> toTerms(final FacetsConfig.DimConfig dimConfig, final String dimension,
+          final String[]... paths) {
+    final Collection<Term> terms = new ArrayList<>(paths.length);
+    for (String[] path : paths)
+      terms.add(FacetQuery.toTerm(dimConfig, dimension, path));
+    return terms;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/835296f2/lucene/facet/src/test/org/apache/lucene/facet/TestFacetQuery.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/TestFacetQuery.java b/lucene/facet/src/test/org/apache/lucene/facet/TestFacetQuery.java
new file mode 100644
index 0000000..f3aa079
--- /dev/null
+++ b/lucene/facet/src/test/org/apache/lucene/facet/TestFacetQuery.java
@@ -0,0 +1,98 @@
+/*
+ * 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.facet;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.analysis.MockTokenizer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetField;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.IOUtils;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+
+public class TestFacetQuery extends FacetTestCase {
+
+  private static Directory indexDirectory;
+  private static RandomIndexWriter indexWriter;
+  private static IndexReader indexReader;
+  private static IndexSearcher searcher;
+  private static FacetsConfig config;
+
+  private static final IndexableField[] DOC_SINGLEVALUED =
+          new IndexableField[] { new SortedSetDocValuesFacetField("Author", "Mark Twain") };
+
+  private static final IndexableField[] DOC_MULTIVALUED =
+          new SortedSetDocValuesFacetField[] { new SortedSetDocValuesFacetField("Author", "Kurt Vonnegut") };
+
+  private static final IndexableField[] DOC_NOFACET =
+          new IndexableField[] { new TextField("Hello", "World", Field.Store.YES) };
+
+  @BeforeClass
+  public static void createTestIndex() throws IOException {
+    indexDirectory = newDirectory();
+    // create and open an index writer
+    indexWriter = new RandomIndexWriter(random(), indexDirectory,
+            newIndexWriterConfig(new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false)));
+
+    config = new FacetsConfig();
+
+    indexDocuments(DOC_SINGLEVALUED, DOC_MULTIVALUED, DOC_NOFACET);
+
+    indexReader = indexWriter.getReader();
+    // prepare searcher to search against
+    searcher = newSearcher(indexReader);
+  }
+
+  private static void indexDocuments(IndexableField[]... docs) throws IOException {
+    for (IndexableField[] fields : docs) {
+      for (IndexableField field : fields) {
+        Document doc = new Document();
+        doc.add(field);
+        indexWriter.addDocument(config.build(doc));
+      }
+    }
+  }
+
+  @AfterClass
+  public static void closeTestIndex() throws IOException {
+    IOUtils.close(indexReader, indexWriter, indexDirectory);
+  }
+
+  @Test
+  public void testSingleValued() throws Exception {
+    TopDocs topDocs = searcher.search(new FacetQuery("Author", "Mark Twain"), 10);
+    assertEquals(1, topDocs.totalHits);
+  }
+
+  @Test
+  public void testMultiValued() throws Exception {
+    TopDocs topDocs = searcher.search(
+            new MultiFacetQuery("Author", new String[] { "Mark Twain" }, new String[] { "Kurt Vonnegut" }), 10);
+    assertEquals(2, topDocs.totalHits);
+  }
+}


[08/23] lucene-solr:feature/metrics: SOLR-9844: Display fc total size only when field entries asked for

Posted by ab...@apache.org.
SOLR-9844: Display fc total size only when field entries asked for


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

Branch: refs/heads/feature/metrics
Commit: 512374384a8984c56c91f47dcac4aaf0490eda54
Parents: 7dec783
Author: Varun Thacker <va...@apache.org>
Authored: Tue Dec 13 15:52:17 2016 -0800
Committer: Varun Thacker <va...@apache.org>
Committed: Wed Dec 14 10:52:26 2016 -0800

----------------------------------------------------------------------
 .../apache/solr/search/SolrFieldCacheMBean.java |  6 ++---
 .../solr/uninverting/UninvertingReader.java     | 25 +++++++++++++-------
 .../solr/search/TestSolrFieldCacheMBean.java    |  3 ++-
 3 files changed, 22 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/51237438/solr/core/src/java/org/apache/solr/search/SolrFieldCacheMBean.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/SolrFieldCacheMBean.java b/solr/core/src/java/org/apache/solr/search/SolrFieldCacheMBean.java
index 70781e9..642b708 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrFieldCacheMBean.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrFieldCacheMBean.java
@@ -62,15 +62,15 @@ public class SolrFieldCacheMBean implements JmxAugmentedSolrInfoMBean {
   private NamedList getStats(boolean listEntries) {
     NamedList stats = new SimpleOrderedMap();
     if (listEntries) {
-      String[] entries = UninvertingReader.getUninvertedStats();
+      UninvertingReader.FieldCacheStats fieldCacheStats = UninvertingReader.getUninvertedStats();
+      String[] entries = fieldCacheStats.info;
       stats.add("entries_count", entries.length);
-      stats.add("total_size", UninvertingReader.getTotalSize());
+      stats.add("total_size", fieldCacheStats.totalSize);
       for (int i = 0; i < entries.length; i++) {
         stats.add("entry#" + i, entries[i]);
       }
     } else {
       stats.add("entries_count", UninvertingReader.getUninvertedStatsSize());
-      stats.add("total_size", UninvertingReader.getTotalSize());
     }
     return stats;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/51237438/solr/core/src/java/org/apache/solr/uninverting/UninvertingReader.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/uninverting/UninvertingReader.java b/solr/core/src/java/org/apache/solr/uninverting/UninvertingReader.java
index 87fb7a6..5276ca9 100644
--- a/solr/core/src/java/org/apache/solr/uninverting/UninvertingReader.java
+++ b/solr/core/src/java/org/apache/solr/uninverting/UninvertingReader.java
@@ -375,25 +375,34 @@ public class UninvertingReader extends FilterLeafReader {
    * Return information about the backing cache
    * @lucene.internal 
    */
-  public static String[] getUninvertedStats() {
+  public static FieldCacheStats getUninvertedStats() {
     CacheEntry[] entries = FieldCache.DEFAULT.getCacheEntries();
+    long totalBytesUsed = 0;
     String[] info = new String[entries.length];
     for (int i = 0; i < entries.length; i++) {
       info[i] = entries[i].toString();
+      totalBytesUsed += entries[i].getValue().ramBytesUsed();
     }
-    return info;
+    String totalSize = RamUsageEstimator.humanReadableUnits(totalBytesUsed);
+    return new FieldCacheStats(totalSize, info);
   }
 
   public static int getUninvertedStatsSize() {
     return FieldCache.DEFAULT.getCacheEntries().length;
   }
 
-  public static String getTotalSize() {
-    CacheEntry[] entries = FieldCache.DEFAULT.getCacheEntries();
-    long totalBytesUsed = 0;
-    for (int i = 0; i < entries.length; i++) {
-      totalBytesUsed += entries[i].getValue().ramBytesUsed();
+  /**
+   * Return information about the backing cache
+   * @lucene.internal
+   */
+  public static class FieldCacheStats {
+    public String totalSize;
+    public String[] info;
+
+    public FieldCacheStats(String totalSize, String[] info) {
+      this.totalSize = totalSize;
+      this.info = info;
     }
-    return RamUsageEstimator.humanReadableUnits(totalBytesUsed);
+
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/51237438/solr/core/src/test/org/apache/solr/search/TestSolrFieldCacheMBean.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestSolrFieldCacheMBean.java b/solr/core/src/test/org/apache/solr/search/TestSolrFieldCacheMBean.java
index a705e1e..35bdec6 100644
--- a/solr/core/src/test/org/apache/solr/search/TestSolrFieldCacheMBean.java
+++ b/solr/core/src/test/org/apache/solr/search/TestSolrFieldCacheMBean.java
@@ -69,6 +69,7 @@ public class TestSolrFieldCacheMBean extends SolrTestCaseJ4 {
     SolrFieldCacheMBean mbean = new SolrFieldCacheMBean();
     NamedList stats = checkJmx ? mbean.getStatisticsForJmx() : mbean.getStatistics();
     assert(new Integer(stats.get("entries_count").toString()) > 0);
+    assertNotNull(stats.get("total_size"));
     assertNotNull(stats.get("entry#0"));
   }
 
@@ -76,7 +77,7 @@ public class TestSolrFieldCacheMBean extends SolrTestCaseJ4 {
     SolrFieldCacheMBean mbean = new SolrFieldCacheMBean();
     NamedList stats = checkJmx ? mbean.getStatisticsForJmx() : mbean.getStatistics();
     assert(new Integer(stats.get("entries_count").toString()) > 0);
-    assertNotNull(stats.get("total_size"));
+    assertNull(stats.get("total_size"));
     assertNull(stats.get("entry#0"));
   }
 }


[20/23] lucene-solr:feature/metrics: LUCENE-7590: add Sorted(Set)DocValuesStats

Posted by ab...@apache.org.
LUCENE-7590: add Sorted(Set)DocValuesStats


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

Branch: refs/heads/feature/metrics
Commit: 23206caabd09310cb23a2b5302ce41af62b5c9cc
Parents: 944b8e0
Author: Shai Erera <sh...@apache.org>
Authored: Sun Dec 18 12:20:42 2016 +0200
Committer: Shai Erera <sh...@apache.org>
Committed: Sun Dec 18 21:50:03 2016 +0200

----------------------------------------------------------------------
 .../apache/lucene/search/DocValuesStats.java    |  83 ++++++++++++
 .../search/TestDocValuesStatsCollector.java     | 127 +++++++++++++++++--
 2 files changed, 196 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/23206caa/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java b/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
index 9dd97a6..b6449cc 100644
--- a/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
+++ b/lucene/misc/src/java/org/apache/lucene/search/DocValuesStats.java
@@ -20,7 +20,11 @@ import java.io.IOException;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.SortedNumericDocValues;
+import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
 
 /** Holds statistics for a DocValues field. */
 public abstract class DocValuesStats<T> {
@@ -323,4 +327,83 @@ public abstract class DocValuesStats<T> {
     }
   }
 
+  private static BytesRef copyFrom(BytesRef src, BytesRef dest) {
+    if (dest == null) {
+      return BytesRef.deepCopyOf(src);
+    }
+
+    dest.bytes = ArrayUtil.grow(dest.bytes, src.length);
+    System.arraycopy(src.bytes, src.offset, dest.bytes, 0, src.length);
+    dest.offset = 0;
+    dest.length = src.length;
+    return dest;
+  }
+
+  /** Holds statistics for a sorted DocValues field. */
+  public static class SortedDocValuesStats extends DocValuesStats<BytesRef> {
+
+    protected SortedDocValues sdv;
+
+    protected SortedDocValuesStats(String field) {
+      super(field, null, null);
+    }
+
+    @Override
+    protected final boolean init(LeafReaderContext context) throws IOException {
+      sdv = context.reader().getSortedDocValues(field);
+      return sdv != null;
+    }
+
+    @Override
+    protected final boolean hasValue(int doc) throws IOException {
+      return sdv.advanceExact(doc);
+    }
+
+    @Override
+    protected void doAccumulate(int count) throws IOException {
+      BytesRef val = sdv.binaryValue();
+      if (max == null || val.compareTo(max) > 0) {
+        max = copyFrom(val, max);
+      }
+      if (min == null || val.compareTo(min) < 0) {
+        min = copyFrom(val, min);
+      }
+    }
+  }
+
+  /** Holds statistics for a sorted-set DocValues field. */
+  public static class SortedSetDocValuesStats extends DocValuesStats<BytesRef> {
+
+    protected SortedSetDocValues ssdv;
+
+    protected SortedSetDocValuesStats(String field) {
+      super(field, null, null);
+    }
+
+    @Override
+    protected final boolean init(LeafReaderContext context) throws IOException {
+      ssdv = context.reader().getSortedSetDocValues(field);
+      return ssdv != null;
+    }
+
+    @Override
+    protected final boolean hasValue(int doc) throws IOException {
+      return ssdv.advanceExact(doc);
+    }
+
+    @Override
+    protected void doAccumulate(int count) throws IOException {
+      long ord;
+      while ((ord = ssdv.nextOrd()) != SortedSetDocValues.NO_MORE_ORDS) {
+        BytesRef val = ssdv.lookupOrd(ord);
+        if (max == null || val.compareTo(max) > 0) {
+          max = copyFrom(val, max);
+        }
+        if (min == null || val.compareTo(min) < 0) {
+          min = copyFrom(val, min);
+        }
+      }
+    }
+  }
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/23206caa/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java b/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
index 5fa4b04..3471c03 100644
--- a/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
+++ b/lucene/misc/src/test/org/apache/lucene/search/TestDocValuesStatsCollector.java
@@ -20,6 +20,7 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.DoubleSummaryStatistics;
 import java.util.LongSummaryStatistics;
+import java.util.Objects;
 import java.util.function.Predicate;
 import java.util.stream.DoubleStream;
 import java.util.stream.LongStream;
@@ -29,16 +30,21 @@ import org.apache.lucene.document.Document;
 import org.apache.lucene.document.DoubleDocValuesField;
 import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.document.SortedNumericDocValuesField;
+import org.apache.lucene.document.SortedSetDocValuesField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.DocValuesStats.DoubleDocValuesStats;
 import org.apache.lucene.search.DocValuesStats.LongDocValuesStats;
+import org.apache.lucene.search.DocValuesStats.SortedDocValuesStats;
 import org.apache.lucene.search.DocValuesStats.SortedDoubleDocValuesStats;
 import org.apache.lucene.search.DocValuesStats.SortedLongDocValuesStats;
+import org.apache.lucene.search.DocValuesStats.SortedSetDocValuesStats;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 
@@ -227,8 +233,8 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
         SortedLongDocValuesStats stats = new SortedLongDocValuesStats(field);
         searcher.search(new MatchAllDocsQuery(), new DocValuesStatsCollector(stats));
 
-        assertEquals(filterValues(docValues, (v) -> v != null).count(), stats.count());
-        assertEquals(filterValues(docValues, (v) -> v == null).count() - reader.numDeletedDocs(), stats.missing());
+        assertEquals(nonNull(docValues).count(), stats.count());
+        assertEquals(isNull(docValues).count() - reader.numDeletedDocs(), stats.missing());
         if (stats.count() > 0) {
           LongSummaryStatistics sumStats = filterAndFlatValues(docValues, (v) -> v != null).summaryStatistics();
           assertEquals(sumStats.getMax(), stats.max().longValue());
@@ -281,8 +287,8 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
         SortedDoubleDocValuesStats stats = new SortedDoubleDocValuesStats(field);
         searcher.search(new MatchAllDocsQuery(), new DocValuesStatsCollector(stats));
 
-        assertEquals(filterValues(docValues, (v) -> v != null).count(), stats.count());
-        assertEquals(filterValues(docValues, (v) -> v == null).count() - reader.numDeletedDocs(), stats.missing());
+        assertEquals(nonNull(docValues).count(), stats.count());
+        assertEquals(isNull(docValues).count() - reader.numDeletedDocs(), stats.missing());
         if (stats.count() > 0) {
           DoubleSummaryStatistics sumStats = filterAndFlatValues(docValues, (v) -> v != null).summaryStatistics();
           assertEquals(sumStats.getMax(), stats.max().longValue(), 0.00001);
@@ -298,6 +304,96 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
     }
   }
 
+  public void testDocsWithSortedValues() throws IOException {
+    try (Directory dir = newDirectory();
+        IndexWriter indexWriter = new IndexWriter(dir, newIndexWriterConfig())) {
+      String field = "sorted";
+      int numDocs = TestUtil.nextInt(random(), 1, 100);
+      BytesRef[] docValues = new BytesRef[numDocs];
+      for (int i = 0; i < numDocs; i++) {
+        Document doc = new Document();
+        if (random().nextBoolean()) { // not all documents have a value
+          BytesRef val = TestUtil.randomBinaryTerm(random());
+          doc.add(new SortedDocValuesField(field, val));
+          doc.add(new StringField("id", "doc" + i, Store.NO));
+          docValues[i] = val;
+        }
+        indexWriter.addDocument(doc);
+      }
+
+      // 20% of cases delete some docs
+      if (random().nextDouble() < 0.2) {
+        for (int i = 0; i < numDocs; i++) {
+          if (random().nextBoolean()) {
+            indexWriter.deleteDocuments(new Term("id", "doc" + i));
+            docValues[i] = null;
+          }
+        }
+      }
+
+      try (DirectoryReader reader = DirectoryReader.open(indexWriter)) {
+        IndexSearcher searcher = new IndexSearcher(reader);
+        SortedDocValuesStats stats = new SortedDocValuesStats(field);
+        searcher.search(new MatchAllDocsQuery(), new DocValuesStatsCollector(stats));
+
+        int expCount = (int) nonNull(docValues).count();
+        assertEquals(expCount, stats.count());
+        assertEquals(isNull(docValues).count() - reader.numDeletedDocs(), stats.missing());
+        if (stats.count() > 0) {
+          assertEquals(nonNull(docValues).min(BytesRef::compareTo).get(), stats.min());
+          assertEquals(nonNull(docValues).max(BytesRef::compareTo).get(), stats.max());
+        }
+      }
+    }
+  }
+
+  public void testDocsWithSortedSetValues() throws IOException {
+    try (Directory dir = newDirectory();
+        IndexWriter indexWriter = new IndexWriter(dir, newIndexWriterConfig())) {
+      String field = "sorted";
+      int numDocs = TestUtil.nextInt(random(), 1, 100);
+      BytesRef[][] docValues = new BytesRef[numDocs][];
+      for (int i = 0; i < numDocs; i++) {
+        Document doc = new Document();
+        if (random().nextBoolean()) { // not all documents have a value
+          int numValues = TestUtil.nextInt(random(), 1, 5);
+          docValues[i] = new BytesRef[numValues];
+          for (int j = 0; j < numValues; j++) {
+            BytesRef val = TestUtil.randomBinaryTerm(random());
+            doc.add(new SortedSetDocValuesField(field, val));
+            docValues[i][j] = val;
+          }
+          doc.add(new StringField("id", "doc" + i, Store.NO));
+        }
+        indexWriter.addDocument(doc);
+      }
+
+      // 20% of cases delete some docs
+      if (random().nextDouble() < 0.2) {
+        for (int i = 0; i < numDocs; i++) {
+          if (random().nextBoolean()) {
+            indexWriter.deleteDocuments(new Term("id", "doc" + i));
+            docValues[i] = null;
+          }
+        }
+      }
+
+      try (DirectoryReader reader = DirectoryReader.open(indexWriter)) {
+        IndexSearcher searcher = new IndexSearcher(reader);
+        SortedSetDocValuesStats stats = new SortedSetDocValuesStats(field);
+        searcher.search(new MatchAllDocsQuery(), new DocValuesStatsCollector(stats));
+
+        int expCount = (int) nonNull(docValues).count();
+        assertEquals(expCount, stats.count());
+        assertEquals(isNull(docValues).count() - reader.numDeletedDocs(), stats.missing());
+        if (stats.count() > 0) {
+          assertEquals(nonNull(docValues).flatMap(Arrays::stream).min(BytesRef::compareTo).get(), stats.min());
+          assertEquals(nonNull(docValues).flatMap(Arrays::stream).max(BytesRef::compareTo).get(), stats.max());
+        }
+      }
+    }
+  }
+
   private static LongStream getPositiveValues(long[] values) {
     return Arrays.stream(values).filter(v -> v > 0);
   }
@@ -314,14 +410,6 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
     return Arrays.stream(values).filter(v -> v == 0);
   }
 
-  private static Stream<long[]> filterValues(long[][] values, Predicate<? super long[]> p) {
-    return Arrays.stream(values).filter(p);
-  }
-
-  private static Stream<double[]> filterValues(double[][] values, Predicate<? super double[]> p) {
-    return Arrays.stream(values).filter(p);
-  }
-
   private static double computeVariance(long[] values, double mean, int count) {
     return getPositiveValues(values).mapToDouble(v -> (v - mean) * (v-mean)).sum() / count;
   }
@@ -331,11 +419,11 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
   }
 
   private static LongStream filterAndFlatValues(long[][] values, Predicate<? super long[]> p) {
-    return filterValues(values, (v) -> v != null).flatMapToLong(Arrays::stream);
+    return nonNull(values).flatMapToLong(Arrays::stream);
   }
 
   private static DoubleStream filterAndFlatValues(double[][] values, Predicate<? super double[]> p) {
-    return filterValues(values, (v) -> v != null).flatMapToDouble(Arrays::stream);
+    return nonNull(values).flatMapToDouble(Arrays::stream);
   }
 
   private static double computeVariance(LongStream values, double mean, int count) {
@@ -346,4 +434,15 @@ public class TestDocValuesStatsCollector extends LuceneTestCase {
     return values.map(v -> (v - mean) * (v-mean)).sum() / count;
   }
 
+  private static <T> Stream<T> nonNull(T[] values) {
+    return filterValues(values, Objects::nonNull);
+  }
+
+  private static <T> Stream<T> isNull(T[] values) {
+    return filterValues(values, Objects::isNull);
+  }
+
+  private static <T> Stream<T> filterValues(T[] values, Predicate<? super T> p) {
+    return Arrays.stream(values).filter(p);
+  }
 }