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 2020/02/12 17:14:26 UTC

[lucenenet] branch master updated (f9e8fc0 -> 08e35ce)

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

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


    from f9e8fc0  BUG: Lucene.Net.Search.BooleanClause::Equals(BooleanClause): Fixed potential null reference exception when Query is set to null
     new d359f83  BUG: Added missing call to FileStream::Flush() in FSIndexOutput::Flush() that was preventing persistence to disk from occuring at the necessary phase (fixes LUCENENET-645)
     new a75a3a3  SWEEP: Pass Random instance to all Shuffle() method calls to ensure the same psuedo-random sequence is used based on the seed.
     new b5a61c8  BUG: Lucene.Net.Index.DocumentsWriterDeleteQueue::DeleteSlice(): Extra Debug.Assert() statement that wasn't in Lucene causes Lucene.Net.Index.TestIndexWriterUnicode::TestEmbeddedFFFF() test to fail when running in debug mode
     new 03f3bb9  Lucene.Net.Tests.Index (TestBagOfPositions + TestBagOfPostings): Fixed performance issue due to converting int to string and string to int in the current culture
     new 08e35ce  Lucene.Net.Store (FSDirectory + BufferedIndexOutput): Refactored FSDirectory.FSIndexOutput to utilize the FileStream buffer only, rather than using both a FileStream buffer and the buffer in BufferedIndexOutput.

The 5 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../Index/BaseStoredFieldsFormatTestCase.cs        |  2 +-
 .../Util/LuceneTestCase.cs                         |  4 +-
 .../Taxonomy/TestTaxonomyFacetCounts2.cs           |  4 +-
 src/Lucene.Net.Tests.Join/TestBlockJoin.cs         |  6 +-
 .../Index/Sorter/SorterTestBase.cs                 |  2 +-
 src/Lucene.Net.Tests.Queries/TermsFilterTest.cs    |  4 +-
 .../Suggest/Analyzing/AnalyzingSuggesterTest.cs    |  2 +-
 .../Suggest/Analyzing/FuzzySuggesterTest.cs        |  2 +-
 .../Suggest/LookupBenchmarkTest.cs                 |  4 +-
 .../Codecs/Lucene3x/TestTermInfosReaderIndex.cs    |  2 +-
 .../Codecs/Lucene40/TestLucene40PostingsReader.cs  |  2 +-
 .../Codecs/Lucene41/TestBlockPostingsFormat3.cs    |  2 +-
 src/Lucene.Net.Tests/Index/Test2BTerms.cs          |  2 +-
 src/Lucene.Net.Tests/Index/TestBagOfPositions.cs   |  7 +-
 src/Lucene.Net.Tests/Index/TestBagOfPostings.cs    |  7 +-
 .../Index/TestIndexWriterDelete.cs                 |  4 +-
 src/Lucene.Net.Tests/Index/TestMaxTermFrequency.cs |  2 +-
 src/Lucene.Net.Tests/Index/TestTermsEnum2.cs       |  2 +-
 src/Lucene.Net.Tests/Search/TestMinShouldMatch2.cs |  6 +-
 .../Search/TestSameScoresWithThreads.cs            |  2 +-
 src/Lucene.Net.Tests/Search/TestShardSearching.cs  |  2 +-
 src/Lucene.Net.Tests/Search/TestSort.cs            | 12 +--
 src/Lucene.Net.Tests/Util/Fst/TestFSTs.cs          |  2 +-
 src/Lucene.Net/Index/DocumentsWriterDeleteQueue.cs |  1 -
 src/Lucene.Net/Store/BufferedIndexOutput.cs        | 27 +++---
 src/Lucene.Net/Store/FSDirectory.cs                | 98 +++++++++++++++-------
 26 files changed, 124 insertions(+), 86 deletions(-)


[lucenenet] 05/05: Lucene.Net.Store (FSDirectory + BufferedIndexOutput): Refactored FSDirectory.FSIndexOutput to utilize the FileStream buffer only, rather than using both a FileStream buffer and the buffer in BufferedIndexOutput.

Posted by ni...@apache.org.
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

commit 08e35ce8ae4ed5d47da37bf1157ad3a17928cc98
Author: Shad Storhaug <sh...@shadstorhaug.com>
AuthorDate: Tue Feb 11 16:57:53 2020 +0700

    Lucene.Net.Store (FSDirectory + BufferedIndexOutput): Refactored FSDirectory.FSIndexOutput to utilize the FileStream buffer only, rather than using both a FileStream buffer and the buffer in BufferedIndexOutput.
---
 src/Lucene.Net/Store/BufferedIndexOutput.cs | 27 ++++++-----
 src/Lucene.Net/Store/FSDirectory.cs         | 75 +++++++++++++++++++++++------
 2 files changed, 74 insertions(+), 28 deletions(-)

diff --git a/src/Lucene.Net/Store/BufferedIndexOutput.cs b/src/Lucene.Net/Store/BufferedIndexOutput.cs
index 6f3f12d..4b59b01 100644
--- a/src/Lucene.Net/Store/BufferedIndexOutput.cs
+++ b/src/Lucene.Net/Store/BufferedIndexOutput.cs
@@ -30,10 +30,10 @@ namespace Lucene.Net.Store
         public const int DEFAULT_BUFFER_SIZE = 16384;
 
         private readonly int bufferSize;
-        private readonly byte[] buffer;
+        private byte[] buffer;
         private long bufferStart = 0; // position in file of buffer
         private int bufferPosition = 0; // position in buffer
-        private readonly CRC32 crc = new CRC32();
+        private readonly CRC32 crc;
 
         /// <summary>
         /// Creates a new <see cref="BufferedIndexOutput"/> with the default buffer size
@@ -48,18 +48,24 @@ namespace Lucene.Net.Store
         /// Creates a new <see cref="BufferedIndexOutput"/> with the given buffer size. </summary>
         /// <param name="bufferSize"> the buffer size in bytes used to buffer writes internally. </param>
         /// <exception cref="ArgumentException"> if the given buffer size is less or equal to <c>0</c> </exception>
-        public BufferedIndexOutput(int bufferSize)
+        public BufferedIndexOutput(int bufferSize) : this(bufferSize, new CRC32()) { }
+
+        // LUCENENET specific - added constructor overload so FSDirectory can still subclass BufferedIndexOutput, but
+        // utilize its own buffer, since FileStream is already buffered in .NET.
+        internal BufferedIndexOutput(int bufferSize, CRC32 crc)
         {
             if (bufferSize <= 0)
             {
                 throw new ArgumentException("bufferSize must be greater than 0 (got " + bufferSize + ")");
             }
             this.bufferSize = bufferSize;
-            buffer = new byte[bufferSize];
+            // LUCENENET: We lazy-load the buffer, so we don't force all subclasses to allocate it
+            this.crc = crc;
         }
 
         public override void WriteByte(byte b)
         {
+            if (buffer == null) buffer = new byte[bufferSize]; // LUCENENET: Lazy-load the buffer, so we don't force all subclasses to allocate it
             if (bufferPosition >= bufferSize)
             {
                 Flush();
@@ -69,6 +75,7 @@ namespace Lucene.Net.Store
 
         public override void WriteBytes(byte[] b, int offset, int length)
         {
+            if (buffer == null) buffer = new byte[bufferSize]; // LUCENENET: Lazy-load the buffer, so we don't force all subclasses to allocate it
             int bytesLeft = bufferSize - bufferPosition;
             // is there enough space in the buffer?
             if (bytesLeft >= length)
@@ -120,9 +127,11 @@ namespace Lucene.Net.Store
             }
         }
 
+        /// <inheritdoc/>
         [MethodImpl(MethodImplOptions.NoInlining)]
         public override void Flush()
         {
+            if (buffer == null) return; // LUCENENET: Lazy-load the buffer, so we don't force all subclasses to allocate it
             crc.Update(buffer, 0, bufferPosition);
             FlushBuffer(buffer, bufferPosition);
             bufferStart += bufferPosition;
@@ -147,6 +156,7 @@ namespace Lucene.Net.Store
         /// <param name="len"> the number of bytes to write </param>
         protected internal abstract void FlushBuffer(byte[] b, int offset, int len);
 
+        /// <inheritdoc/>
         protected override void Dispose(bool disposing)
         {
             if (disposing)
@@ -171,15 +181,8 @@ namespace Lucene.Net.Store
 
         /// <summary>
         /// Returns size of the used output buffer in bytes.
-        ///
         /// </summary>
-        public int BufferSize
-        {
-            get
-            {
-                return bufferSize;
-            }
-        }
+        public int BufferSize => bufferSize;
 
         public override long Checksum
         {
diff --git a/src/Lucene.Net/Store/FSDirectory.cs b/src/Lucene.Net/Store/FSDirectory.cs
index a07021e..00ddcd3 100644
--- a/src/Lucene.Net/Store/FSDirectory.cs
+++ b/src/Lucene.Net/Store/FSDirectory.cs
@@ -1,3 +1,4 @@
+using Lucene.Net.Support;
 using Lucene.Net.Support.IO;
 using System;
 using System.Collections.Generic;
@@ -5,6 +6,7 @@ using System.Diagnostics;
 using System.Globalization;
 using System.IO;
 using System.Linq;// Used only for WRITE_LOCK_NAME in deprecated create=true case:
+using System.Runtime.CompilerServices;
 
 namespace Lucene.Net.Store
 {
@@ -462,51 +464,81 @@ namespace Lucene.Net.Store
             get { return chunkSize; }
         }
 
-        /// <summary>
+        ///// <summary>
         /// Writes output with <see cref="FileStream.Write(byte[], int, int)"/>
         /// </summary>
+        // LUCENENET specific: Since FileStream does its own buffering, this class was refactored
+        // to do all checksum operations as well as writing to the FileStream. By doing this we elminate
+        // the extra set of buffers that were only creating unnecessary memory allocations and copy operations.
         protected class FSIndexOutput : BufferedIndexOutput
         {
-            // LUCENENET specific: chunk size not needed
+            private const int CHUNK_SIZE = DEFAULT_BUFFER_SIZE;
 
             private readonly FSDirectory parent;
             internal readonly string name;
             private readonly FileStream file;
             private volatile bool isOpen; // remember if the file is open, so that we don't try to close it more than once
+            private readonly CRC32 crc = new CRC32();
 
             public FSIndexOutput(FSDirectory parent, string name)
-                : base(/*CHUNK_SIZE*/)
+                : base(CHUNK_SIZE, null)
             {
                 this.parent = parent;
                 this.name = name;
-                file = new FileStream(Path.Combine(parent.m_directory.FullName, name), FileMode.OpenOrCreate, FileAccess.ReadWrite, FileShare.ReadWrite);
+                file = new FileStream(
+                    path: Path.Combine(parent.m_directory.FullName, name),
+                    mode: FileMode.OpenOrCreate,
+                    access: FileAccess.Write,
+                    share: FileShare.ReadWrite,
+                    bufferSize: CHUNK_SIZE);
                 isOpen = true;
             }
 
+            /// <inheritdoc/>
+            public override void WriteByte(byte b)
+            {
+                if (!isOpen)
+                    throw new ObjectDisposedException(nameof(FSIndexOutput));
+
+                crc.Update(b);
+                file.WriteByte(b);
+            }
+
+            /// <inheritdoc/>
+            public override void WriteBytes(byte[] b, int offset, int length)
+            {
+                if (!isOpen)
+                    throw new ObjectDisposedException(nameof(FSIndexOutput));
+
+                crc.Update(b, offset, length);
+                file.Write(b, offset, length);
+            }
+
+            /// <inheritdoc/>
             protected internal override void FlushBuffer(byte[] b, int offset, int size)
             {
-                Debug.Assert(isOpen);
+                if (!isOpen)
+                    throw new ObjectDisposedException(nameof(FSIndexOutput));
 
-                // LUCENENET specific: FileStream is already optimized to write natively
-                // if over the buffer size that is passed through its constructor. So,
-                // all we need to do is Write().
+                crc.Update(b, offset, size);
                 file.Write(b, offset, size);
-
-                //Debug.Assert(size == 0);
             }
 
+            /// <inheritdoc/>
+            [MethodImpl(MethodImplOptions.NoInlining)]
             public override void Flush()
             {
-                base.Flush();
-                // LUCENENET specific - writing bytes into the FileStream (in FlushBuffer()) does not immediately
-                // persist them on disk. We need to explicitly call FileStream.Flush() to move them there.
+                if (!isOpen)
+                    throw new ObjectDisposedException(nameof(FSIndexOutput));
+
                 file.Flush();
             }
 
+            /// <inheritdoc/>
             protected override void Dispose(bool disposing)
             {
                 if (disposing)
-                { 
+                {
                     parent.OnIndexOutputClosed(this);
                     // only close the file if it has not been closed yet
                     if (isOpen)
@@ -514,7 +546,6 @@ namespace Lucene.Net.Store
                         IOException priorE = null;
                         try
                         {
-                            base.Dispose(disposing); // LUCENENET NOTE: This handles Flush() for us automatically, but we need to call Flush(true) to ensure everything persists
                             file.Flush(flushToDisk: true);
                         }
                         catch (IOException ioe)
@@ -535,13 +566,25 @@ namespace Lucene.Net.Store
             [Obsolete("(4.1) this method will be removed in Lucene 5.0")]
             public override void Seek(long pos)
             {
-                base.Seek(pos);
+                if (!isOpen)
+                    throw new ObjectDisposedException(nameof(FSIndexOutput));
+
                 file.Seek(pos, SeekOrigin.Begin);
             }
 
+            /// <inheritdoc/>
             public override long Length => file.Length;
 
             // LUCENENET NOTE: FileStream doesn't have a way to set length
+
+            /// <inheritdoc/>
+            public override long Checksum => crc.Value; // LUCENENET specific - need to override, since we are buffering locally
+
+            /// <inheritdoc/>
+            public override long GetFilePointer() // LUCENENET specific - need to override, since we are buffering locally
+            {
+                return file.Position;
+            }
         }
 
         // LUCENENET specific: Fsync is pointless in .NET, since we are 


[lucenenet] 01/05: BUG: Added missing call to FileStream::Flush() in FSIndexOutput::Flush() that was preventing persistence to disk from occuring at the necessary phase (fixes LUCENENET-645)

Posted by ni...@apache.org.
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

commit d359f83eb11b1d793f516f7fb24820ee2773eae5
Author: Shad Storhaug <sh...@shadstorhaug.com>
AuthorDate: Tue Feb 11 17:07:11 2020 +0700

    BUG: Added missing call to FileStream::Flush() in FSIndexOutput::Flush() that was preventing persistence to disk from occuring at the necessary phase (fixes LUCENENET-645)
---
 src/Lucene.Net/Store/FSDirectory.cs | 33 ++++++++++++---------------------
 1 file changed, 12 insertions(+), 21 deletions(-)

diff --git a/src/Lucene.Net/Store/FSDirectory.cs b/src/Lucene.Net/Store/FSDirectory.cs
index ff99dac..a07021e 100644
--- a/src/Lucene.Net/Store/FSDirectory.cs
+++ b/src/Lucene.Net/Store/FSDirectory.cs
@@ -468,11 +468,6 @@ namespace Lucene.Net.Store
         protected class FSIndexOutput : BufferedIndexOutput
         {
             // LUCENENET specific: chunk size not needed
-            ///// <summary>
-            ///// The maximum chunk size is 8192 bytes, because <seealso cref="RandomAccessFile"/> mallocs
-            ///// a native buffer outside of stack if the write buffer size is larger.
-            ///// </summary>
-            //private const int CHUNK_SIZE = 8192;
 
             private readonly FSDirectory parent;
             internal readonly string name;
@@ -490,14 +485,7 @@ namespace Lucene.Net.Store
 
             protected internal override void FlushBuffer(byte[] b, int offset, int size)
             {
-                //Debug.Assert(isOpen);
-                //while (size > 0)
-                //{
-                //    int toWrite = Math.Min(CHUNK_SIZE, size);
-                //    file.Write(b, offset, toWrite);
-                //    offset += toWrite;
-                //    size -= toWrite;
-                //}
+                Debug.Assert(isOpen);
 
                 // LUCENENET specific: FileStream is already optimized to write natively
                 // if over the buffer size that is passed through its constructor. So,
@@ -507,6 +495,14 @@ namespace Lucene.Net.Store
                 //Debug.Assert(size == 0);
             }
 
+            public override void Flush()
+            {
+                base.Flush();
+                // LUCENENET specific - writing bytes into the FileStream (in FlushBuffer()) does not immediately
+                // persist them on disk. We need to explicitly call FileStream.Flush() to move them there.
+                file.Flush();
+            }
+
             protected override void Dispose(bool disposing)
             {
                 if (disposing)
@@ -518,9 +514,7 @@ namespace Lucene.Net.Store
                         IOException priorE = null;
                         try
                         {
-                            base.Dispose(disposing);
-                            // LUCENENET specific - file.Flush(flushToDisk: true) required in .NET for concurrency
-                            // Part of a solution suggested by Vincent Van Den Berghe: http://apache.markmail.org/message/hafnuhq2ydhfjmi2
+                            base.Dispose(disposing); // LUCENENET NOTE: This handles Flush() for us automatically, but we need to call Flush(true) to ensure everything persists
                             file.Flush(flushToDisk: true);
                         }
                         catch (IOException ioe)
@@ -545,10 +539,7 @@ namespace Lucene.Net.Store
                 file.Seek(pos, SeekOrigin.Begin);
             }
 
-            public override long Length
-            {
-                get { return file.Length; }
-            }
+            public override long Length => file.Length;
 
             // LUCENENET NOTE: FileStream doesn't have a way to set length
         }
@@ -558,7 +549,7 @@ namespace Lucene.Net.Store
         // which means we never need it at the point in Java where it is called.
         //protected virtual void Fsync(string name)
         //{
-        //    IOUtils.Fsync(Path.Combine(m_directory.FullName, name), false);            
+        //    IOUtils.Fsync(Path.Combine(m_directory.FullName, name), false);
         //}
     }
 }
\ No newline at end of file


[lucenenet] 02/05: SWEEP: Pass Random instance to all Shuffle() method calls to ensure the same psuedo-random sequence is used based on the seed.

Posted by ni...@apache.org.
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

commit a75a3a339c030e68f2db30d4022440cb78499a8b
Author: Shad Storhaug <sh...@shadstorhaug.com>
AuthorDate: Mon Feb 10 20:11:04 2020 +0700

    SWEEP: Pass Random instance to all Shuffle() method calls to ensure the same psuedo-random sequence is used based on the seed.
---
 .../Index/BaseStoredFieldsFormatTestCase.cs                  |  2 +-
 src/Lucene.Net.TestFramework/Util/LuceneTestCase.cs          |  4 ++--
 .../Taxonomy/TestTaxonomyFacetCounts2.cs                     |  4 ++--
 src/Lucene.Net.Tests.Join/TestBlockJoin.cs                   |  6 +++---
 src/Lucene.Net.Tests.Misc/Index/Sorter/SorterTestBase.cs     |  2 +-
 src/Lucene.Net.Tests.Queries/TermsFilterTest.cs              |  4 ++--
 .../Suggest/Analyzing/AnalyzingSuggesterTest.cs              |  2 +-
 .../Suggest/Analyzing/FuzzySuggesterTest.cs                  |  2 +-
 src/Lucene.Net.Tests.Suggest/Suggest/LookupBenchmarkTest.cs  |  4 ++--
 .../Codecs/Lucene3x/TestTermInfosReaderIndex.cs              |  2 +-
 .../Codecs/Lucene40/TestLucene40PostingsReader.cs            |  2 +-
 .../Codecs/Lucene41/TestBlockPostingsFormat3.cs              |  2 +-
 src/Lucene.Net.Tests/Index/Test2BTerms.cs                    |  2 +-
 src/Lucene.Net.Tests/Index/TestBagOfPositions.cs             |  2 +-
 src/Lucene.Net.Tests/Index/TestBagOfPostings.cs              |  2 +-
 src/Lucene.Net.Tests/Index/TestIndexWriterDelete.cs          |  4 ++--
 src/Lucene.Net.Tests/Index/TestMaxTermFrequency.cs           |  2 +-
 src/Lucene.Net.Tests/Index/TestTermsEnum2.cs                 |  2 +-
 src/Lucene.Net.Tests/Search/TestMinShouldMatch2.cs           |  6 +++---
 src/Lucene.Net.Tests/Search/TestSameScoresWithThreads.cs     |  2 +-
 src/Lucene.Net.Tests/Search/TestShardSearching.cs            |  2 +-
 src/Lucene.Net.Tests/Search/TestSort.cs                      | 12 ++++++------
 src/Lucene.Net.Tests/Util/Fst/TestFSTs.cs                    |  2 +-
 23 files changed, 37 insertions(+), 37 deletions(-)

diff --git a/src/Lucene.Net.TestFramework/Index/BaseStoredFieldsFormatTestCase.cs b/src/Lucene.Net.TestFramework/Index/BaseStoredFieldsFormatTestCase.cs
index 7a7d31c..da7f856 100644
--- a/src/Lucene.Net.TestFramework/Index/BaseStoredFieldsFormatTestCase.cs
+++ b/src/Lucene.Net.TestFramework/Index/BaseStoredFieldsFormatTestCase.cs
@@ -138,7 +138,7 @@ namespace Lucene.Net.Index
                         if (rand.Next(50) == 17)
                         {
                             // mixup binding of field name -> Number every so often
-                            fieldIDs.Shuffle();
+                            fieldIDs.Shuffle(Random);
                         }
                         if (rand.Next(5) == 3 && i > 0)
                         {
diff --git a/src/Lucene.Net.TestFramework/Util/LuceneTestCase.cs b/src/Lucene.Net.TestFramework/Util/LuceneTestCase.cs
index ff36d6e..e2af615 100644
--- a/src/Lucene.Net.TestFramework/Util/LuceneTestCase.cs
+++ b/src/Lucene.Net.TestFramework/Util/LuceneTestCase.cs
@@ -1835,7 +1835,7 @@ namespace Lucene.Net.Util
                             {
                                 allFields.Add(fi.Name);
                             }
-                            allFields.Shuffle();
+                            allFields.Shuffle(Random);
                             int end = allFields.Count == 0 ? 0 : random.Next(allFields.Count);
                             ISet<string> fields = new JCG.HashSet<string>(allFields.SubList(0, end));
                             // will create no FC insanity as ParallelAtomicReader has own cache key:
@@ -2622,7 +2622,7 @@ namespace Lucene.Net.Util
             rightEnum = rightTerms.GetIterator(rightEnum);
 
             IList<BytesRef> shuffledTests = new List<BytesRef>(tests);
-            shuffledTests.Shuffle();
+            shuffledTests.Shuffle(Random);
 
             foreach (BytesRef b in shuffledTests)
             {
diff --git a/src/Lucene.Net.Tests.Facet/Taxonomy/TestTaxonomyFacetCounts2.cs b/src/Lucene.Net.Tests.Facet/Taxonomy/TestTaxonomyFacetCounts2.cs
index bdfe28c..aca39cd 100644
--- a/src/Lucene.Net.Tests.Facet/Taxonomy/TestTaxonomyFacetCounts2.cs
+++ b/src/Lucene.Net.Tests.Facet/Taxonomy/TestTaxonomyFacetCounts2.cs
@@ -99,8 +99,8 @@ namespace Lucene.Net.Facet.Taxonomy
             categories_a.AddRange(CATEGORIES_A);
             List<FacetField> categories_b = new List<FacetField>();
             categories_b.AddRange(CATEGORIES_B);
-            categories_a.Shuffle();
-            categories_b.Shuffle();
+            categories_a.Shuffle(Random);
+            categories_b.Shuffle(Random);
 
             List<FacetField> categories = new List<FacetField>();
             categories.AddRange(categories_a.SubList(0, numFacetsA));
diff --git a/src/Lucene.Net.Tests.Join/TestBlockJoin.cs b/src/Lucene.Net.Tests.Join/TestBlockJoin.cs
index 0a8159d..751f318 100644
--- a/src/Lucene.Net.Tests.Join/TestBlockJoin.cs
+++ b/src/Lucene.Net.Tests.Join/TestBlockJoin.cs
@@ -297,13 +297,13 @@ namespace Lucene.Net.Tests.Join
             IList<Document> docs = new List<Document>();
             docs.Add(MakeJob("java", 2007));
             docs.Add(MakeJob("python", 2010));
-            docs.Shuffle();
+            docs.Shuffle(Random);
             docs.Add(MakeResume("Lisa", "United Kingdom"));
 
             IList<Document> docs2 = new List<Document>();
             docs2.Add(MakeJob("ruby", 2005));
             docs2.Add(MakeJob("java", 2006));
-            docs2.Shuffle();
+            docs2.Shuffle(Random);
             docs2.Add(MakeResume("Frank", "United States"));
 
             AddSkillless(w);
@@ -1320,7 +1320,7 @@ namespace Lucene.Net.Tests.Join
             docs.Add(MakeJob("java", 2006));
             docs.Add(MakeJob("java", 2010));
             docs.Add(MakeJob("java", 2012));
-            docs.Shuffle();
+            docs.Shuffle(Random);
             docs.Add(MakeResume("Frank", "United States"));
 
             AddSkillless(w);
diff --git a/src/Lucene.Net.Tests.Misc/Index/Sorter/SorterTestBase.cs b/src/Lucene.Net.Tests.Misc/Index/Sorter/SorterTestBase.cs
index d63b643..8152467 100644
--- a/src/Lucene.Net.Tests.Misc/Index/Sorter/SorterTestBase.cs
+++ b/src/Lucene.Net.Tests.Misc/Index/Sorter/SorterTestBase.cs
@@ -182,7 +182,7 @@ namespace Lucene.Net.Index.Sorter
                 ids.Add(i * 10);
             }
             // shuffle them for indexing
-            ids.Shuffle();
+            ids.Shuffle(Random);
 
             if (VERBOSE)
             {
diff --git a/src/Lucene.Net.Tests.Queries/TermsFilterTest.cs b/src/Lucene.Net.Tests.Queries/TermsFilterTest.cs
index 88d512a..3a1dc4c 100644
--- a/src/Lucene.Net.Tests.Queries/TermsFilterTest.cs
+++ b/src/Lucene.Net.Tests.Queries/TermsFilterTest.cs
@@ -259,7 +259,7 @@ namespace Lucene.Net.Tests.Queries
             int numQueries = AtLeast(10);
             for (int i = 0; i < numQueries; i++)
             {
-                terms.Shuffle();
+                terms.Shuffle(Random);
                 int numTerms = 1 + Random.Next(Math.Min(BooleanQuery.MaxClauseCount, terms.Count));
                 BooleanQuery bq = new BooleanQuery();
                 for (int j = 0; j < numTerms; j++)
@@ -325,7 +325,7 @@ namespace Lucene.Net.Tests.Queries
                 terms.Add(new Term(field, @string));
                 uniqueTerms.Add(new Term(field, @string));
                 TermsFilter left = TermsFilter(singleField && Random.NextBoolean(), uniqueTerms);
-                terms.Shuffle();
+                terms.Shuffle(Random);
                 TermsFilter right = TermsFilter(singleField && Random.NextBoolean(), terms);
                 assertEquals(right, left);
                 assertEquals(right.GetHashCode(), left.GetHashCode());
diff --git a/src/Lucene.Net.Tests.Suggest/Suggest/Analyzing/AnalyzingSuggesterTest.cs b/src/Lucene.Net.Tests.Suggest/Suggest/Analyzing/AnalyzingSuggesterTest.cs
index 248f489..83b5324 100644
--- a/src/Lucene.Net.Tests.Suggest/Suggest/Analyzing/AnalyzingSuggesterTest.cs
+++ b/src/Lucene.Net.Tests.Suggest/Suggest/Analyzing/AnalyzingSuggesterTest.cs
@@ -1502,7 +1502,7 @@ namespace Lucene.Net.Search.Suggest.Analyzing
             {
                 asList.Add(value);
             }
-            asList.Shuffle();
+            asList.Shuffle(Random);
             return asList;
         }
     }
diff --git a/src/Lucene.Net.Tests.Suggest/Suggest/Analyzing/FuzzySuggesterTest.cs b/src/Lucene.Net.Tests.Suggest/Suggest/Analyzing/FuzzySuggesterTest.cs
index c960f6d..06a972d 100644
--- a/src/Lucene.Net.Tests.Suggest/Suggest/Analyzing/FuzzySuggesterTest.cs
+++ b/src/Lucene.Net.Tests.Suggest/Suggest/Analyzing/FuzzySuggesterTest.cs
@@ -1154,7 +1154,7 @@ namespace Lucene.Net.Search.Suggest.Analyzing
                 Console.WriteLine("TEST: maxEdits=" + maxEdits + " prefixLen=" + prefixLen + " transpositions=" + transpositions + " num=" + NUM);
             }
 
-            answers.Shuffle();
+            answers.Shuffle(Random);
             suggest.Build(new InputArrayIterator(answers.ToArray()));
 
             int ITERS = AtLeast(100);
diff --git a/src/Lucene.Net.Tests.Suggest/Suggest/LookupBenchmarkTest.cs b/src/Lucene.Net.Tests.Suggest/Suggest/LookupBenchmarkTest.cs
index 175d5fa..15094cc 100644
--- a/src/Lucene.Net.Tests.Suggest/Suggest/LookupBenchmarkTest.cs
+++ b/src/Lucene.Net.Tests.Suggest/Suggest/LookupBenchmarkTest.cs
@@ -77,9 +77,9 @@ namespace Lucene.Net.Search.Suggest
         {
             Debug.Assert(false, "disable assertions before running benchmarks!");
             IList<Input> input = ReadTop50KWiki();
-            input.Shuffle();
+            input.Shuffle(Random);
             dictionaryInput = input.ToArray();
-            input.Shuffle();
+            input.Shuffle(Random);
             benchmarkInput = input;
         }
 
diff --git a/src/Lucene.Net.Tests/Codecs/Lucene3x/TestTermInfosReaderIndex.cs b/src/Lucene.Net.Tests/Codecs/Lucene3x/TestTermInfosReaderIndex.cs
index 46134aa..a708dde 100644
--- a/src/Lucene.Net.Tests/Codecs/Lucene3x/TestTermInfosReaderIndex.cs
+++ b/src/Lucene.Net.Tests/Codecs/Lucene3x/TestTermInfosReaderIndex.cs
@@ -187,7 +187,7 @@ namespace Lucene.Net.Codecs.Lucene3x
                     }
                 }
             }
-            sample.Shuffle();
+            sample.Shuffle(Random);
             return sample;
         }
 
diff --git a/src/Lucene.Net.Tests/Codecs/Lucene40/TestLucene40PostingsReader.cs b/src/Lucene.Net.Tests/Codecs/Lucene40/TestLucene40PostingsReader.cs
index 2ab8c24..744895a 100644
--- a/src/Lucene.Net.Tests/Codecs/Lucene40/TestLucene40PostingsReader.cs
+++ b/src/Lucene.Net.Tests/Codecs/Lucene40/TestLucene40PostingsReader.cs
@@ -153,7 +153,7 @@ namespace Lucene.Net.Codecs.Lucene40
                 }
                 i++;
             }
-            shuffled.Shuffle();
+            shuffled.Shuffle(Random);
             foreach (string term in shuffled)
             {
                 sb.Append(term);
diff --git a/src/Lucene.Net.Tests/Codecs/Lucene41/TestBlockPostingsFormat3.cs b/src/Lucene.Net.Tests/Codecs/Lucene41/TestBlockPostingsFormat3.cs
index 20f4593..1b85750 100644
--- a/src/Lucene.Net.Tests/Codecs/Lucene41/TestBlockPostingsFormat3.cs
+++ b/src/Lucene.Net.Tests/Codecs/Lucene41/TestBlockPostingsFormat3.cs
@@ -274,7 +274,7 @@ namespace Lucene.Net.Codecs.Lucene41
             }
 
             List<BytesRef> shuffledTests = new List<BytesRef>(tests);
-            shuffledTests.Shuffle();
+            shuffledTests.Shuffle(Random);
 
             foreach (BytesRef b in shuffledTests)
             {
diff --git a/src/Lucene.Net.Tests/Index/Test2BTerms.cs b/src/Lucene.Net.Tests/Index/Test2BTerms.cs
index e1f09a2..58903f7 100644
--- a/src/Lucene.Net.Tests/Index/Test2BTerms.cs
+++ b/src/Lucene.Net.Tests/Index/Test2BTerms.cs
@@ -282,7 +282,7 @@ namespace Lucene.Net.Index
         {
             Console.WriteLine("TEST: run " + terms.Count + " terms on reader=" + r);
             IndexSearcher s = NewSearcher(r);
-            terms.Shuffle();
+            terms.Shuffle(Random);
             TermsEnum termsEnum = MultiFields.GetTerms(r, "field").GetIterator(null);
             bool failed = false;
             for (int iter = 0; iter < 10 * terms.Count; iter++)
diff --git a/src/Lucene.Net.Tests/Index/TestBagOfPositions.cs b/src/Lucene.Net.Tests/Index/TestBagOfPositions.cs
index 901229b..6a93bfb 100644
--- a/src/Lucene.Net.Tests/Index/TestBagOfPositions.cs
+++ b/src/Lucene.Net.Tests/Index/TestBagOfPositions.cs
@@ -79,7 +79,7 @@ namespace Lucene.Net.Index
                 }
             }
 
-            postingsList.Shuffle();
+            postingsList.Shuffle(Random);
 
             ConcurrentQueue<string> postings = new ConcurrentQueue<string>(postingsList);
 
diff --git a/src/Lucene.Net.Tests/Index/TestBagOfPostings.cs b/src/Lucene.Net.Tests/Index/TestBagOfPostings.cs
index 3dd9274..68806a3 100644
--- a/src/Lucene.Net.Tests/Index/TestBagOfPostings.cs
+++ b/src/Lucene.Net.Tests/Index/TestBagOfPostings.cs
@@ -79,7 +79,7 @@ namespace Lucene.Net.Index
                     postingsList.Add(term);
                 }
             }
-            postingsList.Shuffle();
+            postingsList.Shuffle(Random);
 
             ConcurrentQueue<string> postings = new ConcurrentQueue<string>(postingsList);
 
diff --git a/src/Lucene.Net.Tests/Index/TestIndexWriterDelete.cs b/src/Lucene.Net.Tests/Index/TestIndexWriterDelete.cs
index e9586cc..8477174 100644
--- a/src/Lucene.Net.Tests/Index/TestIndexWriterDelete.cs
+++ b/src/Lucene.Net.Tests/Index/TestIndexWriterDelete.cs
@@ -1123,14 +1123,14 @@ namespace Lucene.Net.Index
             {
                 ids.Add(id);
             }
-            ids.Shuffle();
+            ids.Shuffle(Random);
             foreach (int id in ids)
             {
                 Document doc = new Document();
                 doc.Add(NewStringField("id", "" + id, Field.Store.NO));
                 w.AddDocument(doc);
             }
-            ids.Shuffle();
+            ids.Shuffle(Random);
             int upto = 0;
             while (upto < ids.Count)
             {
diff --git a/src/Lucene.Net.Tests/Index/TestMaxTermFrequency.cs b/src/Lucene.Net.Tests/Index/TestMaxTermFrequency.cs
index 25fc445..69a6072 100644
--- a/src/Lucene.Net.Tests/Index/TestMaxTermFrequency.cs
+++ b/src/Lucene.Net.Tests/Index/TestMaxTermFrequency.cs
@@ -108,7 +108,7 @@ namespace Lucene.Net.Index
             }
             Expected.Add(max);
 
-            terms.Shuffle();
+            terms.Shuffle(Random);
             return Arrays.ToString(terms.ToArray());
         }
 
diff --git a/src/Lucene.Net.Tests/Index/TestTermsEnum2.cs b/src/Lucene.Net.Tests/Index/TestTermsEnum2.cs
index eda537e..7609376 100644
--- a/src/Lucene.Net.Tests/Index/TestTermsEnum2.cs
+++ b/src/Lucene.Net.Tests/Index/TestTermsEnum2.cs
@@ -129,7 +129,7 @@ namespace Lucene.Net.Index
                 Automaton automaton = (new RegExp(reg, RegExpSyntax.NONE)).ToAutomaton();
                 TermsEnum te = MultiFields.GetTerms(Reader, "field").GetIterator(null);
                 IList<BytesRef> unsortedTerms = new List<BytesRef>(Terms);
-                unsortedTerms.Shuffle();
+                unsortedTerms.Shuffle(Random);
 
                 foreach (BytesRef term in unsortedTerms)
                 {
diff --git a/src/Lucene.Net.Tests/Search/TestMinShouldMatch2.cs b/src/Lucene.Net.Tests/Search/TestMinShouldMatch2.cs
index 222073c..84e92cd 100644
--- a/src/Lucene.Net.Tests/Search/TestMinShouldMatch2.cs
+++ b/src/Lucene.Net.Tests/Search/TestMinShouldMatch2.cs
@@ -135,7 +135,7 @@ namespace Lucene.Net.Search
         private static void AddSome(Document doc, string[] values)
         {
             IList<string> list = values.ToArray();
-            list.Shuffle();
+            list.Shuffle(Random);
             int howMany = TestUtil.NextInt32(Random, 1, list.Count);
             for (int i = 0; i < howMany; i++)
             {
@@ -296,7 +296,7 @@ namespace Lucene.Net.Search
             termsList.AddRange(CommonTerms);
             termsList.AddRange(MediumTerms);
             termsList.AddRange(RareTerms);
-            termsList.Shuffle();
+            termsList.Shuffle(Random);
 
             for (int numTerms = 2; numTerms <= termsList.Count; numTerms++)
             {
@@ -319,7 +319,7 @@ namespace Lucene.Net.Search
             termsList.AddRange(CommonTerms);
             termsList.AddRange(MediumTerms);
             termsList.AddRange(RareTerms);
-            termsList.Shuffle();
+            termsList.Shuffle(Random);
 
             for (int amount = 25; amount < 200; amount += 25)
             {
diff --git a/src/Lucene.Net.Tests/Search/TestSameScoresWithThreads.cs b/src/Lucene.Net.Tests/Search/TestSameScoresWithThreads.cs
index 2420619..7b2d295 100644
--- a/src/Lucene.Net.Tests/Search/TestSameScoresWithThreads.cs
+++ b/src/Lucene.Net.Tests/Search/TestSameScoresWithThreads.cs
@@ -138,7 +138,7 @@ namespace Lucene.Net.Search
                     for (int i = 0; i < 20; i++)
                     {
                         IList<KeyValuePair<BytesRef, TopDocs>> shuffled = new List<KeyValuePair<BytesRef, TopDocs>>(Answers);
-                        shuffled.Shuffle();
+                        shuffled.Shuffle(Random);
                         foreach (KeyValuePair<BytesRef, TopDocs> ent in shuffled)
                         {
                             TopDocs actual = s.Search(new TermQuery(new Term("body", ent.Key)), 100);
diff --git a/src/Lucene.Net.Tests/Search/TestShardSearching.cs b/src/Lucene.Net.Tests/Search/TestShardSearching.cs
index 50385b9..df881d2 100644
--- a/src/Lucene.Net.Tests/Search/TestShardSearching.cs
+++ b/src/Lucene.Net.Tests/Search/TestShardSearching.cs
@@ -325,7 +325,7 @@ namespace Lucene.Net.Search
                     priorSearches.Add(searchState);
                     if (priorSearches.Count > 200)
                     {
-                        priorSearches.Shuffle();
+                        priorSearches.Shuffle(Random);
                         priorSearches.SubList(100, priorSearches.Count).Clear();
                     }
                 }
diff --git a/src/Lucene.Net.Tests/Search/TestSort.cs b/src/Lucene.Net.Tests/Search/TestSort.cs
index bc7cc65..a47b7bd 100644
--- a/src/Lucene.Net.Tests/Search/TestSort.cs
+++ b/src/Lucene.Net.Tests/Search/TestSort.cs
@@ -1707,7 +1707,7 @@ namespace Lucene.Net.Search
         public virtual void TestCustomIntParser()
         {
             IList<string> letters = new List<string> { "A", "B", "C", "D", "E", "F", "G", "H", "I", "J" };
-            letters.Shuffle();
+            letters.Shuffle(Random);
 
             Directory dir = NewDirectory();
             RandomIndexWriter iw = new RandomIndexWriter(
@@ -1772,7 +1772,7 @@ namespace Lucene.Net.Search
         public virtual void TestCustomByteParser()
         {
             IList<string> letters = new List<string> { "A", "B", "C", "D", "E", "F", "G", "H", "I", "J" };
-            letters.Shuffle();
+            letters.Shuffle(Random);
 
             Directory dir = NewDirectory();
             RandomIndexWriter iw = new RandomIndexWriter(
@@ -1836,7 +1836,7 @@ namespace Lucene.Net.Search
         public virtual void TestCustomShortParser()
         {
             IList<string> letters = new List<string> { "A", "B", "C", "D", "E", "F", "G", "H", "I", "J" };
-            letters.Shuffle();
+            letters.Shuffle(Random);
 
             Directory dir = NewDirectory();
             RandomIndexWriter iw = new RandomIndexWriter(
@@ -1903,7 +1903,7 @@ namespace Lucene.Net.Search
         public virtual void TestCustomLongParser()
         {
             IList<string> letters = new List<string> { "A", "B", "C", "D", "E", "F", "G", "H", "I", "J" };
-            letters.Shuffle();
+            letters.Shuffle(Random);
 
             Directory dir = NewDirectory();
             RandomIndexWriter iw = new RandomIndexWriter(
@@ -1968,7 +1968,7 @@ namespace Lucene.Net.Search
         public virtual void TestCustomFloatParser()
         {
             IList<string> letters = new List<string> { "A", "B", "C", "D", "E", "F", "G", "H", "I", "J" };
-            letters.Shuffle();
+            letters.Shuffle(Random);
 
             Directory dir = NewDirectory();
             RandomIndexWriter iw = new RandomIndexWriter(
@@ -2033,7 +2033,7 @@ namespace Lucene.Net.Search
         public virtual void TestCustomDoubleParser()
         {
             IList<string> letters = new List<string> { "A", "B", "C", "D", "E", "F", "G", "H", "I", "J" };
-            letters.Shuffle();
+            letters.Shuffle(Random);
 
             Directory dir = NewDirectory();
             RandomIndexWriter iw = new RandomIndexWriter(
diff --git a/src/Lucene.Net.Tests/Util/Fst/TestFSTs.cs b/src/Lucene.Net.Tests/Util/Fst/TestFSTs.cs
index d5efc6f..d61cb59 100644
--- a/src/Lucene.Net.Tests/Util/Fst/TestFSTs.cs
+++ b/src/Lucene.Net.Tests/Util/Fst/TestFSTs.cs
@@ -1234,7 +1234,7 @@ namespace Lucene.Net.Util.Fst
             w.Dispose();
 
             IList<string> allTermsList = new List<string>(allTerms);
-            allTermsList.Shuffle();
+            allTermsList.Shuffle(Random);
 
             // verify exact lookup
             foreach (string term in allTermsList)


[lucenenet] 03/05: BUG: Lucene.Net.Index.DocumentsWriterDeleteQueue::DeleteSlice(): Extra Debug.Assert() statement that wasn't in Lucene causes Lucene.Net.Index.TestIndexWriterUnicode::TestEmbeddedFFFF() test to fail when running in debug mode

Posted by ni...@apache.org.
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

commit b5a61c81ba1808d0969e5b6ae8a76070a8a92b70
Author: Shad Storhaug <sh...@shadstorhaug.com>
AuthorDate: Tue Feb 11 05:10:01 2020 +0700

    BUG: Lucene.Net.Index.DocumentsWriterDeleteQueue::DeleteSlice(): Extra Debug.Assert() statement that wasn't in Lucene causes Lucene.Net.Index.TestIndexWriterUnicode::TestEmbeddedFFFF() test to fail when running in debug mode
---
 src/Lucene.Net/Index/DocumentsWriterDeleteQueue.cs | 1 -
 1 file changed, 1 deletion(-)

diff --git a/src/Lucene.Net/Index/DocumentsWriterDeleteQueue.cs b/src/Lucene.Net/Index/DocumentsWriterDeleteQueue.cs
index 50c6666..b293828 100644
--- a/src/Lucene.Net/Index/DocumentsWriterDeleteQueue.cs
+++ b/src/Lucene.Net/Index/DocumentsWriterDeleteQueue.cs
@@ -294,7 +294,6 @@ namespace Lucene.Net.Index
             internal DeleteSlice(Node currentTail)
             {
                 Debug.Assert(currentTail != null);
-                Debug.Assert(currentTail.next == null);
                 /*
                  * Initially this is a 0 length slice pointing to the 'current' tail of
                  * the queue. Once we update the slice we only need to assign the tail and


[lucenenet] 04/05: Lucene.Net.Tests.Index (TestBagOfPositions + TestBagOfPostings): Fixed performance issue due to converting int to string and string to int in the current culture

Posted by ni...@apache.org.
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

commit 03f3bb97169d9c9e5208f6f74040461b03c06050
Author: Shad Storhaug <sh...@shadstorhaug.com>
AuthorDate: Tue Feb 11 14:13:27 2020 +0700

    Lucene.Net.Tests.Index (TestBagOfPositions + TestBagOfPostings): Fixed performance issue due to converting int to string and string to int in the current culture
---
 src/Lucene.Net.Tests/Index/TestBagOfPositions.cs | 5 +++--
 src/Lucene.Net.Tests/Index/TestBagOfPostings.cs  | 5 +++--
 2 files changed, 6 insertions(+), 4 deletions(-)

diff --git a/src/Lucene.Net.Tests/Index/TestBagOfPositions.cs b/src/Lucene.Net.Tests/Index/TestBagOfPositions.cs
index 6a93bfb..f152c98 100644
--- a/src/Lucene.Net.Tests/Index/TestBagOfPositions.cs
+++ b/src/Lucene.Net.Tests/Index/TestBagOfPositions.cs
@@ -6,6 +6,7 @@ using NUnit.Framework;
 using System;
 using System.Collections.Concurrent;
 using System.Collections.Generic;
+using System.Globalization;
 using System.Text;
 using System.Threading;
 using Console = Lucene.Net.Util.SystemConsole;
@@ -72,7 +73,7 @@ namespace Lucene.Net.Index
             }
             for (int i = 0; i < numTerms; i++)
             {
-                string term = Convert.ToString(i);
+                string term = Convert.ToString(i, CultureInfo.InvariantCulture);
                 for (int j = 0; j < i; j++)
                 {
                     postingsList.Add(term);
@@ -141,7 +142,7 @@ namespace Lucene.Net.Index
             BytesRef termBR;
             while ((termBR = termsEnum.Next()) != null)
             {
-                int value = Convert.ToInt32(termBR.Utf8ToString());
+                int value = Convert.ToInt32(termBR.Utf8ToString(), CultureInfo.InvariantCulture);
                 Assert.AreEqual(value, termsEnum.TotalTermFreq);
                 // don't really need to check more than this, as CheckIndex
                 // will verify that totalTermFreq == total number of positions seen
diff --git a/src/Lucene.Net.Tests/Index/TestBagOfPostings.cs b/src/Lucene.Net.Tests/Index/TestBagOfPostings.cs
index 68806a3..c9d82cf 100644
--- a/src/Lucene.Net.Tests/Index/TestBagOfPostings.cs
+++ b/src/Lucene.Net.Tests/Index/TestBagOfPostings.cs
@@ -10,6 +10,7 @@ using System.Threading;
 using JCG = J2N.Collections.Generic;
 using Console = Lucene.Net.Util.SystemConsole;
 using J2N.Collections.Generic.Extensions;
+using System.Globalization;
 
 namespace Lucene.Net.Index
 {
@@ -73,7 +74,7 @@ namespace Lucene.Net.Index
 
             for (int i = 0; i < numTerms; i++)
             {
-                string term = Convert.ToString(i);
+                string term = Convert.ToString(i, CultureInfo.InvariantCulture);
                 for (int j = 0; j < i; j++)
                 {
                     postingsList.Add(term);
@@ -124,7 +125,7 @@ namespace Lucene.Net.Index
             BytesRef term_;
             while ((term_ = termsEnum.Next()) != null)
             {
-                int value = Convert.ToInt32(term_.Utf8ToString());
+                int value = Convert.ToInt32(term_.Utf8ToString(), CultureInfo.InvariantCulture);
                 Assert.AreEqual(value, termsEnum.DocFreq);
                 // don't really need to check more than this, as CheckIndex
                 // will verify that docFreq == actual number of documents seen