You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2018/05/16 16:05:46 UTC

[6/6] lucene-solr:branch_7x: LUCENE-8309: Live docs are no longer backed by mutable bits.

LUCENE-8309: Live docs are no longer backed by mutable bits.


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

Branch: refs/heads/branch_7x
Commit: 6bfc38f078096f47c0536c43c11b1021dc246a1b
Parents: b5bfcf0
Author: Adrien Grand <jp...@gmail.com>
Authored: Wed May 16 17:26:02 2018 +0200
Committer: Adrien Grand <jp...@gmail.com>
Committed: Wed May 16 18:01:19 2018 +0200

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  4 ++
 .../codecs/lucene50/Lucene50LiveDocsFormat.java |  2 +-
 .../org/apache/lucene/index/PendingDeletes.java |  2 +-
 .../org/apache/lucene/util/FixedBitSet.java     | 18 ++++++++
 .../java/org/apache/lucene/util/FixedBits.java  | 47 ++++++++++++++++++++
 .../org/apache/lucene/util/TestFixedBitSet.java | 37 +++++++++++----
 6 files changed, 99 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6bfc38f0/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index cb9c5d5..960e91e 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -17,6 +17,10 @@ API Changes
 * LUCENE-8303: LiveDocsFormat is now only responsible for (de)serialization of
   live docs. (Adrien Grand)
 
+Changes in Runtime Behavior
+
+* LUCENE-8309: Live docs are no longer backed by a FixedBitSet. (Adrien Grand)
+
 New Features
 
 * LUCENE-8200: Allow doc-values to be updated atomically together

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6bfc38f0/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50LiveDocsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50LiveDocsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50LiveDocsFormat.java
index 65cf0f7..2343de2 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50LiveDocsFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50LiveDocsFormat.java
@@ -80,7 +80,7 @@ public final class Lucene50LiveDocsFormat extends LiveDocsFormat {
           throw new CorruptIndexException("bits.deleted=" + (fbs.length() - fbs.cardinality()) + 
                                           " info.delcount=" + info.getDelCount(), input);
         }
-        return fbs;
+        return fbs.asReadOnlyBits();
       } catch (Throwable exception) {
         priorE = exception;
       } finally {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6bfc38f0/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java b/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java
index 8e3b7e0..506d397 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java
@@ -76,7 +76,7 @@ class PendingDeletes {
         writeableLiveDocs = new FixedBitSet(info.info.maxDoc());
         writeableLiveDocs.set(0, info.info.maxDoc());
       }
-      liveDocs = writeableLiveDocs;
+      liveDocs = writeableLiveDocs.asReadOnlyBits();
     }
     return writeableLiveDocs;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6bfc38f0/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java b/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java
index 0936488..92d1878 100644
--- a/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java
+++ b/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java
@@ -515,6 +515,12 @@ public final class FixedBitSet extends BitSet implements Bits, Accountable {
    * Make a copy of the given bits.
    */
   public static FixedBitSet copyOf(Bits bits) {
+    if (bits instanceof FixedBits) {
+      // restore the original FixedBitSet
+      FixedBits fixedBits = (FixedBits) bits;
+      bits = new FixedBitSet(fixedBits.bits, fixedBits.length);
+    }
+
     if (bits instanceof FixedBitSet) {
       return ((FixedBitSet)bits).clone();
     } else {
@@ -529,4 +535,16 @@ public final class FixedBitSet extends BitSet implements Bits, Accountable {
       return bitSet;
     }
   }
+
+  /**
+   * Convert this instance to read-only {@link Bits}.
+   * This is useful in the case that this {@link FixedBitSet} is returned as a
+   * {@link Bits} instance, to make sure that consumers may not get write access
+   * back by casting to a {@link FixedBitSet}.
+   * NOTE: Changes to this {@link FixedBitSet} will be reflected on the returned
+   * {@link Bits}.
+   */
+  public Bits asReadOnlyBits() {
+    return new FixedBits(bits, numBits);
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6bfc38f0/lucene/core/src/java/org/apache/lucene/util/FixedBits.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/FixedBits.java b/lucene/core/src/java/org/apache/lucene/util/FixedBits.java
new file mode 100644
index 0000000..372ba3a
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/util/FixedBits.java
@@ -0,0 +1,47 @@
+/*
+ * 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;
+
+/**
+ * Immutable twin of FixedBitSet.
+ */
+final class FixedBits implements Bits {
+
+  final long[] bits;
+  final int length;
+
+  FixedBits(long[] bits, int length) {
+    this.bits = bits;
+    this.length = length;
+  }
+
+  @Override
+  public boolean get(int index) {
+    assert index >= 0 && index < length: "index=" + index + ", numBits=" + length;
+    int i = index >> 6;               // div 64
+    // signed shift will keep a negative index and force an
+    // array-index-out-of-bounds-exception, removing the need for an explicit check.
+    long bitmask = 1L << index;
+    return (bits[i] & bitmask) != 0;
+  }
+
+  @Override
+  public int length() {
+    return length;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6bfc38f0/lucene/core/src/test/org/apache/lucene/util/TestFixedBitSet.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/TestFixedBitSet.java b/lucene/core/src/test/org/apache/lucene/util/TestFixedBitSet.java
index 25c7e41..f043577 100644
--- a/lucene/core/src/test/org/apache/lucene/util/TestFixedBitSet.java
+++ b/lucene/core/src/test/org/apache/lucene/util/TestFixedBitSet.java
@@ -503,11 +503,14 @@ public class TestFixedBitSet extends BaseBitSetTestCase<FixedBitSet> {
     for (int e : bits) {
       fixedBitSet.set(e);
     }
-    FixedBitSet mutableCopy = FixedBitSet.copyOf(fixedBitSet);
-    assertNotSame(mutableCopy, fixedBitSet);
-    assertEquals(mutableCopy, fixedBitSet);
+    for (boolean readOnly : new boolean[] {false, true}) {
+      Bits bitsToCopy = readOnly ? fixedBitSet.asReadOnlyBits() : fixedBitSet;
+      FixedBitSet mutableCopy = FixedBitSet.copyOf(bitsToCopy);
+      assertNotSame(mutableCopy, bitsToCopy);
+      assertEquals(mutableCopy, fixedBitSet);
+    }
 
-    FixedBitSet mutableCopy1 = FixedBitSet.copyOf(new Bits() {
+    final Bits bitsToCopy = new Bits() {
 
       @Override
       public boolean get(int index) {
@@ -518,11 +521,27 @@ public class TestFixedBitSet extends BaseBitSetTestCase<FixedBitSet> {
       public int length() {
         return fixedBitSet.length();
       }
-    });
+    };
+    FixedBitSet mutableCopy = FixedBitSet.copyOf(bitsToCopy);
+
+    assertNotSame(bitsToCopy, mutableCopy);
+    assertNotSame(fixedBitSet, mutableCopy);
+    assertEquals(mutableCopy, fixedBitSet);
+  }
 
-    assertNotSame(mutableCopy, mutableCopy1);
-    assertNotSame(fixedBitSet, mutableCopy1);
-    assertEquals(mutableCopy1, mutableCopy);
-    assertEquals(mutableCopy1, fixedBitSet);
+  public void testAsBits() {
+    FixedBitSet set = new FixedBitSet(10);
+    set.set(3);
+    set.set(4);
+    set.set(9);
+    Bits bits = set.asReadOnlyBits();
+    assertFalse(bits instanceof FixedBitSet);
+    assertEquals(set.length(), bits.length());
+    for (int i = 0; i < set.length(); ++i) {
+      assertEquals(set.get(i), bits.get(i));
+    }
+    // Further changes are reflected
+    set.set(5);
+    assertTrue(bits.get(5));
   }
 }