You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucenenet.apache.org by ni...@apache.org on 2021/12/13 12:10:32 UTC

[lucenenet] branch master updated: BREAKING: Lucene.Net.Index.DocValuesFieldUpdates: Refactored so the subclasses will handle getting the values from DocValuesFieldUpdatesIterator or DocValuesUpdate via a cast rather than boxing the value. Also marked internal (as well as all members of BufferedUpdates), since this was not supposed to be part of the public API.

This is an automated email from the ASF dual-hosted git repository.

nightowl888 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/lucenenet.git


The following commit(s) were added to refs/heads/master by this push:
     new dd58a1c  BREAKING: Lucene.Net.Index.DocValuesFieldUpdates: Refactored so the subclasses will handle getting the values from DocValuesFieldUpdatesIterator or DocValuesUpdate via a cast rather than boxing the value. Also marked internal (as well as all members of BufferedUpdates), since this was not supposed to be part of the public API.
dd58a1c is described below

commit dd58a1ce8aa69d4f9cee7c9f0f83bd34d43d5860
Author: Shad Storhaug <sh...@shadstorhaug.com>
AuthorDate: Mon Dec 13 11:53:03 2021 +0700

    BREAKING: Lucene.Net.Index.DocValuesFieldUpdates: Refactored so the subclasses will handle getting the values from DocValuesFieldUpdatesIterator or DocValuesUpdate via a cast rather than boxing the value. Also marked internal (as well as all members of BufferedUpdates), since this was not supposed to be part of the public API.
---
 .../Index/BinaryDocValuesFieldUpdates.cs           |  25 +++--
 src/Lucene.Net/Index/BufferedUpdates.cs            |  22 ++---
 src/Lucene.Net/Index/BufferedUpdatesStream.cs      |   4 +-
 src/Lucene.Net/Index/CoalescedUpdates.cs           |   4 +-
 src/Lucene.Net/Index/DocValuesFieldUpdates.cs      | 109 ++++++++++++++-------
 src/Lucene.Net/Index/DocValuesUpdate.cs            |  83 +++++++++-------
 src/Lucene.Net/Index/DocumentsWriter.cs            |   2 -
 src/Lucene.Net/Index/DocumentsWriterDeleteQueue.cs |   7 +-
 src/Lucene.Net/Index/FrozenBufferedUpdates.cs      |  11 +--
 src/Lucene.Net/Index/IndexWriter.cs                |  15 +--
 .../Index/NumericDocValuesFieldUpdates.cs          |  38 ++++---
 src/Lucene.Net/Index/ReadersAndUpdates.cs          |   2 +-
 12 files changed, 187 insertions(+), 135 deletions(-)

diff --git a/src/Lucene.Net/Index/BinaryDocValuesFieldUpdates.cs b/src/Lucene.Net/Index/BinaryDocValuesFieldUpdates.cs
index 926db4a..da6b218 100644
--- a/src/Lucene.Net/Index/BinaryDocValuesFieldUpdates.cs
+++ b/src/Lucene.Net/Index/BinaryDocValuesFieldUpdates.cs
@@ -1,4 +1,4 @@
-using Lucene.Net.Documents;
+using Lucene.Net.Documents;
 using System;
 using System.Runtime.CompilerServices;
 
@@ -22,7 +22,6 @@ namespace Lucene.Net.Index
      */
 
     using BinaryDocValuesField = BinaryDocValuesField;
-    using BinaryDocValuesUpdate = Lucene.Net.Index.DocValuesUpdate.BinaryDocValuesUpdate;
     using BytesRef = Lucene.Net.Util.BytesRef;
     using DocIdSetIterator = Lucene.Net.Search.DocIdSetIterator;
     using FixedBitSet = Lucene.Net.Util.FixedBitSet;
@@ -39,7 +38,7 @@ namespace Lucene.Net.Index
     /// </summary>
     internal class BinaryDocValuesFieldUpdates : DocValuesFieldUpdates
     {
-        new internal sealed class Iterator : DocValuesFieldUpdates.Iterator
+        internal sealed class Iterator : DocValuesFieldUpdatesIterator<BytesRef>
         {
             private readonly PagedGrowableWriter offsets;
             private readonly int size;
@@ -61,7 +60,7 @@ namespace Lucene.Net.Index
                 value = (BytesRef)values.Clone();
             }
 
-            public override object Value
+            public override BytesRef Value
             {
                 get
                 {
@@ -135,7 +134,19 @@ namespace Lucene.Net.Index
             size = 0;
         }
 
-        public override void Add(int doc, object value)
+        // LUCENENET specific: Pass iterator instead of the value, since this class knows the type to retrieve, but the caller does not.
+        public override void AddFromIterator(int doc, DocValuesFieldUpdatesIterator iterator)
+        {
+            Add(doc, ((Iterator)iterator).Value);
+        }
+
+        // LUCENENET specific: Pass DocValuesUpdate instead of the value, since this class knows the type to retrieve, but the caller does not.
+        public override void AddFromUpdate(int doc, DocValuesUpdate update)
+        {
+            Add(doc, ((BinaryDocValuesUpdate)update).value);
+        }
+
+        private void Add(int doc, BytesRef value) // LUCENENET specific: Marked private instead of public and changed the value parameter type
         {
             // TODO: if the Sorter interface changes to take long indexes, we can remove that limitation
             if (size == int.MaxValue)
@@ -143,7 +154,7 @@ namespace Lucene.Net.Index
                 throw IllegalStateException.Create("cannot support more than System.Int32.MaxValue doc/value entries");
             }
 
-            BytesRef val = (BytesRef)value;
+            BytesRef val = value;
             if (val == null)
             {
                 val = BinaryDocValuesUpdate.MISSING;
@@ -171,7 +182,7 @@ namespace Lucene.Net.Index
             ++size;
         }
 
-        public override DocValuesFieldUpdates.Iterator GetIterator()
+        public override DocValuesFieldUpdatesIterator GetIterator()
         {
             PagedMutable docs = this.docs;
             PagedGrowableWriter offsets = this.offsets;
diff --git a/src/Lucene.Net/Index/BufferedUpdates.cs b/src/Lucene.Net/Index/BufferedUpdates.cs
index 8db9286..8584028 100644
--- a/src/Lucene.Net/Index/BufferedUpdates.cs
+++ b/src/Lucene.Net/Index/BufferedUpdates.cs
@@ -1,5 +1,7 @@
 using J2N.Collections.Generic;
 using J2N.Threading.Atomic;
+using Lucene.Net.Search;
+using Lucene.Net.Util;
 using JCG = J2N.Collections.Generic;
 using SCG = System.Collections.Generic;
 
@@ -22,12 +24,6 @@ namespace Lucene.Net.Index
      * limitations under the License.
      */
 
-    using BinaryDocValuesUpdate = Lucene.Net.Index.DocValuesUpdate.BinaryDocValuesUpdate;
-    using NumericDocValuesUpdate = Lucene.Net.Index.DocValuesUpdate.NumericDocValuesUpdate;
-    using Query = Lucene.Net.Search.Query;
-    using RamUsageEstimator = Lucene.Net.Util.RamUsageEstimator;
-
-
     /// <summary>
     /// Holds buffered deletes and updates, by docID, term or query for a
     /// single segment. this is used to hold buffered pending
@@ -148,7 +144,7 @@ namespace Lucene.Net.Index
         /// <summary>
         /// NOTE: This was MAX_INT in Lucene
         /// </summary>
-        public static readonly int MAX_INT32 = int.MaxValue;
+        internal static readonly int MAX_INT32 = int.MaxValue; // LUCENENET specific - Made internal rather than public, since this class is intended to be internal but couldn't be because it is exposed through a public API
 
         internal readonly AtomicInt64 bytesUsed;
 
@@ -158,7 +154,7 @@ namespace Lucene.Net.Index
 
         internal long gen;
 
-        internal BufferedUpdates() // LUCENENET NOTE: Made internal rather than public, since this class is intended to be internal but couldn't be because it is exposed through a public API
+        internal BufferedUpdates() // LUCENENET specific - Made internal rather than public, since this class is intended to be internal but couldn't be because it is exposed through a public API
         {
             this.bytesUsed = new AtomicInt64();
         }
@@ -204,7 +200,7 @@ namespace Lucene.Net.Index
             }
         }
 
-        public virtual void AddQuery(Query query, int docIDUpto)
+        internal virtual void AddQuery(Query query, int docIDUpto) // LUCENENET specific - Made internal rather than public, since this class is intended to be internal but couldn't be because it is exposed through a public API
         {
             bool prevExists = queries.TryGetValue(query, out _);
             queries[query] = docIDUpto;
@@ -215,13 +211,13 @@ namespace Lucene.Net.Index
             }
         }
 
-        public virtual void AddDocID(int docID)
+        internal virtual void AddDocID(int docID) // LUCENENET specific - Made internal rather than public, since this class is intended to be internal but couldn't be because it is exposed through a public API
         {
             docIDs.Add(docID);
             bytesUsed.AddAndGet(BYTES_PER_DEL_DOCID);
         }
 
-        public virtual void AddTerm(Term term, int docIDUpto)
+        internal virtual void AddTerm(Term term, int docIDUpto) // LUCENENET specific - Made internal rather than public, since this class is intended to be internal but couldn't be because it is exposed through a public API
         {
             bool currentExists = terms.TryGetValue(term, out int current);
             if (currentExists && docIDUpto < current)
@@ -247,7 +243,7 @@ namespace Lucene.Net.Index
             }
         }
 
-        public virtual void AddNumericUpdate(NumericDocValuesUpdate update, int docIDUpto)
+        internal virtual void AddNumericUpdate(NumericDocValuesUpdate update, int docIDUpto) // LUCENENET specific - Made internal rather than public, since this class is intended to be internal but couldn't be because it is exposed through a public API
         {
             if (!numericUpdates.TryGetValue(update.field, out LinkedDictionary<Term, NumericDocValuesUpdate> fieldUpdates))
             {
@@ -280,7 +276,7 @@ namespace Lucene.Net.Index
             }
         }
 
-        public virtual void AddBinaryUpdate(BinaryDocValuesUpdate update, int docIDUpto)
+        internal virtual void AddBinaryUpdate(BinaryDocValuesUpdate update, int docIDUpto) // LUCENENET specific - Made internal rather than public, since this class is intended to be internal but couldn't be because it is exposed through a public API
         {
             if (!binaryUpdates.TryGetValue(update.field, out LinkedDictionary<Term, BinaryDocValuesUpdate> fieldUpdates))
             {
diff --git a/src/Lucene.Net/Index/BufferedUpdatesStream.cs b/src/Lucene.Net/Index/BufferedUpdatesStream.cs
index 2a6a894..67fa4d7 100644
--- a/src/Lucene.Net/Index/BufferedUpdatesStream.cs
+++ b/src/Lucene.Net/Index/BufferedUpdatesStream.cs
@@ -659,7 +659,9 @@ namespace Lucene.Net.Index
                             {
                                 break; // no more docs that can be updated for this term
                             }
-                            dvUpdates.Add(doc, update.value);
+                            // LUCENENET specific handling - dvUpdates handles getting the value so we don't
+                            // have to deal with boxing/unboxing
+                            dvUpdates.AddFromUpdate(doc, update);
                         }
                     }
                 }
diff --git a/src/Lucene.Net/Index/CoalescedUpdates.cs b/src/Lucene.Net/Index/CoalescedUpdates.cs
index 6270bac..c5acb88 100644
--- a/src/Lucene.Net/Index/CoalescedUpdates.cs
+++ b/src/Lucene.Net/Index/CoalescedUpdates.cs
@@ -22,9 +22,7 @@ namespace Lucene.Net.Index
      * limitations under the License.
      */
 
-    using BinaryDocValuesUpdate = Lucene.Net.Index.DocValuesUpdate.BinaryDocValuesUpdate;
     using BytesRef = Lucene.Net.Util.BytesRef;
-    using NumericDocValuesUpdate = Lucene.Net.Index.DocValuesUpdate.NumericDocValuesUpdate;
     using Query = Lucene.Net.Search.Query;
     using QueryAndLimit = Lucene.Net.Index.BufferedUpdatesStream.QueryAndLimit;
 
@@ -53,7 +51,7 @@ namespace Lucene.Net.Index
 
             foreach (NumericDocValuesUpdate nu in @in.numericDVUpdates)
             {
-                NumericDocValuesUpdate clone = new NumericDocValuesUpdate(nu.term, nu.field, (long?)nu.value);
+                NumericDocValuesUpdate clone = new NumericDocValuesUpdate(nu.term, nu.field, nu.value);
                 clone.docIDUpto = int.MaxValue;
                 numericDVUpdates.Add(clone);
             }
diff --git a/src/Lucene.Net/Index/DocValuesFieldUpdates.cs b/src/Lucene.Net/Index/DocValuesFieldUpdates.cs
index 6a14132..2619c1d 100644
--- a/src/Lucene.Net/Index/DocValuesFieldUpdates.cs
+++ b/src/Lucene.Net/Index/DocValuesFieldUpdates.cs
@@ -1,4 +1,4 @@
-using Lucene.Net.Diagnostics;
+using Lucene.Net.Diagnostics;
 using System;
 using System.Collections.Generic;
 using System.Diagnostics;
@@ -32,36 +32,7 @@ namespace Lucene.Net.Index
     {
         // LUCENENET specific: de-nested Type enum and renamed DocValuesFieldUpdatesType
 
-        /// <summary>
-        /// An iterator over documents and their updated values. Only documents with
-        /// updates are returned by this iterator, and the documents are returned in
-        /// increasing order.
-        /// </summary>
-        public abstract class Iterator
-        {
-            /// <summary>
-            /// Returns the next document which has an update, or
-            /// <see cref="Search.DocIdSetIterator.NO_MORE_DOCS"/> if there are no more documents to
-            /// return.
-            /// </summary>
-            public abstract int NextDoc();
-
-            /// <summary>
-            /// Returns the current document this iterator is on. </summary>
-            public abstract int Doc { get; }
-
-            /// <summary>
-            /// Returns the value of the document returned from <see cref="NextDoc()"/>. A
-            /// <c>null</c> value means that it was unset for this document.
-            /// </summary>
-            public abstract object Value { get; }
-
-            /// <summary>
-            /// Reset the iterator's state. Should be called before <see cref="NextDoc()"/>
-            /// and <see cref="Value"/>.
-            /// </summary>
-            public abstract void Reset();
-        }
+        // LUCENENET specific: de-nested Iterator and renamed DocValuesFieldUpdatesIterator. Also created a generic version that exposes the Value property.
 
         public class Container
         {
@@ -141,23 +112,35 @@ namespace Lucene.Net.Index
         internal readonly string field;
         internal readonly DocValuesFieldUpdatesType type;
 
-        protected internal DocValuesFieldUpdates(string field, DocValuesFieldUpdatesType type)
+        protected DocValuesFieldUpdates(string field, DocValuesFieldUpdatesType type)
         {
             this.field = field;
             this.type = type;
         }
 
+        // LUCENENET specific - use this instance to decide which subclass to cast to, which will expose the
+        // strongly typed value. This allows us to access the long? type without boxing/unboxing.
+        // The Add() method was removed and replaced with the following two.
+
+        /// <summary>
+        /// Add an update to a document from a <see cref="DocValuesFieldUpdatesIterator"/>.
+        /// The <see cref="DocValuesFieldUpdatesIterator"/>'s value should be <c>null</c> to unset a value.
+        /// Note that the value is exposed by casting to the apprpriate <see cref="DocValuesFieldUpdatesIterator"/> subclasss.
+        /// </summary>
+        public abstract void AddFromIterator(int doc, DocValuesFieldUpdatesIterator iterator);
+
         /// <summary>
-        /// Add an update to a document. For unsetting a value you should pass
-        /// <c>null</c>.
+        /// Add an update to a document from a <see cref="DocValuesUpdate"/>.
+        /// The <see cref="DocValuesUpdate"/>'s value should be <c>null</c> to unset a value.
+        /// Note that the value is exposed by casting to the apprpriate <see cref="DocValuesUpdate"/> subclasss.
         /// </summary>
-        public abstract void Add(int doc, object value);
+        public abstract void AddFromUpdate(int doc, DocValuesUpdate update);
 
         /// <summary>
-        /// Returns an <see cref="Iterator"/> over the updated documents and their
+        /// Returns a <see cref="DocValuesFieldUpdatesIterator"/> over the updated documents and their
         /// values.
         /// </summary>
-        public abstract Iterator GetIterator();
+        public abstract DocValuesFieldUpdatesIterator GetIterator();
 
         /// <summary>
         /// Merge with another <see cref="DocValuesFieldUpdates"/>. this is called for a
@@ -173,11 +156,61 @@ namespace Lucene.Net.Index
         public abstract bool Any();
     }
 
+
     // LUCENENET specific - de-nested Type enumeration and renamed DocValuesFieldUpdatesType
     // primarily so it doesn't conflict with System.Type.
-    public enum DocValuesFieldUpdatesType
+    internal enum DocValuesFieldUpdatesType
     {
         NUMERIC,
         BINARY
     }
+
+    /// <summary>
+    /// An iterator over documents. Only documents with
+    /// updates are returned by this iterator, and the documents are returned in
+    /// increasing order.
+    /// </summary>
+    internal abstract class DocValuesFieldUpdatesIterator
+    {
+        /// <summary>
+        /// Returns the next document which has an update, or
+        /// <see cref="Search.DocIdSetIterator.NO_MORE_DOCS"/> if there are no more documents to
+        /// return.
+        /// </summary>
+        public abstract int NextDoc();
+
+        /// <summary>
+        /// Returns the current document this iterator is on. </summary>
+        public abstract int Doc { get; }
+
+        /// <summary>
+        /// Reset the iterator's state. Should be called before <see cref="NextDoc()"/>
+        /// and value.
+        /// </summary>
+        public abstract void Reset();
+    }
+
+    /// <summary>
+    /// An iterator over documents and their updated values. This differs from
+    /// <see cref="DocValuesFieldUpdatesIterator"/> in that it exposes the strongly-typed value.
+    /// Only documents with updates are returned by this iterator, and the documents are returned in
+    /// increasing order.
+    /// </summary>
+    internal abstract class DocValuesFieldUpdatesIterator<T> : DocValuesFieldUpdatesIterator
+    {
+        /// <inheritdoc/>
+        public override abstract int NextDoc();
+
+        /// <inheritdoc/>
+        public override abstract int Doc { get; }
+
+        /// <summary>
+        /// Returns the value of the document returned from <see cref="NextDoc()"/>. A
+        /// <c>null</c> value means that it was unset for this document.
+        /// </summary>
+        public abstract T Value { get; }
+
+        /// <inheritdoc/>
+        public override abstract void Reset();
+    }
 }
\ No newline at end of file
diff --git a/src/Lucene.Net/Index/DocValuesUpdate.cs b/src/Lucene.Net/Index/DocValuesUpdate.cs
index b967d48..60b57f9 100644
--- a/src/Lucene.Net/Index/DocValuesUpdate.cs
+++ b/src/Lucene.Net/Index/DocValuesUpdate.cs
@@ -1,4 +1,4 @@
-using Lucene.Net.Documents;
+using Lucene.Net.Documents;
 
 namespace Lucene.Net.Index
 {
@@ -25,7 +25,7 @@ namespace Lucene.Net.Index
 
     /// <summary>
     /// An in-place update to a <see cref="DocValues"/> field. </summary>
-    public abstract class DocValuesUpdate
+    internal abstract class DocValuesUpdate
     {
         /* Rough logic: OBJ_HEADER + 3*PTR + INT
          * Term: OBJ_HEADER + 2*PTR
@@ -39,7 +39,7 @@ namespace Lucene.Net.Index
         internal readonly DocValuesFieldUpdatesType type;
         internal readonly Term term;
         internal readonly string field;
-        internal readonly object value;
+        // LUCENENET specific - moved value field to appropriate subclass to avoid object/boxing
         internal int docIDUpto = -1; // unassigned until applied, and confusing that it's here, when it's just used in BufferedDeletes...
 
         /// <summary>
@@ -48,13 +48,12 @@ namespace Lucene.Net.Index
         /// <param name="type"> the <see cref="DocValuesFieldUpdatesType"/> </param>
         /// <param name="term"> the <see cref="Term"/> which determines the documents that will be updated </param>
         /// <param name="field"> the <see cref="NumericDocValuesField"/> to update </param>
-        /// <param name="value"> the updated value </param>
-        protected DocValuesUpdate(DocValuesFieldUpdatesType type, Term term, string field, object value)
+        ///// <param name="value"> the updated value </param>
+        protected DocValuesUpdate(DocValuesFieldUpdatesType type, Term term, string field) // LUCENENET: Removed value (will be stored in subclasses with the strong type)
         {
             this.type = type;
             this.term = term;
             this.field = field;
-            this.value = value;
         }
 
         internal abstract long GetValueSizeInBytes();
@@ -71,44 +70,60 @@ namespace Lucene.Net.Index
 
         public override string ToString()
         {
-            return "term=" + term + ",field=" + field + ",value=" + value;
+            return "term=" + term + ",field=" + field /*+ ",value=" + value*/;
         }
+    }
 
-        /// <summary>
-        /// An in-place update to a binary <see cref="DocValues"/> field </summary>
-        public sealed class BinaryDocValuesUpdate : DocValuesUpdate
-        {
-            /* Size of BytesRef: 2*INT + ARRAY_HEADER + PTR */
-            private static readonly long RAW_VALUE_SIZE_IN_BYTES = RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + 2 * RamUsageEstimator.NUM_BYTES_INT32 + RamUsageEstimator.NUM_BYTES_OBJECT_REF;
+    /// <summary>
+    /// An in-place update to a binary <see cref="DocValues"/> field </summary>
+    internal sealed class BinaryDocValuesUpdate : DocValuesUpdate
+    {
+        /* Size of BytesRef: 2*INT + ARRAY_HEADER + PTR */
+        private static readonly long RAW_VALUE_SIZE_IN_BYTES = RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + 2 * RamUsageEstimator.NUM_BYTES_INT32 + RamUsageEstimator.NUM_BYTES_OBJECT_REF;
 
-            internal static readonly BytesRef MISSING = new BytesRef();
+        internal static readonly BytesRef MISSING = new BytesRef();
 
-            internal BinaryDocValuesUpdate(Term term, string field, BytesRef value)
-                : base(DocValuesFieldUpdatesType.BINARY, term, field, value ?? MISSING)
-            {
-            }
+        internal readonly BytesRef value;
 
-            internal override long GetValueSizeInBytes()
-            {
-                return RAW_VALUE_SIZE_IN_BYTES + ((BytesRef)value).Bytes.Length;
-            }
+        internal BinaryDocValuesUpdate(Term term, string field, BytesRef value)
+            : base(DocValuesFieldUpdatesType.BINARY, term, field)
+        {
+            this.value = value ?? MISSING;
         }
 
-        /// <summary>
-        /// An in-place update to a numeric <see cref="DocValues"/> field </summary>
-        public sealed class NumericDocValuesUpdate : DocValuesUpdate // LUCENENET NOTE: Made public rather than internal because it is on a public API
+        internal override long GetValueSizeInBytes()
         {
-            internal static readonly long? MISSING = new long?(0);
+            return RAW_VALUE_SIZE_IN_BYTES + value.Bytes.Length;
+        }
 
-            public NumericDocValuesUpdate(Term term, string field, long? value)
-                : base(DocValuesFieldUpdatesType.NUMERIC, term, field, value == null ? MISSING : value)
-            {
-            }
+        public override string ToString()
+        {
+            return "term=" + term + ",field=" + field + ",value=" + value;
+        }
+    }
 
-            internal override long GetValueSizeInBytes()
-            {
-                return RamUsageEstimator.NUM_BYTES_INT64;
-            }
+    /// <summary>
+    /// An in-place update to a numeric <see cref="DocValues"/> field </summary>
+    internal sealed class NumericDocValuesUpdate : DocValuesUpdate
+    {
+        internal static readonly long MISSING = 0;
+
+        internal readonly long value;
+
+        public NumericDocValuesUpdate(Term term, string field, long? value)
+            : base(DocValuesFieldUpdatesType.NUMERIC, term, field)
+        {
+            this.value = !value.HasValue ? MISSING : value.Value;
+        }
+
+        internal override long GetValueSizeInBytes()
+        {
+            return RamUsageEstimator.NUM_BYTES_INT64;
+        }
+
+        public override string ToString()
+        {
+            return "term=" + term + ",field=" + field + ",value=" + value;
         }
     }
 }
\ No newline at end of file
diff --git a/src/Lucene.Net/Index/DocumentsWriter.cs b/src/Lucene.Net/Index/DocumentsWriter.cs
index 61c26a4..aa975e8 100644
--- a/src/Lucene.Net/Index/DocumentsWriter.cs
+++ b/src/Lucene.Net/Index/DocumentsWriter.cs
@@ -28,13 +28,11 @@ namespace Lucene.Net.Index
      */
 
     using Analyzer = Lucene.Net.Analysis.Analyzer;
-    using BinaryDocValuesUpdate = Lucene.Net.Index.DocValuesUpdate.BinaryDocValuesUpdate;
     using BytesRef = Lucene.Net.Util.BytesRef;
     using Directory = Lucene.Net.Store.Directory;
     using FlushedSegment = Lucene.Net.Index.DocumentsWriterPerThread.FlushedSegment;
     using IEvent = Lucene.Net.Index.IndexWriter.IEvent;
     using InfoStream = Lucene.Net.Util.InfoStream;
-    using NumericDocValuesUpdate = Lucene.Net.Index.DocValuesUpdate.NumericDocValuesUpdate;
     using Query = Lucene.Net.Search.Query;
     using SegmentFlushTicket = Lucene.Net.Index.DocumentsWriterFlushQueue.SegmentFlushTicket;
     using ThreadState = Lucene.Net.Index.DocumentsWriterPerThreadPool.ThreadState;
diff --git a/src/Lucene.Net/Index/DocumentsWriterDeleteQueue.cs b/src/Lucene.Net/Index/DocumentsWriterDeleteQueue.cs
index e378aea..cd2c3b3 100644
--- a/src/Lucene.Net/Index/DocumentsWriterDeleteQueue.cs
+++ b/src/Lucene.Net/Index/DocumentsWriterDeleteQueue.cs
@@ -1,4 +1,5 @@
-using Lucene.Net.Diagnostics;
+using Lucene.Net.Diagnostics;
+using Lucene.Net.Search;
 using Lucene.Net.Support;
 using Lucene.Net.Support.Threading;
 using System;
@@ -23,10 +24,6 @@ namespace Lucene.Net.Index
      * limitations under the License.
      */
 
-    using BinaryDocValuesUpdate = Lucene.Net.Index.DocValuesUpdate.BinaryDocValuesUpdate;
-    using NumericDocValuesUpdate = Lucene.Net.Index.DocValuesUpdate.NumericDocValuesUpdate;
-    using Query = Lucene.Net.Search.Query;
-
     /// <summary>
     /// <see cref="DocumentsWriterDeleteQueue"/> is a non-blocking linked pending deletes
     /// queue. In contrast to other queue implementation we only maintain the
diff --git a/src/Lucene.Net/Index/FrozenBufferedUpdates.cs b/src/Lucene.Net/Index/FrozenBufferedUpdates.cs
index d8a499f..04af265 100644
--- a/src/Lucene.Net/Index/FrozenBufferedUpdates.cs
+++ b/src/Lucene.Net/Index/FrozenBufferedUpdates.cs
@@ -1,9 +1,11 @@
 using J2N.Collections.Generic.Extensions;
 using Lucene.Net.Diagnostics;
-using System;
+using Lucene.Net.Search;
+using Lucene.Net.Util;
 using System.Collections;
 using System.Collections.Generic;
 using JCG = J2N.Collections.Generic;
+using QueryAndLimit = Lucene.Net.Index.BufferedUpdatesStream.QueryAndLimit;
 
 namespace Lucene.Net.Index
 {
@@ -24,13 +26,6 @@ namespace Lucene.Net.Index
      * limitations under the License.
      */
 
-    using ArrayUtil = Lucene.Net.Util.ArrayUtil;
-    using BinaryDocValuesUpdate = Lucene.Net.Index.DocValuesUpdate.BinaryDocValuesUpdate;
-    using NumericDocValuesUpdate = Lucene.Net.Index.DocValuesUpdate.NumericDocValuesUpdate;
-    using Query = Lucene.Net.Search.Query;
-    using QueryAndLimit = Lucene.Net.Index.BufferedUpdatesStream.QueryAndLimit;
-    using RamUsageEstimator = Lucene.Net.Util.RamUsageEstimator;
-
     /// <summary>
     /// Holds buffered deletes and updates by term or query, once pushed. Pushed
     /// deletes/updates are write-once, so we shift to more memory efficient data
diff --git a/src/Lucene.Net/Index/IndexWriter.cs b/src/Lucene.Net/Index/IndexWriter.cs
index fe1beb3..39e3886 100644
--- a/src/Lucene.Net/Index/IndexWriter.cs
+++ b/src/Lucene.Net/Index/IndexWriter.cs
@@ -4355,9 +4355,9 @@ namespace Lucene.Net.Index
             }
         }
 
-        private static void SkipDeletedDoc(DocValuesFieldUpdates.Iterator[] updatesIters, int deletedDoc) // LUCENENET: CA1822: Mark members as static
+        private static void SkipDeletedDoc(DocValuesFieldUpdatesIterator[] updatesIters, int deletedDoc) // LUCENENET: CA1822: Mark members as static
         {
-            foreach (DocValuesFieldUpdates.Iterator iter in updatesIters)
+            foreach (DocValuesFieldUpdatesIterator iter in updatesIters)
             {
                 if (iter.Doc == deletedDoc)
                 {
@@ -4396,12 +4396,12 @@ namespace Lucene.Net.Index
             }
         }
 
-        private void MaybeApplyMergedDVUpdates(MergePolicy.OneMerge merge, MergeState mergeState, int docUpto, MergedDeletesAndUpdates holder, string[] mergingFields, DocValuesFieldUpdates[] dvFieldUpdates, DocValuesFieldUpdates.Iterator[] updatesIters, int curDoc)
+        private void MaybeApplyMergedDVUpdates(MergePolicy.OneMerge merge, MergeState mergeState, int docUpto, MergedDeletesAndUpdates holder, string[] mergingFields, DocValuesFieldUpdates[] dvFieldUpdates, DocValuesFieldUpdatesIterator[] updatesIters, int curDoc)
         {
             int newDoc = -1;
             for (int idx = 0; idx < mergingFields.Length; idx++)
             {
-                DocValuesFieldUpdates.Iterator updatesIter = updatesIters[idx];
+                DocValuesFieldUpdatesIterator updatesIter = updatesIters[idx];
                 if (updatesIter.Doc == curDoc) // document has an update
                 {
                     if (holder.mergedDeletesAndUpdates == null)
@@ -4413,7 +4413,8 @@ namespace Lucene.Net.Index
                         newDoc = holder.docMap.Map(docUpto);
                     }
                     DocValuesFieldUpdates dvUpdates = dvFieldUpdates[idx];
-                    dvUpdates.Add(newDoc, updatesIter.Value);
+                    // LUCENENET specific - dvUpdates handles getting the value so we don't need to deal with boxing/unboxing here.
+                    dvUpdates.AddFromIterator(newDoc, updatesIter);
                     updatesIter.NextDoc(); // advance to next document
                 }
                 else
@@ -4469,7 +4470,7 @@ namespace Lucene.Net.Index
                     IDictionary<string, DocValuesFieldUpdates> mergingFieldUpdates = rld.MergingFieldUpdates;
                     string[] mergingFields;
                     DocValuesFieldUpdates[] dvFieldUpdates;
-                    DocValuesFieldUpdates.Iterator[] updatesIters;
+                    DocValuesFieldUpdatesIterator[] updatesIters;
                     if (mergingFieldUpdates.Count == 0)
                     {
                         mergingFields = null;
@@ -4480,7 +4481,7 @@ namespace Lucene.Net.Index
                     {
                         mergingFields = new string[mergingFieldUpdates.Count];
                         dvFieldUpdates = new DocValuesFieldUpdates[mergingFieldUpdates.Count];
-                        updatesIters = new DocValuesFieldUpdates.Iterator[mergingFieldUpdates.Count];
+                        updatesIters = new DocValuesFieldUpdatesIterator[mergingFieldUpdates.Count];
                         int idx = 0;
                         foreach (KeyValuePair<string, DocValuesFieldUpdates> e in mergingFieldUpdates)
                         {
diff --git a/src/Lucene.Net/Index/NumericDocValuesFieldUpdates.cs b/src/Lucene.Net/Index/NumericDocValuesFieldUpdates.cs
index 900b814..1f40e3a 100644
--- a/src/Lucene.Net/Index/NumericDocValuesFieldUpdates.cs
+++ b/src/Lucene.Net/Index/NumericDocValuesFieldUpdates.cs
@@ -1,6 +1,8 @@
+using Lucene.Net.Diagnostics;
 using Lucene.Net.Documents;
-using System;
-using Lucene.Net.Diagnostics;
+using Lucene.Net.Search;
+using Lucene.Net.Util;
+using Lucene.Net.Util.Packed;
 using System.Runtime.CompilerServices;
 
 namespace Lucene.Net.Index
@@ -22,14 +24,6 @@ namespace Lucene.Net.Index
      * limitations under the License.
      */
 
-    using DocIdSetIterator = Lucene.Net.Search.DocIdSetIterator;
-    using FixedBitSet = Lucene.Net.Util.FixedBitSet;
-    using InPlaceMergeSorter = Lucene.Net.Util.InPlaceMergeSorter;
-    using NumericDocValuesUpdate = Lucene.Net.Index.DocValuesUpdate.NumericDocValuesUpdate;
-    using PackedInt32s = Lucene.Net.Util.Packed.PackedInt32s;
-    using PagedGrowableWriter = Lucene.Net.Util.Packed.PagedGrowableWriter;
-    using PagedMutable = Lucene.Net.Util.Packed.PagedMutable;
-
     /// <summary>
     /// A <see cref="DocValuesFieldUpdates"/> which holds updates of documents, of a single
     /// <see cref="NumericDocValuesField"/>.
@@ -38,7 +32,7 @@ namespace Lucene.Net.Index
     /// </summary>
     internal class NumericDocValuesFieldUpdates : DocValuesFieldUpdates
     {
-        new internal sealed class Iterator : DocValuesFieldUpdates.Iterator
+        internal sealed class Iterator : DocValuesFieldUpdatesIterator<long?>
         {
             private readonly int size;
             private readonly PagedGrowableWriter values;
@@ -56,7 +50,7 @@ namespace Lucene.Net.Index
                 this.docs = docs;
             }
 
-            public override object Value => value;
+            public override long? Value => value;
 
             public override int NextDoc()
             {
@@ -107,7 +101,19 @@ namespace Lucene.Net.Index
             size = 0;
         }
 
-        public override void Add(int doc, object value)
+        // LUCENENET specific: Pass iterator instead of the value, since this class knows the type to retrieve, but the caller does not.
+        public override void AddFromIterator(int doc, DocValuesFieldUpdatesIterator iterator)
+        {
+            Add(doc, ((Iterator)iterator).Value);
+        }
+
+        // LUCENENET specific: Pass DocValuesUpdate instead of the value, since this class knows the type to retrieve, but the caller does not.
+        public override void AddFromUpdate(int doc, DocValuesUpdate update)
+        {
+            Add(doc, ((NumericDocValuesUpdate)update).value);
+        }
+
+        private void Add(int doc, long? value) // LUCENENET specific: Marked private instead of public and changed the value parameter type
         {
             // TODO: if the Sorter interface changes to take long indexes, we can remove that limitation
             if (size == int.MaxValue)
@@ -115,7 +121,7 @@ namespace Lucene.Net.Index
                 throw IllegalStateException.Create("cannot support more than System.Int32.MaxValue doc/value entries");
             }
 
-            long? val = (long?)value;
+            long? val = value;
             if (val == null)
             {
                 val = NumericDocValuesUpdate.MISSING;
@@ -136,11 +142,11 @@ namespace Lucene.Net.Index
             }
 
             docs.Set(size, doc);
-            values.Set(size, (long)val);
+            values.Set(size, val.Value);
             ++size;
         }
 
-        public override DocValuesFieldUpdates.Iterator GetIterator()
+        public override DocValuesFieldUpdatesIterator GetIterator()
         {
             PagedMutable docs = this.docs;
             PagedGrowableWriter values = this.values;
diff --git a/src/Lucene.Net/Index/ReadersAndUpdates.cs b/src/Lucene.Net/Index/ReadersAndUpdates.cs
index e31ed02..92ba646 100644
--- a/src/Lucene.Net/Index/ReadersAndUpdates.cs
+++ b/src/Lucene.Net/Index/ReadersAndUpdates.cs
@@ -761,7 +761,7 @@ namespace Lucene.Net.Index
             {
                 if (curDoc == updateDoc) //document has an updated value
                 {
-                    long? value = (long?)iter.Value; // either null or updated
+                    long? value = iter.Value; // either null or updated
                     updateDoc = iter.NextDoc(); //prepare for next round
                     yield return value;
                 }