You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucenenet.apache.org by sy...@apache.org on 2016/09/01 14:40:09 UTC

[48/52] [abbrv] lucenenet git commit: Fixed bug in Analysis.Hunspell and Core.Util.OfflineSorter where different threads have problems competing for temp file usage.

Fixed bug in Analysis.Hunspell and Core.Util.OfflineSorter where different threads have problems competing for temp file usage.


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

Branch: refs/heads/master
Commit: defcabee7437ed8595bf841a3c332960c8a1ba9c
Parents: 9ed5b8f
Author: Shad Storhaug <sh...@shadstorhaug.com>
Authored: Fri Aug 26 12:29:34 2016 +0700
Committer: Shad Storhaug <sh...@shadstorhaug.com>
Committed: Sat Aug 27 02:20:05 2016 +0700

----------------------------------------------------------------------
 .../Analysis/Hunspell/Dictionary.cs             | 250 +++++++++----------
 src/Lucene.Net.Core/Support/FileSupport.cs      |  70 ++++++
 src/Lucene.Net.Core/Util/OfflineSorter.cs       |  50 ++--
 3 files changed, 220 insertions(+), 150 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucenenet/blob/defcabee/src/Lucene.Net.Analysis.Common/Analysis/Hunspell/Dictionary.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Analysis.Common/Analysis/Hunspell/Dictionary.cs b/src/Lucene.Net.Analysis.Common/Analysis/Hunspell/Dictionary.cs
index 8bab079..e9fc124 100644
--- a/src/Lucene.Net.Analysis.Common/Analysis/Hunspell/Dictionary.cs
+++ b/src/Lucene.Net.Analysis.Common/Analysis/Hunspell/Dictionary.cs
@@ -134,43 +134,41 @@ namespace Lucene.Net.Analysis.Hunspell
             this.needsOutputCleaning = false; // set if we have an OCONV
             flagLookup.Add(new BytesRef()); // no flags -> ord 0
 
-            FileInfo aff = new FileInfo(System.IO.Path.Combine(tempDir.FullName, "affix.aff"));
-            using (Stream @out = aff.Create())
+            FileInfo aff = FileSupport.CreateTempFile("affix", "aff", tempDir);
+            using (Stream @out = aff.Open(FileMode.Open, FileAccess.ReadWrite))
             {
-                Stream aff1 = null;
-                Stream aff2 = null;
-                try
-                {
-                    // copy contents of affix stream to temp file
-                    byte[] buffer = new byte[1024 * 8];
-                    int len;
-                    while ((len = affix.Read(buffer, 0, buffer.Length)) > 0)
-                    {
-                        @out.Write(buffer, 0, len);
-                    }
-                    @out.Close(); // LUCENENET: Release the file handle - we dispose @out later
-
-                    // pass 1: get encoding
-                    aff1 = File.OpenRead(aff.FullName);
-                    string encoding = GetDictionaryEncoding(aff1);
-
-                    // pass 2: parse affixes
-                    Encoding decoder = GetSystemEncoding(encoding);
-                    aff2 = File.OpenRead(aff.FullName);
-                    ReadAffixFile(aff2, decoder);
-
-                    // read dictionary entries
-                    IntSequenceOutputs o = IntSequenceOutputs.Singleton;
-                    Builder<IntsRef> b = new Builder<IntsRef>(FST.INPUT_TYPE.BYTE4, o);
-                    ReadDictionaryFiles(dictionaries, decoder, b);
-                    words = b.Finish();
-                    aliases = null; // no longer needed
-                }
-                finally
-                {
-                    IOUtils.CloseWhileHandlingException(aff1, aff2);
-                    aff.Delete();
-                }
+                // copy contents of affix stream to temp file
+                affix.CopyTo(@out);
+            }
+
+            // pass 1: get encoding
+            string encoding;
+            using (Stream aff1 = aff.Open(FileMode.Open, FileAccess.Read))
+            {
+                encoding = GetDictionaryEncoding(aff1);
+            }
+
+            // pass 2: parse affixes
+            Encoding decoder = GetSystemEncoding(encoding);
+            using (Stream aff2 = aff.Open(FileMode.Open, FileAccess.Read))
+            {
+                ReadAffixFile(aff2, decoder);
+            }
+
+            // read dictionary entries
+            IntSequenceOutputs o = IntSequenceOutputs.Singleton;
+            Builder<IntsRef> b = new Builder<IntsRef>(FST.INPUT_TYPE.BYTE4, o);
+            ReadDictionaryFiles(dictionaries, decoder, b);
+            words = b.Finish();
+            aliases = null; // no longer needed
+
+            try
+            {
+                aff.Delete();
+            }
+            catch
+            {
+                // ignore
             }
         }
 
@@ -744,10 +742,8 @@ namespace Lucene.Net.Analysis.Hunspell
 
             StringBuilder sb = new StringBuilder();
 
-            FileInfo unsorted = new FileInfo(System.IO.Path.Combine(tempDir.FullName, "unsorted.dat"));
-            OfflineSorter.ByteSequencesWriter writer = new OfflineSorter.ByteSequencesWriter(unsorted);
-            bool success = false;
-            try
+            FileInfo unsorted = FileSupport.CreateTempFile("unsorted", "dat", tempDir);
+            using (OfflineSorter.ByteSequencesWriter writer = new OfflineSorter.ByteSequencesWriter(unsorted))
             {
                 foreach (Stream dictionary in dictionaries)
                 {
@@ -784,113 +780,115 @@ namespace Lucene.Net.Analysis.Hunspell
                         }
                     }
                 }
-                success = true;
-            }
-            finally
-            {
-                if (success)
-                {
-                    IOUtils.Close(writer);
-                }
-                else
-                {
-                    IOUtils.CloseWhileHandlingException(writer);
-                }
             }
-            FileInfo sorted = new FileInfo(System.IO.Path.Combine(tempDir.FullName, "sorted.dat"));
-            using (var temp = sorted.Create()) { }
+
+            FileInfo sorted = FileSupport.CreateTempFile("sorted", "dat", tempDir);
 
             OfflineSorter sorter = new OfflineSorter(new ComparatorAnonymousInnerClassHelper(this));
             sorter.Sort(unsorted, sorted);
-            unsorted.Delete();
-
-            OfflineSorter.ByteSequencesReader reader = new OfflineSorter.ByteSequencesReader(sorted);
-            BytesRef scratchLine = new BytesRef();
+            try
+            {
+                unsorted.Delete();
+            }
+            catch
+            {
+                // ignore
+            }
 
-            // TODO: the flags themselves can be double-chars (long) or also numeric
-            // either way the trick is to encode them as char... but they must be parsed differently
+            using (OfflineSorter.ByteSequencesReader reader = new OfflineSorter.ByteSequencesReader(sorted))
+            {
+                BytesRef scratchLine = new BytesRef();
 
-            string currentEntry = null;
-            IntsRef currentOrds = new IntsRef();
+                // TODO: the flags themselves can be double-chars (long) or also numeric
+                // either way the trick is to encode them as char... but they must be parsed differently
 
-            string line2;
-            while (reader.Read(scratchLine))
-            {
-                line2 = scratchLine.Utf8ToString();
-                string entry;
-                char[] wordForm;
+                string currentEntry = null;
+                IntsRef currentOrds = new IntsRef();
 
-                int flagSep = line2.LastIndexOf(FLAG_SEPARATOR);
-                if (flagSep == -1)
+                string line2;
+                while (reader.Read(scratchLine))
                 {
-                    wordForm = NOFLAGS;
-                    entry = line2;
-                }
-                else
-                {
-                    // note, there can be comments (morph description) after a flag.
-                    // we should really look for any whitespace: currently just tab and space
-                    int end = line2.IndexOf('\t', flagSep);
-                    if (end == -1)
+                    line2 = scratchLine.Utf8ToString();
+                    string entry;
+                    char[] wordForm;
+
+                    int flagSep = line2.LastIndexOf(FLAG_SEPARATOR);
+                    if (flagSep == -1)
                     {
-                        end = line2.Length;
+                        wordForm = NOFLAGS;
+                        entry = line2;
                     }
-                    int end2 = line2.IndexOf(' ', flagSep);
-                    if (end2 == -1)
+                    else
                     {
-                        end2 = line2.Length;
-                    }
-                    end = Math.Min(end, end2);
+                        // note, there can be comments (morph description) after a flag.
+                        // we should really look for any whitespace: currently just tab and space
+                        int end = line2.IndexOf('\t', flagSep);
+                        if (end == -1)
+                        {
+                            end = line2.Length;
+                        }
+                        int end2 = line2.IndexOf(' ', flagSep);
+                        if (end2 == -1)
+                        {
+                            end2 = line2.Length;
+                        }
+                        end = Math.Min(end, end2);
 
-                    string flagPart = line2.Substring(flagSep + 1, end - (flagSep + 1));
-                    if (aliasCount > 0)
-                    {
-                        flagPart = GetAliasValue(int.Parse(flagPart, CultureInfo.InvariantCulture));
-                    }
+                        string flagPart = line2.Substring(flagSep + 1, end - (flagSep + 1));
+                        if (aliasCount > 0)
+                        {
+                            flagPart = GetAliasValue(int.Parse(flagPart, CultureInfo.InvariantCulture));
+                        }
 
-                    wordForm = flagParsingStrategy.ParseFlags(flagPart);
-                    Array.Sort(wordForm);
-                    entry = line2.Substring(0, flagSep - 0);
-                }
-                // LUCENENET NOTE: CompareToOrdinal is an extension method that works similarly to
-                // Java's String.compareTo method.
-                int cmp = currentEntry == null ? 1 : entry.CompareToOrdinal(currentEntry);
-                if (cmp < 0)
-                {
-                    throw new System.ArgumentException("out of order: " + entry + " < " + currentEntry);
-                }
-                else
-                {
-                    EncodeFlags(flagsScratch, wordForm);
-                    int ord = flagLookup.Add(flagsScratch);
-                    if (ord < 0)
-                    {
-                        // already exists in our hash
-                        ord = (-ord) - 1;
+                        wordForm = flagParsingStrategy.ParseFlags(flagPart);
+                        Array.Sort(wordForm);
+                        entry = line2.Substring(0, flagSep - 0);
                     }
-                    // finalize current entry, and switch "current" if necessary
-                    if (cmp > 0 && currentEntry != null)
+                    // LUCENENET NOTE: CompareToOrdinal is an extension method that works similarly to
+                    // Java's String.compareTo method.
+                    int cmp = currentEntry == null ? 1 : entry.CompareToOrdinal(currentEntry);
+                    if (cmp < 0)
                     {
-                        Lucene.Net.Util.Fst.Util.ToUTF32(currentEntry, scratchInts);
-                        words.Add(scratchInts, currentOrds);
+                        throw new System.ArgumentException("out of order: " + entry + " < " + currentEntry);
                     }
-                    // swap current
-                    if (cmp > 0 || currentEntry == null)
+                    else
                     {
-                        currentEntry = entry;
-                        currentOrds = new IntsRef(); // must be this way
+                        EncodeFlags(flagsScratch, wordForm);
+                        int ord = flagLookup.Add(flagsScratch);
+                        if (ord < 0)
+                        {
+                            // already exists in our hash
+                            ord = (-ord) - 1;
+                        }
+                        // finalize current entry, and switch "current" if necessary
+                        if (cmp > 0 && currentEntry != null)
+                        {
+                            Lucene.Net.Util.Fst.Util.ToUTF32(currentEntry, scratchInts);
+                            words.Add(scratchInts, currentOrds);
+                        }
+                        // swap current
+                        if (cmp > 0 || currentEntry == null)
+                        {
+                            currentEntry = entry;
+                            currentOrds = new IntsRef(); // must be this way
+                        }
+                        currentOrds.Grow(currentOrds.Length + 1);
+                        currentOrds.Ints[currentOrds.Length++] = ord;
                     }
-                    currentOrds.Grow(currentOrds.Length + 1);
-                    currentOrds.Ints[currentOrds.Length++] = ord;
                 }
-            }
-
-            // finalize last entry
-            Lucene.Net.Util.Fst.Util.ToUTF32(currentEntry, scratchInts);
-            words.Add(scratchInts, currentOrds);
 
-            reader.Dispose();
-            sorted.Delete();
+                // finalize last entry
+                Lucene.Net.Util.Fst.Util.ToUTF32(currentEntry, scratchInts);
+                words.Add(scratchInts, currentOrds);
+            }
+            try
+            {
+                sorted.Delete();
+            }
+            catch
+            {
+                // ignore
+            }
         }
 
         private class ComparatorAnonymousInnerClassHelper : IComparer<BytesRef>

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/defcabee/src/Lucene.Net.Core/Support/FileSupport.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Core/Support/FileSupport.cs b/src/Lucene.Net.Core/Support/FileSupport.cs
index 86c070c..ebc34e2 100644
--- a/src/Lucene.Net.Core/Support/FileSupport.cs
+++ b/src/Lucene.Net.Core/Support/FileSupport.cs
@@ -21,6 +21,8 @@
 
 using System;
 using System.IO;
+using System.Runtime.CompilerServices;
+using System.Text;
 
 namespace Lucene.Net.Support
 {
@@ -117,5 +119,73 @@ namespace Lucene.Net.Support
 
         [System.Runtime.InteropServices.DllImport("kernel32.dll")]
         extern static bool FlushFileBuffers(IntPtr hFile);
+
+
+        /// <summary>
+        /// Creates a new empty file in the specified directory, using the given prefix and suffix strings to generate its name. 
+        /// If this method returns successfully then it is guaranteed that:
+        /// <list type="number">
+        /// <item>The file denoted by the returned abstract pathname did not exist before this method was invoked, and</item>
+        /// <item>Neither this method nor any of its variants will return the same abstract pathname again in the current invocation of the virtual machine.</item>
+        /// </list>
+        /// This method provides only part of a temporary-file facility.To arrange for a file created by this method to be deleted automatically, use the deleteOnExit() method.
+        /// The prefix argument must be at least three characters long. It is recommended that the prefix be a short, meaningful string such as "hjb" or "mail". The suffix argument may be null, in which case the suffix ".tmp" will be used.
+        /// To create the new file, the prefix and the suffix may first be adjusted to fit the limitations of the underlying platform.If the prefix is too long then it will be truncated, but its first three characters will always be preserved.If the suffix is too long then it too will be truncated, but if it begins with a period character ('.') then the period and the first three characters following it will always be preserved.Once these adjustments have been made the name of the new file will be generated by concatenating the prefix, five or more internally-generated characters, and the suffix.
+        /// If the directory argument is null then the system-dependent default temporary-file directory will be used.The default temporary-file directory is specified by the system property java.io.tmpdir.On UNIX systems the default value of this property is typically "/tmp" or "/var/tmp"; on Microsoft Windows systems it is typically "C:\\WINNT\\TEMP". A different value may be given to this system property when the Java virtual machine is invoked, but programmatic changes to this property are not guaranteed to have any effect upon the temporary directory used by this method.
+        /// 
+        /// Ported over from the java.io.File class. Used by the Analysis.Hunspell.Directory
+        /// class, but this can probably be removed when that class is upgraded to a more recent
+        /// version of lucene, where it uses the lucene Store.Directory class to create a temporary
+        /// file.
+        /// </summary>
+        /// <param name="prefix">The prefix string to be used in generating the file's name; must be at least three characters long</param>
+        /// <param name="suffix">The suffix string to be used in generating the file's name; may be null, in which case the suffix ".tmp" will be used</param>
+        /// <param name="directory">The directory in which the file is to be created, or null if the default temporary-file directory is to be used</param>
+        /// <returns></returns>
+        [MethodImpl(MethodImplOptions.Synchronized)]
+        public static FileInfo CreateTempFile(string prefix, string suffix, DirectoryInfo directory)
+        {
+            if (string.IsNullOrEmpty(prefix))
+                throw new ArgumentNullException("prefix");
+            if (prefix.Length < 3)
+                throw new ArgumentException("Prefix string too short");
+            string s = (suffix == null) ? ".tmp" : suffix;
+            if (directory == null)
+            {
+                string tmpDir = Path.GetTempPath();
+                directory = new DirectoryInfo(tmpDir);
+            }
+            int attempt = 0;
+            string extension = suffix.StartsWith(".") ? suffix : '.' + suffix;
+            string fileName = Path.Combine(directory.FullName, string.Concat(prefix, extension));
+            while (true)
+            {
+                try
+                {
+                    if (attempt > 0)
+                    {
+                        fileName = Path.Combine(directory.FullName, string.Concat(prefix, attempt.ToString(), extension));
+                    }
+                    if (File.Exists(fileName))
+                    {
+                        attempt++;
+                        continue;
+                    }
+                    // Create the file
+                    File.WriteAllText(fileName, string.Empty, Encoding.UTF8);
+                    break;
+                }
+                catch (IOException e)
+                {
+                    if (!e.Message.Contains("already exists"))
+                    {
+                        throw e;
+                    }
+
+                    attempt++;
+                }
+            }
+            return new FileInfo(fileName);
+        }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucenenet/blob/defcabee/src/Lucene.Net.Core/Util/OfflineSorter.cs
----------------------------------------------------------------------
diff --git a/src/Lucene.Net.Core/Util/OfflineSorter.cs b/src/Lucene.Net.Core/Util/OfflineSorter.cs
index 4f3084b..0f15405 100644
--- a/src/Lucene.Net.Core/Util/OfflineSorter.cs
+++ b/src/Lucene.Net.Core/Util/OfflineSorter.cs
@@ -4,6 +4,8 @@ using System.Diagnostics;
 using System.IO;
 using Lucene.Net.Store;
 using Lucene.Net.Support.Compatibility;
+using System.Linq;
+using Lucene.Net.Support;
 
 namespace Lucene.Net.Util
 {
@@ -267,7 +269,9 @@ namespace Lucene.Net.Util
         {
             sortInfo = new SortInfo(this) { TotalTime = DateTime.Now.Millisecond };
 
-            output.Delete();
+            // LUCENENET NOTE: Can't do this because another thread could recreate the file before we are done here.
+            // and cause this to bomb. We use the existence of the file as an indicator that we are done using it.
+            //output.Delete(); 
 
             var merges = new List<FileInfo>();
             bool success2 = false;
@@ -369,7 +373,13 @@ namespace Lucene.Net.Util
         /// </summary>
         private static void Copy(FileInfo file, FileInfo output)
         {
-            File.Copy(file.FullName, output.FullName);
+            using (Stream inputStream = file.OpenRead())
+            {
+                using (Stream outputStream = output.OpenWrite())
+                {
+                    inputStream.CopyTo(outputStream);
+                }
+            }
         }
 
         /// <summary>
@@ -377,53 +387,45 @@ namespace Lucene.Net.Util
         internal FileInfo SortPartition(int len)
         {
             var data = this.Buffer;
-            var tempFile = new FileInfo(Path.GetTempFileName());
-            //var tempFile1 = File.Create(new ());
-            //FileInfo tempFile = FileInfo.createTempFile("sort", "partition", TempDirectory);
+            FileInfo tempFile = FileSupport.CreateTempFile("sort", "partition", DefaultTempDir());
 
             long start = DateTime.Now.Millisecond;
             sortInfo.SortTime += (DateTime.Now.Millisecond - start);
 
-            var @out = new ByteSequencesWriter(tempFile);
-            BytesRef spare;
-            try
+            using (var @out = new ByteSequencesWriter(tempFile))
             {
+                BytesRef spare;
+
                 BytesRefIterator iter = Buffer.Iterator(comparator);
                 while ((spare = iter.Next()) != null)
                 {
                     Debug.Assert(spare.Length <= short.MaxValue);
                     @out.Write(spare);
                 }
-
-                @out.Dispose();
-
-                // Clean up the buffer for the next partition.
-                data.Clear();
-                return tempFile;
-            }
-            finally
-            {
-                IOUtils.Close(@out);
             }
+
+            // Clean up the buffer for the next partition.
+            data.Clear();
+            return tempFile;
         }
 
         /// <summary>
         /// Merge a list of sorted temporary files (partitions) into an output file </summary>
-        internal void MergePartitions(IList<FileInfo> merges, FileInfo outputFile)
+        internal void MergePartitions(IEnumerable<FileInfo> merges, FileInfo outputFile)
         {
             long start = DateTime.Now.Millisecond;
 
             var @out = new ByteSequencesWriter(outputFile);
 
-            PriorityQueue<FileAndTop> queue = new PriorityQueueAnonymousInnerClassHelper(this, merges.Count);
+            PriorityQueue<FileAndTop> queue = new PriorityQueueAnonymousInnerClassHelper(this, merges.Count());
 
-            var streams = new ByteSequencesReader[merges.Count];
+            var streams = new ByteSequencesReader[merges.Count()];
             try
             {
                 // Open streams and read the top for each file
-                for (int i = 0; i < merges.Count; i++)
+                for (int i = 0; i < merges.Count(); i++)
                 {
-                    streams[i] = new ByteSequencesReader(merges[i]);
+                    streams[i] = new ByteSequencesReader(merges.ElementAt(i));
                     byte[] line = streams[i].Read();
                     if (line != null)
                     {
@@ -528,7 +530,7 @@ namespace Lucene.Net.Util
             /// <summary>
             /// Constructs a ByteSequencesWriter to the provided File </summary>
             public ByteSequencesWriter(FileInfo file)
-                : this(new BinaryWriterDataOutput(new BinaryWriter(new FileStream(file.FullName, FileMode.OpenOrCreate))))
+                : this(new BinaryWriterDataOutput(new BinaryWriter(new FileStream(file.FullName, FileMode.Open))))
             {
             }