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/03/25 06:47:40 UTC

[lucenenet] branch master updated (1ae9999 -> 6ccde70)

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 1ae9999  docs.ps1: Added validation for LuceneNetVersion argument
     new 19002a4  Lucene.Net.Facet.Taxonomy.WriterCache.Cl2oTaxonomyWriterCache: Removed timeout on ReaderWriterLockSlim as the original lock didn't have one.
     new 889a434  Lucene.Net.Facet.Taxonomy.Directory.DirectoryTaxonomyReader: Restored locks on SetCacheSize() method.
     new 6ccde70  Lucene.Net.Facet.Taxonomy.Directory.DirectoryTaxonomyWriter: Restored locks to the state of Lucene 4.8.0.

The 3 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:
 .../Taxonomy/Directory/DirectoryTaxonomyReader.cs  |  22 +-
 .../Taxonomy/Directory/DirectoryTaxonomyWriter.cs  | 236 ++++++++++-----------
 .../WriterCache/Cl2oTaxonomyWriterCache.cs         |  78 +++----
 3 files changed, 165 insertions(+), 171 deletions(-)

[lucenenet] 03/03: Lucene.Net.Facet.Taxonomy.Directory.DirectoryTaxonomyWriter: Restored locks to the state of Lucene 4.8.0.

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 6ccde70c0883140928a5660f67bc7039d4b34593
Author: Shad Storhaug <sh...@shadstorhaug.com>
AuthorDate: Tue Mar 16 21:56:22 2021 +0700

    Lucene.Net.Facet.Taxonomy.Directory.DirectoryTaxonomyWriter: Restored locks to the state of Lucene 4.8.0.
---
 .../Taxonomy/Directory/DirectoryTaxonomyWriter.cs  | 236 ++++++++++-----------
 1 file changed, 116 insertions(+), 120 deletions(-)

diff --git a/src/Lucene.Net.Facet/Taxonomy/Directory/DirectoryTaxonomyWriter.cs b/src/Lucene.Net.Facet/Taxonomy/Directory/DirectoryTaxonomyWriter.cs
index c45a3f5..cba71c9 100644
--- a/src/Lucene.Net.Facet/Taxonomy/Directory/DirectoryTaxonomyWriter.cs
+++ b/src/Lucene.Net.Facet/Taxonomy/Directory/DirectoryTaxonomyWriter.cs
@@ -370,11 +370,11 @@ namespace Lucene.Net.Facet.Taxonomy.Directory
         /// </summary>
         protected virtual void Dispose(bool disposing)
         {
-            if (disposing && !isClosed)
+            lock (syncLock)
             {
-                lock (syncLock)
+                if (disposing && !isClosed)
                 {
-                    CommitLocked();
+                    Commit();
                     DoClose();
                 }
             }
@@ -391,16 +391,19 @@ namespace Lucene.Net.Facet.Taxonomy.Directory
         /// Closes the <see cref="Index.IndexReader"/> as well as the
         /// <see cref="ITaxonomyWriterCache"/> instances that were used.
         /// </summary>
-        private void CloseResources() // LUCENENET: Made private, since this has the same purpose as Dispose(bool). Removed redundant lock.
+        private void CloseResources() // LUCENENET: Made private, since this has the same purpose as Dispose(bool).
         {
-            if (initializedReaderManager)
+            lock (syncLock)
             {
-                readerManager.Dispose();
-                readerManager = null;
-                initializedReaderManager = false;
+                if (initializedReaderManager)
+                {
+                    readerManager.Dispose();
+                    readerManager = null;
+                    initializedReaderManager = false;
+                }
+                cache?.Dispose();
+                parentStream.Dispose(); // LUCENENET specific
             }
-            cache?.Dispose();
-            parentStream.Dispose(); // LUCENENET specific
         }
 
         /// <summary>
@@ -591,7 +594,7 @@ namespace Lucene.Net.Facet.Taxonomy.Directory
             shouldRefreshReaderManager = true;
 
             // also add to the parent array
-            taxoArrays = GetTaxoArraysLocked().Add(id, parent);
+            taxoArrays = GetTaxoArrays().Add(id, parent);
 
             // NOTE: this line must be executed last, or else the cache gets updated
             // before the parents array (LUCENE-4596)
@@ -666,22 +669,22 @@ namespace Lucene.Net.Facet.Taxonomy.Directory
             }
         }
 
-        // LUCENENET: Removed the lock here because all callers already have a
-        // lock before getting in here and all callers to AddCategoryDocument also
-        // have a lock before calling it.
         private void RefreshReaderManager()
         {
-            // this method is synchronized since it cannot happen concurrently with
-            // addCategoryDocument -- when this method returns, we must know that the
-            // reader manager's state is current. also, it sets shouldRefresh to false, 
-            // and this cannot overlap with addCatDoc too.
-            // NOTE: since this method is sync'ed, it can call maybeRefresh, instead of
-            // maybeRefreshBlocking. If ever this is changed, make sure to change the
-            // call too.
-            if (shouldRefreshReaderManager && initializedReaderManager)
+            lock (syncLock)
             {
-                readerManager.MaybeRefresh();
-                shouldRefreshReaderManager = false;
+                // this method is synchronized since it cannot happen concurrently with
+                // addCategoryDocument -- when this method returns, we must know that the
+                // reader manager's state is current. also, it sets shouldRefresh to false, 
+                // and this cannot overlap with addCatDoc too.
+                // NOTE: since this method is sync'ed, it can call maybeRefresh, instead of
+                // maybeRefreshBlocking. If ever this is changed, make sure to change the
+                // call too.
+                if (shouldRefreshReaderManager && initializedReaderManager)
+                {
+                    readerManager.MaybeRefresh();
+                    shouldRefreshReaderManager = false;
+                }
             }
         }
 
@@ -689,21 +692,16 @@ namespace Lucene.Net.Facet.Taxonomy.Directory
         {
             lock (syncLock)
             {
-                CommitLocked();
-            }
-        }
-
-        private void CommitLocked() // LUCENENET: Added to avoid lock recursion
-        {
-            EnsureOpen();
-            // LUCENE-4972: if we always call setCommitData, we create empty commits
-            if (!indexWriter.CommitData.TryGetValue(INDEX_EPOCH, out string epochStr)
-                || epochStr == null
-                || Convert.ToInt64(epochStr, 16) != indexEpoch)
-            {
-                indexWriter.SetCommitData(CombinedCommitData(indexWriter.CommitData));
+                EnsureOpen();
+                // LUCENE-4972: if we always call setCommitData, we create empty commits
+                if (!indexWriter.CommitData.TryGetValue(INDEX_EPOCH, out string epochStr)
+                    || epochStr == null
+                    || Convert.ToInt64(epochStr, 16) != indexEpoch)
+                {
+                    indexWriter.SetCommitData(CombinedCommitData(indexWriter.CommitData));
+                }
+                indexWriter.Commit();
             }
-            indexWriter.Commit();
         }
 
         /// <summary>
@@ -784,114 +782,112 @@ namespace Lucene.Net.Facet.Taxonomy.Directory
         // we need to guarantee that if several threads call this concurrently, only
         // one executes it, and after it returns, the cache is updated and is either
         // complete or not.
-        private void PerhapsFillCache() // LUCENENET: Ensure the caller has a lock
+        private void PerhapsFillCache()
         {
-            if (cacheMisses < cacheMissesUntilFill)
+            lock (syncLock)
             {
-                return;
-            }
+                if (cacheMisses < cacheMissesUntilFill)
+                {
+                    return;
+                }
 
-            if (!shouldFillCache)
-            {
-                // we already filled the cache once, there's no need to re-fill it
-                return;
-            }
-            shouldFillCache = false;
+                if (!shouldFillCache)
+                {
+                    // we already filled the cache once, there's no need to re-fill it
+                    return;
+                }
+                shouldFillCache = false;
 
-            InitReaderManager();
+                InitReaderManager();
 
-            bool aborted = false;
-            DirectoryReader reader = readerManager.Acquire();
-            try
-            {
-                TermsEnum termsEnum = null;
-                DocsEnum docsEnum = null;
-                foreach (AtomicReaderContext ctx in reader.Leaves)
+                bool aborted = false;
+                DirectoryReader reader = readerManager.Acquire();
+                try
                 {
-                    Terms terms = ctx.AtomicReader.GetTerms(Consts.FULL);
-                    if (terms != null) // cannot really happen, but be on the safe side
+                    TermsEnum termsEnum = null;
+                    DocsEnum docsEnum = null;
+                    foreach (AtomicReaderContext ctx in reader.Leaves)
                     {
-                        termsEnum = terms.GetEnumerator(termsEnum);
-                        while (termsEnum.MoveNext())
+                        Terms terms = ctx.AtomicReader.GetTerms(Consts.FULL);
+                        if (terms != null) // cannot really happen, but be on the safe side
                         {
-                            if (!cache.IsFull)
-                            {
-                                BytesRef t = termsEnum.Term;
-                                // Since we guarantee uniqueness of categories, each term has exactly
-                                // one document. Also, since we do not allow removing categories (and
-                                // hence documents), there are no deletions in the index. Therefore, it
-                                // is sufficient to call next(), and then doc(), exactly once with no
-                                // 'validation' checks.
-                                FacetLabel cp = new FacetLabel(FacetsConfig.StringToPath(t.Utf8ToString()));
-                                docsEnum = termsEnum.Docs(null, docsEnum, DocsFlags.NONE);
-                                bool res = cache.Put(cp, docsEnum.NextDoc() + ctx.DocBase);
-                                if (Debugging.AssertsEnabled) Debugging.Assert(!res, "entries should not have been evicted from the cache");
-                            }
-                            else
+                            termsEnum = terms.GetEnumerator(termsEnum);
+                            while (termsEnum.MoveNext())
                             {
-                                // the cache is full and the next put() will evict entries from it, therefore abort the iteration.
-                                aborted = true;
-                                break;
+                                if (!cache.IsFull)
+                                {
+                                    BytesRef t = termsEnum.Term;
+                                    // Since we guarantee uniqueness of categories, each term has exactly
+                                    // one document. Also, since we do not allow removing categories (and
+                                    // hence documents), there are no deletions in the index. Therefore, it
+                                    // is sufficient to call next(), and then doc(), exactly once with no
+                                    // 'validation' checks.
+                                    FacetLabel cp = new FacetLabel(FacetsConfig.StringToPath(t.Utf8ToString()));
+                                    docsEnum = termsEnum.Docs(null, docsEnum, DocsFlags.NONE);
+                                    bool res = cache.Put(cp, docsEnum.NextDoc() + ctx.DocBase);
+                                    if (Debugging.AssertsEnabled) Debugging.Assert(!res, "entries should not have been evicted from the cache");
+                                }
+                                else
+                                {
+                                    // the cache is full and the next put() will evict entries from it, therefore abort the iteration.
+                                    aborted = true;
+                                    break;
+                                }
                             }
                         }
-                    }
-                    if (aborted)
-                    {
-                        break;
+                        if (aborted)
+                        {
+                            break;
+                        }
                     }
                 }
-            }
-            finally
-            {
-                readerManager.Release(reader);
-            }
-
-            cacheIsComplete = !aborted;
-            if (cacheIsComplete)
-            {
-                //lock (syncLock) // LUCENENET: the caller of this method already has a lock, so we don't need to make it recursive
+                finally
                 {
-                    // everything is in the cache, so no need to keep readerManager open.
-                    // this block is executed in a sync block so that it works well with
-                    // initReaderManager called in parallel.
-                    readerManager.Dispose();
-                    readerManager = null;
-                    initializedReaderManager = false;
+                    readerManager.Release(reader);
                 }
-            }
-        }
 
-        private TaxonomyIndexArrays GetTaxoArrays()
-        {
-            var result = taxoArrays; // LUCENENET: Careful, we need to ensure a parallel thread does't wipe out our return value
-            if (result is null)
-            {
-                lock (syncLock)
+                cacheIsComplete = !aborted;
+                if (cacheIsComplete)
                 {
-                    return GetTaxoArraysLocked();
+                    lock (syncLock)
+                    {
+                        // everything is in the cache, so no need to keep readerManager open.
+                        // this block is executed in a sync block so that it works well with
+                        // initReaderManager called in parallel.
+                        readerManager.Dispose();
+                        readerManager = null;
+                        initializedReaderManager = false;
+                    }
                 }
             }
-            return result;
         }
 
-        private TaxonomyIndexArrays GetTaxoArraysLocked()
+        private TaxonomyIndexArrays GetTaxoArrays()
         {
-            var result = taxoArrays; // LUCENENET: Careful, we need to ensure a parallel thread does't wipe out our return value
-            if (result is null)
+            if (taxoArrays is null)
             {
-                InitReaderManager();
-
-                DirectoryReader reader = readerManager.Acquire();
-                try
-                {
-                    return taxoArrays = new TaxonomyIndexArrays(reader);
-                }
-                finally
+                lock (syncLock)
                 {
-                    readerManager.Release(reader);
+                    if (taxoArrays is null)
+                    {
+                        InitReaderManager();
+                        DirectoryReader reader = readerManager.Acquire();
+                        try
+                        {
+                            // according to Java Concurrency, this might perform better on some
+                            // JVMs, since the object initialization doesn't happen on the
+                            // volatile member.
+                            var tmpArrays = new TaxonomyIndexArrays(reader);
+                            taxoArrays = tmpArrays;
+                        }
+                        finally
+                        {
+                            readerManager.Release(reader);
+                        }
+                    }
                 }
             }
-            return result;
+            return taxoArrays;
         }
 
         public virtual int GetParent(int ordinal)

[lucenenet] 02/03: Lucene.Net.Facet.Taxonomy.Directory.DirectoryTaxonomyReader: Restored locks on SetCacheSize() method.

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 889a4349218a5d789875a3fe9515a8ff199b36d7
Author: Shad Storhaug <sh...@shadstorhaug.com>
AuthorDate: Tue Mar 16 21:54:49 2021 +0700

    Lucene.Net.Facet.Taxonomy.Directory.DirectoryTaxonomyReader: Restored locks on SetCacheSize() method.
---
 .../Taxonomy/Directory/DirectoryTaxonomyReader.cs  | 22 ++++++++++++++++++----
 1 file changed, 18 insertions(+), 4 deletions(-)

diff --git a/src/Lucene.Net.Facet/Taxonomy/Directory/DirectoryTaxonomyReader.cs b/src/Lucene.Net.Facet/Taxonomy/Directory/DirectoryTaxonomyReader.cs
index 1106db8..91f6a81 100644
--- a/src/Lucene.Net.Facet/Taxonomy/Directory/DirectoryTaxonomyReader.cs
+++ b/src/Lucene.Net.Facet/Taxonomy/Directory/DirectoryTaxonomyReader.cs
@@ -422,10 +422,24 @@ namespace Lucene.Net.Facet.Taxonomy.Directory
         public virtual void SetCacheSize(int size)
         {
             EnsureOpen();
-            // LUCENENET specific - removed locking here because these collections
-            // internally use Interlocked.Exchange
-            categoryCache.Limit = size;
-            ordinalCache.Limit = size;
+            categoryCacheLock.EnterWriteLock();
+            try
+            {
+                categoryCache.Limit = size;
+            }
+            finally
+            {
+                categoryCacheLock.ExitWriteLock();
+            }
+            ordinalCacheLock.EnterWriteLock();
+            try
+            {
+                ordinalCache.Limit = size;
+            }
+            finally
+            {
+                ordinalCacheLock.ExitWriteLock();
+            }
         }
 
         /// <summary>

[lucenenet] 01/03: Lucene.Net.Facet.Taxonomy.WriterCache.Cl2oTaxonomyWriterCache: Removed timeout on ReaderWriterLockSlim as the original lock didn't have one.

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 19002a41da77950c5804e0dc6a8db84ec106e5df
Author: Shad Storhaug <sh...@shadstorhaug.com>
AuthorDate: Tue Mar 16 21:53:43 2021 +0700

    Lucene.Net.Facet.Taxonomy.WriterCache.Cl2oTaxonomyWriterCache: Removed timeout on ReaderWriterLockSlim as the original lock didn't have one.
---
 .../WriterCache/Cl2oTaxonomyWriterCache.cs         | 78 +++++++++-------------
 1 file changed, 31 insertions(+), 47 deletions(-)

diff --git a/src/Lucene.Net.Facet/Taxonomy/WriterCache/Cl2oTaxonomyWriterCache.cs b/src/Lucene.Net.Facet/Taxonomy/WriterCache/Cl2oTaxonomyWriterCache.cs
index 3699a45..0239cbf 100644
--- a/src/Lucene.Net.Facet/Taxonomy/WriterCache/Cl2oTaxonomyWriterCache.cs
+++ b/src/Lucene.Net.Facet/Taxonomy/WriterCache/Cl2oTaxonomyWriterCache.cs
@@ -31,12 +31,14 @@ namespace Lucene.Net.Facet.Taxonomy.WriterCache
     /// </summary>
     public class Cl2oTaxonomyWriterCache : ITaxonomyWriterCache
     {
-        private const int LOCK_TIMEOUT = 1000;
-        private readonly ReaderWriterLockSlim @lock = new ReaderWriterLockSlim();
         private readonly int initialCapcity, numHashArrays;
         private readonly float loadFactor;
 
         private volatile CompactLabelToOrdinal cache;
+
+        // LUCENENET specific - use ReaderWriterLockSlim for better throughput
+        private readonly ReaderWriterLockSlim syncLock = new ReaderWriterLockSlim();
+        private readonly object disposalLock = new object();
         private bool isDisposed = false;
 
         /// <summary>
@@ -52,20 +54,14 @@ namespace Lucene.Net.Facet.Taxonomy.WriterCache
 
         public virtual void Clear()
         {
-            if (@lock.TryEnterWriteLock(LOCK_TIMEOUT))
+            syncLock.EnterWriteLock();
+            try
             {
-                try
-                {
-                    cache = new CompactLabelToOrdinal(initialCapcity, loadFactor, numHashArrays);
-                }
-                finally
-                {
-                    @lock.ExitWriteLock();
-                }
+                cache = new CompactLabelToOrdinal(initialCapcity, loadFactor, numHashArrays);
             }
-            else {
-                //Throwing ArguementException to maintain behavoir with ReaderWriterLock.AquireWriteLock.
-                throw new ArgumentException();
+            finally
+            {
+                syncLock.ExitWriteLock();
             }
         }
 
@@ -77,26 +73,28 @@ namespace Lucene.Net.Facet.Taxonomy.WriterCache
 
         protected virtual void Dispose(bool disposing) // LUCENENET specific - use proper dispose pattern
         {
+            
             if (disposing)
             {
                 if (isDisposed) return;
-                if (@lock.TryEnterWriteLock(LOCK_TIMEOUT))
+
+                // LUCENENET: Use additional lock to ensure our ReaderWriterLockSlim only gets
+                // disposed by the first caller.
+                lock (disposalLock)
                 {
                     if (isDisposed) return;
+                    syncLock.EnterWriteLock();
                     try
                     {
                         cache = null;
                     }
                     finally
                     {
+                        syncLock.ExitWriteLock();
                         isDisposed = true;
-                        @lock.ExitWriteLock();
-                        @lock.Dispose();
+                        syncLock.Dispose();
                     }
                 }
-                else
-                    //Throwing ArguementException to maintain behavoir with ReaderWriterLock.AquireWriteLock.
-                    throw new ArgumentException();
             }
         }
 
@@ -106,44 +104,30 @@ namespace Lucene.Net.Facet.Taxonomy.WriterCache
 
         public virtual int Get(FacetLabel categoryPath)
         {
-            if (@lock.TryEnterReadLock(LOCK_TIMEOUT))
+            syncLock.EnterReadLock();
+            try
             {
-                try
-                {
-                    return cache.GetOrdinal(categoryPath);
-                }
-                finally
-                {
-                    @lock.ExitReadLock();
-                }
+                return cache.GetOrdinal(categoryPath);
             }
-            else
+            finally
             {
-                //Throwing ArguementException to maintain behavoir with ReaderWriterLock.AquireWriteLock.
-                throw new ArgumentException();
+                syncLock.ExitReadLock();
             }
         }
 
         public virtual bool Put(FacetLabel categoryPath, int ordinal)
         {
-            if (@lock.TryEnterWriteLock(LOCK_TIMEOUT))
+            syncLock.EnterWriteLock();
+            try
             {
-                try
-                {
-                    cache.AddLabel(categoryPath, ordinal);
-                    // Tell the caller we didn't clear part of the cache, so it doesn't
-                    // have to flush its on-disk index now
-                    return false;
-                }
-                finally
-                {
-                    @lock.ExitWriteLock();
-                }
+                cache.AddLabel(categoryPath, ordinal);
+                // Tell the caller we didn't clear part of the cache, so it doesn't
+                // have to flush its on-disk index now
+                return false;
             }
-            else
+            finally
             {
-                //Throwing ArguementException to maintain behavoir with ReaderWriterLock.AquireWriteLock.
-                throw new ArgumentException();
+                syncLock.ExitWriteLock();
             }
         }