You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2021/11/15 01:56:40 UTC

[GitHub] [lucene] gf2121 opened a new pull request #438: LUCENE-10233: Store docIds as bitset when leafCardinality = 1 to speed up addAll

gf2121 opened a new pull request #438:
URL: https://github.com/apache/lucene/pull/438


   In low cardinality points cases, id blocks will usually store doc ids that have the same point value, and `intersect` will get into `addAll` logic. If we store ids as bitset when the leafCadinality = 1, and give the IntersectVisitor bulk visiting ability, we can speed up addAll because we can just execute the 'or' logic between the result and the block ids.
   
   I mocked a field that has 10,000,000 docs per value and search it with a PointInSetQuery with 1 term, the build scorer time decreased from 71ms to 8ms.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] gf2121 commented on a change in pull request #438: LUCENE-10233: Store docIds as bitset to speed up addAll

Posted by GitBox <gi...@apache.org>.
gf2121 commented on a change in pull request #438:
URL: https://github.com/apache/lucene/pull/438#discussion_r761690720



##########
File path: lucene/core/src/java/org/apache/lucene/util/bkd/DocIdsWriter.java
##########
@@ -18,23 +18,32 @@
 
 import java.io.IOException;
 import org.apache.lucene.index.PointValues.IntersectVisitor;
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.DocBaseBitSetIterator;
+import org.apache.lucene.util.FixedBitSet;
 
 class DocIdsWriter {
 
   private DocIdsWriter() {}
 
-  static void writeDocIds(int[] docIds, int start, int count, DataOutput out) throws IOException {
+  static void writeDocIds(
+      int[] docIds, int start, int count, DataOutput out, boolean consistentValue)
+      throws IOException {
+    if (consistentValue

Review comment:
       @jpountz @iverase Hi! For scenes that index is sorted on the field, blocks with continuous ids may be a common situation. In this case we can handle this situation more efficiently. We only need to check `stritylysorted && (docIds[start+count-1]-docids[start]+1) = = count` to check if ids is continuous (this is fast). If so, we can just write the first id of this block. I wonder if this optimization will make sense to you?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] gf2121 commented on a change in pull request #438: LUCENE-10233: Store docIds as bitset when leafCardinality = 1 to speed up addAll

Posted by GitBox <gi...@apache.org>.
gf2121 commented on a change in pull request #438:
URL: https://github.com/apache/lucene/pull/438#discussion_r760082305



##########
File path: lucene/core/src/java/org/apache/lucene/util/DocBaseBitSetIterator.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.util;
+
+import org.apache.lucene.search.DocIdSetIterator;
+
+/**
+ * A @{@link DocIdSetIterator} like {@link BitSetIterator} but has a doc base in onder to avoid
+ * storing previous 0s.
+ */
+public class DocBaseBitSetIterator extends DocIdSetIterator {
+
+  private final FixedBitSet bits;
+  private final int length;
+  private final long cost;
+  private final int docBase;
+  private int doc = -1;
+
+  public DocBaseBitSetIterator(FixedBitSet bits, long cost, int docBase) {
+    if (cost < 0) {
+      throw new IllegalArgumentException("cost must be >= 0, got " + cost);
+    }
+    if ((docBase & 63) != 0) {
+      throw new IllegalArgumentException("docBase need to be a multiple of 64");
+    }
+    this.bits = bits;
+    this.length = bits.length() + docBase;
+    this.cost = cost;
+    this.docBase = docBase;
+  }
+
+  public FixedBitSet getBitSet() {

Review comment:
       done

##########
File path: lucene/core/src/java/org/apache/lucene/util/DocBaseBitSetIterator.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.util;
+
+import org.apache.lucene.search.DocIdSetIterator;
+
+/**
+ * A @{@link DocIdSetIterator} like {@link BitSetIterator} but has a doc base in onder to avoid
+ * storing previous 0s.
+ */
+public class DocBaseBitSetIterator extends DocIdSetIterator {
+
+  private final FixedBitSet bits;
+  private final int length;
+  private final long cost;
+  private final int docBase;
+  private int doc = -1;
+
+  public DocBaseBitSetIterator(FixedBitSet bits, long cost, int docBase) {
+    if (cost < 0) {
+      throw new IllegalArgumentException("cost must be >= 0, got " + cost);
+    }
+    if ((docBase & 63) != 0) {
+      throw new IllegalArgumentException("docBase need to be a multiple of 64");
+    }
+    this.bits = bits;
+    this.length = bits.length() + docBase;
+    this.cost = cost;
+    this.docBase = docBase;
+  }
+
+  public FixedBitSet getBitSet() {
+    return bits;
+  }
+
+  @Override
+  public int docID() {
+    return doc;
+  }
+
+  public int getDocBase() {

Review comment:
       done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] gf2121 commented on pull request #438: LUCENE-10233: Store docIds as bitset when leafCardinality = 1 to speed up addAll

Posted by GitBox <gi...@apache.org>.
gf2121 commented on pull request #438:
URL: https://github.com/apache/lucene/pull/438#issuecomment-983562338


   @jpountz Thanks a lot for the great advices! I have fixed them and added a CHANGES entry.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jpountz commented on pull request #438: LUCENE-10233: Store docIds as bitset to speed up addAll

Posted by GitBox <gi...@apache.org>.
jpountz commented on pull request #438:
URL: https://github.com/apache/lucene/pull/438#issuecomment-983728887


   The backport has some test failures that look related to the change of endianness, I'm seeing DocIdsWriter write longs in a given byte order and then read them in the opposite byte order on the read side, I'm unsure why we are seeing this.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] gf2121 commented on a change in pull request #438: LUCENE-10233: Store docIds as bitset to speed up addAll

Posted by GitBox <gi...@apache.org>.
gf2121 commented on a change in pull request #438:
URL: https://github.com/apache/lucene/pull/438#discussion_r761690720



##########
File path: lucene/core/src/java/org/apache/lucene/util/bkd/DocIdsWriter.java
##########
@@ -18,23 +18,32 @@
 
 import java.io.IOException;
 import org.apache.lucene.index.PointValues.IntersectVisitor;
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.DocBaseBitSetIterator;
+import org.apache.lucene.util.FixedBitSet;
 
 class DocIdsWriter {
 
   private DocIdsWriter() {}
 
-  static void writeDocIds(int[] docIds, int start, int count, DataOutput out) throws IOException {
+  static void writeDocIds(
+      int[] docIds, int start, int count, DataOutput out, boolean consistentValue)
+      throws IOException {
+    if (consistentValue

Review comment:
       @jpountz @iverase Hi! For scenes that index is sorted on the field, blocks having continuous ids will be a common situation. Maybe we can handle this situation more efficiently:  If continuous, we can just write the first id of this block. 
   
   To check if ids are continuous, we just need to check `stritylysorted && (docIds[start+count-1]-docids[start]+1) == count`, this check is very fast :).




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jpountz merged pull request #438: LUCENE-10233: Store docIds as bitset to speed up addAll

Posted by GitBox <gi...@apache.org>.
jpountz merged pull request #438:
URL: https://github.com/apache/lucene/pull/438


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jpountz commented on a change in pull request #438: LUCENE-10233: Store docIds as bitset when leafCardinality = 1 to speed up addAll

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #438:
URL: https://github.com/apache/lucene/pull/438#discussion_r759955885



##########
File path: lucene/core/src/java/org/apache/lucene/util/DocBaseBitSetIterator.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.util;
+
+import org.apache.lucene.search.DocIdSetIterator;
+
+/**
+ * A @{@link DocIdSetIterator} like {@link BitSetIterator} but has a doc base in onder to avoid
+ * storing previous 0s.
+ */
+public class DocBaseBitSetIterator extends DocIdSetIterator {
+
+  private final FixedBitSet bits;
+  private final int length;
+  private final long cost;
+  private final int docBase;
+  private int doc = -1;
+
+  public DocBaseBitSetIterator(FixedBitSet bits, long cost, int docBase) {
+    if (cost < 0) {
+      throw new IllegalArgumentException("cost must be >= 0, got " + cost);
+    }
+    if ((docBase & 63) != 0) {
+      throw new IllegalArgumentException("docBase need to be a multiple of 64");
+    }
+    this.bits = bits;
+    this.length = bits.length() + docBase;
+    this.cost = cost;
+    this.docBase = docBase;
+  }
+
+  public FixedBitSet getBitSet() {

Review comment:
       Can you add javadocs?

##########
File path: lucene/core/src/java/org/apache/lucene/util/DocBaseBitSetIterator.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.util;
+
+import org.apache.lucene.search.DocIdSetIterator;
+
+/**
+ * A @{@link DocIdSetIterator} like {@link BitSetIterator} but has a doc base in onder to avoid

Review comment:
       ```suggestion
    * A {@link DocIdSetIterator} like {@link BitSetIterator} but has a doc base in onder to avoid
   ```

##########
File path: lucene/core/src/java/org/apache/lucene/util/bkd/DocIdsWriter.java
##########
@@ -85,10 +94,65 @@ static void writeDocIds(int[] docIds, int start, int count, DataOutput out) thro
     }
   }
 
+  private static boolean isSorted(int[] docIds, int start, int count) {
+    for (int i = 1; i < count; ++i) {
+      if (docIds[start + i - 1] > docIds[start + i]) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  private static boolean isStrictlySorted(int[] docIds, int start, int count) {
+    assert isSorted(docIds, start, count);
+    for (int i = 1; i < count; ++i) {
+      if (docIds[start + i - 1] == docIds[start + i]) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  private static void writeIdsAsBitSet(int[] docIds, int start, int count, DataOutput out)
+      throws IOException {
+    int min = docIds[start];
+    int max = docIds[start + count - 1];
+
+    final int offsetWords = min >> 6;
+    final int offsetBits = offsetWords << 6;
+    final int totalWordCount = FixedBitSet.bits2words(max - offsetBits + 1);
+    long currentWord = 0;
+    int currentWordIndex = 0;
+
+    out.writeVInt(offsetBits);

Review comment:
       let's write offsetWords instead since it's smaller?

##########
File path: lucene/core/src/java/org/apache/lucene/util/DocBaseBitSetIterator.java
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.util;
+
+import org.apache.lucene.search.DocIdSetIterator;
+
+/**
+ * A @{@link DocIdSetIterator} like {@link BitSetIterator} but has a doc base in onder to avoid
+ * storing previous 0s.
+ */
+public class DocBaseBitSetIterator extends DocIdSetIterator {
+
+  private final FixedBitSet bits;
+  private final int length;
+  private final long cost;
+  private final int docBase;
+  private int doc = -1;
+
+  public DocBaseBitSetIterator(FixedBitSet bits, long cost, int docBase) {
+    if (cost < 0) {
+      throw new IllegalArgumentException("cost must be >= 0, got " + cost);
+    }
+    if ((docBase & 63) != 0) {
+      throw new IllegalArgumentException("docBase need to be a multiple of 64");
+    }
+    this.bits = bits;
+    this.length = bits.length() + docBase;
+    this.cost = cost;
+    this.docBase = docBase;
+  }
+
+  public FixedBitSet getBitSet() {
+    return bits;
+  }
+
+  @Override
+  public int docID() {
+    return doc;
+  }
+
+  public int getDocBase() {

Review comment:
       Can you add javadocs?

##########
File path: lucene/core/src/java/org/apache/lucene/util/bkd/DocIdsWriter.java
##########
@@ -18,23 +18,32 @@
 
 import java.io.IOException;
 import org.apache.lucene.index.PointValues.IntersectVisitor;
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.DocBaseBitSetIterator;
+import org.apache.lucene.util.FixedBitSet;
 
 class DocIdsWriter {
 
   private DocIdsWriter() {}
 
-  static void writeDocIds(int[] docIds, int start, int count, DataOutput out) throws IOException {
+  static void writeDocIds(
+      int[] docIds, int start, int count, DataOutput out, boolean consistentValue)
+      throws IOException {
+    if (consistentValue

Review comment:
       So we only check this boolean as a way to avoid running `isStrictlyOrdered` since fields with low cardinality are the most likely to trigger this optimization, is my understanding correct?
   
   I'm tempted to remove this check so that your new optimization would also kick in when the index is sorted by this field, or when index order matches a field's order. We already perform some heavier operations at the leaf level like sorting by value.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] gf2121 commented on a change in pull request #438: LUCENE-10233: Store docIds as bitset to speed up addAll

Posted by GitBox <gi...@apache.org>.
gf2121 commented on a change in pull request #438:
URL: https://github.com/apache/lucene/pull/438#discussion_r761690720



##########
File path: lucene/core/src/java/org/apache/lucene/util/bkd/DocIdsWriter.java
##########
@@ -18,23 +18,32 @@
 
 import java.io.IOException;
 import org.apache.lucene.index.PointValues.IntersectVisitor;
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.DocBaseBitSetIterator;
+import org.apache.lucene.util.FixedBitSet;
 
 class DocIdsWriter {
 
   private DocIdsWriter() {}
 
-  static void writeDocIds(int[] docIds, int start, int count, DataOutput out) throws IOException {
+  static void writeDocIds(
+      int[] docIds, int start, int count, DataOutput out, boolean consistentValue)
+      throws IOException {
+    if (consistentValue

Review comment:
       @jpountz @iverase Hi! For scenes that index is sorted on the field, blocks with continuous ids may be a common situation. Maybe we can handle this situation more efficiently: We only need to check `stritylysorted && (docIds[start+count-1]-docids[start]+1) = = count` to see if ids are continuous. If continuous, we can just write the first id of this block. I wonder if this optimization will make sense to you?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] sonatype-lift[bot] commented on a change in pull request #438: LUCENE-10233: Store docIds as bitset when leafCardinality = 1 to speed up addAll

Posted by GitBox <gi...@apache.org>.
sonatype-lift[bot] commented on a change in pull request #438:
URL: https://github.com/apache/lucene/pull/438#discussion_r751055798



##########
File path: lucene/core/src/java/org/apache/lucene/util/SparseFixedBitSet.java
##########
@@ -530,4 +530,33 @@ public long ramBytesUsed() {
   public String toString() {
     return "SparseFixedBitSet(size=" + length + ",cardinality=~" + approximateCardinality();
   }
+
+  public static SparseFixedBitSet fromWords(int maxDoc, int base, long[] words) {
+    assert (base & 63) == 0;
+    SparseFixedBitSet sparseFixedBitSet = new SparseFixedBitSet(maxDoc + 1);
+    final int baseWord = base >> 6;
+    final long[] indices = sparseFixedBitSet.indices;
+    final long[][] bits = sparseFixedBitSet.bits;
+    int nonZeroWordCounts = 0;
+    int lastIndex = -1;
+    int bitsPos = 0;
+    long[] currentBits = null;
+    for (int i = 0; i < words.length; i++) {
+      long word = words[i];
+      if (word != 0) {
+        int actualI = i + baseWord;
+        int index = actualI >> 6;
+        if (index != lastIndex) {
+          currentBits = bits[index] = new long[64];
+          bitsPos = 0;
+          lastIndex = index;
+        }
+        nonZeroWordCounts++;
+        indices[index] |= 1L << actualI;
+        currentBits[bitsPos++] = word;

Review comment:
       *NULL_DEREFERENCE:*  object `currentBits` last assigned on line 543 could be null and is dereferenced at line 556.
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] gf2121 commented on a change in pull request #438: LUCENE-10233: Store docIds as bitset to speed up addAll

Posted by GitBox <gi...@apache.org>.
gf2121 commented on a change in pull request #438:
URL: https://github.com/apache/lucene/pull/438#discussion_r761691414



##########
File path: lucene/core/src/java/org/apache/lucene/util/bkd/DocIdsWriter.java
##########
@@ -18,23 +18,32 @@
 
 import java.io.IOException;
 import org.apache.lucene.index.PointValues.IntersectVisitor;
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.DocBaseBitSetIterator;
+import org.apache.lucene.util.FixedBitSet;
 
 class DocIdsWriter {
 
   private DocIdsWriter() {}
 
-  static void writeDocIds(int[] docIds, int start, int count, DataOutput out) throws IOException {
+  static void writeDocIds(
+      int[] docIds, int start, int count, DataOutput out, boolean consistentValue)
+      throws IOException {
+    if (consistentValue

Review comment:
       I raised an ISSUE about this: https://issues.apache.org/jira/browse/LUCENE-10280




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] gf2121 commented on a change in pull request #438: LUCENE-10233: Store docIds as bitset when leafCardinality = 1 to speed up addAll

Posted by GitBox <gi...@apache.org>.
gf2121 commented on a change in pull request #438:
URL: https://github.com/apache/lucene/pull/438#discussion_r760082798



##########
File path: lucene/core/src/java/org/apache/lucene/util/bkd/DocIdsWriter.java
##########
@@ -18,23 +18,32 @@
 
 import java.io.IOException;
 import org.apache.lucene.index.PointValues.IntersectVisitor;
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.DocBaseBitSetIterator;
+import org.apache.lucene.util.FixedBitSet;
 
 class DocIdsWriter {
 
   private DocIdsWriter() {}
 
-  static void writeDocIds(int[] docIds, int start, int count, DataOutput out) throws IOException {
+  static void writeDocIds(
+      int[] docIds, int start, int count, DataOutput out, boolean consistentValue)
+      throws IOException {
+    if (consistentValue

Review comment:
       +1 to remove this check




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] jpountz commented on pull request #438: LUCENE-10233: Store docIds as bitset when leafCardinality = 1 to speed up addAll

Posted by GitBox <gi...@apache.org>.
jpountz commented on pull request #438:
URL: https://github.com/apache/lucene/pull/438#issuecomment-983424947


   Can you also add a CHANGES entry?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] sonatype-lift[bot] commented on a change in pull request #438: LUCENE-10233: Store docIds as bitset to speed up addAll

Posted by GitBox <gi...@apache.org>.
sonatype-lift[bot] commented on a change in pull request #438:
URL: https://github.com/apache/lucene/pull/438#discussion_r760188396



##########
File path: lucene/core/src/java/org/apache/lucene/util/bkd/DocIdsWriter.java
##########
@@ -85,10 +103,64 @@ static void writeDocIds(int[] docIds, int start, int count, DataOutput out) thro
     }
   }
 
+  private static boolean isSorted(int[] docIds, int start, int count) {

Review comment:
       *UnusedMethod:*  Private method 'isSorted' is never used. [(details)](https://errorprone.info/bugpattern/UnusedMethod)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: lucene/core/src/java/org/apache/lucene/util/bkd/DocIdsWriter.java
##########
@@ -85,10 +103,64 @@ static void writeDocIds(int[] docIds, int start, int count, DataOutput out) thro
     }
   }
 
+  private static boolean isSorted(int[] docIds, int start, int count) {
+    for (int i = 1; i < count; ++i) {
+      if (docIds[start + i - 1] > docIds[start + i]) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  private static boolean isStrictlySorted(int[] docIds, int start, int count) {

Review comment:
       *UnusedMethod:*  Private method 'isStrictlySorted' is never used. [(details)](https://errorprone.info/bugpattern/UnusedMethod)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene] gf2121 commented on a change in pull request #438: LUCENE-10233: Store docIds as bitset to speed up addAll

Posted by GitBox <gi...@apache.org>.
gf2121 commented on a change in pull request #438:
URL: https://github.com/apache/lucene/pull/438#discussion_r761690720



##########
File path: lucene/core/src/java/org/apache/lucene/util/bkd/DocIdsWriter.java
##########
@@ -18,23 +18,32 @@
 
 import java.io.IOException;
 import org.apache.lucene.index.PointValues.IntersectVisitor;
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.DocBaseBitSetIterator;
+import org.apache.lucene.util.FixedBitSet;
 
 class DocIdsWriter {
 
   private DocIdsWriter() {}
 
-  static void writeDocIds(int[] docIds, int start, int count, DataOutput out) throws IOException {
+  static void writeDocIds(
+      int[] docIds, int start, int count, DataOutput out, boolean consistentValue)
+      throws IOException {
+    if (consistentValue

Review comment:
       @jpountz @iverase Hi! For scenes that index is sorted on the field, blocks with continuous ids can be a common situation. Maybe we can handle this situation more efficiently: We only need to check `stritylysorted && (docIds[start+count-1]-docids[start]+1) = = count` to see if ids are continuous. If continuous, we can just write the first id of this block. I wonder if this optimization will make sense to you?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org