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/09 13:40:33 UTC

[2/2] lucene-solr:master: LUCENE-8303: Make LiveDocsFormat only responsible for serialization/deserialization of live docs.

LUCENE-8303: Make LiveDocsFormat only responsible for serialization/deserialization of live docs.


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

Branch: refs/heads/master
Commit: 8dc69428e34d894bb5fdd6d2b0e3ec1835e87b5f
Parents: b53ee58
Author: Adrien Grand <jp...@gmail.com>
Authored: Wed May 9 14:31:23 2018 +0200
Committer: Adrien Grand <jp...@gmail.com>
Committed: Wed May 9 15:40:14 2018 +0200

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   3 +
 .../simpletext/SimpleTextLiveDocsFormat.java    |  42 +-----
 .../TestSimpleTextLiveDocsFormat.java           |  29 ++++
 .../apache/lucene/codecs/LiveDocsFormat.java    |   9 +-
 .../codecs/lucene50/Lucene50LiveDocsFormat.java |  40 +++---
 .../lucene/index/DocumentsWriterPerThread.java  |  18 +--
 .../lucene/index/FreqProxTermsWriter.java       |   4 +-
 .../org/apache/lucene/index/PendingDeletes.java |  25 ++--
 .../apache/lucene/index/PendingSoftDeletes.java |   6 +-
 .../apache/lucene/index/SegmentWriteState.java  |   6 +-
 .../src/java/org/apache/lucene/util/BitSet.java |   5 +-
 .../org/apache/lucene/util/FixedBitSet.java     |   2 +-
 .../org/apache/lucene/util/MutableBits.java     |  31 -----
 .../lucene50/TestLucene50LiveDocsFormat.java    |  30 ++++
 .../asserting/AssertingLiveDocsFormat.java      |  46 +-----
 .../codecs/cranky/CrankyLiveDocsFormat.java     |  13 +-
 .../index/BaseLiveDocsFormatTestCase.java       | 139 +++++++++++++++++++
 17 files changed, 267 insertions(+), 181 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8dc69428/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 42c758d..7467093 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -112,6 +112,9 @@ API Changes
 * LUCENE-8248: MergePolicyWrapper is renamed to FilterMergePolicy and now
   also overrides getMaxCFSSegmentSizeMB (Mike Sokolov via Mike McCandless)
 
+* LUCENE-8303: LiveDocsFormat is now only responsible for (de)serialization of
+  live docs. (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/8dc69428/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextLiveDocsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextLiveDocsFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextLiveDocsFormat.java
index 8f01ebf..d158ede 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextLiveDocsFormat.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextLiveDocsFormat.java
@@ -34,7 +34,6 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.CharsRefBuilder;
 import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.MutableBits;
 import org.apache.lucene.util.StringHelper;
 
 /**
@@ -50,17 +49,6 @@ public class SimpleTextLiveDocsFormat extends LiveDocsFormat {
   final static BytesRef SIZE             = new BytesRef("size ");
   final static BytesRef DOC              = new BytesRef("  doc ");
   final static BytesRef END              = new BytesRef("END");
-  
-  @Override
-  public MutableBits newLiveDocs(int size) throws IOException {
-    return new SimpleTextMutableBits(size);
-  }
-
-  @Override
-  public MutableBits newLiveDocs(Bits existing) throws IOException {
-    final SimpleTextBits bits = (SimpleTextBits) existing;
-    return new SimpleTextMutableBits((BitSet)bits.bits.clone(), bits.size);
-  }
 
   @Override
   public Bits readLiveDocs(Directory dir, SegmentCommitInfo info, IOContext context) throws IOException {
@@ -107,8 +95,7 @@ public class SimpleTextLiveDocsFormat extends LiveDocsFormat {
   }
 
   @Override
-  public void writeLiveDocs(MutableBits bits, Directory dir, SegmentCommitInfo info, int newDelCount, IOContext context) throws IOException {
-    BitSet set = ((SimpleTextBits) bits).bits;
+  public void writeLiveDocs(Bits bits, Directory dir, SegmentCommitInfo info, int newDelCount, IOContext context) throws IOException {
     int size = bits.length();
     BytesRefBuilder scratch = new BytesRefBuilder();
     
@@ -121,10 +108,12 @@ public class SimpleTextLiveDocsFormat extends LiveDocsFormat {
       SimpleTextUtil.write(out, Integer.toString(size), scratch);
       SimpleTextUtil.writeNewline(out);
       
-      for (int i = set.nextSetBit(0); i >= 0; i=set.nextSetBit(i + 1)) { 
-        SimpleTextUtil.write(out, DOC);
-        SimpleTextUtil.write(out, Integer.toString(i), scratch);
-        SimpleTextUtil.writeNewline(out);
+      for (int i = 0; i < size; ++i) {
+        if (bits.get(i)) {
+          SimpleTextUtil.write(out, DOC);
+          SimpleTextUtil.write(out, Integer.toString(i), scratch);
+          SimpleTextUtil.writeNewline(out);
+        }
       }
       
       SimpleTextUtil.write(out, END);
@@ -167,22 +156,5 @@ public class SimpleTextLiveDocsFormat extends LiveDocsFormat {
       return size;
     }
   }
-  
-  // read-write
-  static class SimpleTextMutableBits extends SimpleTextBits implements MutableBits {
 
-    SimpleTextMutableBits(int size) {
-      this(new BitSet(size), size);
-      bits.set(0, size);
-    }
-    
-    SimpleTextMutableBits(BitSet bits, int size) {
-      super(bits, size);
-    }
-    
-    @Override
-    public void clear(int bit) {
-      bits.clear(bit);
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8dc69428/lucene/codecs/src/test/org/apache/lucene/codecs/simpletext/TestSimpleTextLiveDocsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/simpletext/TestSimpleTextLiveDocsFormat.java b/lucene/codecs/src/test/org/apache/lucene/codecs/simpletext/TestSimpleTextLiveDocsFormat.java
new file mode 100644
index 0000000..bb2ff69
--- /dev/null
+++ b/lucene/codecs/src/test/org/apache/lucene/codecs/simpletext/TestSimpleTextLiveDocsFormat.java
@@ -0,0 +1,29 @@
+/*
+ * 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.codecs.simpletext;
+
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.index.BaseLiveDocsFormatTestCase;
+
+public class TestSimpleTextLiveDocsFormat extends BaseLiveDocsFormatTestCase {
+  private final Codec codec = new SimpleTextCodec();
+
+  @Override
+  protected Codec getCodec() {
+    return codec;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8dc69428/lucene/core/src/java/org/apache/lucene/codecs/LiveDocsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/LiveDocsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/LiveDocsFormat.java
index 40e9865..286b75d 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/LiveDocsFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/LiveDocsFormat.java
@@ -24,7 +24,6 @@ import org.apache.lucene.index.SegmentCommitInfo;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.MutableBits;
 
 /** Format for live/deleted documents
  * @lucene.experimental */
@@ -35,19 +34,13 @@ public abstract class LiveDocsFormat {
   protected LiveDocsFormat() {
   }
 
-  /** Creates a new MutableBits, with all bits set, for the specified size. */
-  public abstract MutableBits newLiveDocs(int size) throws IOException;
-
-  /** Creates a new mutablebits of the same bits set and size of existing. */
-  public abstract MutableBits newLiveDocs(Bits existing) throws IOException;
-
   /** Read live docs bits. */
   public abstract Bits readLiveDocs(Directory dir, SegmentCommitInfo info, IOContext context) throws IOException;
 
   /** Persist live docs bits.  Use {@link
    *  SegmentCommitInfo#getNextDelGen} to determine the
    *  generation of the deletes file you should write to. */
-  public abstract void writeLiveDocs(MutableBits bits, Directory dir, SegmentCommitInfo info, int newDelCount, IOContext context) throws IOException;
+  public abstract void writeLiveDocs(Bits bits, Directory dir, SegmentCommitInfo info, int newDelCount, IOContext context) throws IOException;
 
   /** Records all files in use by this {@link SegmentCommitInfo} into the files argument. */
   public abstract void files(SegmentCommitInfo info, Collection<String> files) throws IOException;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8dc69428/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 aac7375..65cf0f7 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
@@ -32,7 +32,6 @@ import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.FixedBitSet;
-import org.apache.lucene.util.MutableBits;
 
 /** 
  * Lucene 5.0 live docs format 
@@ -63,19 +62,6 @@ public final class Lucene50LiveDocsFormat extends LiveDocsFormat {
   private static final int VERSION_CURRENT = VERSION_START;
 
   @Override
-  public MutableBits newLiveDocs(int size) throws IOException {
-    FixedBitSet bits = new FixedBitSet(size);
-    bits.set(0, size);
-    return bits;
-  }
-
-  @Override
-  public MutableBits newLiveDocs(Bits existing) throws IOException {
-    FixedBitSet fbs = (FixedBitSet) existing;
-    return fbs.clone();
-  }
-
-  @Override
   public Bits readLiveDocs(Directory dir, SegmentCommitInfo info, IOContext context) throws IOException {
     long gen = info.getDelGen();
     String name = IndexFileNames.fileNameFromGeneration(info.info.name, EXTENSION, gen);
@@ -105,22 +91,30 @@ public final class Lucene50LiveDocsFormat extends LiveDocsFormat {
   }
 
   @Override
-  public void writeLiveDocs(MutableBits bits, Directory dir, SegmentCommitInfo info, int newDelCount, IOContext context) throws IOException {
+  public void writeLiveDocs(Bits bits, Directory dir, SegmentCommitInfo info, int newDelCount, IOContext context) throws IOException {
     long gen = info.getNextDelGen();
     String name = IndexFileNames.fileNameFromGeneration(info.info.name, EXTENSION, gen);
-    FixedBitSet fbs = (FixedBitSet) bits;
-    if (fbs.length() - fbs.cardinality() != info.getDelCount() + newDelCount) {
-      throw new CorruptIndexException("bits.deleted=" + (fbs.length() - fbs.cardinality()) + 
-                                      " info.delcount=" + info.getDelCount() + " newdelcount=" + newDelCount, name);
-    }
-    long data[] = fbs.getBits();
+    int delCount = 0;
     try (IndexOutput output = dir.createOutput(name, context)) {
       CodecUtil.writeIndexHeader(output, CODEC_NAME, VERSION_CURRENT, info.info.getId(), Long.toString(gen, Character.MAX_RADIX));
-      for (int i = 0; i < data.length; i++) {
-        output.writeLong(data[i]);
+      final int longCount = FixedBitSet.bits2words(bits.length());
+      for (int i = 0; i < longCount; ++i) {
+        long currentBits = 0;
+        for (int j = i << 6, end = Math.min(j + 63, bits.length() - 1); j <= end; ++j) {
+          if (bits.get(j)) {
+            currentBits |= 1L << j; // mod 64
+          } else {
+            delCount += 1;
+          }
+        }
+        output.writeLong(currentBits);
       }
       CodecUtil.writeFooter(output);
     }
+    if (delCount != info.getDelCount() + newDelCount) {
+      throw new CorruptIndexException("bits.deleted=" + delCount + 
+          " info.delcount=" + info.getDelCount() + " newdelcount=" + newDelCount, name);
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8dc69428/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
index da45875..094afc5 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
@@ -37,9 +37,9 @@ import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.ByteBlockPool.Allocator;
 import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
 import org.apache.lucene.util.Counter;
+import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.InfoStream;
 import org.apache.lucene.util.IntBlockPool;
-import org.apache.lucene.util.MutableBits;
 import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.Version;
 
@@ -106,12 +106,12 @@ final class DocumentsWriterPerThread {
     final SegmentCommitInfo segmentInfo;
     final FieldInfos fieldInfos;
     final FrozenBufferedUpdates segmentUpdates;
-    final MutableBits liveDocs;
+    final FixedBitSet liveDocs;
     final Sorter.DocMap sortMap;
     final int delCount;
 
     private FlushedSegment(InfoStream infoStream, SegmentCommitInfo segmentInfo, FieldInfos fieldInfos,
-                           BufferedUpdates segmentUpdates, MutableBits liveDocs, int delCount, Sorter.DocMap sortMap)
+                           BufferedUpdates segmentUpdates, FixedBitSet liveDocs, int delCount, Sorter.DocMap sortMap)
       throws IOException {
       this.segmentInfo = segmentInfo;
       this.fieldInfos = fieldInfos;
@@ -436,7 +436,8 @@ final class DocumentsWriterPerThread {
     // happens when an exception is hit processing that
     // doc, eg if analyzer has some problem w/ the text):
     if (pendingUpdates.deleteDocIDs.size() > 0) {
-      flushState.liveDocs = codec.liveDocsFormat().newLiveDocs(numDocsInRAM);
+      flushState.liveDocs = new FixedBitSet(numDocsInRAM);
+      flushState.liveDocs.set(0, numDocsInRAM);
       for(int delDocID : pendingUpdates.deleteDocIDs) {
         flushState.liveDocs.clear(delDocID);
       }
@@ -529,9 +530,10 @@ final class DocumentsWriterPerThread {
     return filesToDelete;
   }
 
-  private MutableBits sortLiveDocs(Bits liveDocs, Sorter.DocMap sortMap) throws IOException {
+  private FixedBitSet sortLiveDocs(Bits liveDocs, Sorter.DocMap sortMap) throws IOException {
     assert liveDocs != null && sortMap != null;
-    MutableBits sortedLiveDocs = codec.liveDocsFormat().newLiveDocs(liveDocs.length());
+    FixedBitSet sortedLiveDocs = new FixedBitSet(liveDocs.length());
+    sortedLiveDocs.set(0, liveDocs.length());
     for (int i = 0; i < liveDocs.length(); i++) {
       if (liveDocs.get(i) == false) {
         sortedLiveDocs.clear(sortMap.oldToNew(i));
@@ -542,7 +544,7 @@ final class DocumentsWriterPerThread {
 
   /**
    * Seals the {@link SegmentInfo} for the new flushed segment and persists
-   * the deleted documents {@link MutableBits}.
+   * the deleted documents {@link FixedBitSet}.
    */
   void sealFlushedSegment(FlushedSegment flushedSegment, Sorter.DocMap sortMap, DocumentsWriter.FlushNotifications flushNotifications) throws IOException {
     assert flushedSegment != null;
@@ -593,7 +595,7 @@ final class DocumentsWriterPerThread {
           
         SegmentCommitInfo info = flushedSegment.segmentInfo;
         Codec codec = info.info.getCodec();
-        final MutableBits bits;
+        final FixedBitSet bits;
         if (sortMap == null) {
           bits = flushedSegment.liveDocs;
         } else {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8dc69428/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java b/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
index ac70669..2ebb13d 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
@@ -26,6 +26,7 @@ import java.util.Map;
 import org.apache.lucene.codecs.FieldsConsumer;
 import org.apache.lucene.codecs.NormsProducer;
 import org.apache.lucene.util.CollectionUtil;
+import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.IOUtils;
 
 final class FreqProxTermsWriter extends TermsHash {
@@ -63,7 +64,8 @@ final class FreqProxTermsWriter extends TermsHash {
             int doc = postingsEnum.nextDoc();
             if (doc < delDocLimit) {
               if (state.liveDocs == null) {
-                state.liveDocs = state.segmentInfo.getCodec().liveDocsFormat().newLiveDocs(state.segmentInfo.maxDoc());
+                state.liveDocs = new FixedBitSet(state.segmentInfo.maxDoc());
+                state.liveDocs.set(0, state.segmentInfo.maxDoc());
               }
               if (state.liveDocs.get(doc)) {
                 state.delCountOnFlush++;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8dc69428/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 2dc0858..354f9d1 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java
@@ -20,14 +20,13 @@ package org.apache.lucene.index;
 import java.io.IOException;
 
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.LiveDocsFormat;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.TrackingDirectoryWrapper;
 import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.IOSupplier;
 import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.MutableBits;
 
 /**
  * This class handles accounting and applying pending deletes for live segment readers
@@ -67,7 +66,7 @@ class PendingDeletes {
   }
 
 
-  protected MutableBits getMutableBits() throws IOException {
+  protected FixedBitSet getMutableBits() throws IOException {
     // if we pull mutable bits but we haven't been initialized something is completely off.
     // this means we receive deletes without having the bitset that is on-disk ready to be cloned
     assert liveDocsInitialized : "can't delete if liveDocs are not initialized";
@@ -76,17 +75,19 @@ class PendingDeletes {
       // SegmentReader sharing the current liveDocs
       // instance; must now make a private clone so we can
       // change it:
-      LiveDocsFormat liveDocsFormat = info.info.getCodec().liveDocsFormat();
-      MutableBits mutableBits;
-      if (liveDocs == null) {
-        mutableBits = liveDocsFormat.newLiveDocs(info.info.maxDoc());
-      } else {
-        mutableBits = liveDocsFormat.newLiveDocs(liveDocs);
+      FixedBitSet mutableBits = new FixedBitSet(info.info.maxDoc());
+      mutableBits.set(0, info.info.maxDoc());
+      if (liveDocs != null) {
+        for (int i = 0; i < liveDocs.length(); ++i) {
+          if (liveDocs.get(i) == false) {
+            mutableBits.clear(i);
+          }
+        }
       }
       liveDocs = mutableBits;
       liveDocsShared = false;
     }
-    return (MutableBits) liveDocs;
+    return (FixedBitSet) liveDocs;
   }
 
 
@@ -96,7 +97,7 @@ class PendingDeletes {
    */
   boolean delete(int docID) throws IOException {
     assert info.info.maxDoc() > 0;
-    MutableBits mutableBits = getMutableBits();
+    FixedBitSet mutableBits = getMutableBits();
     assert mutableBits != null;
     assert docID >= 0 && docID < mutableBits.length() : "out of bounds: docid=" + docID + " liveDocsLength=" + mutableBits.length() + " seg=" + info.info.name + " maxDoc=" + info.info.maxDoc();
     assert !liveDocsShared;
@@ -202,7 +203,7 @@ class PendingDeletes {
     boolean success = false;
     try {
       Codec codec = info.info.getCodec();
-      codec.liveDocsFormat().writeLiveDocs((MutableBits)liveDocs, trackingDir, info, pendingDeleteCount, IOContext.DEFAULT);
+      codec.liveDocsFormat().writeLiveDocs(liveDocs, trackingDir, info, pendingDeleteCount, IOContext.DEFAULT);
       success = true;
     } finally {
       if (!success) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8dc69428/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java b/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java
index 41eebd2..3809304 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java
@@ -26,9 +26,9 @@ import org.apache.lucene.search.DocValuesFieldExistsQuery;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.IOSupplier;
 import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.MutableBits;
 
 final class PendingSoftDeletes extends PendingDeletes {
 
@@ -50,7 +50,7 @@ final class PendingSoftDeletes extends PendingDeletes {
 
   @Override
   boolean delete(int docID) throws IOException {
-    MutableBits mutableBits = getMutableBits(); // we need to fetch this first it might be a shared instance with hardDeletes
+    FixedBitSet mutableBits = getMutableBits(); // we need to fetch this first it might be a shared instance with hardDeletes
     if (hardDeletes.delete(docID)) {
       if (mutableBits.get(docID)) { // delete it here too!
         mutableBits.clear(docID);
@@ -105,7 +105,7 @@ final class PendingSoftDeletes extends PendingDeletes {
    * @param bits the bit set to apply the deletes to
    * @return the number of bits changed by this function
    */
-  static int applySoftDeletes(DocIdSetIterator iterator, MutableBits bits) throws IOException {
+  static int applySoftDeletes(DocIdSetIterator iterator, FixedBitSet bits) throws IOException {
     assert iterator != null;
     int newDeletes = 0;
     int docID;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8dc69428/lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java b/lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java
index 4cd020e..f569701 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SegmentWriteState.java
@@ -21,8 +21,8 @@ import org.apache.lucene.codecs.PostingsFormat; // javadocs
 import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; // javadocs
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
+import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.InfoStream;
-import org.apache.lucene.util.MutableBits;
 
 /**
  * Holder class for common parameters used during write.
@@ -56,9 +56,9 @@ public class SegmentWriteState {
    */
   public final BufferedUpdates segUpdates;
 
-  /** {@link MutableBits} recording live documents; this is
+  /** {@link FixedBitSet} recording live documents; this is
    *  only set if there is one or more deleted documents. */
-  public MutableBits liveDocs;
+  public FixedBitSet liveDocs;
 
   /** Unique suffix for any postings files written for this
    *  segment.  {@link PerFieldPostingsFormat} sets this for

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8dc69428/lucene/core/src/java/org/apache/lucene/util/BitSet.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/BitSet.java b/lucene/core/src/java/org/apache/lucene/util/BitSet.java
index b86219b..a07fc43 100644
--- a/lucene/core/src/java/org/apache/lucene/util/BitSet.java
+++ b/lucene/core/src/java/org/apache/lucene/util/BitSet.java
@@ -25,7 +25,7 @@ import org.apache.lucene.search.DocIdSetIterator;
  * Base implementation for a bit set.
  * @lucene.internal
  */
-public abstract class BitSet implements MutableBits, Accountable {
+public abstract class BitSet implements Bits, Accountable {
 
   /** Build a {@link BitSet} from the content of the provided {@link DocIdSetIterator}.
    *  NOTE: this will fully consume the {@link DocIdSetIterator}. */
@@ -45,6 +45,9 @@ public abstract class BitSet implements MutableBits, Accountable {
   /** Set the bit at <code>i</code>. */
   public abstract void set(int i);
 
+  /** Clear the bit at <code>i</code>. */
+  public abstract void clear(int i);
+
   /** Clears a range of bits.
    *
    * @param startIndex lower index

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8dc69428/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 143b10b..f9804d2 100644
--- a/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java
+++ b/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java
@@ -31,7 +31,7 @@ import org.apache.lucene.search.DocIdSetIterator;
  * 
  * @lucene.internal
  */
-public final class FixedBitSet extends BitSet implements MutableBits, Accountable {
+public final class FixedBitSet extends BitSet implements Bits, Accountable {
 
   private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(FixedBitSet.class);
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8dc69428/lucene/core/src/java/org/apache/lucene/util/MutableBits.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/MutableBits.java b/lucene/core/src/java/org/apache/lucene/util/MutableBits.java
deleted file mode 100644
index 3927cac..0000000
--- a/lucene/core/src/java/org/apache/lucene/util/MutableBits.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * 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;
-
-
-/**
- * Extension of Bits for live documents.
- */
-public interface MutableBits extends Bits {
-  /** 
-   * Sets the bit specified by <code>index</code> to false. 
-   * @param index index, should be non-negative and &lt; {@link #length()}.
-   *        The result of passing negative or out of bounds values is undefined
-   *        by this interface, <b>just don't do it!</b>
-   */
-  public void clear(int index);
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8dc69428/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50LiveDocsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50LiveDocsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50LiveDocsFormat.java
new file mode 100644
index 0000000..eead6b2
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene50/TestLucene50LiveDocsFormat.java
@@ -0,0 +1,30 @@
+/*
+ * 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.codecs.lucene50;
+
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.index.BaseLiveDocsFormatTestCase;
+import org.apache.lucene.util.TestUtil;
+
+public class TestLucene50LiveDocsFormat extends BaseLiveDocsFormatTestCase {
+
+  @Override
+  protected Codec getCodec() {
+    return TestUtil.getDefaultCodec();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8dc69428/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingLiveDocsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingLiveDocsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingLiveDocsFormat.java
index e02164b..127437b 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingLiveDocsFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingLiveDocsFormat.java
@@ -24,7 +24,6 @@ import org.apache.lucene.index.SegmentCommitInfo;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.MutableBits;
 import org.apache.lucene.util.TestUtil;
 
 /**
@@ -34,31 +33,6 @@ public class AssertingLiveDocsFormat extends LiveDocsFormat {
   private final LiveDocsFormat in = TestUtil.getDefaultCodec().liveDocsFormat();
 
   @Override
-  public MutableBits newLiveDocs(int size) throws IOException {
-    assert size >= 0;
-    MutableBits raw = in.newLiveDocs(size);
-    assert raw != null;
-    assert raw.length() == size;
-    for (int i = 0; i < raw.length(); i++) {
-      assert raw.get(i);
-    }
-    return new AssertingMutableBits(raw);
-  }
-
-  @Override
-  public MutableBits newLiveDocs(Bits existing) throws IOException {
-    assert existing instanceof AssertingBits;
-    Bits rawExisting = ((AssertingBits)existing).in;
-    MutableBits raw = in.newLiveDocs(rawExisting);
-    assert raw != null;
-    assert raw.length() == rawExisting.length();
-    for (int i = 0; i < raw.length(); i++) {
-      assert rawExisting.get(i) == raw.get(i);
-    }
-    return new AssertingMutableBits(raw);
-  }
-
-  @Override
   public Bits readLiveDocs(Directory dir, SegmentCommitInfo info, IOContext context) throws IOException {
     Bits raw = in.readLiveDocs(dir, info, context);
     assert raw != null;
@@ -67,11 +41,9 @@ public class AssertingLiveDocsFormat extends LiveDocsFormat {
   }
 
   @Override
-  public void writeLiveDocs(MutableBits bits, Directory dir, SegmentCommitInfo info, int newDelCount, IOContext context) throws IOException {
-    assert bits instanceof AssertingMutableBits;
-    MutableBits raw = (MutableBits) ((AssertingMutableBits)bits).in;
-    check(raw, info.info.maxDoc(), info.getDelCount() + newDelCount);
-    in.writeLiveDocs(raw, dir, info, newDelCount, context);
+  public void writeLiveDocs(Bits bits, Directory dir, SegmentCommitInfo info, int newDelCount, IOContext context) throws IOException {
+    check(bits, info.info.maxDoc(), info.getDelCount() + newDelCount);
+    in.writeLiveDocs(bits, dir, info, newDelCount, context);
   }
   
   private void check(Bits bits, int expectedLength, int expectedDeleteCount) {
@@ -120,17 +92,5 @@ public class AssertingLiveDocsFormat extends LiveDocsFormat {
       return "Asserting(" + in + ")";
     }
   }
-  
-  static class AssertingMutableBits extends AssertingBits implements MutableBits {   
-    AssertingMutableBits(MutableBits in) {
-      super(in);
-    }
 
-    @Override
-    public void clear(int index) {
-      assert index >= 0;
-      assert index < in.length();
-      ((MutableBits)in).clear(index);
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8dc69428/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyLiveDocsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyLiveDocsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyLiveDocsFormat.java
index e879524..027050e 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyLiveDocsFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyLiveDocsFormat.java
@@ -25,7 +25,6 @@ import org.apache.lucene.index.SegmentCommitInfo;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.MutableBits;
 
 class CrankyLiveDocsFormat extends LiveDocsFormat {
   final LiveDocsFormat delegate;
@@ -37,22 +36,12 @@ class CrankyLiveDocsFormat extends LiveDocsFormat {
   }
 
   @Override
-  public MutableBits newLiveDocs(int size) throws IOException {
-    return delegate.newLiveDocs(size);
-  }
-
-  @Override
-  public MutableBits newLiveDocs(Bits existing) throws IOException {
-    return delegate.newLiveDocs(existing);
-  }
-
-  @Override
   public Bits readLiveDocs(Directory dir, SegmentCommitInfo info, IOContext context) throws IOException {
     return delegate.readLiveDocs(dir, info, context);
   }
 
   @Override
-  public void writeLiveDocs(MutableBits bits, Directory dir, SegmentCommitInfo info, int newDelCount, IOContext context) throws IOException {
+  public void writeLiveDocs(Bits bits, Directory dir, SegmentCommitInfo info, int newDelCount, IOContext context) throws IOException {
     if (random.nextInt(100) == 0) {
       throw new IOException("Fake IOException from LiveDocsFormat.writeLiveDocs()");
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8dc69428/lucene/test-framework/src/java/org/apache/lucene/index/BaseLiveDocsFormatTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseLiveDocsFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseLiveDocsFormatTestCase.java
new file mode 100644
index 0000000..7e0af18
--- /dev/null
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseLiveDocsFormatTestCase.java
@@ -0,0 +1,139 @@
+/*
+ * 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.index;
+
+import java.io.IOException;
+import java.util.Collections;
+
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.LiveDocsFormat;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.StringHelper;
+import org.apache.lucene.util.TestUtil;
+import org.apache.lucene.util.Version;
+
+/**
+ * Abstract class that performs basic testing of a codec's
+ * {@link LiveDocsFormat}.
+ */
+public abstract class BaseLiveDocsFormatTestCase extends LuceneTestCase {
+
+  /** Returns the codec to run tests against */
+  protected abstract Codec getCodec();
+
+  private Codec savedCodec;
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    // set the default codec, so adding test cases to this isn't fragile
+    savedCodec = Codec.getDefault();
+    Codec.setDefault(getCodec());
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    Codec.setDefault(savedCodec); // restore
+    super.tearDown();
+  }
+
+  public void testDenseLiveDocs() throws IOException {
+    final int maxDoc = TestUtil.nextInt(random(), 3, 1000);
+    testSerialization(maxDoc, maxDoc - 1, false);
+    testSerialization(maxDoc, maxDoc - 1, true);
+  }
+
+  public void testEmptyLiveDocs() throws IOException {
+    final int maxDoc = TestUtil.nextInt(random(), 3, 1000);
+    testSerialization(maxDoc, 0, false);
+    testSerialization(maxDoc, 0, true);
+  }
+
+  public void testSparseLiveDocs() throws IOException {
+    final int maxDoc = TestUtil.nextInt(random(), 3, 1000);
+    testSerialization(maxDoc, 1, false);
+    testSerialization(maxDoc, 1, true);
+  }
+
+  @Nightly
+  public void testOverflow() throws IOException {
+    testSerialization(IndexWriter.MAX_DOCS, IndexWriter.MAX_DOCS - 7, false);
+  }
+
+  private void testSerialization(int maxDoc, int numLiveDocs, boolean fixedBitSet) throws IOException {
+    final Codec codec = Codec.getDefault();
+    final LiveDocsFormat format = codec.liveDocsFormat();
+
+    final FixedBitSet liveDocs = new FixedBitSet(maxDoc);
+    if (numLiveDocs > maxDoc / 2) {
+      liveDocs.set(0, maxDoc);
+      for (int i = 0; i < maxDoc - numLiveDocs; ++i) {
+        int clearBit;
+        do {
+          clearBit = random().nextInt(maxDoc);
+        } while (liveDocs.get(clearBit) == false);
+        liveDocs.clear(clearBit);
+      }
+    } else {
+      for (int i = 0; i < numLiveDocs; ++i) {
+        int setBit;
+        do {
+          setBit = random().nextInt(maxDoc);
+        } while (liveDocs.get(setBit));
+        liveDocs.set(setBit);
+      }
+    }
+
+    final Bits bits;
+    if (fixedBitSet) {
+      bits = liveDocs;
+    } else {
+      // Make sure the impl doesn't only work with a FixedBitSet
+      bits = new Bits() {
+
+        @Override
+        public boolean get(int index) {
+          return liveDocs.get(index);
+        }
+
+        @Override
+        public int length() {
+          return liveDocs.length();
+        }
+
+      };
+    }
+
+    final Directory dir = newDirectory();
+    final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, Version.LATEST, "foo", maxDoc, random().nextBoolean(),
+        codec, Collections.emptyMap(), StringHelper.randomId(), Collections.emptyMap(), null);
+    SegmentCommitInfo sci = new SegmentCommitInfo(si, 0, 0, -1, -1);
+    format.writeLiveDocs(bits, dir, sci, maxDoc - numLiveDocs, IOContext.DEFAULT);
+
+    sci = new SegmentCommitInfo(si, maxDoc - numLiveDocs, 1, -1, -1);
+    final Bits bits2 = format.readLiveDocs(dir, sci, IOContext.READONCE);
+    assertEquals(maxDoc, bits2.length());
+    for (int i = 0; i < maxDoc; ++i) {
+      assertEquals(bits.get(i), bits2.get(i));
+    }
+    dir.close();
+  }
+}