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 2018/05/15 20:04:31 UTC

[14/39] lucene-solr:jira/solr-11779: LUCENE-8298: Allow DocValues updates to reset a value

LUCENE-8298: Allow DocValues updates to reset a value

Today once a document has a value in a certain DV field this values
can only be changed but not removed. While resetting / removing a value
from a field is certainly a corner case it can be used to undelete a
soft-deleted document unless it's merged away.
This allows to rollback changes without rolling back to another commitpoint
or trashing all uncommitted changes. In certain cenarios it can be used to
"repair" history of documents in distributed systems.


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

Branch: refs/heads/jira/solr-11779
Commit: a3c86373e47ddd43368c5800eea63b50d2064759
Parents: 7873cf8
Author: Simon Willnauer <si...@apache.org>
Authored: Mon May 7 11:52:51 2018 +0200
Committer: Simon Willnauer <si...@apache.org>
Committed: Wed May 9 18:57:57 2018 +0200

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   5 +
 .../simpletext/SimpleTextLiveDocsFormat.java    |   1 -
 .../lucene/document/NumericDocValuesField.java  |  12 +-
 .../lucene/index/DefaultIndexingChain.java      |   3 +
 .../lucene/index/DocValuesFieldUpdates.java     |  45 +++++++-
 .../apache/lucene/index/DocValuesUpdate.java    |  46 ++++++--
 .../lucene/index/FrozenBufferedUpdates.java     |  21 +++-
 .../org/apache/lucene/index/IndexWriter.java    |  42 ++++---
 .../org/apache/lucene/index/PendingDeletes.java |  13 +--
 .../apache/lucene/index/PendingSoftDeletes.java |  19 +++-
 .../apache/lucene/index/ReadersAndUpdates.java  |  53 +++++----
 .../org/apache/lucene/util/FixedBitSet.java     |  19 ++++
 .../org/apache/lucene/index/TestDocValues.java  |  17 +++
 .../lucene/index/TestMixedDocValuesUpdates.java | 110 +++++++++++++++++--
 .../lucene/index/TestPendingSoftDeletes.java    |  74 ++++++++++++-
 .../TestSoftDeletesRetentionMergePolicy.java    |  89 +++++++++++++++
 .../org/apache/lucene/util/TestFixedBitSet.java |  30 +++++
 .../asserting/AssertingLiveDocsFormat.java      |  15 ++-
 18 files changed, 522 insertions(+), 92 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a3c86373/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 7467093..baede74 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -167,6 +167,11 @@ New Features
   fields for individual documents. This allows for instance to soft-delete
   individual documents. (Simon Willnauer)
 
+* LUCENE-8298: Allow DocValues updates to reset a value. Passing a DV field with a null
+  value to IW#updateDocValues or IW#tryUpdateDocValues will now remove the value from the 
+  provided document. This allows to undelete a soft-deleted document unless it's been claimed
+  by a merge. (Simon Willnauer) 
+
 Bug Fixes
 
 * LUCENE-8266: Detect bogus tiles when creating a standard polygon and

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a3c86373/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 d158ede..2754fcb 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
@@ -156,5 +156,4 @@ public class SimpleTextLiveDocsFormat extends LiveDocsFormat {
       return size;
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a3c86373/lucene/core/src/java/org/apache/lucene/document/NumericDocValuesField.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/NumericDocValuesField.java b/lucene/core/src/java/org/apache/lucene/document/NumericDocValuesField.java
index 89210db..0b20ca9 100644
--- a/lucene/core/src/java/org/apache/lucene/document/NumericDocValuesField.java
+++ b/lucene/core/src/java/org/apache/lucene/document/NumericDocValuesField.java
@@ -59,8 +59,18 @@ public class NumericDocValuesField extends Field {
    * @throws IllegalArgumentException if the field name is null
    */
   public NumericDocValuesField(String name, long value) {
+    this(name, Long.valueOf(value));
+  }
+
+  /**
+   * Creates a new DocValues field with the specified 64-bit long value
+   * @param name field name
+   * @param value 64-bit long value or <code>null</code> if the existing fields value should be removed on update
+   * @throws IllegalArgumentException if the field name is null
+   */
+  public NumericDocValuesField(String name, Long value) {
     super(name, TYPE);
-    fieldsData = Long.valueOf(value);
+    fieldsData = value;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a3c86373/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java b/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
index 4541f4a..705d7bc 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
@@ -542,6 +542,9 @@ final class DefaultIndexingChain extends DocConsumer {
         if (fp.docValuesWriter == null) {
           fp.docValuesWriter = new NumericDocValuesWriter(fp.fieldInfo, bytesUsed);
         }
+        if (field.numericValue() == null) {
+          throw new IllegalArgumentException("field=\"" + fp.fieldInfo.name + "\": null value not allowed");
+        }
         ((NumericDocValuesWriter) fp.docValuesWriter).addValue(docID, field.numericValue().longValue());
         break;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a3c86373/lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java b/lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java
index f4e15f0..093a428 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java
@@ -35,6 +35,9 @@ import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
 abstract class DocValuesFieldUpdates implements Accountable {
   
   protected static final int PAGE_SIZE = 1024;
+  private static final long HAS_VALUE_MASK = 1;
+  private static final long HAS_NO_VALUE_MASK = 0;
+  private static final int SHIFT = 1; // we use the first bit of each value to mark if the doc has a value or not
 
   /**
    * An iterator over documents and their updated values. Only documents with
@@ -74,6 +77,11 @@ abstract class DocValuesFieldUpdates implements Accountable {
     abstract long delGen();
 
     /**
+     * Returns true if this doc has a value
+     */
+    abstract boolean hasValue();
+
+    /**
      * Wraps the given iterator as a BinaryDocValues instance.
      */
     static BinaryDocValues asBinaryDocValues(Iterator iterator) {
@@ -217,6 +225,11 @@ abstract class DocValuesFieldUpdates implements Accountable {
       public long delGen() {
         throw new UnsupportedOperationException();
       }
+
+      @Override
+      boolean hasValue() {
+        return queue.top().hasValue();
+      }
     };
   }
 
@@ -237,7 +250,7 @@ abstract class DocValuesFieldUpdates implements Accountable {
       throw new NullPointerException("DocValuesType must not be null");
     }
     this.type = type;
-    bitsPerValue = PackedInts.bitsRequired(maxDoc - 1);
+    bitsPerValue = PackedInts.bitsRequired(maxDoc - 1) + SHIFT;
     docs = new PagedMutable(1, PAGE_SIZE, bitsPerValue, PackedInts.COMPACT);
   }
 
@@ -299,7 +312,18 @@ abstract class DocValuesFieldUpdates implements Accountable {
     return size;
   }
 
+  /**
+   * Adds an update that resets the documents value.
+   * @param doc the doc to update
+   */
+  final synchronized void reset(int doc) {
+    addInternal(doc, HAS_NO_VALUE_MASK);
+  }
   final synchronized int add(int doc) {
+    return addInternal(doc, HAS_VALUE_MASK);
+  }
+
+  private synchronized int addInternal(int doc, long hasValueMask) {
     if (finished) {
       throw new IllegalStateException("already finished");
     }
@@ -313,8 +337,7 @@ abstract class DocValuesFieldUpdates implements Accountable {
     if (docs.size() == size) {
       grow(size+1);
     }
-
-    docs.set(size, doc);
+    docs.set(size, (((long)doc) << SHIFT) | hasValueMask);
     ++size;
     return size-1;
   }
@@ -354,6 +377,7 @@ abstract class DocValuesFieldUpdates implements Accountable {
     private long idx = 0; // long so we don't overflow if size == Integer.MAX_VALUE
     private int doc = -1;
     private final long delGen;
+    private boolean hasValue;
 
     AbstractIterator(int size, PagedMutable docs, long delGen) {
       this.size = size;
@@ -366,13 +390,17 @@ abstract class DocValuesFieldUpdates implements Accountable {
       if (idx >= size) {
         return doc = DocIdSetIterator.NO_MORE_DOCS;
       }
-      doc = (int) docs.get(idx);
+      long longDoc = docs.get(idx);
       ++idx;
-      while (idx < size && docs.get(idx) == doc) {
+      while (idx < size && docs.get(idx) == longDoc) {
         // scan forward to last update to this doc
         ++idx;
       }
-      set(idx-1);
+      hasValue = (longDoc & HAS_VALUE_MASK) >  0;
+      if (hasValue) {
+        set(idx - 1);
+      }
+      doc = (int)(longDoc >> SHIFT);
       return doc;
     }
 
@@ -391,5 +419,10 @@ abstract class DocValuesFieldUpdates implements Accountable {
     final long delGen() {
       return delGen;
     }
+
+    @Override
+    final boolean hasValue() {
+      return hasValue;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a3c86373/lucene/core/src/java/org/apache/lucene/index/DocValuesUpdate.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocValuesUpdate.java b/lucene/core/src/java/org/apache/lucene/index/DocValuesUpdate.java
index c8bc8fb..a818def 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocValuesUpdate.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocValuesUpdate.java
@@ -46,6 +46,7 @@ abstract class DocValuesUpdate {
   // used in BufferedDeletes to apply this update only to a slice of docs. It's initialized to BufferedUpdates.MAX_INT
   // since it's safe and most often used this way we safe object creations.
   final int docIDUpto;
+  final boolean hasValue;
 
   /**
    * Constructor.
@@ -53,12 +54,13 @@ abstract class DocValuesUpdate {
    * @param term the {@link Term} which determines the documents that will be updated
    * @param field the {@link NumericDocValuesField} to update
    */
-  protected DocValuesUpdate(DocValuesType type, Term term, String field, int docIDUpto) {
+  protected DocValuesUpdate(DocValuesType type, Term term, String field, int docIDUpto, boolean hasValue) {
     assert docIDUpto >= 0 : docIDUpto + "must be >= 0";
     this.type = type;
     this.term = term;
     this.field = field;
     this.docIDUpto = docIDUpto;
+    this.hasValue = hasValue;
   }
 
   abstract long valueSizeInBytes();
@@ -69,12 +71,17 @@ abstract class DocValuesUpdate {
     sizeInBytes += term.bytes.bytes.length;
     sizeInBytes += field.length() * Character.BYTES;
     sizeInBytes += valueSizeInBytes();
+    sizeInBytes += 1; // hasValue
     return sizeInBytes;
   }
 
   protected abstract String valueToString();
 
   abstract void writeTo(DataOutput output) throws IOException;
+
+  boolean hasValue() {
+    return hasValue;
+  }
   
   @Override
   public String toString() {
@@ -83,8 +90,8 @@ abstract class DocValuesUpdate {
   
   /** An in-place update to a binary DocValues field */
   static final class BinaryDocValuesUpdate extends DocValuesUpdate {
-    final BytesRef value;
-    
+    private final BytesRef value;
+
     /* Size of BytesRef: 2*INT + ARRAY_HEADER + PTR */
     private static final long RAW_VALUE_SIZE_IN_BYTES = NUM_BYTES_ARRAY_HEADER + 2*Integer.BYTES + NUM_BYTES_OBJECT_REF;
 
@@ -93,7 +100,7 @@ abstract class DocValuesUpdate {
     }
     
     private BinaryDocValuesUpdate(Term term, String field, BytesRef value, int docIDUpTo) {
-      super(DocValuesType.BINARY, term, field, docIDUpTo);
+      super(DocValuesType.BINARY, term, field, docIDUpTo, value != null);
       this.value = value;
     }
 
@@ -106,7 +113,7 @@ abstract class DocValuesUpdate {
 
     @Override
     long valueSizeInBytes() {
-      return RAW_VALUE_SIZE_IN_BYTES + value.bytes.length;
+      return RAW_VALUE_SIZE_IN_BYTES + (value == null ? 0 : value.bytes.length);
     }
 
     @Override
@@ -114,8 +121,14 @@ abstract class DocValuesUpdate {
       return value.toString();
     }
 
+    BytesRef getValue() {
+      assert hasValue : "getValue should only be called if this update has a value";
+      return value;
+    }
+
     @Override
     void writeTo(DataOutput out) throws IOException {
+      assert hasValue;
       out.writeVInt(value.length);
       out.writeBytes(value.bytes, value.offset, value.length);
     }
@@ -132,14 +145,19 @@ abstract class DocValuesUpdate {
 
   /** An in-place update to a numeric DocValues field */
   static final class NumericDocValuesUpdate extends DocValuesUpdate {
-    final long value;
+    private final long value;
 
     NumericDocValuesUpdate(Term term, String field, long value) {
-      this(term, field, value, BufferedUpdates.MAX_INT);
+      this(term, field, value, BufferedUpdates.MAX_INT, true);
+    }
+
+    NumericDocValuesUpdate(Term term, String field, Long value) {
+      this(term, field, value != null ? value.longValue() : -1, BufferedUpdates.MAX_INT, value != null);
     }
 
-    private NumericDocValuesUpdate(Term term, String field, long value, int docIDUpTo) {
-      super(DocValuesType.NUMERIC, term, field, docIDUpTo);
+
+    private NumericDocValuesUpdate(Term term, String field, long value, int docIDUpTo, boolean hasValue) {
+      super(DocValuesType.NUMERIC, term, field, docIDUpTo, hasValue);
       this.value = value;
     }
 
@@ -147,7 +165,7 @@ abstract class DocValuesUpdate {
       if (docIDUpto == this.docIDUpto) {
         return this;
       }
-      return new NumericDocValuesUpdate(term, field, value, docIDUpto);
+      return new NumericDocValuesUpdate(term, field, value, docIDUpto, hasValue);
     }
 
     @Override
@@ -157,16 +175,22 @@ abstract class DocValuesUpdate {
 
     @Override
     protected String valueToString() {
-      return Long.toString(value);
+      return hasValue ? Long.toString(value) : "null";
     }
 
     @Override
     void writeTo(DataOutput out) throws IOException {
+      assert hasValue;
       out.writeZLong(value);
     }
 
     static long readFrom(DataInput in) throws IOException {
       return in.readZLong();
     }
+
+    long getValue() {
+      assert hasValue : "getValue should only be called if this update has a value";
+      return value;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a3c86373/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java b/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java
index 6e4b6bd..c69f212 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java
@@ -157,7 +157,7 @@ final class FrozenBufferedUpdates {
       for (LinkedHashMap<Term, T> updates : dvUpdates.values()) {
         updateSizeConsumer.accept(updates.size());
         for (T update : updates.values()) {
-          int code = update.term.bytes().length << 2;
+          int code = update.term.bytes().length << 3;
 
           String termField = update.term.field();
           if (termField.equals(lastTermField) == false) {
@@ -167,6 +167,9 @@ final class FrozenBufferedUpdates {
           if (updateField.equals(lastUpdateField) == false) {
             code |= 2;
           }
+          if (update.hasValue()) {
+            code |= 4;
+          }
           out.writeVInt(code);
           out.writeVInt(update.docIDUpto);
           if (termField.equals(lastTermField) == false) {
@@ -178,7 +181,9 @@ final class FrozenBufferedUpdates {
             lastUpdateField = updateField;
           }
           out.writeBytes(update.term.bytes().bytes, update.term.bytes().offset, update.term.bytes().length);
-          update.writeTo(out);
+          if (update.hasValue()) {
+            update.writeTo(out);
+          }
         }
       }
       byte[] bytes = new byte[(int) out.getFilePointer()];
@@ -538,7 +543,7 @@ final class FrozenBufferedUpdates {
     while (in.getPosition() != updates.length) {
       int code = in.readVInt();
       int docIDUpto = in.readVInt();
-      term.length = code >> 2;
+      term.length = code >> 3;
       
       if ((code & 1) != 0) {
         termField = in.readString();
@@ -546,6 +551,7 @@ final class FrozenBufferedUpdates {
       if ((code & 2) != 0) {
         updateField = in.readString();
       }
+      boolean hasValue = (code & 4) != 0;
 
       if (term.bytes.length < term.length) {
         term.bytes = ArrayUtil.grow(term.bytes, term.length);
@@ -584,7 +590,10 @@ final class FrozenBufferedUpdates {
 
       final BytesRef binaryValue;
       final long longValue;
-      if (isNumeric) {
+      if (hasValue == false) {
+        longValue = -1;
+        binaryValue = null;
+      } else if (isNumeric) {
         longValue = NumericDocValuesUpdate.readFrom(in);
         binaryValue = null;
       } else {
@@ -611,7 +620,9 @@ final class FrozenBufferedUpdates {
         }
         final IntConsumer docIdConsumer;
         final DocValuesFieldUpdates update = dvUpdates;
-        if (isNumeric) {
+        if (hasValue == false) {
+          docIdConsumer = doc -> update.reset(doc);
+        } else if (isNumeric) {
           docIdConsumer = doc -> update.add(doc, longValue);
         } else {
           docIdConsumer = doc -> update.add(doc, binaryValue);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a3c86373/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
index d8ef5c0..0a57123 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -1370,6 +1370,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
    *  the update succeeds and this method returns a valid (&gt; 0) sequence
    *  number; else, it returns -1 and the caller must then
    *  either retry the update and resolve the document again.
+   *  If a doc values fields data is <code>null</code> the existing
+   *  value is removed from all documents matching the term. This can be used
+   *  to un-delete a soft-deleted document since this method will apply the
+   *  field update even if the document is marked as deleted.
    *
    *  <b>NOTE</b>: this method can only updates documents
    *  visible to the currently open NRT reader.  If you need
@@ -1393,15 +1397,19 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
                 throw new AssertionError("type: " + update.type + " is not supported");
             }
           });
-          switch (update.type) {
-            case NUMERIC:
-              docValuesFieldUpdates.add(leafDocId, ((NumericDocValuesUpdate) update).value);
-              break;
-            case BINARY:
-              docValuesFieldUpdates.add(leafDocId, ((BinaryDocValuesUpdate) update).value);
-              break;
-            default:
-              throw new AssertionError("type: " + update.type + " is not supported");
+          if (update.hasValue()) {
+            switch (update.type) {
+              case NUMERIC:
+                docValuesFieldUpdates.add(leafDocId, ((NumericDocValuesUpdate) update).getValue());
+                break;
+              case BINARY:
+                docValuesFieldUpdates.add(leafDocId, ((BinaryDocValuesUpdate) update).getValue());
+                break;
+              default:
+                throw new AssertionError("type: " + update.type + " is not supported");
+            }
+          } else {
+            docValuesFieldUpdates.reset(leafDocId);
           }
         }
         for (DocValuesFieldUpdates updates : fieldUpdatesMap.values()) {
@@ -1726,7 +1734,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
    * Updates documents' DocValues fields to the given values. Each field update
    * is applied to the set of documents that are associated with the
    * {@link Term} to the same value. All updates are atomically applied and
-   * flushed together.
+   * flushed together. If a doc values fields data is <code>null</code> the existing
+   * value is removed from all documents matching the term.
+   *
    * 
    * @param updates
    *          the updates to apply
@@ -1772,9 +1782,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
       if (config.getIndexSortFields().contains(f.name())) {
         throw new IllegalArgumentException("cannot update docvalues field involved in the index sort, field=" + f.name() + ", sort=" + config.getIndexSort());
       }
+
       switch (dvType) {
         case NUMERIC:
-          dvUpdates[i] = new NumericDocValuesUpdate(term, f.name(), (Long) f.numericValue());
+          Long value = (Long)f.numericValue();
+          dvUpdates[i] = new NumericDocValuesUpdate(term, f.name(), value);
           break;
         case BINARY:
           dvUpdates[i] = new BinaryDocValuesUpdate(term, f.name(), f.binaryValue());
@@ -3740,8 +3752,12 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
           while ((doc = it.nextDoc()) != NO_MORE_DOCS) {
             int mappedDoc = segDocMap.get(segLeafDocMap.get(doc));
             if (mappedDoc != -1) {
-              // not deleted
-              mappedUpdates.add(mappedDoc, it);
+              if (it.hasValue()) {
+                // not deleted
+                mappedUpdates.add(mappedDoc, it);
+              } else {
+                mappedUpdates.reset(mappedDoc);
+              }
               anyDVUpdates = true;
             }
           }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a3c86373/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 4bd90cb..8e3b7e0 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java
@@ -61,7 +61,7 @@ class PendingDeletes {
   }
 
 
-  protected FixedBitSet getMutableBits() throws IOException {
+  protected FixedBitSet getMutableBits() {
     // 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";
@@ -70,14 +70,11 @@ class PendingDeletes {
       // SegmentReader sharing the current liveDocs
       // instance; must now make a private clone so we can
       // change it:
-      writeableLiveDocs = new FixedBitSet(info.info.maxDoc());
-      writeableLiveDocs.set(0, info.info.maxDoc());
       if (liveDocs != null) {
-        for (int i = 0; i < liveDocs.length(); ++i) {
-          if (liveDocs.get(i) == false) {
-            writeableLiveDocs.clear(i);
-          }
-        }
+        writeableLiveDocs = FixedBitSet.copyOf(liveDocs);
+      } else {
+        writeableLiveDocs = new FixedBitSet(info.info.maxDoc());
+        writeableLiveDocs.set(0, info.info.maxDoc());
       }
       liveDocs = writeableLiveDocs;
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a3c86373/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 637a90c..fe012f6 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java
@@ -109,12 +109,21 @@ final class PendingSoftDeletes extends PendingDeletes {
     assert iterator != null;
     int newDeletes = 0;
     int docID;
+    DocValuesFieldUpdates.Iterator hasValue = iterator instanceof DocValuesFieldUpdates.Iterator
+        ? (DocValuesFieldUpdates.Iterator) iterator : null;
     while ((docID = iterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
-      if (bits.get(docID)) { // doc is live - clear it
-        bits.clear(docID);
-        newDeletes++;
-        // now that we know we deleted it and we fully control the hard deletes we can do correct accounting
-        // below.
+      if (hasValue == null || hasValue.hasValue()) {
+        if (bits.get(docID)) { // doc is live - clear it
+          bits.clear(docID);
+          newDeletes++;
+          // now that we know we deleted it and we fully control the hard deletes we can do correct accounting
+          // below.
+        }
+      } else {
+        if (bits.get(docID) == false) {
+          bits.set(docID);
+          newDeletes--;
+        }
       }
     }
     return newDeletes;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a3c86373/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java b/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
index 1a96c13..96740a7 100644
--- a/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
+++ b/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
@@ -350,9 +350,10 @@ final class ReadersAndUpdates {
           fieldsConsumer.addBinaryField(fieldInfo, new EmptyDocValuesProducer() {
             @Override
             public BinaryDocValues getBinary(FieldInfo fieldInfoIn) throws IOException {
+              DocValuesFieldUpdates.Iterator iterator = updateSupplier.apply(fieldInfo);
               final MergedDocValues<BinaryDocValues> mergedDocValues = new MergedDocValues<>(
                   reader.getBinaryDocValues(field),
-                  DocValuesFieldUpdates.Iterator.asBinaryDocValues(updateSupplier.apply(fieldInfo)));
+                  DocValuesFieldUpdates.Iterator.asBinaryDocValues(iterator), iterator);
               // Merge sort of the original doc values with updated doc values:
               return new BinaryDocValues() {
                 @Override
@@ -387,9 +388,10 @@ final class ReadersAndUpdates {
           fieldsConsumer.addNumericField(fieldInfo, new EmptyDocValuesProducer() {
             @Override
             public NumericDocValues getNumeric(FieldInfo fieldInfoIn) throws IOException {
+              DocValuesFieldUpdates.Iterator iterator = updateSupplier.apply(fieldInfo);
               final MergedDocValues<NumericDocValues> mergedDocValues = new MergedDocValues<>(
                   reader.getNumericDocValues(field),
-                  DocValuesFieldUpdates.Iterator.asNumericDocValues(updateSupplier.apply(fieldInfo)));
+                  DocValuesFieldUpdates.Iterator.asNumericDocValues(iterator), iterator);
               // Merge sort of the original doc values with updated doc values:
               return new NumericDocValues() {
                 @Override
@@ -433,6 +435,7 @@ final class ReadersAndUpdates {
    * wins over the on-disk version.
    */
   static final class MergedDocValues<DocValuesInstance extends DocValuesIterator> extends DocValuesIterator {
+    private final DocValuesFieldUpdates.Iterator updateIterator;
     // merged docID
     private int docIDOut = -1;
     // docID from our original doc values
@@ -444,9 +447,10 @@ final class ReadersAndUpdates {
     private final DocValuesInstance updateDocValues;
     DocValuesInstance currentValuesSupplier;
 
-    protected MergedDocValues(DocValuesInstance onDiskDocValues, DocValuesInstance updateDocValues) {
+    protected MergedDocValues(DocValuesInstance onDiskDocValues, DocValuesInstance updateDocValues, DocValuesFieldUpdates.Iterator updateIterator) {
       this.onDiskDocValues = onDiskDocValues;
       this.updateDocValues = updateDocValues;
+      this.updateIterator = updateIterator;
     }
 
     @Override
@@ -471,26 +475,33 @@ final class ReadersAndUpdates {
 
     @Override
     public int nextDoc() throws IOException {
-      if (docIDOnDisk == docIDOut) {
-        if (onDiskDocValues == null) {
-          docIDOnDisk = NO_MORE_DOCS;
-        } else {
-          docIDOnDisk = onDiskDocValues.nextDoc();
+      boolean hasValue = false;
+      do {
+        if (docIDOnDisk == docIDOut) {
+          if (onDiskDocValues == null) {
+            docIDOnDisk = NO_MORE_DOCS;
+          } else {
+            docIDOnDisk = onDiskDocValues.nextDoc();
+          }
         }
-      }
-      if (updateDocID == docIDOut) {
-        updateDocID = updateDocValues.nextDoc();
-      }
-      if (docIDOnDisk < updateDocID) {
-        // no update to this doc - we use the on-disk values
-        docIDOut = docIDOnDisk;
-        currentValuesSupplier = onDiskDocValues;
-      } else {
-        docIDOut = updateDocID;
-        if (docIDOut != NO_MORE_DOCS) {
-          currentValuesSupplier = updateDocValues;
+        if (updateDocID == docIDOut) {
+          updateDocID = updateDocValues.nextDoc();
         }
-      }
+        if (docIDOnDisk < updateDocID) {
+          // no update to this doc - we use the on-disk values
+          docIDOut = docIDOnDisk;
+          currentValuesSupplier = onDiskDocValues;
+          hasValue = true;
+        } else {
+          docIDOut = updateDocID;
+          if (docIDOut != NO_MORE_DOCS) {
+            currentValuesSupplier = updateDocValues;
+            hasValue = updateIterator.hasValue();
+          } else {
+            hasValue = true;
+          }
+        }
+      } while (hasValue == false);
       return docIDOut;
     }
   };

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a3c86373/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 f9804d2..0936488 100644
--- a/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java
+++ b/lucene/core/src/java/org/apache/lucene/util/FixedBitSet.java
@@ -510,4 +510,23 @@ public final class FixedBitSet extends BitSet implements Bits, Accountable {
     // empty sets from returning 0, which is too common.
     return (int) ((h>>32) ^ h) + 0x98761234;
   }
+
+  /**
+   * Make a copy of the given bits.
+   */
+  public static FixedBitSet copyOf(Bits bits) {
+    if (bits instanceof FixedBitSet) {
+      return ((FixedBitSet)bits).clone();
+    } else {
+      int length = bits.length();
+      FixedBitSet bitSet = new FixedBitSet(length);
+      bitSet.set(0, length);
+      for (int i = 0; i < length; ++i) {
+        if (bits.get(i) == false) {
+          bitSet.clear(i);
+        }
+      }
+      return bitSet;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a3c86373/lucene/core/src/test/org/apache/lucene/index/TestDocValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDocValues.java b/lucene/core/src/test/org/apache/lucene/index/TestDocValues.java
index 1f7ef30..0214e54 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestDocValues.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestDocValues.java
@@ -17,6 +17,8 @@
 package org.apache.lucene.index;
 
 
+import java.io.IOException;
+
 import org.apache.lucene.document.BinaryDocValuesField;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
@@ -27,6 +29,7 @@ import org.apache.lucene.document.SortedSetDocValuesField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 
 /** Tests helper methods in DocValues */
@@ -252,4 +255,18 @@ public class TestDocValues extends LuceneTestCase {
     iw.close();
     dir.close();
   }
+
+  public void testAddNullNumericDocValues() throws IOException {
+    Directory dir = newDirectory();
+    IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig(null));
+    Document doc = new Document();
+    if (random().nextBoolean()) {
+      doc.add(new NumericDocValuesField("foo", null));
+    } else {
+      doc.add(new BinaryDocValuesField("foo", null));
+    }
+    IllegalArgumentException iae = expectThrows(IllegalArgumentException.class, () -> iw.addDocument(doc));
+    assertEquals("field=\"foo\": null value not allowed", iae.getMessage());
+    IOUtils.close(iw, dir);
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a3c86373/lucene/core/src/test/org/apache/lucene/index/TestMixedDocValuesUpdates.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestMixedDocValuesUpdates.java b/lucene/core/src/test/org/apache/lucene/index/TestMixedDocValuesUpdates.java
index 401de4d..0927dcd 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestMixedDocValuesUpdates.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestMixedDocValuesUpdates.java
@@ -36,6 +36,9 @@ import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.DocValuesFieldExistsQuery;
+import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.Bits;
@@ -456,11 +459,12 @@ public class TestMixedDocValuesUpdates extends LuceneTestCase {
     IndexWriterConfig conf = newIndexWriterConfig();
     IndexWriter writer = new IndexWriter(dir, conf);
     ReentrantLock[] locks = new ReentrantLock[25 + random().nextInt(50)];
-    int[] values = new int[locks.length];
+    Long[] values = new Long[locks.length];
+
     for (int i = 0; i < locks.length; i++) {
       locks[i] = new ReentrantLock();
       Document doc = new Document();
-      values[i] = random().nextInt();
+      values[i] = random().nextLong();
       doc.add(new StringField("id", Integer.toString(i), Store.NO));
       doc.add(new NumericDocValuesField("value", values[i]));
       writer.addDocument(doc);
@@ -476,7 +480,7 @@ public class TestMixedDocValuesUpdates extends LuceneTestCase {
             int docId = random().nextInt(locks.length);
             locks[docId].lock();
             try {
-              int value = random().nextInt();
+              Long value = rarely() ? null : random().nextLong(); // sometimes reset it
               if (random().nextBoolean()) {
                 writer.updateDocValues(new Term("id", docId + ""), new NumericDocValuesField("value", value));
               } else {
@@ -488,7 +492,6 @@ public class TestMixedDocValuesUpdates extends LuceneTestCase {
             } finally {
               locks[docId].unlock();
             }
-
             if (rarely()) {
               writer.flush();
             }
@@ -508,7 +511,7 @@ public class TestMixedDocValuesUpdates extends LuceneTestCase {
       for (int i = 0; i < locks.length; i++) {
         locks[i].lock();
         try {
-          int value = values[i];
+          Long value = values[i];
           TopDocs topDocs = new IndexSearcher(reader).search(new TermQuery(new Term("id", "" + i)), 10);
           assertEquals(topDocs.totalHits, 1);
           int docID = topDocs.scoreDocs[0].doc;
@@ -517,12 +520,15 @@ public class TestMixedDocValuesUpdates extends LuceneTestCase {
           LeafReader leafReader = leaves.get(subIndex).reader();
           docID -= leaves.get(subIndex).docBase;
           NumericDocValues numericDocValues = leafReader.getNumericDocValues("value");
-          assertEquals(docID, numericDocValues.advance(docID));
-          assertEquals(numericDocValues.longValue(), value);
+          if (value == null) {
+            assertFalse("docID: " + docID, numericDocValues.advanceExact(docID));
+          } else {
+            assertTrue("docID: " + docID, numericDocValues.advanceExact(docID));
+            assertEquals(numericDocValues.longValue(), value.longValue());
+          }
         } finally {
           locks[i].unlock();
         }
-
       }
     }
 
@@ -540,5 +546,93 @@ public class TestMixedDocValuesUpdates extends LuceneTestCase {
       }
     } while (seqId == -1);
   }
+
+  public void testResetValue() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
+    IndexWriter writer = new IndexWriter(dir, conf);
+    Document doc = new Document();
+    doc.add(new StringField("id", "0", Store.NO));
+    doc.add(new NumericDocValuesField("val", 5));
+    doc.add(new BinaryDocValuesField("val-bin", new BytesRef(new byte[] {(byte)5})));
+    writer.addDocument(doc);
+
+    if (random().nextBoolean()) {
+      writer.commit();
+    }
+    try(DirectoryReader reader = writer.getReader()) {
+      assertEquals(1, reader.leaves().size());
+      LeafReader r = reader.leaves().get(0).reader();
+      NumericDocValues ndv = r.getNumericDocValues("val");
+      assertEquals(0, ndv.nextDoc());
+      assertEquals(5, ndv.longValue());
+      assertEquals(DocIdSetIterator.NO_MORE_DOCS, ndv.nextDoc());
+
+      BinaryDocValues bdv = r.getBinaryDocValues("val-bin");
+      assertEquals(0, bdv.nextDoc());
+      assertEquals(new BytesRef(new byte[]{(byte) 5}), bdv.binaryValue());
+      assertEquals(DocIdSetIterator.NO_MORE_DOCS, bdv.nextDoc());
+    }
+
+    writer.updateDocValues(new Term("id", "0"), new BinaryDocValuesField("val-bin", null));
+    try(DirectoryReader reader = writer.getReader()) {
+      assertEquals(1, reader.leaves().size());
+      LeafReader r = reader.leaves().get(0).reader();
+      NumericDocValues ndv = r.getNumericDocValues("val");
+      assertEquals(0, ndv.nextDoc());
+      assertEquals(5, ndv.longValue());
+      assertEquals(DocIdSetIterator.NO_MORE_DOCS, ndv.nextDoc());
+
+      BinaryDocValues bdv = r.getBinaryDocValues("val-bin");
+      assertEquals(DocIdSetIterator.NO_MORE_DOCS, bdv.nextDoc());
+    }
+    IOUtils.close(writer, dir);
+  }
+
+  public void testResetValueMultipleDocs() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
+    IndexWriter writer = new IndexWriter(dir, conf);
+    int numDocs = 10 + random().nextInt(50);
+    int currentSeqId = 0;
+    int[] seqId = new int[5];
+    for (int i = 0; i < numDocs; i++) {
+      Document doc = new Document();
+      int id = random().nextInt(5);
+      seqId[id] = currentSeqId;
+      doc.add(new StringField("id",  "" + id, Store.YES));
+      doc.add(new NumericDocValuesField("seqID", currentSeqId++));
+      doc.add(new NumericDocValuesField("is_live", 1));
+      if (i > 0) {
+        writer.updateDocValues(new Term("id", "" + id), new NumericDocValuesField("is_live", null));
+      }
+      writer.addDocument(doc);
+      if (random().nextBoolean()) {
+        writer.flush();
+      }
+    }
+
+    if (random().nextBoolean()) {
+      writer.commit();
+    }
+    try(DirectoryReader reader = writer.getReader()) {
+      IndexSearcher searcher = new IndexSearcher(reader);
+
+      TopDocs is_live = searcher.search(new DocValuesFieldExistsQuery("is_live"), 5);
+      assertEquals(5, is_live.totalHits);
+      for (ScoreDoc doc : is_live.scoreDocs) {
+        int id = Integer.parseInt(reader.document(doc.doc).get("id"));
+        int i = ReaderUtil.subIndex(doc.doc, reader.leaves());
+        assertTrue(i >= 0);
+        LeafReaderContext leafReaderContext = reader.leaves().get(i);
+        NumericDocValues seqID = leafReaderContext.reader().getNumericDocValues("seqID");
+        assertNotNull(seqID);
+        assertTrue(seqID.advanceExact(doc.doc - leafReaderContext.docBase));
+        assertEquals(seqId[id], seqID.longValue());
+      }
+    }
+    IOUtils.close(writer, dir);
+  }
+  
 }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a3c86373/lucene/core/src/test/org/apache/lucene/index/TestPendingSoftDeletes.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestPendingSoftDeletes.java b/lucene/core/src/test/org/apache/lucene/index/TestPendingSoftDeletes.java
index 903f847..827c6e3 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestPendingSoftDeletes.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestPendingSoftDeletes.java
@@ -118,7 +118,7 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
     writer.close();
     FieldInfo fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, 0, Collections.emptyMap(), 0, 0);
     List<Integer> docsDeleted = Arrays.asList(1, 3, 7, 8, DocIdSetIterator.NO_MORE_DOCS);
-    List<DocValuesFieldUpdates> updates = Arrays.asList(singleUpdate(docsDeleted, 10));
+    List<DocValuesFieldUpdates> updates = Arrays.asList(singleUpdate(docsDeleted, 10, true));
     for (DocValuesFieldUpdates update : updates) {
       deletes.onDocValuesUpdate(fieldInfo, update.iterator());
     }
@@ -135,7 +135,7 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
     assertTrue(deletes.getLiveDocs().get(9));
 
     docsDeleted = Arrays.asList(1, 2, DocIdSetIterator.NO_MORE_DOCS);
-    updates = Arrays.asList(singleUpdate(docsDeleted, 10));
+    updates = Arrays.asList(singleUpdate(docsDeleted, 10, true));
     fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, 1, Collections.emptyMap(), 0, 0);
     for (DocValuesFieldUpdates update : updates) {
       deletes.onDocValuesUpdate(fieldInfo, update.iterator());
@@ -180,7 +180,7 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
     deletes.onNewReader(segmentReader, segmentInfo);
     FieldInfo fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, segmentInfo.getNextDocValuesGen(), Collections.emptyMap(), 0, 0);
     List<Integer> docsDeleted = Arrays.asList(1, DocIdSetIterator.NO_MORE_DOCS);
-    List<DocValuesFieldUpdates> updates = Arrays.asList(singleUpdate(docsDeleted, 3));
+    List<DocValuesFieldUpdates> updates = Arrays.asList(singleUpdate(docsDeleted, 3, true));
     for (DocValuesFieldUpdates update : updates) {
       deletes.onDocValuesUpdate(fieldInfo, update.iterator());
     }
@@ -199,7 +199,66 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
     IOUtils.close(reader, writer, dir);
   }
 
-  private DocValuesFieldUpdates singleUpdate(List<Integer> docsDeleted, int maxDoc) {
+  public void testResetOnUpdate() throws IOException {
+    Directory dir = newDirectory();
+    IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig()
+        .setSoftDeletesField("_soft_deletes")
+        .setMaxBufferedDocs(3) // make sure we write one segment
+        .setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH));
+    Document doc = new Document();
+    doc.add(new StringField("id", "1", Field.Store.YES));
+    writer.softUpdateDocument(new Term("id", "1"), doc,
+        new NumericDocValuesField("_soft_deletes", 1));
+    doc = new Document();
+    doc.add(new StringField("id", "2", Field.Store.YES));
+    writer.softUpdateDocument(new Term("id", "2"), doc,
+        new NumericDocValuesField("_soft_deletes", 1));
+    doc = new Document();
+    doc.add(new StringField("id", "2", Field.Store.YES));
+    writer.softUpdateDocument(new Term("id", "2"), doc,
+        new NumericDocValuesField("_soft_deletes", 1));
+    writer.commit();
+    DirectoryReader reader = writer.getReader();
+    assertEquals(1, reader.leaves().size());
+    SegmentReader segmentReader = (SegmentReader) reader.leaves().get(0).reader();
+    SegmentCommitInfo segmentInfo = segmentReader.getSegmentInfo();
+    PendingDeletes deletes = newPendingDeletes(segmentInfo);
+    deletes.onNewReader(segmentReader, segmentInfo);
+    FieldInfo fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, segmentInfo.getNextDocValuesGen(), Collections.emptyMap(), 0, 0);
+    List<DocValuesFieldUpdates> updates = Arrays.asList(singleUpdate(Arrays.asList(0, 1, DocIdSetIterator.NO_MORE_DOCS), 3, false));
+    for (DocValuesFieldUpdates update : updates) {
+      deletes.onDocValuesUpdate(fieldInfo, update.iterator());
+    }
+    assertEquals(0, deletes.numPendingDeletes());
+    assertTrue(deletes.getLiveDocs().get(0));
+    assertTrue(deletes.getLiveDocs().get(1));
+    assertTrue(deletes.getLiveDocs().get(2));
+    Bits liveDocs = deletes.getLiveDocs();
+    deletes.onNewReader(segmentReader, segmentInfo);
+    // no changes we keep this update
+    assertSame(liveDocs, deletes.getLiveDocs());
+    assertTrue(deletes.getLiveDocs().get(0));
+    assertTrue(deletes.getLiveDocs().get(1));
+    assertTrue(deletes.getLiveDocs().get(2));
+    assertEquals(0, deletes.numPendingDeletes());
+
+    segmentInfo.advanceDocValuesGen();
+    fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, segmentInfo.getNextDocValuesGen(), Collections.emptyMap(), 0, 0);
+    updates = Arrays.asList(singleUpdate(Arrays.asList(1, DocIdSetIterator.NO_MORE_DOCS), 3, true));
+    for (DocValuesFieldUpdates update : updates) {
+      deletes.onDocValuesUpdate(fieldInfo, update.iterator());
+    }
+    // no changes we keep this update
+    assertNotSame(liveDocs, deletes.getLiveDocs());
+    assertTrue(deletes.getLiveDocs().get(0));
+    assertFalse(deletes.getLiveDocs().get(1));
+    assertTrue(deletes.getLiveDocs().get(2));
+    assertEquals(1, deletes.numPendingDeletes());
+    IOUtils.close(reader, writer, dir);
+
+  }
+
+  private DocValuesFieldUpdates singleUpdate(List<Integer> docsChanged, int maxDoc, boolean hasValue) {
     return new DocValuesFieldUpdates(maxDoc, 0, "_soft_deletes", DocValuesType.NUMERIC) {
       @Override
       public void add(int doc, long value) {
@@ -219,7 +278,7 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
       @Override
       public Iterator iterator() {
         return new Iterator() {
-          java.util.Iterator<Integer> iter = docsDeleted.iterator();
+          java.util.Iterator<Integer> iter = docsChanged.iterator();
           int doc = -1;
 
           @Override
@@ -246,6 +305,11 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
           long delGen() {
             return 0;
           }
+
+          @Override
+          boolean hasValue() {
+            return hasValue;
+          }
         };
       }
     };

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a3c86373/lucene/core/src/test/org/apache/lucene/index/TestSoftDeletesRetentionMergePolicy.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestSoftDeletesRetentionMergePolicy.java b/lucene/core/src/test/org/apache/lucene/index/TestSoftDeletesRetentionMergePolicy.java
index c9b22a5..be1f7ac 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestSoftDeletesRetentionMergePolicy.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestSoftDeletesRetentionMergePolicy.java
@@ -538,4 +538,93 @@ public class TestSoftDeletesRetentionMergePolicy extends LuceneTestCase {
     assertFalse(delete.get());
     IOUtils.close(reader, writer, dir);
   }
+
+  public void testUndeleteDocument() throws IOException {
+    Directory dir = newDirectory();
+    String softDelete = "soft_delete";
+    IndexWriterConfig config = newIndexWriterConfig()
+        .setSoftDeletesField(softDelete)
+        .setMergePolicy(new SoftDeletesRetentionMergePolicy("soft_delete",
+        MatchAllDocsQuery::new, new LogDocMergePolicy()));
+    config.setReaderPooling(true);
+    config.setMergePolicy(new LogDocMergePolicy());
+    IndexWriter writer = new IndexWriter(dir, config);
+    Document d = new Document();
+    d.add(new StringField("id", "0", Field.Store.YES));
+    d.add(new StringField("seq_id", "0", Field.Store.YES));
+    writer.addDocument(d);
+    d = new Document();
+    d.add(new StringField("id", "1", Field.Store.YES));
+    writer.addDocument(d);
+    writer.updateDocValues(new Term("id", "0"), new NumericDocValuesField("soft_delete", 1));
+    try (IndexReader reader = writer.getReader()) {
+      assertEquals(2, reader.maxDoc());
+      assertEquals(1, reader.numDocs());
+    }
+    doUpdate(new Term("id", "0"), writer, new NumericDocValuesField("soft_delete", null));
+    try (IndexReader reader = writer.getReader()) {
+      assertEquals(2, reader.maxDoc());
+      assertEquals(2, reader.numDocs());
+    }
+    IOUtils.close(writer, dir);
+  }
+
+  static void doUpdate(Term doc, IndexWriter writer, Field... fields) throws IOException {
+    long seqId = -1;
+    do { // retry if we just committing a merge
+      try (DirectoryReader reader = writer.getReader()) {
+        TopDocs topDocs = new IndexSearcher(new NoDeletesWrapper(reader)).search(new TermQuery(doc), 10);
+        assertEquals(1, topDocs.totalHits);
+        int theDoc = topDocs.scoreDocs[0].doc;
+        seqId = writer.tryUpdateDocValue(reader, theDoc, fields);
+      }
+    } while (seqId == -1);
+  }
+
+  private static final class NoDeletesSubReaderWrapper extends FilterDirectoryReader.SubReaderWrapper {
+
+    @Override
+    public LeafReader wrap(LeafReader reader) {
+      return new FilterLeafReader(reader) {
+
+        @Override
+        public int numDocs() {
+          return maxDoc();
+        }
+
+        @Override
+        public Bits getLiveDocs() {
+          return null;
+        }
+
+        @Override
+        public CacheHelper getCoreCacheHelper() {
+          return null;
+        }
+
+        @Override
+        public CacheHelper getReaderCacheHelper() {
+          return null;
+        }
+      };
+    }
+  }
+
+  private static final class NoDeletesWrapper extends FilterDirectoryReader {
+
+    NoDeletesWrapper(DirectoryReader in) throws IOException {
+      super(in, new NoDeletesSubReaderWrapper());
+    }
+
+    @Override
+    protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
+      return new NoDeletesWrapper(in);
+    }
+
+
+    @Override
+    public CacheHelper getReaderCacheHelper() {
+      return null;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a3c86373/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 4c91187..3b80540 100644
--- a/lucene/core/src/test/org/apache/lucene/util/TestFixedBitSet.java
+++ b/lucene/core/src/test/org/apache/lucene/util/TestFixedBitSet.java
@@ -493,4 +493,34 @@ public class TestFixedBitSet extends BaseBitSetTestCase<FixedBitSet> {
     
     assertEquals(bitSet1.cardinality(), andNotCount);
   }
+
+  public void testCopyOf() {
+    Random random = random();
+    int numBits = TestUtil.nextInt(random, 1000, 2000);
+    int count = TestUtil.nextInt(random, 0, numBits - 1);
+    int[] bits = makeIntArray(random, count, 0, numBits - 1);
+    FixedBitSet fixedBitSet = makeFixedBitSet(bits, numBits);
+
+    FixedBitSet mutableCopy = FixedBitSet.copyOf(fixedBitSet);
+    assertNotSame(mutableCopy, fixedBitSet);
+    assertEquals(mutableCopy, fixedBitSet);
+
+    FixedBitSet mutableCopy1 = FixedBitSet.copyOf(new Bits() {
+
+      @Override
+      public boolean get(int index) {
+        return fixedBitSet.get(index);
+      }
+
+      @Override
+      public int length() {
+        return fixedBitSet.length();
+      }
+    });
+
+    assertNotSame(mutableCopy, mutableCopy1);
+    assertNotSame(fixedBitSet, mutableCopy1);
+    assertEquals(mutableCopy1, mutableCopy);
+    assertEquals(mutableCopy1, fixedBitSet);
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/a3c86373/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 127437b..6cf511a 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
@@ -45,7 +45,7 @@ public class AssertingLiveDocsFormat extends LiveDocsFormat {
     check(bits, info.info.maxDoc(), info.getDelCount() + newDelCount);
     in.writeLiveDocs(bits, dir, info, newDelCount, context);
   }
-  
+
   private void check(Bits bits, int expectedLength, int expectedDeleteCount) {
     assert bits.length() == expectedLength;
     int deletedCount = 0;
@@ -61,7 +61,7 @@ public class AssertingLiveDocsFormat extends LiveDocsFormat {
   public void files(SegmentCommitInfo info, Collection<String> files) throws IOException {
     in.files(info, files);
   }
-  
+
   @Override
   public String toString() {
     return "Asserting(" + in + ")";
@@ -69,28 +69,27 @@ public class AssertingLiveDocsFormat extends LiveDocsFormat {
 
   static class AssertingBits implements Bits {
     final Bits in;
-    
+
     AssertingBits(Bits in) {
       this.in = in;
       assert in.length() >= 0;
     }
-    
+
     @Override
     public boolean get(int index) {
       assert index >= 0;
-      assert index < in.length(): "index=" + index + " vs in.length()=" + in.length();
+      assert index < in.length() : "index=" + index + " vs in.length()=" + in.length();
       return in.get(index);
     }
-    
+
     @Override
     public int length() {
       return in.length();
     }
-    
+
     @Override
     public String toString() {
       return "Asserting(" + in + ")";
     }
   }
-
 }