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:58 UTC

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

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")));