You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@lucenenet.apache.org by "Van Den Berghe, Vincent" <Vi...@bvdinfo.com> on 2017/07/21 15:01:53 UTC

An alternative NativeFSLockFactory

Hello everyone,

I spent an inordinate amount of time looking at the implementation of NativeFSLockFactory, and I see a potential problem.

Consider the IsLocked() implementation: this method determines if a  lock is available, but only does so by trying to Obtain the lock and releasing it, reporting false if it succeeds.
If the lock is free and two processes A and B both call IsLocked() to determine if it is, there is a nonzero chance that only one of them will report success, since the second process may try to obtain the lock that the first process is temporarily holding, causing its method to return true. Normally, both should return false if the lock is free.

Since I'm in a complaining mood: I find the Dipose() pattern on the locks very confusing, since the conventional implementation pattern specifies that an IDisposable object is not to be used anymore after Dispose() is called. However, after disposing a lock you can call Obtain() again with no ill consequences. The cycle can be repeated ad nauseam, a fact that used in some tests.

Recent implementations of the native locks in java get rid of the "Obtain", and the IsLocked (a Good Thing), so in the far far future these problems will solve themselves.

For now, I'd like to submit an implementation that attempts to corrects the IsLocked problem, without giving any guarantee that it won't introduce other problems or make existing ones go away. Sadly, the implementation corrects them only in regular .NET. In the version of .NET Core we are using, file locking is not implemented, and the implementation falls back to its old behavior.

It is much closer to the java method of doing things. Let me know what you think and feel free to improve (and correct bugs).

using Lucene.Net.Util;
using System;
using System.IO;
using System.Collections.Generic;

namespace Lucene.Net.Store
{
                /*
                * Licensed to the Apache Software Foundation (ASF) under one or more
                * contributor license agreements.  See the NOTICE file distributed with
                * this work for additional information regarding copyright ownership.
                * The ASF licenses this file to You under the Apache License, Version 2.0
                * (the "License"); you may not use this file except in compliance with
                * the License.  You may obtain a copy of the License at
                *
                *     http://www.apache.org/licenses/LICENSE-2.0
                *
                * Unless required by applicable law or agreed to in writing, software
                * distributed under the License is distributed on an "AS IS" BASIS,
                * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
                * See the License for the specific language governing permissions and
                * limitations under the License.
                */

                /// <summary>
                /// <para>Implements <see cref="LockFactory"/> using native OS file
                /// locks.  For NFS based access to an index, it's
                /// recommended that you try <see cref="SimpleFSLockFactory"/>
                /// first and work around the one limitation that a lock file
                /// could be left when the runtime exits abnormally.</para>
                ///
                /// <para>The primary benefit of <see cref="NativeFSLockFactory"/> is
                /// that locks (not the lock file itsself) will be properly
                /// removed (by the OS) if the runtime has an abnormal exit.</para>
                ///
                /// <para>Note that, unlike <see cref="SimpleFSLockFactory"/>, the existence of
                /// leftover lock files in the filesystem is fine because the OS
                /// will free the locks held against these files even though the
                /// files still remain. Lucene will never actively remove the lock
                /// files, so although you see them, the index may not be locked.</para>
                ///
                /// <para>Special care needs to be taken if you change the locking
                /// implementation: First be certain that no writer is in fact
                /// writing to the index otherwise you can easily corrupt
                /// your index. Be sure to do the <see cref="LockFactory"/> change on all Lucene
                /// instances and clean up all leftover lock files before starting
                /// the new configuration for the first time. Different implementations
                /// can not work together!</para>
                ///
                /// <para>If you suspect that this or any other <see cref="LockFactory"/> is
                /// not working properly in your environment, you can easily
                /// test it by using <see cref="VerifyingLockFactory"/>,
                /// <see cref="LockVerifyServer"/> and <see cref="LockStressTest"/>.</para>
                /// </summary>
                /// <seealso cref="LockFactory"/>
                public class NativeFSLockFactory : FSLockFactory
                {
                                /// <summary>
                                /// Create a <see cref="NativeFSLockFactory"/> instance, with <c>null</c> (unset)
                                /// lock directory. When you pass this factory to a <see cref="FSDirectory"/>
                                /// subclass, the lock directory is automatically set to the
                                /// directory itself. Be sure to create one instance for each directory
                                /// your create!
                                /// </summary>
                                public NativeFSLockFactory()
                                                : this((DirectoryInfo)null)
                                {
                                }

                                /// <summary>
                                /// Create a <see cref="NativeFSLockFactory"/> instance, storing lock
                                /// files into the specified <paramref name="lockDirName"/>
                                /// </summary>
                                /// <param name="lockDirName"> where lock files are created. </param>
                                public NativeFSLockFactory(string lockDirName)
                                                : this(new DirectoryInfo(lockDirName))
                                {
                                }

                                /// <summary>
                                /// Create a <see cref="NativeFSLockFactory"/> instance, storing lock
                                /// files into the specified <paramref name="lockDir"/>
                                /// </summary>
                                /// <param name="lockDir"> where lock files are created. </param>
                                public NativeFSLockFactory(DirectoryInfo lockDir)
                                {
                                                SetLockDir(lockDir);
                                }

                                // LUCENENET: NativeFSLocks in Java are infact singletons; this is how we mimick that to track instances and make sure
                                // IW.Unlock and IW.IsLocked works correctly
                                internal static readonly Dictionary<string, NativeFSLock> _locks = new Dictionary<string, NativeFSLock>();

                                /// <summary>
                                /// Given a lock name, return the full prefixed path of the actual lock file.
                                /// </summary>
                                /// <param name="lockName"></param>
                                /// <returns></returns>
                                private string GetPathOfLockFile(string lockName)
                                {
                                                if (m_lockPrefix != null)
                                                {
                                                                lockName = m_lockPrefix + "-" + lockName;
                                                }
                                                return Path.Combine(m_lockDir.FullName, lockName);
                                }

                                public override Lock MakeLock(string lockName)
                                {
                                                var path = GetPathOfLockFile(lockName);
                                                NativeFSLock l;
                                                lock(_locks)
                                                                if (!_locks.TryGetValue(path, out l))
                                                                                _locks.Add(path, l = new NativeFSLock(this, m_lockDir, path));
                                                return l;
                                }

                                public override void ClearLock(string lockName)
                                {
                                                var path = GetPathOfLockFile(lockName);
                                                NativeFSLock l;
                                                // this is the reason why we can't use ConcurrentDictionary: we need the removal and disposal of the lock to be atomic
                                                // otherwise it may clash with MakeLock making a lock and ClearLock disposing of it in another thread.
                                                lock (_locks)
                                                                if (_locks.TryGetValue(path, out l))
                                                                {
                                                                                _locks.Remove(path);
                                                                                l.Dispose();
                                                                }
                                }
                }

                internal class NativeFSLock : Lock
                {
#if NETSTANDARD
                                private const int ERROR_SHARE_VIOLATION = 0x20;
#else
                                private const int ERROR_LOCK_VIOLATION = 0x21;
#endif

                                private readonly NativeFSLockFactory outerInstance;

                                private FileStream channel;
                                private readonly string path;
                                private readonly DirectoryInfo lockDir;

                                public NativeFSLock(NativeFSLockFactory outerInstance, DirectoryInfo lockDir, string path)
                                {
                                                this.outerInstance = outerInstance;
                                                this.lockDir = lockDir;
                                                this.path = path;
                                }

                                /// <summary>
                                /// Return true if the <see cref="IOException"/> is the result of a lock violation
                                /// </summary>
                                /// <param name="e"></param>
                                /// <returns></returns>
                                private static bool IsLockOrShareViolation(IOException e)
                                {
                                                var result = e.HResult & 0xFFFF;
#if NETSTANDARD
                                                return result == ERROR_SHARE_VIOLATION;
#else
                                                return result == ERROR_LOCK_VIOLATION;
#endif
                                }

                                private FileStream GetLockFileStream(FileMode mode)
                                {
                                                if (!System.IO.Directory.Exists(lockDir.FullName))
                                                {
                                                                try
                                                                {
                                                                                System.IO.Directory.CreateDirectory(lockDir.FullName);
                                                                }
                                                                catch (Exception e)
                                                                {
                                                                                // note that several processes might have been trying to create the same directory at the same time.
                                                                                // if one succeeded, the directory will exist and the exception can be ignored. In all other cases we should report it.
                                                                                if (!System.IO.Directory.Exists(lockDir.FullName))
                                                                                                throw new IOException("Cannot create directory: " + lockDir.FullName, e);
                                                                }
                                                }
                                                else if (File.Exists(lockDir.FullName))
                                                {
                                                                throw new IOException("Found regular file where directory expected: " + lockDir.FullName);
                                                }

#if NETSTANDARD
                                                return new FileStream(path, mode, FileAccess.Write, FileShare.None, 1, mode == FileMode.Open ? FileOptions.None : FileOptions.DeleteOnClose);
#else
                                                return new FileStream(path, mode, FileAccess.Write, FileShare.ReadWrite);
#endif
                                }

                                public override bool Obtain()
                                {
                                                lock (this)
                                                {
                                                                FailureReason = null;

                                                                if (channel != null)
                                                                {
                                                                                // Our instance is already locked:
                                                                                return false;
                                                                }

#if NETSTANDARD
                                                                try
                                                                {
                                                                                channel = GetLockFileStream(FileMode.OpenOrCreate);
                                                                }
                                                                catch (IOException e) when(IsLockOrShareViolation(e))
                                                                {
                                                                                // no failure reason to be recorded, since this is the expected error if a lock exists
                                                                }
                                                                catch (Exception e)
                                                                {
                                                                                // all other exceptions need to be recorded
                                                                                FailureReason = e;
                                                                }
#else
                                                                FileStream stream = null;
                                                                try
                                                                {
                                                                                stream = GetLockFileStream(FileMode.OpenOrCreate);
                                                                }
                                                                catch (IOException e)
                                                                {
                                                                                FailureReason = e;
                                                                }
                                                                // LUCENENET: UnauthorizedAccessException does not derive from IOException like in java
                                                                catch (UnauthorizedAccessException e)
                                                                {
                                                                                // On Windows, we can get intermittent "Access
                                                                                // Denied" here.  So, we treat this as failure to
                                                                                // acquire the lock, but, store the reason in case
                                                                                // there is in fact a real error case.
                                                                                FailureReason = e;
                                                                }

                                                                if (stream != null)
                                                                                try
                                                                                {
                                                                                                stream.Lock(0, 1);
                                                                                                // only assign the channel if the lock succeeds
                                                                                                channel = stream;
                                                                                }
                                                                                catch (Exception e)
                                                                                {
                                                                                                FailureReason = e;
                                                                                                IOUtils.DisposeWhileHandlingException(stream);
                                                                                }
#endif
                                                                return channel != null;
                                                }
                                }

                                protected override void Dispose(bool disposing)
                                {
                                                if (disposing)
                                                {
                                                                lock (this)
                                                                {
                                                                                if (channel != null)
                                                                                {
                                                                                                try
                                                                                                {
                                                                                                                NativeFSLockFactory._locks.Remove(path);
                                                                                                }
                                                                                                finally
                                                                                                {
                                                                                                                IOUtils.DisposeWhileHandlingException(channel);
                                                                                                                channel = null;
                                                                                                }
#if !NETSTANDARD
                                                                                                // try to delete the file if we created it, but it's not an error if we can't.
                                                                                                                try
                                                                                                                {
                                                                                                                                File.Delete(path);
                                                                                                                }
                                                                                                                catch
                                                                                                                {
                                                                                                                }
#endif
                                                                                }
                                                                }
                                                }
                                }

                                public override bool IsLocked()
                                {
                                                lock (this)
                                                {
                                                                // First a shortcut, if a lock reference in this instance is available
                                                                if (channel != null)
                                                                {
                                                                                return true;
                                                                }

#if NETSTANDARD
                                                                try
                                                                {
                                                                                using (var stream = GetLockFileStream(FileMode.Open))
                                                                                {
                                                                                }
                                                                                return false;
                                                                }
                                                                catch (IOException e) when (IsLockOrShareViolation(e))
                                                                {
                                                                                return true;
                                                                }
                                                                catch (FileNotFoundException)
                                                                {
                                                                                // if the file doesn't exists, there can be no lock active
                                                                                return false;
                                                                }
#else
                                                                try
                                                                {
                                                                                using (var stream = GetLockFileStream(FileMode.Open))
                                                                                {
                                                                                                // try to find out if the file is locked by writing a byte. Note that we need to flush the stream to find out.
                                                                                                stream.WriteByte(0);
                                                                                                stream.Flush();   // this *may* throw an IOException if the file is locked, but...
                                                                                                                                                                                                // ... closing the stream is the real test
                                                                                }
                                                                                return false;
                                                                }
                                                                catch (IOException e) when (IsLockOrShareViolation(e))
                                                                {
                                                                                return true;
                                                                }
                                                                catch (FileNotFoundException)
                                                                {
                                                                                // if the file doesn't exists, there can be no lock active
                                                                                return false;
                                                                }
#endif
                                                }
                                }

                                public override string ToString()
                                {
                                                return "NativeFSLock@" + path;
                                }
                }
}



RE: An alternative NativeFSLockFactory

Posted by "Van Den Berghe, Vincent" <Vi...@bvdinfo.com>.
Hello Shad,

You are correct: catch(Exception e) is perhaps a bit too general. You can change:

                catch (Exception e)
                {
                    // all other exceptions need to be recorded
                    FailureReason = e;
                }

By:

catch (IOException e) 
                 { 
                     FailureReason = e; 
                 } 
                 // LUCENENET: UnauthorizedAccessException does not derive from IOException like in java 
                 catch (UnauthorizedAccessException e) 
                 { 
                     // On Windows, we can get intermittent "Access 
                     // Denied" here.  So, we treat this as failure to 
                     // acquire the lock, but, store the reason in case 
                     // there is in fact a real error case. 
                     FailureReason = e; 
                 }

... to reinstate the previous behavior and soothe the nerves. Note that the first catch(IOException e) when(...) must remain as is, since it's integral to the new approach.

I noticed the Replicator PR, but this will have to wait for a bit: I'm still trying to come to grips with the intermittent failures of TestSearcherManager_Mem. I'm almost ready to give up, though. But not yet. It is fortunate that the window I'm facing doesn't open (and the other one isn't high enough)  since I've been meaning to jump out of it many times now. 

If I give up in disgust, I'll report my findings and delegate to a higher power.

Vincent


-----Original Message-----
From: Shad Storhaug [mailto:shad@shadstorhaug.com] 
Sent: Monday, July 24, 2017 1:41 PM
To: Van Den Berghe, Vincent <Vi...@bvdinfo.com>
Cc: dev@lucenenet.apache.org
Subject: RE: An alternative NativeFSLockFactory

Vincent,

I took a look and indeed FileStream.Lock is coming back in .NET Standard 2.0 (https://apisof.net/catalog/System.IO.FileStream.Lock(Int64,Int64)), so I have created a FEATURE_FILESTREAM_LOCK so we can put it back in when the time comes.

Also, I noticed that in the Obtain method under the NETSTANDARD symbol, you have:

                try
                {
                    channel = GetLockFileStream(FileMode.OpenOrCreate);
                }
                catch (IOException e) when (IsLockOrShareViolation(e))
                {
                    // no failure reason to be recorded, since this is the expected error if a lock exists
                }
                catch (Exception e)
                {
                    // all other exceptions need to be recorded
                    FailureReason = e;
                }

In the original (https://github.com/apache/lucenenet/blob/c3f60b29f54ac1c5cc2d3e94f17a27208c13683c/src/Lucene.Net/Store/NativeFSLockFactory.cs#L149-L176), the code does not swallow all exceptions, but catches specific exceptions. Is this intentional or a bug?

Note that Lucene's design often depends on specific exceptions being thrown to go down the right execution path, so I just wanted to be sure we aren't shooting ourselves in the foot by catching everything here.


BTW - Jens has submitted the Replicator PR (https://github.com/apache/lucenenet/pull/209). I would appreciate any feedback you may have.

Thanks,
Shad Storhaug (NightOwl888)


-----Original Message-----
From: Shad Storhaug [mailto:shad@shadstorhaug.com] 
Sent: Saturday, July 22, 2017 1:58 AM
To: Van Den Berghe, Vincent
Cc: dev@lucenenet.apache.org
Subject: RE: An alternative NativeFSLockFactory

Vincent,

Thanks for doing the legwork. 

Tests

To make sure we can maintain the same functionality through future porting efforts, we should have a test or two that fail without this patched locking (at least part of the time) and succeed with the fix in place. Maybe those tests will just need to be skipped on .NET Core (at least for now), but at least going forward we can detect this issue if repeated again.

Features

If you take a look and it turns out file locking is coming back in .NET Standard 2.0 (you can drill into the "2.0" link here https://docs.microsoft.com/en-us/dotnet/standard/net-standard to see a diff of the APIs that are being added), we should change this to FEATURE_FILE_LOCKING instead of NETSTANDARD so we can put this feature back in when we support .NET Standard 2.0.

You can look at https://github.com/apache/lucenenet/pull/191 for a list of features that we support (it may be out of date a little), which we can hopefully add back in for .NET Standard 2.0.

Thanks,
Shad Storhaug (NightOwl888)



-----Original Message-----
From: Van Den Berghe, Vincent [mailto:Vincent.VanDenBerghe@bvdinfo.com] 
Sent: Friday, July 21, 2017 10:02 PM
To: dev@lucenenet.apache.org
Subject: An alternative NativeFSLockFactory

Hello everyone,

I spent an inordinate amount of time looking at the implementation of NativeFSLockFactory, and I see a potential problem.

Consider the IsLocked() implementation: this method determines if a  lock is available, but only does so by trying to Obtain the lock and releasing it, reporting false if it succeeds.
If the lock is free and two processes A and B both call IsLocked() to determine if it is, there is a nonzero chance that only one of them will report success, since the second process may try to obtain the lock that the first process is temporarily holding, causing its method to return true. Normally, both should return false if the lock is free.

Since I'm in a complaining mood: I find the Dipose() pattern on the locks very confusing, since the conventional implementation pattern specifies that an IDisposable object is not to be used anymore after Dispose() is called. However, after disposing a lock you can call Obtain() again with no ill consequences. The cycle can be repeated ad nauseam, a fact that used in some tests.

Recent implementations of the native locks in java get rid of the "Obtain", and the IsLocked (a Good Thing), so in the far far future these problems will solve themselves.

For now, I'd like to submit an implementation that attempts to corrects the IsLocked problem, without giving any guarantee that it won't introduce other problems or make existing ones go away. Sadly, the implementation corrects them only in regular .NET. In the version of .NET Core we are using, file locking is not implemented, and the implementation falls back to its old behavior.

It is much closer to the java method of doing things. Let me know what you think and feel free to improve (and correct bugs).

using Lucene.Net.Util;
using System;
using System.IO;
using System.Collections.Generic;

namespace Lucene.Net.Store
{
                /*
                * Licensed to the Apache Software Foundation (ASF) under one or more
                * contributor license agreements.  See the NOTICE file distributed with
                * this work for additional information regarding copyright ownership.
                * The ASF licenses this file to You under the Apache License, Version 2.0
                * (the "License"); you may not use this file except in compliance with
                * the License.  You may obtain a copy of the License at
                *
                *     http://www.apache.org/licenses/LICENSE-2.0
                *
                * Unless required by applicable law or agreed to in writing, software
                * distributed under the License is distributed on an "AS IS" BASIS,
                * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
                * See the License for the specific language governing permissions and
                * limitations under the License.
                */

                /// <summary>
                /// <para>Implements <see cref="LockFactory"/> using native OS file
                /// locks.  For NFS based access to an index, it's
                /// recommended that you try <see cref="SimpleFSLockFactory"/>
                /// first and work around the one limitation that a lock file
                /// could be left when the runtime exits abnormally.</para>
                ///
                /// <para>The primary benefit of <see cref="NativeFSLockFactory"/> is
                /// that locks (not the lock file itsself) will be properly
                /// removed (by the OS) if the runtime has an abnormal exit.</para>
                ///
                /// <para>Note that, unlike <see cref="SimpleFSLockFactory"/>, the existence of
                /// leftover lock files in the filesystem is fine because the OS
                /// will free the locks held against these files even though the
                /// files still remain. Lucene will never actively remove the lock
                /// files, so although you see them, the index may not be locked.</para>
                ///
                /// <para>Special care needs to be taken if you change the locking
                /// implementation: First be certain that no writer is in fact
                /// writing to the index otherwise you can easily corrupt
                /// your index. Be sure to do the <see cref="LockFactory"/> change on all Lucene
                /// instances and clean up all leftover lock files before starting
                /// the new configuration for the first time. Different implementations
                /// can not work together!</para>
                ///
                /// <para>If you suspect that this or any other <see cref="LockFactory"/> is
                /// not working properly in your environment, you can easily
                /// test it by using <see cref="VerifyingLockFactory"/>,
                /// <see cref="LockVerifyServer"/> and <see cref="LockStressTest"/>.</para>
                /// </summary>
                /// <seealso cref="LockFactory"/>
                public class NativeFSLockFactory : FSLockFactory
                {
                                /// <summary>
                                /// Create a <see cref="NativeFSLockFactory"/> instance, with <c>null</c> (unset)
                                /// lock directory. When you pass this factory to a <see cref="FSDirectory"/>
                                /// subclass, the lock directory is automatically set to the
                                /// directory itself. Be sure to create one instance for each directory
                                /// your create!
                                /// </summary>
                                public NativeFSLockFactory()
                                                : this((DirectoryInfo)null)
                                {
                                }

                                /// <summary>
                                /// Create a <see cref="NativeFSLockFactory"/> instance, storing lock
                                /// files into the specified <paramref name="lockDirName"/>
                                /// </summary>
                                /// <param name="lockDirName"> where lock files are created. </param>
                                public NativeFSLockFactory(string lockDirName)
                                                : this(new DirectoryInfo(lockDirName))
                                {
                                }

                                /// <summary>
                                /// Create a <see cref="NativeFSLockFactory"/> instance, storing lock
                                /// files into the specified <paramref name="lockDir"/>
                                /// </summary>
                                /// <param name="lockDir"> where lock files are created. </param>
                                public NativeFSLockFactory(DirectoryInfo lockDir)
                                {
                                                SetLockDir(lockDir);
                                }

                                // LUCENENET: NativeFSLocks in Java are infact singletons; this is how we mimick that to track instances and make sure
                                // IW.Unlock and IW.IsLocked works correctly
                                internal static readonly Dictionary<string, NativeFSLock> _locks = new Dictionary<string, NativeFSLock>();

                                /// <summary>
                                /// Given a lock name, return the full prefixed path of the actual lock file.
                                /// </summary>
                                /// <param name="lockName"></param>
                                /// <returns></returns>
                                private string GetPathOfLockFile(string lockName)
                                {
                                                if (m_lockPrefix != null)
                                                {
                                                                lockName = m_lockPrefix + "-" + lockName;
                                                }
                                                return Path.Combine(m_lockDir.FullName, lockName);
                                }

                                public override Lock MakeLock(string lockName)
                                {
                                                var path = GetPathOfLockFile(lockName);
                                                NativeFSLock l;
                                                lock(_locks)
                                                                if (!_locks.TryGetValue(path, out l))
                                                                                _locks.Add(path, l = new NativeFSLock(this, m_lockDir, path));
                                                return l;
                                }

                                public override void ClearLock(string lockName)
                                {
                                                var path = GetPathOfLockFile(lockName);
                                                NativeFSLock l;
                                                // this is the reason why we can't use ConcurrentDictionary: we need the removal and disposal of the lock to be atomic
                                                // otherwise it may clash with MakeLock making a lock and ClearLock disposing of it in another thread.
                                                lock (_locks)
                                                                if (_locks.TryGetValue(path, out l))
                                                                {
                                                                                _locks.Remove(path);
                                                                                l.Dispose();
                                                                }
                                }
                }

                internal class NativeFSLock : Lock
                {
#if NETSTANDARD
                                private const int ERROR_SHARE_VIOLATION = 0x20; #else
                                private const int ERROR_LOCK_VIOLATION = 0x21; #endif

                                private readonly NativeFSLockFactory outerInstance;

                                private FileStream channel;
                                private readonly string path;
                                private readonly DirectoryInfo lockDir;

                                public NativeFSLock(NativeFSLockFactory outerInstance, DirectoryInfo lockDir, string path)
                                {
                                                this.outerInstance = outerInstance;
                                                this.lockDir = lockDir;
                                                this.path = path;
                                }

                                /// <summary>
                                /// Return true if the <see cref="IOException"/> is the result of a lock violation
                                /// </summary>
                                /// <param name="e"></param>
                                /// <returns></returns>
                                private static bool IsLockOrShareViolation(IOException e)
                                {
                                                var result = e.HResult & 0xFFFF; #if NETSTANDARD
                                                return result == ERROR_SHARE_VIOLATION; #else
                                                return result == ERROR_LOCK_VIOLATION; #endif
                                }

                                private FileStream GetLockFileStream(FileMode mode)
                                {
                                                if (!System.IO.Directory.Exists(lockDir.FullName))
                                                {
                                                                try
                                                                {
                                                                                System.IO.Directory.CreateDirectory(lockDir.FullName);
                                                                }
                                                                catch (Exception e)
                                                                {
                                                                                // note that several processes might have been trying to create the same directory at the same time.
                                                                                // if one succeeded, the directory will exist and the exception can be ignored. In all other cases we should report it.
                                                                                if (!System.IO.Directory.Exists(lockDir.FullName))
                                                                                                throw new IOException("Cannot create directory: " + lockDir.FullName, e);
                                                                }
                                                }
                                                else if (File.Exists(lockDir.FullName))
                                                {
                                                                throw new IOException("Found regular file where directory expected: " + lockDir.FullName);
                                                }

#if NETSTANDARD
                                                return new FileStream(path, mode, FileAccess.Write, FileShare.None, 1, mode == FileMode.Open ? FileOptions.None : FileOptions.DeleteOnClose); #else
                                                return new FileStream(path, mode, FileAccess.Write, FileShare.ReadWrite); #endif
                                }

                                public override bool Obtain()
                                {
                                                lock (this)
                                                {
                                                                FailureReason = null;

                                                                if (channel != null)
                                                                {
                                                                                // Our instance is already locked:
                                                                                return false;
                                                                }

#if NETSTANDARD
                                                                try
                                                                {
                                                                                channel = GetLockFileStream(FileMode.OpenOrCreate);
                                                                }
                                                                catch (IOException e) when(IsLockOrShareViolation(e))
                                                                {
                                                                                // no failure reason to be recorded, since this is the expected error if a lock exists
                                                                }
                                                                catch (Exception e)
                                                                {
                                                                                // all other exceptions need to be recorded
                                                                                FailureReason = e;
                                                                } #else
                                                                FileStream stream = null;
                                                                try
                                                                {
                                                                                stream = GetLockFileStream(FileMode.OpenOrCreate);
                                                                }
                                                                catch (IOException e)
                                                                {
                                                                                FailureReason = e;
                                                                }
                                                                // LUCENENET: UnauthorizedAccessException does not derive from IOException like in java
                                                                catch (UnauthorizedAccessException e)
                                                                {
                                                                                // On Windows, we can get intermittent "Access
                                                                                // Denied" here.  So, we treat this as failure to
                                                                                // acquire the lock, but, store the reason in case
                                                                                // there is in fact a real error case.
                                                                                FailureReason = e;
                                                                }

                                                                if (stream != null)
                                                                                try
                                                                                {
                                                                                                stream.Lock(0, 1);
                                                                                                // only assign the channel if the lock succeeds
                                                                                                channel = stream;
                                                                                }
                                                                                catch (Exception e)
                                                                                {
                                                                                                FailureReason = e;
                                                                                                IOUtils.DisposeWhileHandlingException(stream);
                                                                                } #endif
                                                                return channel != null;
                                                }
                                }

                                protected override void Dispose(bool disposing)
                                {
                                                if (disposing)
                                                {
                                                                lock (this)
                                                                {
                                                                                if (channel != null)
                                                                                {
                                                                                                try
                                                                                                {
                                                                                                                NativeFSLockFactory._locks.Remove(path);
                                                                                                }
                                                                                                finally
                                                                                                {
                                                                                                                IOUtils.DisposeWhileHandlingException(channel);
                                                                                                                channel = null;
                                                                                                } #if !NETSTANDARD
                                                                                                // try to delete the file if we created it, but it's not an error if we can't.
                                                                                                                try
                                                                                                                {
                                                                                                                                File.Delete(path);
                                                                                                                }
                                                                                                                catch
                                                                                                                {
                                                                                                                } #endif
                                                                                }
                                                                }
                                                }
                                }

                                public override bool IsLocked()
                                {
                                                lock (this)
                                                {
                                                                // First a shortcut, if a lock reference in this instance is available
                                                                if (channel != null)
                                                                {
                                                                                return true;
                                                                }

#if NETSTANDARD
                                                                try
                                                                {
                                                                                using (var stream = GetLockFileStream(FileMode.Open))
                                                                                {
                                                                                }
                                                                                return false;
                                                                }
                                                                catch (IOException e) when (IsLockOrShareViolation(e))
                                                                {
                                                                                return true;
                                                                }
                                                                catch (FileNotFoundException)
                                                                {
                                                                                // if the file doesn't exists, there can be no lock active
                                                                                return false;
                                                                } #else
                                                                try
                                                                {
                                                                                using (var stream = GetLockFileStream(FileMode.Open))
                                                                                {
                                                                                                // try to find out if the file is locked by writing a byte. Note that we need to flush the stream to find out.
                                                                                                stream.WriteByte(0);
                                                                                                stream.Flush();   // this *may* throw an IOException if the file is locked, but...
                                                                                                                                                                                                // ... closing the stream is the real test
                                                                                }
                                                                                return false;
                                                                }
                                                                catch (IOException e) when (IsLockOrShareViolation(e))
                                                                {
                                                                                return true;
                                                                }
                                                                catch (FileNotFoundException)
                                                                {
                                                                                // if the file doesn't exists, there can be no lock active
                                                                                return false;
                                                                } #endif
                                                }
                                }

                                public override string ToString()
                                {
                                                return "NativeFSLock@" + path;
                                }
                }
}


RE: An alternative NativeFSLockFactory

Posted by Shad Storhaug <sh...@shadstorhaug.com>.
Vincent,

I took a look and indeed FileStream.Lock is coming back in .NET Standard 2.0 (https://apisof.net/catalog/System.IO.FileStream.Lock(Int64,Int64)), so I have created a FEATURE_FILESTREAM_LOCK so we can put it back in when the time comes.

Also, I noticed that in the Obtain method under the NETSTANDARD symbol, you have:

                try
                {
                    channel = GetLockFileStream(FileMode.OpenOrCreate);
                }
                catch (IOException e) when (IsLockOrShareViolation(e))
                {
                    // no failure reason to be recorded, since this is the expected error if a lock exists
                }
                catch (Exception e)
                {
                    // all other exceptions need to be recorded
                    FailureReason = e;
                }

In the original (https://github.com/apache/lucenenet/blob/c3f60b29f54ac1c5cc2d3e94f17a27208c13683c/src/Lucene.Net/Store/NativeFSLockFactory.cs#L149-L176), the code does not swallow all exceptions, but catches specific exceptions. Is this intentional or a bug?

Note that Lucene's design often depends on specific exceptions being thrown to go down the right execution path, so I just wanted to be sure we aren't shooting ourselves in the foot by catching everything here.


BTW - Jens has submitted the Replicator PR (https://github.com/apache/lucenenet/pull/209). I would appreciate any feedback you may have.

Thanks,
Shad Storhaug (NightOwl888)


-----Original Message-----
From: Shad Storhaug [mailto:shad@shadstorhaug.com] 
Sent: Saturday, July 22, 2017 1:58 AM
To: Van Den Berghe, Vincent
Cc: dev@lucenenet.apache.org
Subject: RE: An alternative NativeFSLockFactory

Vincent,

Thanks for doing the legwork. 

Tests

To make sure we can maintain the same functionality through future porting efforts, we should have a test or two that fail without this patched locking (at least part of the time) and succeed with the fix in place. Maybe those tests will just need to be skipped on .NET Core (at least for now), but at least going forward we can detect this issue if repeated again.

Features

If you take a look and it turns out file locking is coming back in .NET Standard 2.0 (you can drill into the "2.0" link here https://docs.microsoft.com/en-us/dotnet/standard/net-standard to see a diff of the APIs that are being added), we should change this to FEATURE_FILE_LOCKING instead of NETSTANDARD so we can put this feature back in when we support .NET Standard 2.0.

You can look at https://github.com/apache/lucenenet/pull/191 for a list of features that we support (it may be out of date a little), which we can hopefully add back in for .NET Standard 2.0.

Thanks,
Shad Storhaug (NightOwl888)



-----Original Message-----
From: Van Den Berghe, Vincent [mailto:Vincent.VanDenBerghe@bvdinfo.com] 
Sent: Friday, July 21, 2017 10:02 PM
To: dev@lucenenet.apache.org
Subject: An alternative NativeFSLockFactory

Hello everyone,

I spent an inordinate amount of time looking at the implementation of NativeFSLockFactory, and I see a potential problem.

Consider the IsLocked() implementation: this method determines if a  lock is available, but only does so by trying to Obtain the lock and releasing it, reporting false if it succeeds.
If the lock is free and two processes A and B both call IsLocked() to determine if it is, there is a nonzero chance that only one of them will report success, since the second process may try to obtain the lock that the first process is temporarily holding, causing its method to return true. Normally, both should return false if the lock is free.

Since I'm in a complaining mood: I find the Dipose() pattern on the locks very confusing, since the conventional implementation pattern specifies that an IDisposable object is not to be used anymore after Dispose() is called. However, after disposing a lock you can call Obtain() again with no ill consequences. The cycle can be repeated ad nauseam, a fact that used in some tests.

Recent implementations of the native locks in java get rid of the "Obtain", and the IsLocked (a Good Thing), so in the far far future these problems will solve themselves.

For now, I'd like to submit an implementation that attempts to corrects the IsLocked problem, without giving any guarantee that it won't introduce other problems or make existing ones go away. Sadly, the implementation corrects them only in regular .NET. In the version of .NET Core we are using, file locking is not implemented, and the implementation falls back to its old behavior.

It is much closer to the java method of doing things. Let me know what you think and feel free to improve (and correct bugs).

using Lucene.Net.Util;
using System;
using System.IO;
using System.Collections.Generic;

namespace Lucene.Net.Store
{
                /*
                * Licensed to the Apache Software Foundation (ASF) under one or more
                * contributor license agreements.  See the NOTICE file distributed with
                * this work for additional information regarding copyright ownership.
                * The ASF licenses this file to You under the Apache License, Version 2.0
                * (the "License"); you may not use this file except in compliance with
                * the License.  You may obtain a copy of the License at
                *
                *     http://www.apache.org/licenses/LICENSE-2.0
                *
                * Unless required by applicable law or agreed to in writing, software
                * distributed under the License is distributed on an "AS IS" BASIS,
                * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
                * See the License for the specific language governing permissions and
                * limitations under the License.
                */

                /// <summary>
                /// <para>Implements <see cref="LockFactory"/> using native OS file
                /// locks.  For NFS based access to an index, it's
                /// recommended that you try <see cref="SimpleFSLockFactory"/>
                /// first and work around the one limitation that a lock file
                /// could be left when the runtime exits abnormally.</para>
                ///
                /// <para>The primary benefit of <see cref="NativeFSLockFactory"/> is
                /// that locks (not the lock file itsself) will be properly
                /// removed (by the OS) if the runtime has an abnormal exit.</para>
                ///
                /// <para>Note that, unlike <see cref="SimpleFSLockFactory"/>, the existence of
                /// leftover lock files in the filesystem is fine because the OS
                /// will free the locks held against these files even though the
                /// files still remain. Lucene will never actively remove the lock
                /// files, so although you see them, the index may not be locked.</para>
                ///
                /// <para>Special care needs to be taken if you change the locking
                /// implementation: First be certain that no writer is in fact
                /// writing to the index otherwise you can easily corrupt
                /// your index. Be sure to do the <see cref="LockFactory"/> change on all Lucene
                /// instances and clean up all leftover lock files before starting
                /// the new configuration for the first time. Different implementations
                /// can not work together!</para>
                ///
                /// <para>If you suspect that this or any other <see cref="LockFactory"/> is
                /// not working properly in your environment, you can easily
                /// test it by using <see cref="VerifyingLockFactory"/>,
                /// <see cref="LockVerifyServer"/> and <see cref="LockStressTest"/>.</para>
                /// </summary>
                /// <seealso cref="LockFactory"/>
                public class NativeFSLockFactory : FSLockFactory
                {
                                /// <summary>
                                /// Create a <see cref="NativeFSLockFactory"/> instance, with <c>null</c> (unset)
                                /// lock directory. When you pass this factory to a <see cref="FSDirectory"/>
                                /// subclass, the lock directory is automatically set to the
                                /// directory itself. Be sure to create one instance for each directory
                                /// your create!
                                /// </summary>
                                public NativeFSLockFactory()
                                                : this((DirectoryInfo)null)
                                {
                                }

                                /// <summary>
                                /// Create a <see cref="NativeFSLockFactory"/> instance, storing lock
                                /// files into the specified <paramref name="lockDirName"/>
                                /// </summary>
                                /// <param name="lockDirName"> where lock files are created. </param>
                                public NativeFSLockFactory(string lockDirName)
                                                : this(new DirectoryInfo(lockDirName))
                                {
                                }

                                /// <summary>
                                /// Create a <see cref="NativeFSLockFactory"/> instance, storing lock
                                /// files into the specified <paramref name="lockDir"/>
                                /// </summary>
                                /// <param name="lockDir"> where lock files are created. </param>
                                public NativeFSLockFactory(DirectoryInfo lockDir)
                                {
                                                SetLockDir(lockDir);
                                }

                                // LUCENENET: NativeFSLocks in Java are infact singletons; this is how we mimick that to track instances and make sure
                                // IW.Unlock and IW.IsLocked works correctly
                                internal static readonly Dictionary<string, NativeFSLock> _locks = new Dictionary<string, NativeFSLock>();

                                /// <summary>
                                /// Given a lock name, return the full prefixed path of the actual lock file.
                                /// </summary>
                                /// <param name="lockName"></param>
                                /// <returns></returns>
                                private string GetPathOfLockFile(string lockName)
                                {
                                                if (m_lockPrefix != null)
                                                {
                                                                lockName = m_lockPrefix + "-" + lockName;
                                                }
                                                return Path.Combine(m_lockDir.FullName, lockName);
                                }

                                public override Lock MakeLock(string lockName)
                                {
                                                var path = GetPathOfLockFile(lockName);
                                                NativeFSLock l;
                                                lock(_locks)
                                                                if (!_locks.TryGetValue(path, out l))
                                                                                _locks.Add(path, l = new NativeFSLock(this, m_lockDir, path));
                                                return l;
                                }

                                public override void ClearLock(string lockName)
                                {
                                                var path = GetPathOfLockFile(lockName);
                                                NativeFSLock l;
                                                // this is the reason why we can't use ConcurrentDictionary: we need the removal and disposal of the lock to be atomic
                                                // otherwise it may clash with MakeLock making a lock and ClearLock disposing of it in another thread.
                                                lock (_locks)
                                                                if (_locks.TryGetValue(path, out l))
                                                                {
                                                                                _locks.Remove(path);
                                                                                l.Dispose();
                                                                }
                                }
                }

                internal class NativeFSLock : Lock
                {
#if NETSTANDARD
                                private const int ERROR_SHARE_VIOLATION = 0x20; #else
                                private const int ERROR_LOCK_VIOLATION = 0x21; #endif

                                private readonly NativeFSLockFactory outerInstance;

                                private FileStream channel;
                                private readonly string path;
                                private readonly DirectoryInfo lockDir;

                                public NativeFSLock(NativeFSLockFactory outerInstance, DirectoryInfo lockDir, string path)
                                {
                                                this.outerInstance = outerInstance;
                                                this.lockDir = lockDir;
                                                this.path = path;
                                }

                                /// <summary>
                                /// Return true if the <see cref="IOException"/> is the result of a lock violation
                                /// </summary>
                                /// <param name="e"></param>
                                /// <returns></returns>
                                private static bool IsLockOrShareViolation(IOException e)
                                {
                                                var result = e.HResult & 0xFFFF; #if NETSTANDARD
                                                return result == ERROR_SHARE_VIOLATION; #else
                                                return result == ERROR_LOCK_VIOLATION; #endif
                                }

                                private FileStream GetLockFileStream(FileMode mode)
                                {
                                                if (!System.IO.Directory.Exists(lockDir.FullName))
                                                {
                                                                try
                                                                {
                                                                                System.IO.Directory.CreateDirectory(lockDir.FullName);
                                                                }
                                                                catch (Exception e)
                                                                {
                                                                                // note that several processes might have been trying to create the same directory at the same time.
                                                                                // if one succeeded, the directory will exist and the exception can be ignored. In all other cases we should report it.
                                                                                if (!System.IO.Directory.Exists(lockDir.FullName))
                                                                                                throw new IOException("Cannot create directory: " + lockDir.FullName, e);
                                                                }
                                                }
                                                else if (File.Exists(lockDir.FullName))
                                                {
                                                                throw new IOException("Found regular file where directory expected: " + lockDir.FullName);
                                                }

#if NETSTANDARD
                                                return new FileStream(path, mode, FileAccess.Write, FileShare.None, 1, mode == FileMode.Open ? FileOptions.None : FileOptions.DeleteOnClose); #else
                                                return new FileStream(path, mode, FileAccess.Write, FileShare.ReadWrite); #endif
                                }

                                public override bool Obtain()
                                {
                                                lock (this)
                                                {
                                                                FailureReason = null;

                                                                if (channel != null)
                                                                {
                                                                                // Our instance is already locked:
                                                                                return false;
                                                                }

#if NETSTANDARD
                                                                try
                                                                {
                                                                                channel = GetLockFileStream(FileMode.OpenOrCreate);
                                                                }
                                                                catch (IOException e) when(IsLockOrShareViolation(e))
                                                                {
                                                                                // no failure reason to be recorded, since this is the expected error if a lock exists
                                                                }
                                                                catch (Exception e)
                                                                {
                                                                                // all other exceptions need to be recorded
                                                                                FailureReason = e;
                                                                } #else
                                                                FileStream stream = null;
                                                                try
                                                                {
                                                                                stream = GetLockFileStream(FileMode.OpenOrCreate);
                                                                }
                                                                catch (IOException e)
                                                                {
                                                                                FailureReason = e;
                                                                }
                                                                // LUCENENET: UnauthorizedAccessException does not derive from IOException like in java
                                                                catch (UnauthorizedAccessException e)
                                                                {
                                                                                // On Windows, we can get intermittent "Access
                                                                                // Denied" here.  So, we treat this as failure to
                                                                                // acquire the lock, but, store the reason in case
                                                                                // there is in fact a real error case.
                                                                                FailureReason = e;
                                                                }

                                                                if (stream != null)
                                                                                try
                                                                                {
                                                                                                stream.Lock(0, 1);
                                                                                                // only assign the channel if the lock succeeds
                                                                                                channel = stream;
                                                                                }
                                                                                catch (Exception e)
                                                                                {
                                                                                                FailureReason = e;
                                                                                                IOUtils.DisposeWhileHandlingException(stream);
                                                                                } #endif
                                                                return channel != null;
                                                }
                                }

                                protected override void Dispose(bool disposing)
                                {
                                                if (disposing)
                                                {
                                                                lock (this)
                                                                {
                                                                                if (channel != null)
                                                                                {
                                                                                                try
                                                                                                {
                                                                                                                NativeFSLockFactory._locks.Remove(path);
                                                                                                }
                                                                                                finally
                                                                                                {
                                                                                                                IOUtils.DisposeWhileHandlingException(channel);
                                                                                                                channel = null;
                                                                                                } #if !NETSTANDARD
                                                                                                // try to delete the file if we created it, but it's not an error if we can't.
                                                                                                                try
                                                                                                                {
                                                                                                                                File.Delete(path);
                                                                                                                }
                                                                                                                catch
                                                                                                                {
                                                                                                                } #endif
                                                                                }
                                                                }
                                                }
                                }

                                public override bool IsLocked()
                                {
                                                lock (this)
                                                {
                                                                // First a shortcut, if a lock reference in this instance is available
                                                                if (channel != null)
                                                                {
                                                                                return true;
                                                                }

#if NETSTANDARD
                                                                try
                                                                {
                                                                                using (var stream = GetLockFileStream(FileMode.Open))
                                                                                {
                                                                                }
                                                                                return false;
                                                                }
                                                                catch (IOException e) when (IsLockOrShareViolation(e))
                                                                {
                                                                                return true;
                                                                }
                                                                catch (FileNotFoundException)
                                                                {
                                                                                // if the file doesn't exists, there can be no lock active
                                                                                return false;
                                                                } #else
                                                                try
                                                                {
                                                                                using (var stream = GetLockFileStream(FileMode.Open))
                                                                                {
                                                                                                // try to find out if the file is locked by writing a byte. Note that we need to flush the stream to find out.
                                                                                                stream.WriteByte(0);
                                                                                                stream.Flush();   // this *may* throw an IOException if the file is locked, but...
                                                                                                                                                                                                // ... closing the stream is the real test
                                                                                }
                                                                                return false;
                                                                }
                                                                catch (IOException e) when (IsLockOrShareViolation(e))
                                                                {
                                                                                return true;
                                                                }
                                                                catch (FileNotFoundException)
                                                                {
                                                                                // if the file doesn't exists, there can be no lock active
                                                                                return false;
                                                                } #endif
                                                }
                                }

                                public override string ToString()
                                {
                                                return "NativeFSLock@" + path;
                                }
                }
}


RE: An alternative NativeFSLockFactory

Posted by Shad Storhaug <sh...@shadstorhaug.com>.
Vincent,

Thanks for doing the legwork. 

Tests

To make sure we can maintain the same functionality through future porting efforts, we should have a test or two that fail without this patched locking (at least part of the time) and succeed with the fix in place. Maybe those tests will just need to be skipped on .NET Core (at least for now), but at least going forward we can detect this issue if repeated again.

Features

If you take a look and it turns out file locking is coming back in .NET Standard 2.0 (you can drill into the "2.0" link here https://docs.microsoft.com/en-us/dotnet/standard/net-standard to see a diff of the APIs that are being added), we should change this to FEATURE_FILE_LOCKING instead of NETSTANDARD so we can put this feature back in when we support .NET Standard 2.0.

You can look at https://github.com/apache/lucenenet/pull/191 for a list of features that we support (it may be out of date a little), which we can hopefully add back in for .NET Standard 2.0.

Thanks,
Shad Storhaug (NightOwl888)



-----Original Message-----
From: Van Den Berghe, Vincent [mailto:Vincent.VanDenBerghe@bvdinfo.com] 
Sent: Friday, July 21, 2017 10:02 PM
To: dev@lucenenet.apache.org
Subject: An alternative NativeFSLockFactory

Hello everyone,

I spent an inordinate amount of time looking at the implementation of NativeFSLockFactory, and I see a potential problem.

Consider the IsLocked() implementation: this method determines if a  lock is available, but only does so by trying to Obtain the lock and releasing it, reporting false if it succeeds.
If the lock is free and two processes A and B both call IsLocked() to determine if it is, there is a nonzero chance that only one of them will report success, since the second process may try to obtain the lock that the first process is temporarily holding, causing its method to return true. Normally, both should return false if the lock is free.

Since I'm in a complaining mood: I find the Dipose() pattern on the locks very confusing, since the conventional implementation pattern specifies that an IDisposable object is not to be used anymore after Dispose() is called. However, after disposing a lock you can call Obtain() again with no ill consequences. The cycle can be repeated ad nauseam, a fact that used in some tests.

Recent implementations of the native locks in java get rid of the "Obtain", and the IsLocked (a Good Thing), so in the far far future these problems will solve themselves.

For now, I'd like to submit an implementation that attempts to corrects the IsLocked problem, without giving any guarantee that it won't introduce other problems or make existing ones go away. Sadly, the implementation corrects them only in regular .NET. In the version of .NET Core we are using, file locking is not implemented, and the implementation falls back to its old behavior.

It is much closer to the java method of doing things. Let me know what you think and feel free to improve (and correct bugs).

using Lucene.Net.Util;
using System;
using System.IO;
using System.Collections.Generic;

namespace Lucene.Net.Store
{
                /*
                * Licensed to the Apache Software Foundation (ASF) under one or more
                * contributor license agreements.  See the NOTICE file distributed with
                * this work for additional information regarding copyright ownership.
                * The ASF licenses this file to You under the Apache License, Version 2.0
                * (the "License"); you may not use this file except in compliance with
                * the License.  You may obtain a copy of the License at
                *
                *     http://www.apache.org/licenses/LICENSE-2.0
                *
                * Unless required by applicable law or agreed to in writing, software
                * distributed under the License is distributed on an "AS IS" BASIS,
                * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
                * See the License for the specific language governing permissions and
                * limitations under the License.
                */

                /// <summary>
                /// <para>Implements <see cref="LockFactory"/> using native OS file
                /// locks.  For NFS based access to an index, it's
                /// recommended that you try <see cref="SimpleFSLockFactory"/>
                /// first and work around the one limitation that a lock file
                /// could be left when the runtime exits abnormally.</para>
                ///
                /// <para>The primary benefit of <see cref="NativeFSLockFactory"/> is
                /// that locks (not the lock file itsself) will be properly
                /// removed (by the OS) if the runtime has an abnormal exit.</para>
                ///
                /// <para>Note that, unlike <see cref="SimpleFSLockFactory"/>, the existence of
                /// leftover lock files in the filesystem is fine because the OS
                /// will free the locks held against these files even though the
                /// files still remain. Lucene will never actively remove the lock
                /// files, so although you see them, the index may not be locked.</para>
                ///
                /// <para>Special care needs to be taken if you change the locking
                /// implementation: First be certain that no writer is in fact
                /// writing to the index otherwise you can easily corrupt
                /// your index. Be sure to do the <see cref="LockFactory"/> change on all Lucene
                /// instances and clean up all leftover lock files before starting
                /// the new configuration for the first time. Different implementations
                /// can not work together!</para>
                ///
                /// <para>If you suspect that this or any other <see cref="LockFactory"/> is
                /// not working properly in your environment, you can easily
                /// test it by using <see cref="VerifyingLockFactory"/>,
                /// <see cref="LockVerifyServer"/> and <see cref="LockStressTest"/>.</para>
                /// </summary>
                /// <seealso cref="LockFactory"/>
                public class NativeFSLockFactory : FSLockFactory
                {
                                /// <summary>
                                /// Create a <see cref="NativeFSLockFactory"/> instance, with <c>null</c> (unset)
                                /// lock directory. When you pass this factory to a <see cref="FSDirectory"/>
                                /// subclass, the lock directory is automatically set to the
                                /// directory itself. Be sure to create one instance for each directory
                                /// your create!
                                /// </summary>
                                public NativeFSLockFactory()
                                                : this((DirectoryInfo)null)
                                {
                                }

                                /// <summary>
                                /// Create a <see cref="NativeFSLockFactory"/> instance, storing lock
                                /// files into the specified <paramref name="lockDirName"/>
                                /// </summary>
                                /// <param name="lockDirName"> where lock files are created. </param>
                                public NativeFSLockFactory(string lockDirName)
                                                : this(new DirectoryInfo(lockDirName))
                                {
                                }

                                /// <summary>
                                /// Create a <see cref="NativeFSLockFactory"/> instance, storing lock
                                /// files into the specified <paramref name="lockDir"/>
                                /// </summary>
                                /// <param name="lockDir"> where lock files are created. </param>
                                public NativeFSLockFactory(DirectoryInfo lockDir)
                                {
                                                SetLockDir(lockDir);
                                }

                                // LUCENENET: NativeFSLocks in Java are infact singletons; this is how we mimick that to track instances and make sure
                                // IW.Unlock and IW.IsLocked works correctly
                                internal static readonly Dictionary<string, NativeFSLock> _locks = new Dictionary<string, NativeFSLock>();

                                /// <summary>
                                /// Given a lock name, return the full prefixed path of the actual lock file.
                                /// </summary>
                                /// <param name="lockName"></param>
                                /// <returns></returns>
                                private string GetPathOfLockFile(string lockName)
                                {
                                                if (m_lockPrefix != null)
                                                {
                                                                lockName = m_lockPrefix + "-" + lockName;
                                                }
                                                return Path.Combine(m_lockDir.FullName, lockName);
                                }

                                public override Lock MakeLock(string lockName)
                                {
                                                var path = GetPathOfLockFile(lockName);
                                                NativeFSLock l;
                                                lock(_locks)
                                                                if (!_locks.TryGetValue(path, out l))
                                                                                _locks.Add(path, l = new NativeFSLock(this, m_lockDir, path));
                                                return l;
                                }

                                public override void ClearLock(string lockName)
                                {
                                                var path = GetPathOfLockFile(lockName);
                                                NativeFSLock l;
                                                // this is the reason why we can't use ConcurrentDictionary: we need the removal and disposal of the lock to be atomic
                                                // otherwise it may clash with MakeLock making a lock and ClearLock disposing of it in another thread.
                                                lock (_locks)
                                                                if (_locks.TryGetValue(path, out l))
                                                                {
                                                                                _locks.Remove(path);
                                                                                l.Dispose();
                                                                }
                                }
                }

                internal class NativeFSLock : Lock
                {
#if NETSTANDARD
                                private const int ERROR_SHARE_VIOLATION = 0x20; #else
                                private const int ERROR_LOCK_VIOLATION = 0x21; #endif

                                private readonly NativeFSLockFactory outerInstance;

                                private FileStream channel;
                                private readonly string path;
                                private readonly DirectoryInfo lockDir;

                                public NativeFSLock(NativeFSLockFactory outerInstance, DirectoryInfo lockDir, string path)
                                {
                                                this.outerInstance = outerInstance;
                                                this.lockDir = lockDir;
                                                this.path = path;
                                }

                                /// <summary>
                                /// Return true if the <see cref="IOException"/> is the result of a lock violation
                                /// </summary>
                                /// <param name="e"></param>
                                /// <returns></returns>
                                private static bool IsLockOrShareViolation(IOException e)
                                {
                                                var result = e.HResult & 0xFFFF; #if NETSTANDARD
                                                return result == ERROR_SHARE_VIOLATION; #else
                                                return result == ERROR_LOCK_VIOLATION; #endif
                                }

                                private FileStream GetLockFileStream(FileMode mode)
                                {
                                                if (!System.IO.Directory.Exists(lockDir.FullName))
                                                {
                                                                try
                                                                {
                                                                                System.IO.Directory.CreateDirectory(lockDir.FullName);
                                                                }
                                                                catch (Exception e)
                                                                {
                                                                                // note that several processes might have been trying to create the same directory at the same time.
                                                                                // if one succeeded, the directory will exist and the exception can be ignored. In all other cases we should report it.
                                                                                if (!System.IO.Directory.Exists(lockDir.FullName))
                                                                                                throw new IOException("Cannot create directory: " + lockDir.FullName, e);
                                                                }
                                                }
                                                else if (File.Exists(lockDir.FullName))
                                                {
                                                                throw new IOException("Found regular file where directory expected: " + lockDir.FullName);
                                                }

#if NETSTANDARD
                                                return new FileStream(path, mode, FileAccess.Write, FileShare.None, 1, mode == FileMode.Open ? FileOptions.None : FileOptions.DeleteOnClose); #else
                                                return new FileStream(path, mode, FileAccess.Write, FileShare.ReadWrite); #endif
                                }

                                public override bool Obtain()
                                {
                                                lock (this)
                                                {
                                                                FailureReason = null;

                                                                if (channel != null)
                                                                {
                                                                                // Our instance is already locked:
                                                                                return false;
                                                                }

#if NETSTANDARD
                                                                try
                                                                {
                                                                                channel = GetLockFileStream(FileMode.OpenOrCreate);
                                                                }
                                                                catch (IOException e) when(IsLockOrShareViolation(e))
                                                                {
                                                                                // no failure reason to be recorded, since this is the expected error if a lock exists
                                                                }
                                                                catch (Exception e)
                                                                {
                                                                                // all other exceptions need to be recorded
                                                                                FailureReason = e;
                                                                } #else
                                                                FileStream stream = null;
                                                                try
                                                                {
                                                                                stream = GetLockFileStream(FileMode.OpenOrCreate);
                                                                }
                                                                catch (IOException e)
                                                                {
                                                                                FailureReason = e;
                                                                }
                                                                // LUCENENET: UnauthorizedAccessException does not derive from IOException like in java
                                                                catch (UnauthorizedAccessException e)
                                                                {
                                                                                // On Windows, we can get intermittent "Access
                                                                                // Denied" here.  So, we treat this as failure to
                                                                                // acquire the lock, but, store the reason in case
                                                                                // there is in fact a real error case.
                                                                                FailureReason = e;
                                                                }

                                                                if (stream != null)
                                                                                try
                                                                                {
                                                                                                stream.Lock(0, 1);
                                                                                                // only assign the channel if the lock succeeds
                                                                                                channel = stream;
                                                                                }
                                                                                catch (Exception e)
                                                                                {
                                                                                                FailureReason = e;
                                                                                                IOUtils.DisposeWhileHandlingException(stream);
                                                                                } #endif
                                                                return channel != null;
                                                }
                                }

                                protected override void Dispose(bool disposing)
                                {
                                                if (disposing)
                                                {
                                                                lock (this)
                                                                {
                                                                                if (channel != null)
                                                                                {
                                                                                                try
                                                                                                {
                                                                                                                NativeFSLockFactory._locks.Remove(path);
                                                                                                }
                                                                                                finally
                                                                                                {
                                                                                                                IOUtils.DisposeWhileHandlingException(channel);
                                                                                                                channel = null;
                                                                                                } #if !NETSTANDARD
                                                                                                // try to delete the file if we created it, but it's not an error if we can't.
                                                                                                                try
                                                                                                                {
                                                                                                                                File.Delete(path);
                                                                                                                }
                                                                                                                catch
                                                                                                                {
                                                                                                                } #endif
                                                                                }
                                                                }
                                                }
                                }

                                public override bool IsLocked()
                                {
                                                lock (this)
                                                {
                                                                // First a shortcut, if a lock reference in this instance is available
                                                                if (channel != null)
                                                                {
                                                                                return true;
                                                                }

#if NETSTANDARD
                                                                try
                                                                {
                                                                                using (var stream = GetLockFileStream(FileMode.Open))
                                                                                {
                                                                                }
                                                                                return false;
                                                                }
                                                                catch (IOException e) when (IsLockOrShareViolation(e))
                                                                {
                                                                                return true;
                                                                }
                                                                catch (FileNotFoundException)
                                                                {
                                                                                // if the file doesn't exists, there can be no lock active
                                                                                return false;
                                                                } #else
                                                                try
                                                                {
                                                                                using (var stream = GetLockFileStream(FileMode.Open))
                                                                                {
                                                                                                // try to find out if the file is locked by writing a byte. Note that we need to flush the stream to find out.
                                                                                                stream.WriteByte(0);
                                                                                                stream.Flush();   // this *may* throw an IOException if the file is locked, but...
                                                                                                                                                                                                // ... closing the stream is the real test
                                                                                }
                                                                                return false;
                                                                }
                                                                catch (IOException e) when (IsLockOrShareViolation(e))
                                                                {
                                                                                return true;
                                                                }
                                                                catch (FileNotFoundException)
                                                                {
                                                                                // if the file doesn't exists, there can be no lock active
                                                                                return false;
                                                                } #endif
                                                }
                                }

                                public override string ToString()
                                {
                                                return "NativeFSLock@" + path;
                                }
                }
}



RE: [Possibly spoofed] Re: An alternative NativeFSLockFactory

Posted by "Van Den Berghe, Vincent" <Vi...@bvdinfo.com>.
Thanks for the feedback, but I don't see what problem this would solve: this isn't thread synchronization, it's process synchronization.
Care to elaborate on a scenario where TryEnter would handle a case that the code doesn't?

Thanks,
Vincent

-----Original Message-----
From: Srini V [mailto:vgeneral7@gmail.com] 
Sent: Friday, July 21, 2017 5:08 PM
To: dev@lucenenet.apache.org
Subject: [Possibly spoofed] Re: An alternative NativeFSLockFactory

I haven't quite looked in to the code but based on the description in the email, probably we should use Monitor.TryEnter that way a thread enters the critical section code otherwise tells that another thread is already executing that code

Thanks & Regards

On Fri, Jul 21, 2017 at 10:01 AM, Van Den Berghe, Vincent < Vincent.VanDenBerghe@bvdinfo.com> wrote:

> Hello everyone,
>
> I spent an inordinate amount of time looking at the implementation of 
> NativeFSLockFactory, and I see a potential problem.
>
> Consider the IsLocked() implementation: this method determines if a  
> lock is available, but only does so by trying to Obtain the lock and 
> releasing it, reporting false if it succeeds.
> If the lock is free and two processes A and B both call IsLocked() to 
> determine if it is, there is a nonzero chance that only one of them 
> will report success, since the second process may try to obtain the 
> lock that the first process is temporarily holding, causing its method 
> to return true. Normally, both should return false if the lock is free.
>
> Since I'm in a complaining mood: I find the Dipose() pattern on the 
> locks very confusing, since the conventional implementation pattern 
> specifies that an IDisposable object is not to be used anymore after 
> Dispose() is called. However, after disposing a lock you can call 
> Obtain() again with no ill consequences. The cycle can be repeated ad 
> nauseam, a fact that used in some tests.
>
> Recent implementations of the native locks in java get rid of the 
> "Obtain", and the IsLocked (a Good Thing), so in the far far future 
> these problems will solve themselves.
>
> For now, I'd like to submit an implementation that attempts to 
> corrects the IsLocked problem, without giving any guarantee that it 
> won't introduce other problems or make existing ones go away. Sadly, 
> the implementation corrects them only in regular .NET. In the version 
> of .NET Core we are using, file locking is not implemented, and the 
> implementation falls back to its old behavior.
>
> It is much closer to the java method of doing things. Let me know what 
> you think and feel free to improve (and correct bugs).
>
> using Lucene.Net.Util;
> using System;
> using System.IO;
> using System.Collections.Generic;
>
> namespace Lucene.Net.Store
> {
>                 /*
>                 * Licensed to the Apache Software Foundation (ASF) 
> under one or more
>                 * contributor license agreements.  See the NOTICE file 
> distributed with
>                 * this work for additional information regarding 
> copyright ownership.
>                 * The ASF licenses this file to You under the Apache 
> License, Version 2.0
>                 * (the "License"); you may not use this file except in 
> compliance with
>                 * the License.  You may obtain a copy of the License at
>                 *
>                 *     http://www.apache.org/licenses/LICENSE-2.0
>                 *
>                 * Unless required by applicable law or agreed to in 
> writing, software
>                 * distributed under the License is distributed on an 
> "AS IS" BASIS,
>                 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either 
> express or implied.
>                 * See the License for the specific language governing 
> permissions and
>                 * limitations under the License.
>                 */
>
>                 /// <summary>
>                 /// <para>Implements <see cref="LockFactory"/> using 
> native OS file
>                 /// locks.  For NFS based access to an index, it's
>                 /// recommended that you try <see 
> cref="SimpleFSLockFactory"/>
>                 /// first and work around the one limitation that a 
> lock file
>                 /// could be left when the runtime exits abnormally.</para>
>                 ///
>                 /// <para>The primary benefit of <see 
> cref="NativeFSLockFactory"/> is
>                 /// that locks (not the lock file itsself) will be properly
>                 /// removed (by the OS) if the runtime has an abnormal 
> exit.</para>
>                 ///
>                 /// <para>Note that, unlike <see 
> cref="SimpleFSLockFactory"/>, the existence of
>                 /// leftover lock files in the filesystem is fine 
> because the OS
>                 /// will free the locks held against these files even 
> though the
>                 /// files still remain. Lucene will never actively 
> remove the lock
>                 /// files, so although you see them, the index may not 
> be locked.</para>
>                 ///
>                 /// <para>Special care needs to be taken if you change 
> the locking
>                 /// implementation: First be certain that no writer is 
> in fact
>                 /// writing to the index otherwise you can easily corrupt
>                 /// your index. Be sure to do the <see 
> cref="LockFactory"/> change on all Lucene
>                 /// instances and clean up all leftover lock files 
> before starting
>                 /// the new configuration for the first time. 
> Different implementations
>                 /// can not work together!</para>
>                 ///
>                 /// <para>If you suspect that this or any other <see 
> cref="LockFactory"/> is
>                 /// not working properly in your environment, you can 
> easily
>                 /// test it by using <see cref="VerifyingLockFactory"/>,
>                 /// <see cref="LockVerifyServer"/> and <see 
> cref="LockStressTest"/>.</para>
>                 /// </summary>
>                 /// <seealso cref="LockFactory"/>
>                 public class NativeFSLockFactory : FSLockFactory
>                 {
>                                 /// <summary>
>                                 /// Create a <see 
> cref="NativeFSLockFactory"/> instance, with <c>null</c> (unset)
>                                 /// lock directory. When you pass this 
> factory to a <see cref="FSDirectory"/>
>                                 /// subclass, the lock directory is 
> automatically set to the
>                                 /// directory itself. Be sure to 
> create one instance for each directory
>                                 /// your create!
>                                 /// </summary>
>                                 public NativeFSLockFactory()
>                                                 : this((DirectoryInfo)null)
>                                 {
>                                 }
>
>                                 /// <summary>
>                                 /// Create a <see 
> cref="NativeFSLockFactory"/> instance, storing lock
>                                 /// files into the specified <paramref 
> name="lockDirName"/>
>                                 /// </summary>
>                                 /// <param name="lockDirName"> where 
> lock files are created. </param>
>                                 public NativeFSLockFactory(string
> lockDirName)
>                                                 : this(new
> DirectoryInfo(lockDirName))
>                                 {
>                                 }
>
>                                 /// <summary>
>                                 /// Create a <see 
> cref="NativeFSLockFactory"/> instance, storing lock
>                                 /// files into the specified <paramref 
> name="lockDir"/>
>                                 /// </summary>
>                                 /// <param name="lockDir"> where lock 
> files are created. </param>
>                                 public 
> NativeFSLockFactory(DirectoryInfo
> lockDir)
>                                 {
>                                                 SetLockDir(lockDir);
>                                 }
>
>                                 // LUCENENET: NativeFSLocks in Java 
> are infact singletons; this is how we mimick that to track instances 
> and make sure
>                                 // IW.Unlock and IW.IsLocked works 
> correctly
>                                 internal static readonly 
> Dictionary<string, NativeFSLock> _locks = new Dictionary<string,
> NativeFSLock>();
>
>                                 /// <summary>
>                                 /// Given a lock name, return the full 
> prefixed path of the actual lock file.
>                                 /// </summary>
>                                 /// <param name="lockName"></param>
>                                 /// <returns></returns>
>                                 private string 
> GetPathOfLockFile(string
> lockName)
>                                 {
>                                                 if (m_lockPrefix != null)
>                                                 {
>                                                                 
> lockName = m_lockPrefix + "-" + lockName;
>                                                 }
>                                                 return 
> Path.Combine(m_lockDir.FullName, lockName);
>                                 }
>
>                                 public override Lock MakeLock(string
> lockName)
>                                 {
>                                                 var path = 
> GetPathOfLockFile(lockName);
>                                                 NativeFSLock l;
>                                                 lock(_locks)
>                                                                 if 
> (!_locks.TryGetValue(path, out l))
>
>       _locks.Add(path, l = new NativeFSLock(this, m_lockDir, path));
>                                                 return l;
>                                 }
>
>                                 public override void ClearLock(string
> lockName)
>                                 {
>                                                 var path = 
> GetPathOfLockFile(lockName);
>                                                 NativeFSLock l;
>                                                 // this is the reason 
> why we can't use ConcurrentDictionary: we need the removal and 
> disposal of the lock to be atomic
>                                                 // otherwise it may 
> clash with MakeLock making a lock and ClearLock disposing of it in another thread.
>                                                 lock (_locks)
>                                                                 if 
> (_locks.TryGetValue(path, out l))
>                                                                 {
>
>       _locks.Remove(path);
>
>       l.Dispose();
>                                                                 }
>                                 }
>                 }
>
>                 internal class NativeFSLock : Lock
>                 {
> #if NETSTANDARD
>                                 private const int 
> ERROR_SHARE_VIOLATION = 0x20; #else
>                                 private const int ERROR_LOCK_VIOLATION 
> = 0x21; #endif
>
>                                 private readonly NativeFSLockFactory 
> outerInstance;
>
>                                 private FileStream channel;
>                                 private readonly string path;
>                                 private readonly DirectoryInfo 
> lockDir;
>
>                                 public 
> NativeFSLock(NativeFSLockFactory outerInstance, DirectoryInfo lockDir, string path)
>                                 {
>                                                 this.outerInstance = 
> outerInstance;
>                                                 this.lockDir = lockDir;
>                                                 this.path = path;
>                                 }
>
>                                 /// <summary>
>                                 /// Return true if the <see 
> cref="IOException"/> is the result of a lock violation
>                                 /// </summary>
>                                 /// <param name="e"></param>
>                                 /// <returns></returns>
>                                 private static bool 
> IsLockOrShareViolation(IOException
> e)
>                                 {
>                                                 var result = e.HResult 
> & 0xFFFF; #if NETSTANDARD
>                                                 return result == 
> ERROR_SHARE_VIOLATION; #else
>                                                 return result == 
> ERROR_LOCK_VIOLATION; #endif
>                                 }
>
>                                 private FileStream 
> GetLockFileStream(FileMode mode)
>                                 {
>                                                 if
> (!System.IO.Directory.Exists(lockDir.FullName))
>                                                 {
>                                                                 try
>                                                                 {
>
>       System.IO.Directory.CreateDirectory(lockDir.FullName);
>                                                                 }
>                                                                 catch 
> (Exception e)
>                                                                 {
>
>       // note that several processes might have been trying to create 
> the same directory at the same time.
>
>       // if one succeeded, the directory will exist and the exception 
> can be ignored. In all other cases we should report it.
>
>       if (!System.IO.Directory.Exists(lockDir.FullName))
>
>                       throw new IOException("Cannot create directory: 
> " + lockDir.FullName, e);
>                                                                 }
>                                                 }
>                                                 else if
> (File.Exists(lockDir.FullName))
>                                                 {
>                                                                 throw 
> new IOException("Found regular file where directory expected: " + 
> lockDir.FullName);
>                                                 }
>
> #if NETSTANDARD
>                                                 return new 
> FileStream(path, mode, FileAccess.Write, FileShare.None, 1, mode == 
> FileMode.Open ? FileOptions.None : FileOptions.DeleteOnClose); #else
>                                                 return new 
> FileStream(path, mode, FileAccess.Write, FileShare.ReadWrite); #endif
>                                 }
>
>                                 public override bool Obtain()
>                                 {
>                                                 lock (this)
>                                                 {
>
> FailureReason = null;
>
>                                                                 if 
> (channel != null)
>                                                                 {
>
>       // Our instance is already locked:
>
>       return false;
>                                                                 }
>
> #if NETSTANDARD
>                                                                 try
>                                                                 {
>
>       channel = GetLockFileStream(FileMode.OpenOrCreate);
>                                                                 }
>                                                                 catch 
> (IOException e) when(IsLockOrShareViolation(e))
>                                                                 {
>
>       // no failure reason to be recorded, since this is the expected 
> error if a lock exists
>                                                                 }
>                                                                 catch 
> (Exception e)
>                                                                 {
>
>       // all other exceptions need to be recorded
>
>       FailureReason = e;
>                                                                 } 
> #else
>                                                                 
> FileStream stream = null;
>                                                                 try
>                                                                 {
>
>       stream = GetLockFileStream(FileMode.OpenOrCreate);
>                                                                 }
>                                                                 catch 
> (IOException e)
>                                                                 {
>
>       FailureReason = e;
>                                                                 }
>                                                                 //
> LUCENENET: UnauthorizedAccessException does not derive from 
> IOException like in java
>                                                                 catch 
> (UnauthorizedAccessException e)
>                                                                 {
>
>       // On Windows, we can get intermittent "Access
>
>       // Denied" here.  So, we treat this as failure to
>
>       // acquire the lock, but, store the reason in case
>
>       // there is in fact a real error case.
>
>       FailureReason = e;
>                                                                 }
>
>                                                                 if 
> (stream != null)
>
>       try
>
>       {
>
>                       stream.Lock(0, 1);
>
>                       // only assign the channel if the lock succeeds
>
>                       channel = stream;
>
>       }
>
>       catch (Exception e)
>
>       {
>
>                       FailureReason = e;
>
>                       IOUtils.DisposeWhileHandlingException(stream);
>
>       }
> #endif
>                                                                 return 
> channel != null;
>                                                 }
>                                 }
>
>                                 protected override void Dispose(bool
> disposing)
>                                 {
>                                                 if (disposing)
>                                                 {
>                                                                 lock (this)
>                                                                 {
>
>       if (channel != null)
>
>       {
>
>                       try
>
>                       {
>
>                                       NativeFSLockFactory._locks.
> Remove(path);
>
>                       }
>
>                       finally
>
>                       {
>
>                                       IOUtils.
> DisposeWhileHandlingException(channel);
>
>                                       channel = null;
>
>                       }
> #if !NETSTANDARD
>
>                       // try to delete the file if we created it, but 
> it's not an error if we can't.
>
>                                       try
>
>                                       {
>
>                                                       
> File.Delete(path);
>
>                                       }
>
>                                       catch
>
>                                       {
>
>                                       } #endif
>
>       }
>                                                                 }
>                                                 }
>                                 }
>
>                                 public override bool IsLocked()
>                                 {
>                                                 lock (this)
>                                                 {
>                                                                 // 
> First a shortcut, if a lock reference in this instance is available
>                                                                 if 
> (channel != null)
>                                                                 {
>
>       return true;
>                                                                 }
>
> #if NETSTANDARD
>                                                                 try
>                                                                 {
>
>       using (var stream = GetLockFileStream(FileMode.Open))
>
>       {
>
>       }
>
>       return false;
>                                                                 }
>                                                                 catch 
> (IOException e) when (IsLockOrShareViolation(e))
>                                                                 {
>
>       return true;
>                                                                 }
>                                                                 catch
> (FileNotFoundException)
>                                                                 {
>
>       // if the file doesn't exists, there can be no lock active
>
>       return false;
>                                                                 } 
> #else
>                                                                 try
>                                                                 {
>
>       using (var stream = GetLockFileStream(FileMode.Open))
>
>       {
>
>                       // try to find out if the file is locked by 
> writing a byte. Note that we need to flush the stream to find out.
>
>                       stream.WriteByte(0);
>
>                       stream.Flush();   // this *may* throw an IOException
> if the file is locked, but...
>
>
>                                           // ... closing the stream is 
> the real test
>
>       }
>
>       return false;
>                                                                 }
>                                                                 catch 
> (IOException e) when (IsLockOrShareViolation(e))
>                                                                 {
>
>       return true;
>                                                                 }
>                                                                 catch
> (FileNotFoundException)
>                                                                 {
>
>       // if the file doesn't exists, there can be no lock active
>
>       return false;
>                                                                 } 
> #endif
>                                                 }
>                                 }
>
>                                 public override string ToString()
>                                 {
>                                                 return "NativeFSLock@" 
> + path;
>                                 }
>                 }
> }
>
>
>

Re: An alternative NativeFSLockFactory

Posted by Srini V <vg...@gmail.com>.
I haven't quite looked in to the code but based on the description in the
email, probably we should use Monitor.TryEnter that way a thread enters the
critical section code otherwise tells that another thread is already
executing that code

Thanks & Regards

On Fri, Jul 21, 2017 at 10:01 AM, Van Den Berghe, Vincent <
Vincent.VanDenBerghe@bvdinfo.com> wrote:

> Hello everyone,
>
> I spent an inordinate amount of time looking at the implementation of
> NativeFSLockFactory, and I see a potential problem.
>
> Consider the IsLocked() implementation: this method determines if a  lock
> is available, but only does so by trying to Obtain the lock and releasing
> it, reporting false if it succeeds.
> If the lock is free and two processes A and B both call IsLocked() to
> determine if it is, there is a nonzero chance that only one of them will
> report success, since the second process may try to obtain the lock that
> the first process is temporarily holding, causing its method to return
> true. Normally, both should return false if the lock is free.
>
> Since I'm in a complaining mood: I find the Dipose() pattern on the locks
> very confusing, since the conventional implementation pattern specifies
> that an IDisposable object is not to be used anymore after Dispose() is
> called. However, after disposing a lock you can call Obtain() again with no
> ill consequences. The cycle can be repeated ad nauseam, a fact that used in
> some tests.
>
> Recent implementations of the native locks in java get rid of the
> "Obtain", and the IsLocked (a Good Thing), so in the far far future these
> problems will solve themselves.
>
> For now, I'd like to submit an implementation that attempts to corrects
> the IsLocked problem, without giving any guarantee that it won't introduce
> other problems or make existing ones go away. Sadly, the implementation
> corrects them only in regular .NET. In the version of .NET Core we are
> using, file locking is not implemented, and the implementation falls back
> to its old behavior.
>
> It is much closer to the java method of doing things. Let me know what you
> think and feel free to improve (and correct bugs).
>
> using Lucene.Net.Util;
> using System;
> using System.IO;
> using System.Collections.Generic;
>
> namespace Lucene.Net.Store
> {
>                 /*
>                 * Licensed to the Apache Software Foundation (ASF) under
> one or more
>                 * contributor license agreements.  See the NOTICE file
> distributed with
>                 * this work for additional information regarding copyright
> ownership.
>                 * The ASF licenses this file to You under the Apache
> License, Version 2.0
>                 * (the "License"); you may not use this file except in
> compliance with
>                 * the License.  You may obtain a copy of the License at
>                 *
>                 *     http://www.apache.org/licenses/LICENSE-2.0
>                 *
>                 * Unless required by applicable law or agreed to in
> writing, software
>                 * distributed under the License is distributed on an "AS
> IS" BASIS,
>                 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
> express or implied.
>                 * See the License for the specific language governing
> permissions and
>                 * limitations under the License.
>                 */
>
>                 /// <summary>
>                 /// <para>Implements <see cref="LockFactory"/> using
> native OS file
>                 /// locks.  For NFS based access to an index, it's
>                 /// recommended that you try <see
> cref="SimpleFSLockFactory"/>
>                 /// first and work around the one limitation that a lock
> file
>                 /// could be left when the runtime exits abnormally.</para>
>                 ///
>                 /// <para>The primary benefit of <see
> cref="NativeFSLockFactory"/> is
>                 /// that locks (not the lock file itsself) will be properly
>                 /// removed (by the OS) if the runtime has an abnormal
> exit.</para>
>                 ///
>                 /// <para>Note that, unlike <see
> cref="SimpleFSLockFactory"/>, the existence of
>                 /// leftover lock files in the filesystem is fine because
> the OS
>                 /// will free the locks held against these files even
> though the
>                 /// files still remain. Lucene will never actively remove
> the lock
>                 /// files, so although you see them, the index may not be
> locked.</para>
>                 ///
>                 /// <para>Special care needs to be taken if you change the
> locking
>                 /// implementation: First be certain that no writer is in
> fact
>                 /// writing to the index otherwise you can easily corrupt
>                 /// your index. Be sure to do the <see
> cref="LockFactory"/> change on all Lucene
>                 /// instances and clean up all leftover lock files before
> starting
>                 /// the new configuration for the first time. Different
> implementations
>                 /// can not work together!</para>
>                 ///
>                 /// <para>If you suspect that this or any other <see
> cref="LockFactory"/> is
>                 /// not working properly in your environment, you can
> easily
>                 /// test it by using <see cref="VerifyingLockFactory"/>,
>                 /// <see cref="LockVerifyServer"/> and <see
> cref="LockStressTest"/>.</para>
>                 /// </summary>
>                 /// <seealso cref="LockFactory"/>
>                 public class NativeFSLockFactory : FSLockFactory
>                 {
>                                 /// <summary>
>                                 /// Create a <see
> cref="NativeFSLockFactory"/> instance, with <c>null</c> (unset)
>                                 /// lock directory. When you pass this
> factory to a <see cref="FSDirectory"/>
>                                 /// subclass, the lock directory is
> automatically set to the
>                                 /// directory itself. Be sure to create
> one instance for each directory
>                                 /// your create!
>                                 /// </summary>
>                                 public NativeFSLockFactory()
>                                                 : this((DirectoryInfo)null)
>                                 {
>                                 }
>
>                                 /// <summary>
>                                 /// Create a <see
> cref="NativeFSLockFactory"/> instance, storing lock
>                                 /// files into the specified <paramref
> name="lockDirName"/>
>                                 /// </summary>
>                                 /// <param name="lockDirName"> where lock
> files are created. </param>
>                                 public NativeFSLockFactory(string
> lockDirName)
>                                                 : this(new
> DirectoryInfo(lockDirName))
>                                 {
>                                 }
>
>                                 /// <summary>
>                                 /// Create a <see
> cref="NativeFSLockFactory"/> instance, storing lock
>                                 /// files into the specified <paramref
> name="lockDir"/>
>                                 /// </summary>
>                                 /// <param name="lockDir"> where lock
> files are created. </param>
>                                 public NativeFSLockFactory(DirectoryInfo
> lockDir)
>                                 {
>                                                 SetLockDir(lockDir);
>                                 }
>
>                                 // LUCENENET: NativeFSLocks in Java are
> infact singletons; this is how we mimick that to track instances and make
> sure
>                                 // IW.Unlock and IW.IsLocked works
> correctly
>                                 internal static readonly
> Dictionary<string, NativeFSLock> _locks = new Dictionary<string,
> NativeFSLock>();
>
>                                 /// <summary>
>                                 /// Given a lock name, return the full
> prefixed path of the actual lock file.
>                                 /// </summary>
>                                 /// <param name="lockName"></param>
>                                 /// <returns></returns>
>                                 private string GetPathOfLockFile(string
> lockName)
>                                 {
>                                                 if (m_lockPrefix != null)
>                                                 {
>                                                                 lockName =
> m_lockPrefix + "-" + lockName;
>                                                 }
>                                                 return
> Path.Combine(m_lockDir.FullName, lockName);
>                                 }
>
>                                 public override Lock MakeLock(string
> lockName)
>                                 {
>                                                 var path =
> GetPathOfLockFile(lockName);
>                                                 NativeFSLock l;
>                                                 lock(_locks)
>                                                                 if
> (!_locks.TryGetValue(path, out l))
>
>       _locks.Add(path, l = new NativeFSLock(this, m_lockDir, path));
>                                                 return l;
>                                 }
>
>                                 public override void ClearLock(string
> lockName)
>                                 {
>                                                 var path =
> GetPathOfLockFile(lockName);
>                                                 NativeFSLock l;
>                                                 // this is the reason why
> we can't use ConcurrentDictionary: we need the removal and disposal of the
> lock to be atomic
>                                                 // otherwise it may clash
> with MakeLock making a lock and ClearLock disposing of it in another thread.
>                                                 lock (_locks)
>                                                                 if
> (_locks.TryGetValue(path, out l))
>                                                                 {
>
>       _locks.Remove(path);
>
>       l.Dispose();
>                                                                 }
>                                 }
>                 }
>
>                 internal class NativeFSLock : Lock
>                 {
> #if NETSTANDARD
>                                 private const int ERROR_SHARE_VIOLATION =
> 0x20;
> #else
>                                 private const int ERROR_LOCK_VIOLATION =
> 0x21;
> #endif
>
>                                 private readonly NativeFSLockFactory
> outerInstance;
>
>                                 private FileStream channel;
>                                 private readonly string path;
>                                 private readonly DirectoryInfo lockDir;
>
>                                 public NativeFSLock(NativeFSLockFactory
> outerInstance, DirectoryInfo lockDir, string path)
>                                 {
>                                                 this.outerInstance =
> outerInstance;
>                                                 this.lockDir = lockDir;
>                                                 this.path = path;
>                                 }
>
>                                 /// <summary>
>                                 /// Return true if the <see
> cref="IOException"/> is the result of a lock violation
>                                 /// </summary>
>                                 /// <param name="e"></param>
>                                 /// <returns></returns>
>                                 private static bool IsLockOrShareViolation(IOException
> e)
>                                 {
>                                                 var result = e.HResult &
> 0xFFFF;
> #if NETSTANDARD
>                                                 return result ==
> ERROR_SHARE_VIOLATION;
> #else
>                                                 return result ==
> ERROR_LOCK_VIOLATION;
> #endif
>                                 }
>
>                                 private FileStream
> GetLockFileStream(FileMode mode)
>                                 {
>                                                 if
> (!System.IO.Directory.Exists(lockDir.FullName))
>                                                 {
>                                                                 try
>                                                                 {
>
>       System.IO.Directory.CreateDirectory(lockDir.FullName);
>                                                                 }
>                                                                 catch
> (Exception e)
>                                                                 {
>
>       // note that several processes might have been trying to create the
> same directory at the same time.
>
>       // if one succeeded, the directory will exist and the exception can
> be ignored. In all other cases we should report it.
>
>       if (!System.IO.Directory.Exists(lockDir.FullName))
>
>                       throw new IOException("Cannot create directory: " +
> lockDir.FullName, e);
>                                                                 }
>                                                 }
>                                                 else if
> (File.Exists(lockDir.FullName))
>                                                 {
>                                                                 throw new
> IOException("Found regular file where directory expected: " +
> lockDir.FullName);
>                                                 }
>
> #if NETSTANDARD
>                                                 return new
> FileStream(path, mode, FileAccess.Write, FileShare.None, 1, mode ==
> FileMode.Open ? FileOptions.None : FileOptions.DeleteOnClose);
> #else
>                                                 return new
> FileStream(path, mode, FileAccess.Write, FileShare.ReadWrite);
> #endif
>                                 }
>
>                                 public override bool Obtain()
>                                 {
>                                                 lock (this)
>                                                 {
>
> FailureReason = null;
>
>                                                                 if
> (channel != null)
>                                                                 {
>
>       // Our instance is already locked:
>
>       return false;
>                                                                 }
>
> #if NETSTANDARD
>                                                                 try
>                                                                 {
>
>       channel = GetLockFileStream(FileMode.OpenOrCreate);
>                                                                 }
>                                                                 catch
> (IOException e) when(IsLockOrShareViolation(e))
>                                                                 {
>
>       // no failure reason to be recorded, since this is the expected error
> if a lock exists
>                                                                 }
>                                                                 catch
> (Exception e)
>                                                                 {
>
>       // all other exceptions need to be recorded
>
>       FailureReason = e;
>                                                                 }
> #else
>                                                                 FileStream
> stream = null;
>                                                                 try
>                                                                 {
>
>       stream = GetLockFileStream(FileMode.OpenOrCreate);
>                                                                 }
>                                                                 catch
> (IOException e)
>                                                                 {
>
>       FailureReason = e;
>                                                                 }
>                                                                 //
> LUCENENET: UnauthorizedAccessException does not derive from IOException
> like in java
>                                                                 catch
> (UnauthorizedAccessException e)
>                                                                 {
>
>       // On Windows, we can get intermittent "Access
>
>       // Denied" here.  So, we treat this as failure to
>
>       // acquire the lock, but, store the reason in case
>
>       // there is in fact a real error case.
>
>       FailureReason = e;
>                                                                 }
>
>                                                                 if (stream
> != null)
>
>       try
>
>       {
>
>                       stream.Lock(0, 1);
>
>                       // only assign the channel if the lock succeeds
>
>                       channel = stream;
>
>       }
>
>       catch (Exception e)
>
>       {
>
>                       FailureReason = e;
>
>                       IOUtils.DisposeWhileHandlingException(stream);
>
>       }
> #endif
>                                                                 return
> channel != null;
>                                                 }
>                                 }
>
>                                 protected override void Dispose(bool
> disposing)
>                                 {
>                                                 if (disposing)
>                                                 {
>                                                                 lock (this)
>                                                                 {
>
>       if (channel != null)
>
>       {
>
>                       try
>
>                       {
>
>                                       NativeFSLockFactory._locks.
> Remove(path);
>
>                       }
>
>                       finally
>
>                       {
>
>                                       IOUtils.
> DisposeWhileHandlingException(channel);
>
>                                       channel = null;
>
>                       }
> #if !NETSTANDARD
>
>                       // try to delete the file if we created it, but it's
> not an error if we can't.
>
>                                       try
>
>                                       {
>
>                                                       File.Delete(path);
>
>                                       }
>
>                                       catch
>
>                                       {
>
>                                       }
> #endif
>
>       }
>                                                                 }
>                                                 }
>                                 }
>
>                                 public override bool IsLocked()
>                                 {
>                                                 lock (this)
>                                                 {
>                                                                 // First a
> shortcut, if a lock reference in this instance is available
>                                                                 if
> (channel != null)
>                                                                 {
>
>       return true;
>                                                                 }
>
> #if NETSTANDARD
>                                                                 try
>                                                                 {
>
>       using (var stream = GetLockFileStream(FileMode.Open))
>
>       {
>
>       }
>
>       return false;
>                                                                 }
>                                                                 catch
> (IOException e) when (IsLockOrShareViolation(e))
>                                                                 {
>
>       return true;
>                                                                 }
>                                                                 catch
> (FileNotFoundException)
>                                                                 {
>
>       // if the file doesn't exists, there can be no lock active
>
>       return false;
>                                                                 }
> #else
>                                                                 try
>                                                                 {
>
>       using (var stream = GetLockFileStream(FileMode.Open))
>
>       {
>
>                       // try to find out if the file is locked by writing a
> byte. Note that we need to flush the stream to find out.
>
>                       stream.WriteByte(0);
>
>                       stream.Flush();   // this *may* throw an IOException
> if the file is locked, but...
>
>
>                                           // ... closing the stream is the
> real test
>
>       }
>
>       return false;
>                                                                 }
>                                                                 catch
> (IOException e) when (IsLockOrShareViolation(e))
>                                                                 {
>
>       return true;
>                                                                 }
>                                                                 catch
> (FileNotFoundException)
>                                                                 {
>
>       // if the file doesn't exists, there can be no lock active
>
>       return false;
>                                                                 }
> #endif
>                                                 }
>                                 }
>
>                                 public override string ToString()
>                                 {
>                                                 return "NativeFSLock@" +
> path;
>                                 }
>                 }
> }
>
>
>